You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "kadirozde (via GitHub)" <gi...@apache.org> on 2023/11/29 01:35:52 UTC

[PR] HBASE-25972 Dual File Compactor [hbase]

kadirozde opened a new pull request, #5545:
URL: https://github.com/apache/hbase/pull/5545

   [Design doc](https://docs.google.com/document/d/1Ea42tEBh2X2fCq0_tXSe1BgEqBz58oswJULEbA8-MfI/edit)


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1851083879

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 10s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   | -0 :warning: |  patch  |   5m 40s |  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  |   2m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 14s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 218m 24s |  hbase-server in the patch passed.  |
   |  |   | 240m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 03a194f97cac 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 78c5ac3725 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/testReport/ |
   | Max. process+thread count | 4729 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   StoreEngine and related interfaces have evolved organically and the current state is maybe not ideal.
   
   If we take the above approach, to refactor the default compactor interface, perhaps these called out compactors and engines can be refactored to take a cleaner approach (imho). If we look at the pattern that other recent refactors have taken there might be cause to consolidate common logic into an abstract class named appropriately and have the store engines inherit from that. This could be follow up work. Not asking for this work to be performed for 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416119851


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   It's possible to refactor this so DefaultCompactor becomes a dual file compactor that only compacts a single file. Can we try that instead? I do not think we need a new Store Engine for this. That introduces some configuration trouble for operators. Ideally operators do not need to change their store engine in order to take advantage of this and other improvements that are generally applicable and are in other respects backwards compatible. 
   
   Related comment from the design doc: 
   https://docs.google.com/document/d/1Ea42tEBh2X2fCq0_tXSe1BgEqBz58oswJULEbA8-MfI/edit?disco=AAABAgKl--o
   > HAS_LATEST_VERSION handling could be introduced into the default store engine and compatibility is assured given how you handle HFiles that lack this metadata. Also older versions that don't know about and ignore HAS_LATEST_VERSION will also function correctly because all HFiles will be examined as before.
   > It seems that we can incrementally upgrade or downgrade from a store engine that understands HAS_LATEST_VERSION and one that does not, unless I am missing something, which is certainly possible. Is my understanding correct? If so I am wondering if we really need a new StoreEngine implementation.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   ... and if we take the above approach, to refactor the default compactor to become a multi file compactor that only compacts to a single file, these called out compactors can possibly be refactored to inherit from `DefaultStoreEngine` to take a cleaner approach (imho), although that would be follow up work. I am not suggesting that be done for 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   StoreEngine and related interfaces have evolved organically and the current state is maybe not ideal.
   
   If we take the above approach, to refactor the default compactor to become a multi file compactor that only compacts to a single file, these called out compactors and engines can possibly be refactored to inherit from `DefaultStoreEngine` to take a cleaner approach (imho), although that would be follow up work. It would be worth taking a look at that. Later. Of course I am not suggesting that be done for this PR.
   Or if we look at the pattern that other recent refactors have taken there might be cause to consolidate common logic into an abstract class named appropriately and have the store engines inherit from 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443550627


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        deleteFamily = cell;
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (!isDeletedByDeleteFamily(cell)) {

Review Comment:
   Just in case, if DeleteFamily and DeleteFamilyVersion both have same timestamp (if at all possible), maybe this could be better handled with `if (!isDeletedByDeleteFamily(cell) && deleteFamily == null)`?
   
   Although from data correctness, this is likely not an issue. It's just to protect DeleteFamilyVersion from going to live version file if DeleteFamily is already present in live version file with same timestamp. Maybe it's just a corner case.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1930646978

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  master passed  |
   | -0 :warning: |  patch  |   6m  9s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 20s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 217m 26s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  13m 53s |  hbase-mapreduce in the patch passed.  |
   |  |   | 255m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e2af0139a9ae 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 16de74c194 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/testReport/ |
   | Max. process+thread count | 5583 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000009812

   > @Apache9 @kadirozde How close is this to ready? I think I'm reading to cut the first RC0 of 2.6.0 on Monday, but this might be a nice addition to the release. We've delayed 2.6.0 for a long time so I'd rather not delay it much longer though.
   
   In the current PR, this feature is enabled for all compaction types but testing is done only using default compaction. Since the change is at the store file writer level, we think it should work for all. I can add checks to disable  it for other compactions as @Apache9 suggested earlier, for the interest of the time.   I can get this done today. However, after the store file writer restructuring done yesterday,  the PR has not been reviewed yet. So, completing the review may some time. 


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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1962855097

   @Apache9 , Thank you for this detailed response. Based your feedback, I will integrate dual file writing into StoreFileWriter. StoreFileWriter will be able to separate cells into live and historical and write them to separate files when dual file writing is enabled by config. By doing so, DualFileWriter will be eliminated. Dual file writing will be supported only for default store engine initially (that is, in this PR). Please let me know if I misunderstood your suggestion or you want to add more things. 


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028937969

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 39s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 11s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 51s |  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  |   5m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 202m 54s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  12m 52s |  hbase-mapreduce in the patch passed.  |
   |  |   | 240m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e40ab7e3a51f 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/testReport/ |
   | Max. process+thread count | 5447 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028489821

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 56s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 48s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 56s |  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  |   5m 48s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 223m 49s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 49s |  hbase-mapreduce in the patch passed.  |
   |  |   | 265m 55s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 04daf4e2b5aa 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/testReport/ |
   | Max. process+thread count | 4470 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558971974


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();

Review Comment:
   Checked the code, besides calling the (old) StoreFileWriter, this method is only called in HFileOutputFormat2, for appending metadata for store files generated by MR job. Seems better to introduce a special method in StoreFileWriter for it. Anyway, not a problem for this PR. Can file another issue for it.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2067813556

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 44s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 10s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 50s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | -0 :warning: |  patch  |   6m 48s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 236m 35s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  15m  8s |  hbase-mapreduce in the patch passed.  |
   |  |   | 278m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8464e44b0505 5.4.0-174-generic #193-Ubuntu SMP Thu Mar 7 14:29:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3539581268 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/testReport/ |
   | Max. process+thread count | 5716 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2115235512

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 41s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 48s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 52s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  1s |  master passed  |
   | -0 :warning: |  patch  |   8m 14s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 33s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 263m  9s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  13m 38s |  hbase-mapreduce in the patch passed.  |
   |  |   | 309m 51s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8a6f4107cf90 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/testReport/ |
   | Max. process+thread count | 4532 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2114088475

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/32/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

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


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998519608

   @Apache9, I have moved historical file generation to StoreFileWriter. Based on a new config param called hbase.enable.historical.compaction.files (please suggest a better name if you like), StoreFileWriter uses a single file (just live file) or two file (live vs historical). This new config param replaces the previous one (hbase.hstore.defaultengine.enable.dualfilewriter). 
   
   I also changed the name of the new store file metadata key. Now it is called HISTORICAL. This is  set only when the file is historical. 
   
   Currently, we generate historical files only for compactions. However, we can enable it for flushes too. We can introduce another config param say base.enable.historical.flush.files to enable it for flushes. It will be a simple change. Please let me know if you like me to include this change too.
   
   Since historical file generated at the store file, it is available for all types of store engines and compactions by default. I have not disabled it for specific types of compactions yet. Either we can disable it for them or add tests to cover them. Let me know how you like to proceed.
   
   I also introduced a comprehensive data integrity checks with a new integration test class called TestStoreFileWriter. The test identified data integrity issues with version delete markers. I verified that these issues are not introduced by this PR and currently exist in hbase. I will file a jira for this. 
   
   After your feedback, I will update the design doc and 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998881646

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 26s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 56s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  master passed  |
   | -0 :warning: |  patch  |   6m  4s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 270m 25s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  18m 45s |  hbase-mapreduce in the patch passed.  |
   |  |   | 313m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3b7be1915754 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/testReport/ |
   | Max. process+thread count | 5269 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837334676

   :broken_heart: **-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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 47s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 59s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 42s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 24s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-server: The patch generated 7 new + 37 unchanged - 0 fixed = 44 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 45s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 33s |  patch has 64 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   1m 30s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux d3579f16db9e 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1871021530

   :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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 20s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 45s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 43s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 20s |  master passed  |
   | -0 :warning: |  patch  |   0m 44s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 18s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 40s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 45s |  hbase-server: The patch generated 2 new + 56 unchanged - 0 fixed = 58 total (was 56)  |
   | -0 :warning: |  checkstyle  |   0m 15s |  hbase-mapreduce: The patch generated 1 new + 27 unchanged - 0 fixed = 28 total (was 27)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 51s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 54s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 46s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  43m 23s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 9eb2d564ba31 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 75801a5291 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/artifact/yetus-general-check/output/diff-checkstyle-hbase-mapreduce.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416119851


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   Is it possible to refactor this so DefaultCompactor becomes a dual file compactor that only compacts a single file? Can we try that instead? I do not think we need a new Store Engine for this. That introduces some configuration trouble for operators. Ideally operators do not need to change their store engine in order to take advantage of this and other improvements that are generally applicable and are in other respects backwards compatible. 
   
   Related comment on design doc: https://docs.google.com/document/d/1Ea42tEBh2X2fCq0_tXSe1BgEqBz58oswJULEbA8-MfI/edit?disco=AAABAgKl--o
   > HAS_LATEST_VERSION handling could be introduced into the default store engine and compatibility is assured given how you handle HFiles that lack this metadata. Also older versions that don't know about and ignore HAS_LATEST_VERSION will also function correctly because all HFiles will be examined as before.
   > It seems that we can incrementally upgrade or downgrade from a store engine that understands HAS_LATEST_VERSION and one that does not, unless I am missing something, which is certainly possible. Is my understanding correct? If so I am wondering if we really need a new StoreEngine implementation.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1424412442


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   Please see the updated version where I refactored the existing code without introducing a new compactor or store engine.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442683796


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -124,7 +124,7 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
    */
   public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
     boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
-    ScanQueryMatcher matcher, long readPt) throws IOException {
+    ScanQueryMatcher matcher, long readPt, boolean onlyLatestVersion) throws IOException {

Review Comment:
   I will fix 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443690909


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   DefaultStoreFileManager needs to read the store files while loading them to find out if they are live or historical files. Doing this at the load time once eliminates it doing at the scan time every time. This was suggested during previous reviews. Reading the files requires handling IOException.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443816313


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        deleteFamily = cell;
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (!isDeletedByDeleteFamily(cell)) {

Review Comment:
   Sounds good, that makes sense.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1487000133


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java:
##########
@@ -70,36 +72,17 @@ protected void createComponents(Configuration conf, HStore store, CellComparator
   }
 
   protected void createCompactor(Configuration conf, HStore store) throws IOException {
-    String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
-    try {
-      compactor = ReflectionUtils.instantiateWithCustomCtor(className,
-        new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
-    } catch (Exception e) {
-      throw new IOException("Unable to load configured compactor '" + className + "'", e);
-    }
+    createCompactor(conf, store, DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
   }
 
   protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException {
-    String className =
-      conf.get(DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
-    try {
-      compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
-        new Class[] { Configuration.class, StoreConfigInformation.class },
-        new Object[] { conf, store });
-    } catch (Exception e) {
-      throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
-    }
+    createCompactionPolicy(conf, store, DEFAULT_COMPACTION_POLICY_CLASS_KEY,

Review Comment:
   Adding 1 to the min value when dual file writing is enabled sounds good to me. If there is no objection for this, I can prepare the change for it.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1916090671

   After fixing checkstyles, I am +1 for the changes in its current form. Unless there is strong opinion in the favor of changing MOB/Strip compactions, let's tackle them on separate sub-tasks.
   
   I have reviewed the entire change and the perf improvement for the mutable usecases is massive. I am in favour of pushing this sooner, sometime around start of next week unless there is any objections.
   
   FYI @apurtell @Apache9 @bbeaudreault @wchevreuil


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558965605


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);

Review Comment:
   Checked the old code, this method is only called in StoreFileWriter.append, so I think you can just remove this method here, only keep it in SIngleStoreFileWriter, and make it 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000437590

   @Apache9, @virajjasani, @bbeaudreault , This PR is not done yet. I just realized that I need to add testing for newVersionBehavior. 
   
   Also, I need to discuss the following case:
   
   Assume that for a given cell, two versions inserted and max versions is set to 1. If memory compaction is not enabled then I expect that both versions will be written to a new hfile (hf1) during flush even though the second version is redundant (is that true? I need to verify this). Now during minor compaction, the latest version will be written to a new live file (hf2) and the redundant version to a new historical file (hf3). Assume that a delete version marker is inserted for the latest version. This delete marker will be written to a new hfile (hf4). This delete marker will mask mask the latest version, and regular scans for the latest versions will not return any of the versions of this cell as latest version is masked by the delete marker and the redundant version is in the historical file (hf3) will be omitted by these scans. When the major compaction happens I expect that the redundant version should be revived and will be written to a new live file (hf5). Now the redundant
  version would be visible to regular scans. Please let me know if any of this incorrect. 
   
   Please note this should not happen with newVersionBehavior as the deleted versions are considered toward total version count.  Should we enable this feature only when newVersionBehavior is enabled?


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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2001352366

   Added tests for new version behavior and found a bug in HBase and create the [jira](https://issues.apache.org/jira/browse/HBASE-28442) for that. One of the tests fails because of this bug. 
   The PR is now ready for your review. 


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545467925


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   By default all files are live file when the dual file writing is not enabled. Dual file writing is not supported for mob in 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028883478

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 34s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  6s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 45s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 52s |  master passed  |
   | -0 :warning: |  patch  |   0m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 39s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 39s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 39s |  hbase-server: The patch generated 1 new + 39 unchanged - 0 fixed = 40 total (was 39)  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m 39s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  6s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m 50s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux e2a7ab773db0 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 80 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "bbeaudreault (via GitHub)" <gi...@apache.org>.
bbeaudreault commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545798434


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   Could you check if mob is enabled and, if so, log a warn and ignore the dual writer config value?



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1577320047


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   Sorry I missed this one before. When DefaultStoreFileManager loads the store files, it also reads the file metadata to see if they are live or not in order to form the list of live store files. Reading file can generate IOException. Please see
   ```
   private List<HStoreFile> getLiveFiles(Collection<HStoreFile> storeFiles) throws IOException {
       List<HStoreFile> liveFiles = new ArrayList<>(storeFiles.size());
       for (HStoreFile file : storeFiles) {
         file.initReader();
         if (!file.isHistorical()) {
           liveFiles.add(file);
         }
       }
       return liveFiles;
     }
   
     @Override
     public void loadFiles(List<HStoreFile> storeFiles) throws IOException {
       this.storeFiles = new StoreFileList(ImmutableList.sortedCopyOf(storeFileComparator, storeFiles),
         enableLiveFileTracking
           ? ImmutableList.sortedCopyOf(storeFileComparator, getLiveFiles(storeFiles))
           : null);
     }
   ```



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2114510060

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 55s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 12s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 52s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 45s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  3s |  master passed  |
   | -0 :warning: |  patch  |   1m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 52s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 11s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 11s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 37s |  hbase-server: The patch generated 1 new + 46 unchanged - 0 fixed = 47 total (was 46)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   4m 57s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 42s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 16s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 20s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 1bf9fde0dcfe 5.4.0-174-generic #193-Ubuntu SMP Thu Mar 7 14:29:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1412493239


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
-      sortedFiles.add(file);
+      if (onlyLatestVersion) {
+        if (file.hasLatestVersion()) {
+          sortedFiles.add(file);
+        }
+      } else {
+        sortedFiles.add(file);
+      }
     }
     boolean succ = false;
     try {
-      for (int i = 0, n = files.size(); i < n; i++) {
+      for (int i = 0, n = files.size(); i < n && !sortedFiles.isEmpty(); i++) {

Review Comment:
   I did want to iterate over over sorted files for the same reason the existing code does not iterate over sorted files since sorted files are modified within the loop. I do not see the NPE issue here.



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

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

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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1831314147

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-server generated 1 new + 22 unchanged - 0 fixed = 23 total (was 22)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 287m 18s |  hbase-server in the patch passed.  |
   |  |   | 309m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c375f9c58772 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / dbfb516a55 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/testReport/ |
   | Max. process+thread count | 3993 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1413023401


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();

Review Comment:
   Good suggestion! I will add DualStoreFileManager.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837471519

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 44s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 15s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 38s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-server generated 1 new + 95 unchanged - 0 fixed = 96 total (was 95)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 214m 16s |  hbase-server in the patch passed.  |
   |  |   | 236m 34s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4b37c6d11f8a 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/testReport/ |
   | Max. process+thread count | 4686 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1851977876

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   | -0 :warning: |  patch  |   5m 19s |  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  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 48s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 274m 27s |  hbase-server in the patch passed.  |
   |  |   | 296m  9s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 234c827e3fe6 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 78c5ac3725 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/testReport/ |
   | Max. process+thread count | 4105 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442383606


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,48 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
   private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);

Review Comment:
   nit: no longer in use



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java:
##########
@@ -478,6 +477,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
             return false;
           }
         }
+        writer.appendAll(cells);

Review Comment:
   With this change, earlier if `hbase.hstore.close.check.interval` limit was reached, we would have appended the cell in the loop but now we would not have appended any cell. Though this change seems correct because even if the limit is reached, we already reset compaction counters with `progress.cancel()`.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        if (cell.getType() == Cell.Type.DeleteFamily) {
+          deleteFamily = cell;
+          addLiveVersion(cell);
+        } else {
+          addHistoricalVersion(cell);
+        }
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (deleteFamily == null) {
+        deleteFamilyVersionList.add(cell);
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.DeleteColumn) {
+      if (!isDeletedByDeleteFamily(cell) && deleteColumn == null) {
+        deleteColumn = cell;
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.Delete) {
+      if (!isDeletedByDeleteFamily(cell) && deleteColumn == null) {
+        deleteColumnVersionList.add(cell);
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.Put) {
+      if (livePutCellCount < maxVersions) {
+        // This is a live put cell (i.e., the latest version) of a column. Is it deleted?
+        if (!isDeleted(cell)) {
+          addLiveVersion(cell);
+          livePutCellCount++;
+        } else {
+          // It is deleted
+          addHistoricalVersion(cell);
+        }
+      } else {
+        // It is an older put cell
+        addHistoricalVersion(cell);
+      }
+    }
+    lastCell = cell;
+  }
+
+  @Override
+  public void appendAll(List<Cell> cellList) throws IOException {
+    if (!dualWriterEnabled) {
+      // If the dual writer is not enabled then all cells are written to one file. We use
+      // the live version file in this case
+      for (Cell cell : cellList) {
+        addLiveVersion(cell);
+      }
+      return;
+    }
+    if (cellList.isEmpty()) {
+      return;
+    }
+    if (lastCell != null && comparator.compareRows(lastCell, cellList.get(0)) != 0) {
+      // It is a new row and thus time to reset the state
+      initRowState();
+    }
+    for (Cell cell : cellList) {
+      appendCell(cell);
+    }
+  }
+
+  @Override
+  public void append(Cell cell) throws IOException {
+    if (!dualWriterEnabled) {
+      // If the dual writer is not enabled then all cells are written to one file. We use
+      // the live version file in this case
+      addLiveVersion(cell);
+      return;
+    }
+    if (lastCell != null && comparator.compareRows(lastCell, cell) != 0) {
+      // It is a new row and thus time to reset the state
+      initRowState();
+    }
+    appendCell(cell);
+  }
+
+  @Override
+  protected Collection<StoreFileWriter> writers() {
+    return writers;
+  }
+
+  @Override
+  protected void preCommitWriters() throws IOException {
+    if (writers.isEmpty()) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    if (!dualWriterEnabled) {
+      return;
+    }
+    if (liveVersionWriter != null) {
+      liveVersionWriter.appendFileInfo(HAS_LIVE_VERSIONS_KEY, Bytes.toBytes(true));
+    }
+    if (historicalVersionWriter != null) {
+      historicalVersionWriter.appendFileInfo(HAS_LIVE_VERSIONS_KEY, Bytes.toBytes(false));
+    }
+  }
+
+  public HFile.Writer getHFileWriter() {

Review Comment:
   nit: instead of `getHFileWriter()`, maybe we can call it `getLiveVersionHFileWriter()`?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java:
##########
@@ -138,6 +140,12 @@ public class HStoreFile implements StoreFile {
   // Indicates if the file got compacted
   private volatile boolean compactedAway = false;
 
+  // Indicate if the file contains live cell versions for a given column
+  // in a row. MemStore flushes generate files with all cell versions. However,
+  // compactions can generate two files, one with the live cell versions and the other
+  // with the remaining (historical) cell versions.
+  private volatile boolean hasLiveVersions = true;

Review Comment:
   I was thinking why this is not kept `false` by default, but now i realized that it is only used by `DefaultStoreFileManager#getLiveVersionFiles` which will only be used by callers if `hbase.hstore.defaultengine.enable.dualfilewriter` is enabled. 
   
   Hence, unless dual file writer is enabled, this is not even used, so it's default value here doesn't matter. Is that correct? If so, maybe we can comment the same here so that no one tries to use this value without checking for `hbase.hstore.defaultengine.enable.dualfilewriter`.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        if (cell.getType() == Cell.Type.DeleteFamily) {
+          deleteFamily = cell;
+          addLiveVersion(cell);
+        } else {
+          addHistoricalVersion(cell);
+        }
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (deleteFamily == null) {
+        deleteFamilyVersionList.add(cell);
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.DeleteColumn) {
+      if (!isDeletedByDeleteFamily(cell) && deleteColumn == null) {
+        deleteColumn = cell;
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.Delete) {
+      if (!isDeletedByDeleteFamily(cell) && deleteColumn == null) {
+        deleteColumnVersionList.add(cell);
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.Put) {
+      if (livePutCellCount < maxVersions) {
+        // This is a live put cell (i.e., the latest version) of a column. Is it deleted?
+        if (!isDeleted(cell)) {
+          addLiveVersion(cell);
+          livePutCellCount++;
+        } else {
+          // It is deleted
+          addHistoricalVersion(cell);
+        }
+      } else {
+        // It is an older put cell
+        addHistoricalVersion(cell);
+      }
+    }
+    lastCell = cell;
+  }
+
+  @Override
+  public void appendAll(List<Cell> cellList) throws IOException {
+    if (!dualWriterEnabled) {
+      // If the dual writer is not enabled then all cells are written to one file. We use
+      // the live version file in this case
+      for (Cell cell : cellList) {
+        addLiveVersion(cell);
+      }
+      return;
+    }
+    if (cellList.isEmpty()) {
+      return;
+    }
+    if (lastCell != null && comparator.compareRows(lastCell, cellList.get(0)) != 0) {
+      // It is a new row and thus time to reset the state
+      initRowState();
+    }
+    for (Cell cell : cellList) {
+      appendCell(cell);
+    }
+  }
+
+  @Override
+  public void append(Cell cell) throws IOException {
+    if (!dualWriterEnabled) {
+      // If the dual writer is not enabled then all cells are written to one file. We use
+      // the live version file in this case
+      addLiveVersion(cell);
+      return;
+    }
+    if (lastCell != null && comparator.compareRows(lastCell, cell) != 0) {
+      // It is a new row and thus time to reset the state
+      initRowState();
+    }
+    appendCell(cell);
+  }
+
+  @Override
+  protected Collection<StoreFileWriter> writers() {
+    return writers;
+  }
+
+  @Override
+  protected void preCommitWriters() throws IOException {
+    if (writers.isEmpty()) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }

Review Comment:
   Is it possible for writers to be empty in precommit phase? If so, we still don't need to initialize `liveVersionWriter` here? or i am missing something?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        if (cell.getType() == Cell.Type.DeleteFamily) {
+          deleteFamily = cell;
+          addLiveVersion(cell);
+        } else {
+          addHistoricalVersion(cell);
+        }
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (deleteFamily == null) {

Review Comment:
   Do we not need check here to see if the DeleteFamilyVersion is deleted by DeleteFamily?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java:
##########
@@ -46,13 +45,16 @@ public class DefaultStoreEngine extends StoreEngine<DefaultStoreFlusher, RatioBa
     "hbase.hstore.defaultengine.compactor.class";
   public static final String DEFAULT_COMPACTION_POLICY_CLASS_KEY =
     "hbase.hstore.defaultengine.compactionpolicy.class";
+  public static final String DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY =
+    "hbase.hstore.defaultengine.enable.dualfilewriter";
 
   private static final Class<? extends DefaultStoreFlusher> DEFAULT_STORE_FLUSHER_CLASS =
     DefaultStoreFlusher.class;
   private static final Class<? extends DefaultCompactor> DEFAULT_COMPACTOR_CLASS =
     DefaultCompactor.class;
-  private static final Class<? extends RatioBasedCompactionPolicy> DEFAULT_COMPACTION_POLICY_CLASS =
+  public static final Class<? extends RatioBasedCompactionPolicy> DEFAULT_COMPACTION_POLICY_CLASS =
     ExploringCompactionPolicy.class;
+  public static final boolean DEFAULT_ENABLE_DUAL_FILE_WRITER = false;

Review Comment:
   I think temporarily we can change this to `true` and see how build results go? after all tests pass, we can change it back to `false` before committing the changes.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -124,7 +124,7 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
    */
   public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
     boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
-    ScanQueryMatcher matcher, long readPt) throws IOException {
+    ScanQueryMatcher matcher, long readPt, boolean onlyLatestVersion) throws IOException {

Review Comment:
   not required?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java:
##########
@@ -223,6 +224,10 @@ private void addCurrentScanners(List<? extends KeyValueScanner> scanners) {
     this.currentScanners.addAll(scanners);
   }
 
+  private static boolean isOnlyLatestVersionScan(Scan scan) {
+    return !scan.isRaw() && scan.getTimeRange().getMax() == HConstants.LATEST_TIMESTAMP;
+  }

Review Comment:
   nit: perhaps good to comment that we don't need to check for Scan#getMaxVersions because live version file generated by dual file writer will anyways retain max versions specified in ColumnFamilyDescriptor for the given CF?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        if (cell.getType() == Cell.Type.DeleteFamily) {
+          deleteFamily = cell;
+          addLiveVersion(cell);
+        } else {
+          addHistoricalVersion(cell);
+        }
+      }

Review Comment:
   `if (cell.getType() == Cell.Type.DeleteFamily)` is repeated, which means we are not adding the cell to historical version.
   
   I think this was meant to be:
   ```
       if (cell.getType() == Cell.Type.DeleteFamily) {
         if (deleteFamily == null) {
           deleteFamily = cell;
           addLiveVersion(cell);
         } else {
           addHistoricalVersion(cell);
         }
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1879666334

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 51s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 47s |  master passed  |
   | -0 :warning: |  patch  |   0m 36s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 17s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 32s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 37s |  hbase-server: The patch generated 2 new + 42 unchanged - 0 fixed = 44 total (was 42)  |
   | -0 :warning: |  checkstyle  |   0m 10s |  hbase-mapreduce: The patch generated 1 new + 27 unchanged - 0 fixed = 28 total (was 27)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m 23s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | -1 :x: |  spotless  |   0m 33s |  patch has 26 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   2m  3s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 464c446c3b6d 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4aeabdcc71 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-general-check/output/diff-checkstyle-hbase-mapreduce.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 83 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1890293446

   @kadirozde are you able to repro these test failures locally? https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/testReport/
   
   e.g. `TestVerifyReplicationCrossDiffHdfs`
   
   ```
   java.lang.NoClassDefFoundError: org/bouncycastle/operator/OperatorCreationException
   	at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceInit(ResourceManager.java:888)
   	at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164)
   	at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.createAndInitActiveServices(ResourceManager.java:1410)
   	at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceInit(ResourceManager.java:344)
   	at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164)
   	at org.apache.hadoop.yarn.server.MiniYARNCluster.initResourceManager(MiniYARNCluster.java:359)
   	at org.apache.hadoop.yarn.server.MiniYARNCluster.access$200(MiniYARNCluster.java:129)
   	at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper.serviceInit(MiniYARNCluster.java:494)
   
   ```


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998576326

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | -0 :warning: |  patch  |   6m  2s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  7s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  11m 15s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 46s |  hbase-mapreduce in the patch passed.  |
   |  |   |  47m 54s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 976fbdbfeb8c 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/testReport/ |
   | Max. process+thread count | 3225 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998556843

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 41s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 52s |  master passed  |
   | -0 :warning: |  patch  |   0m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 58s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 37s |  hbase-server: The patch generated 1 new + 37 unchanged - 0 fixed = 38 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  8s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 2f60f6f09d2b 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1999254901

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 57s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 50s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 41s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 54s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 218m 40s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 28s |  hbase-mapreduce in the patch passed.  |
   |  |   | 259m 26s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 6b6d67620c4f 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/testReport/ |
   | Max. process+thread count | 5216 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2048360084

   > For me the only big problem is how to deal with the updating of store file list.
   
   Please check my response on this. If you still want me to add a lock to protect the store file list updates, I will do 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2067761930

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 57s |  master passed  |
   | -0 :warning: |  patch  |   0m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 20s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 20s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 45s |  hbase-server: The patch generated 1 new + 39 unchanged - 0 fixed = 40 total (was 39)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m 49s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 50s |  patch has no errors when running spotless:check.  |
   | +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.  |
   |  |   |  33m 55s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux b497492f055f 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3539581268 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 81 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2001888007

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 19s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m  9s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  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  |   5m 21s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 201m  1s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  12m 56s |  hbase-mapreduce in the patch passed.  |
   |  |   | 238m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4ba6ecea3fe6 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-jdk17-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/testReport/ |
   | Max. process+thread count | 5703 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545469228


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();

Review Comment:
   I did not change the existing behavior. I refactored the code such that the exiting StoreFileWriter implementation was moved to the new nested class called SingleStoreFileWriter. StoreFileWriter has now two SingleStoreFileWriter objects, liveFileWriter, and historicalFileWriter.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2074177955

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/29/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2114087829

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/32/console |
   | versions | git=2.25.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1602564157


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   I do not know the answer. I was going to debug and find the answer. However, I could not find a test that calls this method with non-empty storeFiles. I need to device a test where I need to generate some files and then close the store, and reopen it while debugging it. I will try to do it later.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2113965661

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/31/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837480931

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 23s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 55s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 55s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 40s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 30s |  hbase-server generated 1 new + 22 unchanged - 0 fixed = 23 total (was 22)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 248m  4s |  hbase-server in the patch passed.  |
   |  |   | 273m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9515f46f3708 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/testReport/ |
   | Max. process+thread count | 4292 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1839394162

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 43s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-server generated 1 new + 95 unchanged - 0 fixed = 96 total (was 95)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 214m 58s |  hbase-server in the patch passed.  |
   |  |   | 237m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux edbc1515060a 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 25e9228e2c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/testReport/ |
   | Max. process+thread count | 4743 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   ... and if we take the above approach, these can be refactored to inherit from  DefaultStoreEngine, although that would be follow up work. I am not suggesting that be done for 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1852568756

   > Just to be clear, based on my understanding of the design (see doc here: https://docs.google.com/document/d/1Ea42tEBh2X2fCq0_tXSe1BgEqBz58oswJULEbA8-MfI/edit ) we can integrate this change into the default store engine without requiring opt in and everyone will benefit from the optimization. Compatibility is assured given how we handle HFiles that lack the new metadata. Older HBase versions that don't know about and ignore HAS_LATEST_VERSION will also function correctly because all HFiles will be examined as before. Upgrade to and downgrade from a HAS_LATEST_VERSION capable version does not pose a correctness problem. Operation with mixed HFiles from different versions is also fine. Its simply that the performance benefit is fully realized once upgraded to HAS_LATEST_VERSION capable version and compaction has run on all live regions.
   > 
   > However, in case someone is concerned about potential impacts, please prepare to make the new behavior opt in via a site configuration setting. Hopefully we can achieve a consensus and avoid that.
   
   @apurtell, as per your feedback, I eliminated DualFileStoreEngine, DualFileCompactor, and DualFileStoreFileManager.  Their functionality is now integrated into DefaultStoreEngine, DefaultCompactor and DefaultStoreFileManager respectively with some refactoring. Dual file compaction can be turned on/off using a config parameter. By default, it is turned on now. I did that to make sure that existing tests will exercise the new code. We can change the default value before we merge the 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1942365621

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 26s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 49s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 49s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 27s |  master passed  |
   | -0 :warning: |  patch  |   0m 47s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 50s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m 34s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 49s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 34s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 19s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 2b90551db0d7 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9656006778 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545470519


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);
+    if (historicalFileWriter != null) {
+      historicalFileWriter.trackTimestamps(cell);
     }
-    timeRangeTracker.includeTimestamp(cell);
   }
 
-  private void appendGeneralBloomfilter(final Cell cell) throws IOException {
-    if (this.generalBloomFilterWriter != null) {
-      /*
-       * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
-       * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + Timestamp 3 Types of
-       * Filtering: 1. Row = Row 2. RowCol = Row + Qualifier 3. RowPrefixFixedLength = Fixed Length
-       * Row Prefix
-       */
-      bloomContext.writeBloom(cell);
+  @Override
+  public void beforeShipped() throws IOException {
+    liveFileWriter.beforeShipped();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.beforeShipped();
     }
   }
 
-  private void appendDeleteFamilyBloomFilter(final Cell cell) throws IOException {
-    if (!PrivateCellUtil.isDeleteFamily(cell) && !PrivateCellUtil.isDeleteFamilyVersion(cell)) {
-      return;
-    }
+  public Path getPath() {
+    return liveFileWriter.getPath();

Review Comment:
   I had to introduce another method called getPaths to get the list of paths, that is , up to two paths, one for the live file and the other for the historical file. getPath is currently called by the compactions that do not support dual file writing yet.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028493399

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 47s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 31s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | -0 :warning: |  patch  |   6m  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 _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 10s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 245m 56s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 55s |  hbase-mapreduce in the patch passed.  |
   |  |   | 285m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4b4d4ed618dd 5.4.0-174-generic #193-Ubuntu SMP Thu Mar 7 14:29:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/testReport/ |
   | Max. process+thread count | 5260 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000699516

   > @kadirozde i just verified with HBase 2.6 (branch-2) that the flush writes only maxVersions versions to the new HFile. Hence, if the max version is 1, and if we write 2 versions of the cell, only the latest cell is written to the HFile.
   > So we should be good for the first case.
   > 
   > At this point, we can add some tests for newVersionBehavior, and we should be good for the above concern.
   
   @virajjasani, Thank you for verifying this.  I had the wrong assumption about the memstore behavior and thus thought that there were issues in hbase with handling delete version markers. I will add new tests and wrap up 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998509877

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 46s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 20s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 11s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  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  |   5m 18s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  11m 51s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  12m 41s |  hbase-mapreduce in the patch passed.  |
   |  |   |  47m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c380e2c286db 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-jdk17-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/testReport/ |
   | Max. process+thread count | 2961 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998843591

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 12s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  4s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 59s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 50s |  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 13s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 23s |  hbase-mapreduce in the patch passed.  |
   |  |   | 259m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 18e8acd4cc88 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/testReport/ |
   | Max. process+thread count | 5088 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2075520201

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  0s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 48s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 41s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 54s |  master passed  |
   | -0 :warning: |  patch  |   0m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 59s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 35s |  hbase-server: The patch generated 1 new + 46 unchanged - 0 fixed = 47 total (was 46)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  9s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 46s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 58s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 25s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 3b9f5a04bc13 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6c6e776eea |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 82 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2075919947

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | -0 :warning: |  patch  |   6m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 33s |  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  |   5m  9s |  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  | 240m 30s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  15m 21s |  hbase-mapreduce in the patch passed.  |
   |  |   | 280m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9c186612914f 5.4.0-176-generic #196-Ubuntu SMP Fri Mar 22 16:46:39 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6c6e776eea |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/testReport/ |
   | Max. process+thread count | 5077 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2113965652

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/31/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2113965649

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  2s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/31/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2114088119

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/32/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2115317652

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 28s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 17s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  master passed  |
   | -0 :warning: |  patch  |   8m 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 _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  5s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 293m 35s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  20m 42s |  hbase-mapreduce in the patch passed.  |
   |  |   | 346m 31s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux aed95bfce225 5.4.0-172-generic #190-Ubuntu SMP Fri Feb 2 23:24:22 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/testReport/ |
   | Max. process+thread count | 4880 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1604229161


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileWriter.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 static org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.NEW_VERSION_BEHAVIOR;
+import static org.apache.hadoop.hbase.regionserver.StoreFileWriter.ENABLE_HISTORICAL_COMPACTION_FILES;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.MemoryCompactionPolicy;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Store file writer does not do any compaction. Each cell written to either the live or historical
+ * file. Regular (i.e., not-raw) scans that reads the latest put cells scans only live files. To
+ * ensure the correctness of store file writer, we need to verify that live files includes all live
+ * cells. This test indirectly verify this as follows. The test creates two tables, each with one
+ * region and one store. The dual file writing (live vs historical) is configured on only one of the
+ * tables. The test generates exact set of mutations on both tables. These mutations include all
+ * types of cells and these cells are written to multiple files using multiple memstore flushes.
+ * After writing all cells, the test first verify that both tables return the same set of cells for
+ * regular and raw scans. Then the same verification is done after tables are minor and finally
+ * major compacted. The test also verifies that flushes do not generate historical files and the
+ * historical files are generated only when historical file generation is enabled (by the config
+ * hbase.enable.historical.compaction.files).
+ */
+@Category({ MediumTests.class, RegionServerTests.class })
+@RunWith(Parameterized.class)
+public class TestStoreFileWriter {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStoreFileWriter.class);
+  private final int ROW_NUM = 100;
+  private final Random RANDOM = new Random(11);
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private HRegion[] regions = new HRegion[2];
+  private final byte[][] qualifiers =
+    { Bytes.toBytes("0"), Bytes.toBytes("1"), Bytes.toBytes("2") };
+  // This keeps track of all cells. It is a list of rows, each row is a list of columns, each
+  // column is a list of CellInfo object
+  private ArrayList<ArrayList<ArrayList<CellInfo>>> insertedCells;
+  private TableName[] tableName = new TableName[2];
+  private final Configuration conf = testUtil.getConfiguration();
+  private int flushCount = 0;
+
+  @Parameterized.Parameter(0)
+  public KeepDeletedCells keepDeletedCells;
+  @Parameterized.Parameter(1)
+  public int maxVersions;
+  @Parameterized.Parameter(2)
+  public boolean newVersionBehavior;
+
+  @Parameterized.Parameters(name = "keepDeletedCells={0}, maxVersions={1}, newVersionBehavior={2}")
+  public static synchronized Collection<Object[]> data() {
+    return Arrays.asList(
+      new Object[][] { { KeepDeletedCells.FALSE, 1, true }, { KeepDeletedCells.FALSE, 2, false },
+        { KeepDeletedCells.FALSE, 3, true }, { KeepDeletedCells.TRUE, 1, false },
+        // { KeepDeletedCells.TRUE, 2, true }, see HBASE-28442
+        { KeepDeletedCells.TRUE, 3, false } });
+  }
+
+  // In memory representation of a cell. We only need to know timestamp and type field for our
+  // testing for cell. Please note the row for the cell is implicit in insertedCells.
+  private static class CellInfo {
+    long timestamp;
+    Cell.Type type;
+
+    CellInfo(long timestamp, Cell.Type type) {
+      this.timestamp = timestamp;
+      this.type = type;
+    }
+  }
+
+  private void createTable(int index, boolean enableDualFileWriter) throws IOException {
+    tableName[index] = TableName.valueOf(getClass().getSimpleName() + "_" + index);
+    ColumnFamilyDescriptor familyDescriptor =
+      ColumnFamilyDescriptorBuilder.newBuilder(HBaseTestingUtil.fam1).setMaxVersions(maxVersions)
+        .setKeepDeletedCells(keepDeletedCells)
+        .setValue(NEW_VERSION_BEHAVIOR, Boolean.toString(newVersionBehavior)).build();
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(tableName[index]).setColumnFamily(familyDescriptor)
+        .setValue(ENABLE_HISTORICAL_COMPACTION_FILES, Boolean.toString(enableDualFileWriter));
+    testUtil.createTable(builder.build(), null);
+    regions[index] = testUtil.getMiniHBaseCluster().getRegions(tableName[index]).get(0);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 6);
+    conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
+      String.valueOf(MemoryCompactionPolicy.NONE));
+    testUtil.startMiniCluster();
+    createTable(0, false);
+    createTable(1, true);
+    insertedCells = new ArrayList<>(ROW_NUM);
+    for (int r = 0; r < ROW_NUM; r++) {
+      insertedCells.add(new ArrayList<>(qualifiers.length));
+      for (int q = 0; q < qualifiers.length; q++) {
+        insertedCells.get(r).add(new ArrayList<>(10));
+      }
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    this.testUtil.shutdownMiniCluster();
+    testUtil.cleanupTestDir();
+  }
+
+  @Test
+  public void testCompactedFiles() throws Exception {
+    for (int i = 0; i < 10; i++) {
+      insertRows(ROW_NUM * maxVersions);
+      deleteRows(ROW_NUM / 8);
+      deleteRowVersions(ROW_NUM / 8);
+      deleteColumns(ROW_NUM / 8);
+      deleteColumnVersions(ROW_NUM / 8);
+      flushRegion();
+    }
+
+    verifyCells();
+
+    HStore[] stores = new HStore[2];
+
+    stores[0] = regions[0].getStore(HBaseTestingUtil.fam1);
+    assertEquals(flushCount, stores[0].getStorefilesCount());
+
+    stores[1] = regions[1].getStore(HBaseTestingUtil.fam1);
+    assertEquals(flushCount, stores[1].getStorefilesCount());
+
+    regions[0].compact(false);
+    assertEquals(flushCount - stores[0].getCompactedFiles().size() + 1,
+      stores[0].getStorefilesCount());
+
+    regions[1].compact(false);
+    assertEquals(flushCount - stores[1].getCompactedFiles().size() + 2,
+      stores[1].getStorefilesCount());
+
+    verifyCells();
+
+    regions[0].compact(true);
+    assertEquals(1, stores[0].getStorefilesCount());
+
+    regions[1].compact(true);
+    assertEquals(keepDeletedCells == KeepDeletedCells.FALSE ? 1 : 2,
+      stores[1].getStorefilesCount());
+
+    verifyCells();
+  }
+
+  private void verifyCells() throws Exception {
+    scanAndCompare(false);
+    scanAndCompare(true);
+  }
+
+  private void flushRegion() throws Exception {
+    regions[0].flush(true);
+    regions[1].flush(true);
+    flushCount++;
+  }
+
+  private Long getRowTimestamp(int row) {
+    Long maxTimestamp = null;
+    for (int q = 0; q < qualifiers.length; q++) {
+      int size = insertedCells.get(row).get(q).size();
+      if (size > 0) {
+        CellInfo mostRecentCellInfo = insertedCells.get(row).get(q).get(size - 1);
+        if (mostRecentCellInfo.type == Cell.Type.Put) {
+          if (maxTimestamp == null || maxTimestamp < mostRecentCellInfo.timestamp) {
+            maxTimestamp = mostRecentCellInfo.timestamp;
+          }
+        }
+      }
+    }
+    return maxTimestamp;
+  }
+
+  private long getNewTimestamp(long timestamp) throws Exception {
+    long newTimestamp = System.currentTimeMillis();
+    if (timestamp == newTimestamp) {
+      Thread.sleep(1);
+      newTimestamp = System.currentTimeMillis();
+      assert (timestamp < newTimestamp);
+    }
+    return newTimestamp;
+  }
+
+  private void insertRows(int rowCount) throws Exception {
+    int row;
+    long timestamp = System.currentTimeMillis();
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Put put = new Put(Bytes.toBytes(String.valueOf(row)), timestamp);
+      for (int q = 0; q < qualifiers.length; q++) {
+        put.addColumn(HBaseTestingUtil.fam1, qualifiers[q],
+          Bytes.toBytes(String.valueOf(timestamp)));
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.Put));
+      }
+      regions[0].put(put);
+      regions[1].put(put);
+      timestamp = getNewTimestamp(timestamp);
+    }
+  }
+
+  private void deleteRows(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      long timestamp = System.currentTimeMillis();
+      row = RANDOM.nextInt(ROW_NUM);
+      Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+      regions[0].delete(delete);
+      regions[1].delete(delete);
+      // For simplicity, the family delete markers are inserted for all columns (instead of
+      // allocating a separate column for them) in the memory representation of the data stored
+      // to HBase
+      for (int q = 0; q < qualifiers.length; q++) {
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteFamily));
+      }
+    }
+  }
+
+  private void deleteSingleRowVersion(int row, long timestamp) throws IOException {
+    Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+    delete.addFamilyVersion(HBaseTestingUtil.fam1, timestamp);
+    regions[0].delete(delete);
+    regions[1].delete(delete);
+    // For simplicity, the family delete version markers are inserted for all columns (instead of
+    // allocating a separate column for them) in the memory representation of the data stored
+    // to HBase
+    for (int q = 0; q < qualifiers.length; q++) {
+      insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteFamilyVersion));
+    }
+  }
+
+  private void deleteRowVersions(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Long timestamp = getRowTimestamp(row);
+      if (timestamp != null) {
+        deleteSingleRowVersion(row, timestamp);
+      }
+    }
+    // Just insert one more delete marker possibly does not delete any row version
+    row = RANDOM.nextInt(ROW_NUM);
+    deleteSingleRowVersion(row, System.currentTimeMillis());
+  }
+
+  private void deleteColumns(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      long timestamp = System.currentTimeMillis();
+      row = RANDOM.nextInt(ROW_NUM);
+      int q = RANDOM.nextInt(qualifiers.length);
+      Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)), timestamp);
+      delete.addColumns(HBaseTestingUtil.fam1, qualifiers[q], timestamp);
+      regions[0].delete(delete);
+      regions[1].delete(delete);
+      insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteColumn));
+    }
+  }
+
+  private void deleteColumnVersions(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Long timestamp = getRowTimestamp(row);
+      if (timestamp != null) {
+        Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+        int q = RANDOM.nextInt(qualifiers.length);
+        delete.addColumn(HBaseTestingUtil.fam1, qualifiers[q], timestamp);
+        regions[0].delete(delete);
+        regions[1].delete(delete);
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.Delete));
+      }
+    }
+  }
+
+  private Scan createScan(boolean raw) {
+    Scan scan = new Scan();
+    scan.readAllVersions();
+    scan.setRaw(raw);
+    return scan;
+  }
+
+  private void scanAndCompare(boolean raw) throws Exception {
+    try (RegionScanner firstRS = regions[0].getScanner(createScan(raw))) {
+      try (RegionScanner secondRS = regions[1].getScanner(createScan(raw))) {
+        boolean firstHasMore;
+        boolean secondHasMore;
+        do {
+          List<Cell> firstRowList = new ArrayList<>();
+          List<Cell> secondRowList = new ArrayList<>();
+          firstHasMore = firstRS.nextRaw(firstRowList);
+          secondHasMore = secondRS.nextRaw(secondRowList);
+          assertEquals(firstRowList.size(), secondRowList.size());
+          int size = firstRowList.size();
+          for (int i = 0; i < size; i++) {
+            Cell firstCell = firstRowList.get(i);
+            Cell secondCell = secondRowList.get(i);
+            assert (CellUtil.matchingRowColumn(firstCell, secondCell));

Review Comment:
   nit: These asserts could be assertTrue or assertEquals, but this is not blocker for merging the 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1604284847


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileWriter.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 static org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.NEW_VERSION_BEHAVIOR;
+import static org.apache.hadoop.hbase.regionserver.StoreFileWriter.ENABLE_HISTORICAL_COMPACTION_FILES;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.MemoryCompactionPolicy;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Store file writer does not do any compaction. Each cell written to either the live or historical
+ * file. Regular (i.e., not-raw) scans that reads the latest put cells scans only live files. To
+ * ensure the correctness of store file writer, we need to verify that live files includes all live
+ * cells. This test indirectly verify this as follows. The test creates two tables, each with one
+ * region and one store. The dual file writing (live vs historical) is configured on only one of the
+ * tables. The test generates exact set of mutations on both tables. These mutations include all
+ * types of cells and these cells are written to multiple files using multiple memstore flushes.
+ * After writing all cells, the test first verify that both tables return the same set of cells for
+ * regular and raw scans. Then the same verification is done after tables are minor and finally
+ * major compacted. The test also verifies that flushes do not generate historical files and the
+ * historical files are generated only when historical file generation is enabled (by the config
+ * hbase.enable.historical.compaction.files).
+ */
+@Category({ MediumTests.class, RegionServerTests.class })
+@RunWith(Parameterized.class)
+public class TestStoreFileWriter {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStoreFileWriter.class);
+  private final int ROW_NUM = 100;
+  private final Random RANDOM = new Random(11);
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private HRegion[] regions = new HRegion[2];
+  private final byte[][] qualifiers =
+    { Bytes.toBytes("0"), Bytes.toBytes("1"), Bytes.toBytes("2") };
+  // This keeps track of all cells. It is a list of rows, each row is a list of columns, each
+  // column is a list of CellInfo object
+  private ArrayList<ArrayList<ArrayList<CellInfo>>> insertedCells;
+  private TableName[] tableName = new TableName[2];
+  private final Configuration conf = testUtil.getConfiguration();
+  private int flushCount = 0;
+
+  @Parameterized.Parameter(0)
+  public KeepDeletedCells keepDeletedCells;
+  @Parameterized.Parameter(1)
+  public int maxVersions;
+  @Parameterized.Parameter(2)
+  public boolean newVersionBehavior;
+
+  @Parameterized.Parameters(name = "keepDeletedCells={0}, maxVersions={1}, newVersionBehavior={2}")
+  public static synchronized Collection<Object[]> data() {
+    return Arrays.asList(
+      new Object[][] { { KeepDeletedCells.FALSE, 1, true }, { KeepDeletedCells.FALSE, 2, false },
+        { KeepDeletedCells.FALSE, 3, true }, { KeepDeletedCells.TRUE, 1, false },
+        // { KeepDeletedCells.TRUE, 2, true }, see HBASE-28442
+        { KeepDeletedCells.TRUE, 3, false } });
+  }
+
+  // In memory representation of a cell. We only need to know timestamp and type field for our
+  // testing for cell. Please note the row for the cell is implicit in insertedCells.
+  private static class CellInfo {
+    long timestamp;
+    Cell.Type type;
+
+    CellInfo(long timestamp, Cell.Type type) {
+      this.timestamp = timestamp;
+      this.type = type;
+    }
+  }
+
+  private void createTable(int index, boolean enableDualFileWriter) throws IOException {
+    tableName[index] = TableName.valueOf(getClass().getSimpleName() + "_" + index);
+    ColumnFamilyDescriptor familyDescriptor =
+      ColumnFamilyDescriptorBuilder.newBuilder(HBaseTestingUtil.fam1).setMaxVersions(maxVersions)
+        .setKeepDeletedCells(keepDeletedCells)
+        .setValue(NEW_VERSION_BEHAVIOR, Boolean.toString(newVersionBehavior)).build();
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(tableName[index]).setColumnFamily(familyDescriptor)
+        .setValue(ENABLE_HISTORICAL_COMPACTION_FILES, Boolean.toString(enableDualFileWriter));
+    testUtil.createTable(builder.build(), null);
+    regions[index] = testUtil.getMiniHBaseCluster().getRegions(tableName[index]).get(0);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 6);
+    conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
+      String.valueOf(MemoryCompactionPolicy.NONE));
+    testUtil.startMiniCluster();
+    createTable(0, false);
+    createTable(1, true);
+    insertedCells = new ArrayList<>(ROW_NUM);
+    for (int r = 0; r < ROW_NUM; r++) {
+      insertedCells.add(new ArrayList<>(qualifiers.length));
+      for (int q = 0; q < qualifiers.length; q++) {
+        insertedCells.get(r).add(new ArrayList<>(10));
+      }
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    this.testUtil.shutdownMiniCluster();
+    testUtil.cleanupTestDir();
+  }
+
+  @Test
+  public void testCompactedFiles() throws Exception {
+    for (int i = 0; i < 10; i++) {
+      insertRows(ROW_NUM * maxVersions);
+      deleteRows(ROW_NUM / 8);
+      deleteRowVersions(ROW_NUM / 8);
+      deleteColumns(ROW_NUM / 8);
+      deleteColumnVersions(ROW_NUM / 8);
+      flushRegion();
+    }
+
+    verifyCells();
+
+    HStore[] stores = new HStore[2];
+
+    stores[0] = regions[0].getStore(HBaseTestingUtil.fam1);
+    assertEquals(flushCount, stores[0].getStorefilesCount());
+
+    stores[1] = regions[1].getStore(HBaseTestingUtil.fam1);
+    assertEquals(flushCount, stores[1].getStorefilesCount());
+
+    regions[0].compact(false);
+    assertEquals(flushCount - stores[0].getCompactedFiles().size() + 1,
+      stores[0].getStorefilesCount());
+
+    regions[1].compact(false);
+    assertEquals(flushCount - stores[1].getCompactedFiles().size() + 2,
+      stores[1].getStorefilesCount());
+
+    verifyCells();
+
+    regions[0].compact(true);
+    assertEquals(1, stores[0].getStorefilesCount());
+
+    regions[1].compact(true);
+    assertEquals(keepDeletedCells == KeepDeletedCells.FALSE ? 1 : 2,
+      stores[1].getStorefilesCount());
+
+    verifyCells();
+  }
+
+  private void verifyCells() throws Exception {
+    scanAndCompare(false);
+    scanAndCompare(true);
+  }
+
+  private void flushRegion() throws Exception {
+    regions[0].flush(true);
+    regions[1].flush(true);
+    flushCount++;
+  }
+
+  private Long getRowTimestamp(int row) {
+    Long maxTimestamp = null;
+    for (int q = 0; q < qualifiers.length; q++) {
+      int size = insertedCells.get(row).get(q).size();
+      if (size > 0) {
+        CellInfo mostRecentCellInfo = insertedCells.get(row).get(q).get(size - 1);
+        if (mostRecentCellInfo.type == Cell.Type.Put) {
+          if (maxTimestamp == null || maxTimestamp < mostRecentCellInfo.timestamp) {
+            maxTimestamp = mostRecentCellInfo.timestamp;
+          }
+        }
+      }
+    }
+    return maxTimestamp;
+  }
+
+  private long getNewTimestamp(long timestamp) throws Exception {
+    long newTimestamp = System.currentTimeMillis();
+    if (timestamp == newTimestamp) {
+      Thread.sleep(1);
+      newTimestamp = System.currentTimeMillis();
+      assert (timestamp < newTimestamp);
+    }
+    return newTimestamp;
+  }
+
+  private void insertRows(int rowCount) throws Exception {
+    int row;
+    long timestamp = System.currentTimeMillis();
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Put put = new Put(Bytes.toBytes(String.valueOf(row)), timestamp);
+      for (int q = 0; q < qualifiers.length; q++) {
+        put.addColumn(HBaseTestingUtil.fam1, qualifiers[q],
+          Bytes.toBytes(String.valueOf(timestamp)));
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.Put));
+      }
+      regions[0].put(put);
+      regions[1].put(put);
+      timestamp = getNewTimestamp(timestamp);
+    }
+  }
+
+  private void deleteRows(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      long timestamp = System.currentTimeMillis();
+      row = RANDOM.nextInt(ROW_NUM);
+      Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+      regions[0].delete(delete);
+      regions[1].delete(delete);
+      // For simplicity, the family delete markers are inserted for all columns (instead of
+      // allocating a separate column for them) in the memory representation of the data stored
+      // to HBase
+      for (int q = 0; q < qualifiers.length; q++) {
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteFamily));
+      }
+    }
+  }
+
+  private void deleteSingleRowVersion(int row, long timestamp) throws IOException {
+    Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+    delete.addFamilyVersion(HBaseTestingUtil.fam1, timestamp);
+    regions[0].delete(delete);
+    regions[1].delete(delete);
+    // For simplicity, the family delete version markers are inserted for all columns (instead of
+    // allocating a separate column for them) in the memory representation of the data stored
+    // to HBase
+    for (int q = 0; q < qualifiers.length; q++) {
+      insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteFamilyVersion));
+    }
+  }
+
+  private void deleteRowVersions(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Long timestamp = getRowTimestamp(row);
+      if (timestamp != null) {
+        deleteSingleRowVersion(row, timestamp);
+      }
+    }
+    // Just insert one more delete marker possibly does not delete any row version
+    row = RANDOM.nextInt(ROW_NUM);
+    deleteSingleRowVersion(row, System.currentTimeMillis());
+  }
+
+  private void deleteColumns(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      long timestamp = System.currentTimeMillis();
+      row = RANDOM.nextInt(ROW_NUM);
+      int q = RANDOM.nextInt(qualifiers.length);
+      Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)), timestamp);
+      delete.addColumns(HBaseTestingUtil.fam1, qualifiers[q], timestamp);
+      regions[0].delete(delete);
+      regions[1].delete(delete);
+      insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteColumn));
+    }
+  }
+
+  private void deleteColumnVersions(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Long timestamp = getRowTimestamp(row);
+      if (timestamp != null) {
+        Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+        int q = RANDOM.nextInt(qualifiers.length);
+        delete.addColumn(HBaseTestingUtil.fam1, qualifiers[q], timestamp);
+        regions[0].delete(delete);
+        regions[1].delete(delete);
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.Delete));
+      }
+    }
+  }
+
+  private Scan createScan(boolean raw) {
+    Scan scan = new Scan();
+    scan.readAllVersions();
+    scan.setRaw(raw);
+    return scan;
+  }
+
+  private void scanAndCompare(boolean raw) throws Exception {
+    try (RegionScanner firstRS = regions[0].getScanner(createScan(raw))) {
+      try (RegionScanner secondRS = regions[1].getScanner(createScan(raw))) {
+        boolean firstHasMore;
+        boolean secondHasMore;
+        do {
+          List<Cell> firstRowList = new ArrayList<>();
+          List<Cell> secondRowList = new ArrayList<>();
+          firstHasMore = firstRS.nextRaw(firstRowList);
+          secondHasMore = secondRS.nextRaw(secondRowList);
+          assertEquals(firstRowList.size(), secondRowList.size());
+          int size = firstRowList.size();
+          for (int i = 0; i < size; i++) {
+            Cell firstCell = firstRowList.get(i);
+            Cell secondCell = secondRowList.get(i);
+            assert (CellUtil.matchingRowColumn(firstCell, secondCell));

Review Comment:
   I made the change for this comment.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2116540777

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 56s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 13s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 52s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 44s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  2s |  master passed  |
   | -0 :warning: |  patch  |   1m 42s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m 20s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m 20s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 47s |  hbase-server: The patch generated 1 new + 46 unchanged - 0 fixed = 47 total (was 46)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m 47s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 58s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 18s |  The patch does not generate ASF License warnings.  |
   |  |   |  37m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux ec4f2513565d 5.4.0-172-generic #190-Ubuntu SMP Fri Feb 2 23:24:22 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1890422318

   > @kadirozde are you able to repro these test failures locally? https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/testReport/
   > 
   > e.g. `TestVerifyReplicationCrossDiffHdfs`
   > 
   > ```
   > java.lang.NoClassDefFoundError: org/bouncycastle/operator/OperatorCreationException
   > 	at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceInit(ResourceManager.java:888)
   > 	at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164)
   > 	at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.createAndInitActiveServices(ResourceManager.java:1410)
   > 	at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceInit(ResourceManager.java:344)
   > 	at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164)
   > 	at org.apache.hadoop.yarn.server.MiniYARNCluster.initResourceManager(MiniYARNCluster.java:359)
   > 	at org.apache.hadoop.yarn.server.MiniYARNCluster.access$200(MiniYARNCluster.java:129)
   > 	at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper.serviceInit(MiniYARNCluster.java:494)
   > ```
   
   I run failed tests locally and all passed. I pushed a change for a java doc comment just to run the tests again.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1866166753

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 28s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 51s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  master passed  |
   | -0 :warning: |  patch  |   5m 50s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 14s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 49s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 49s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 49s |  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  | 233m 40s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  16m  9s |  hbase-mapreduce in the patch passed.  |
   |  |   | 273m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4d3a7abc2bae 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2c07847656 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/testReport/ |
   | Max. process+thread count | 4531 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1831089037

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 58s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 26s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 41s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 29s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 25s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-server: The patch generated 5 new + 28 unchanged - 0 fixed = 33 total (was 28)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 47s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 33s |  patch has 72 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   1m 29s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 10s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux a3fb04ab5665 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / dbfb516a55 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1411227263


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   I tried it but it did not work. The signature of the class requires DefaultCompactor. This means I need to extend DefaultCompactor to implement DualFileCompactor. However, DefaultCompactor supports only StoreFileWriter, a single file writer. So I ended up with a new engine.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1411228804


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
-      sortedFiles.add(file);
+      if (onlyLatestVersion) {
+        if (file.hasLatestVersion()) {
+          sortedFiles.add(file);
+        }
+      } else {
+        sortedFiles.add(file);
+      }
     }
     boolean succ = false;
     try {
-      for (int i = 0, n = files.size(); i < n; i++) {
+      for (int i = 0, n = files.size(); i < n && !sortedFiles.isEmpty(); i++) {

Review Comment:
   Yes, because we skip the files with older cell versions and delete markers for the regular scans with max versions = 1.



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

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

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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1412480023


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   I am still trying but I could not find a good solution yet. The reason is that DefaultCompactor is a single file writer compactor and DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine. That is why I think DualFileStoreEngine should not inherit from DefaultStoreEngine. 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1412480023


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   I am still trying but I could not find a good solution yet. The reason is that DefaultCompactor is a single file writer compactor and DualFileCompactor is a multi file writers as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine. That is why I think DualFileStoreEngine should not inherit from DefaultStoreEngine. 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1831299465

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 18s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 53s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 30s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 59s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-server generated 1 new + 95 unchanged - 0 fixed = 96 total (was 95)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 264m 12s |  hbase-server in the patch failed.  |
   |  |   | 292m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4ccc4c8ea713 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / dbfb516a55 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/testReport/ |
   | Max. process+thread count | 4695 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/1/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442621969


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        if (cell.getType() == Cell.Type.DeleteFamily) {
+          deleteFamily = cell;
+          addLiveVersion(cell);
+        } else {
+          addHistoricalVersion(cell);
+        }
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (deleteFamily == null) {
+        deleteFamilyVersionList.add(cell);
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.DeleteColumn) {
+      if (!isDeletedByDeleteFamily(cell) && deleteColumn == null) {
+        deleteColumn = cell;
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.Delete) {
+      if (!isDeletedByDeleteFamily(cell) && deleteColumn == null) {
+        deleteColumnVersionList.add(cell);
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.Put) {
+      if (livePutCellCount < maxVersions) {
+        // This is a live put cell (i.e., the latest version) of a column. Is it deleted?
+        if (!isDeleted(cell)) {
+          addLiveVersion(cell);
+          livePutCellCount++;
+        } else {
+          // It is deleted
+          addHistoricalVersion(cell);
+        }
+      } else {
+        // It is an older put cell
+        addHistoricalVersion(cell);
+      }
+    }
+    lastCell = cell;
+  }
+
+  @Override
+  public void appendAll(List<Cell> cellList) throws IOException {
+    if (!dualWriterEnabled) {
+      // If the dual writer is not enabled then all cells are written to one file. We use
+      // the live version file in this case
+      for (Cell cell : cellList) {
+        addLiveVersion(cell);
+      }
+      return;
+    }
+    if (cellList.isEmpty()) {
+      return;
+    }
+    if (lastCell != null && comparator.compareRows(lastCell, cellList.get(0)) != 0) {
+      // It is a new row and thus time to reset the state
+      initRowState();
+    }
+    for (Cell cell : cellList) {
+      appendCell(cell);
+    }
+  }
+
+  @Override
+  public void append(Cell cell) throws IOException {
+    if (!dualWriterEnabled) {
+      // If the dual writer is not enabled then all cells are written to one file. We use
+      // the live version file in this case
+      addLiveVersion(cell);
+      return;
+    }
+    if (lastCell != null && comparator.compareRows(lastCell, cell) != 0) {
+      // It is a new row and thus time to reset the state
+      initRowState();
+    }
+    appendCell(cell);
+  }
+
+  @Override
+  protected Collection<StoreFileWriter> writers() {
+    return writers;
+  }
+
+  @Override
+  protected void preCommitWriters() throws IOException {
+    if (writers.isEmpty()) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }

Review Comment:
   When there is no cell to write (empty region or all the cells are deleted), we still generate an HFile for the region (one of the tests specifically checks for that, see TestDateTieredCompactor#testEmptyOutputFile).



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442559511


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java:
##########
@@ -138,6 +140,12 @@ public class HStoreFile implements StoreFile {
   // Indicates if the file got compacted
   private volatile boolean compactedAway = false;
 
+  // Indicate if the file contains live cell versions for a given column
+  // in a row. MemStore flushes generate files with all cell versions. However,
+  // compactions can generate two files, one with the live cell versions and the other
+  // with the remaining (historical) cell versions.
+  private volatile boolean hasLiveVersions = true;

Review Comment:
   I was thinking why this is not kept `false` by default, but now i realized that it is only used by `DefaultStoreFileManager#getLiveVersionFiles` which will only be used by callers if `hbase.hstore.defaultengine.enable.dualfilewriter` is enabled. 
   
   Hence, unless dual file writer is enabled, this is not even used, so it's default value here doesn't matter. Is that correct? If so, maybe we can comment the same here so that no one tries to use this value without checking for `hbase.hstore.defaultengine.enable.dualfilewriter` in future.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1879017714

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   5m 41s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 230m 18s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 39s |  hbase-mapreduce in the patch passed.  |
   |  |   | 268m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 44af8257e7f7 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 119885415c |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/testReport/ |
   | Max. process+thread count | 5414 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1851894732

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 44s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 13s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  master passed  |
   | -0 :warning: |  patch  |   5m 43s |  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  |   2m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 44s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 217m  4s |  hbase-server in the patch passed.  |
   |  |   | 239m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 087fdc9a511b 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 78c5ac3725 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/testReport/ |
   | Max. process+thread count | 4670 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2067815560

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  4s |  master passed  |
   | -0 :warning: |  patch  |   9m 40s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 17s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 43s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 43s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  1s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 241m 18s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 49s |  hbase-mapreduce in the patch passed.  |
   |  |   | 291m  9s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d281983b0048 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3539581268 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/testReport/ |
   | Max. process+thread count | 5569 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2074177958

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  2s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/29/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2073830963

   Ah, sorry, forgot this one...
   
   Will take a look soon.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998515178

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 13s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 52s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 44s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 53s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  13m 20s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 29s |  hbase-mapreduce in the patch passed.  |
   |  |   |  52m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3fe029031303 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/testReport/ |
   | Max. process+thread count | 2948 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2001502697

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 46s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 51s |  master passed  |
   | -0 :warning: |  patch  |   0m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 54s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 33s |  hbase-server: The patch generated 1 new + 37 unchanged - 0 fixed = 38 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  3s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  30m 56s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux ff5eb6c97e4b 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 80 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000166912

   @bbeaudreault, @virajjasani, I updated the PR such that the historical files will be generated only with default store engine and default compactor. In other cases, historical files will not be generated and hbase.enable.historical.compaction.files will be ignored silently. I will file jiras to support historical files for other types of compactions.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000651680

   > i just verified with HBase 2.6 (branch-2) that the flush writes only maxVersions versions to the new HFile.
   
   Btw this also means that raw scan (that reads all versions) can have different results while scanning the cell with multiple versions because if we do raw scan before flush, we will get all versions written by user (regardless of max versions set at the table descriptor), however if we do raw scan after flush, we will get only max versions num of versions for the given cell.
   
   It is expected or perhaps we should document that anyone doing raw scan should read only max versions num of versions during the scan, otherwise (if they read all versions during raw scan) they can observe different results before and after flush operation.
   
   This behavior has nothing to do with this PR though.


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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1559993483


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   I just want to make sure that we are on the same page here. You think we need to make this method atomic even though
   (1) The class is not thread-safe as specified here 
   `/**
    * Default implementation of StoreFileManager. Not thread-safe.
    */`
   (2) There is only one caller which is StoreEngine and it takes a write lock whenever it calls a StoreFileManager methods, for example
   `  /**
      * Add the store files to store file manager, and also record it in the store file tracker.
      * <p/>
      * The {@code actionAfterAdding} will be executed after the insertion to store file manager, under
      * the lock protection. Usually this is for clear the memstore snapshot.
      */
     public void addStoreFiles(Collection<HStoreFile> storeFiles,
       IOExceptionRunnable actionAfterAdding) throws IOException {
       storeFileTracker.add(StoreUtils.toStoreFileInfo(storeFiles));
       writeLock();
       try {
         storeFileManager.insertNewFiles(storeFiles);
         actionAfterAdding.run();
       } finally {
         // We need the lock, as long as we are updating the storeFiles
         // or changing the memstore. Let us release it before calling
         // notifyChangeReadersObservers. See HBASE-4485 for a possible
         // deadlock scenario that could have happened if continue to hold
         // the lock.
         writeUnlock();
       }
     }`
   
   I can easily add a write lock here and to the other methods but given the above points, is not it redundant? 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "bbeaudreault (via GitHub)" <gi...@apache.org>.
bbeaudreault commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545796726


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   Could you check if mob is enabled and, if so, log a warn and ignore the dual writer config value?
   



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545770119


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   Since we moved dual file writing into StoreFileWriter and all compaction methods use StoreFileWriter, there is no way to prevent it. Please note by default dual file writing is disabled, one needs to enable it to use it by setting  hbase.enable.historical.compaction.files to true. 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545554196


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   Is there a way to prevent users enable dual file writing and mob together?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2116510269

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  docker  |   0m  5s |  Docker failed to build run-specific yetus/hbase:tp-876}.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/console |
   | versions | git=2.25.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2111577285

   @Apache9 I think there was only one issue considered a blocker and @kadirozde has addressed it. There are three failing tests in the precommit but all are related to quotas so are unlikely due to this. I would like to merge this interesting change and test it further if there are no objections. 


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1999217480

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 10s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 22s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 12s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 20s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 200m 47s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  12m 50s |  hbase-mapreduce in the patch passed.  |
   |  |   | 238m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 55aa554b6501 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/testReport/ |
   | Max. process+thread count | 5209 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1849420048

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | -0 :warning: |  patch  |   5m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 45s |  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  |   1m 15s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 235m 22s |  hbase-server in the patch failed.  |
   |  |   | 260m 13s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a2267f1e4f44 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 82a2ce10f2 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/testReport/ |
   | Max. process+thread count | 4159 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1878666005

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 43s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 52s |  master passed  |
   | -0 :warning: |  patch  |   0m 37s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 33s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 33s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 35s |  hbase-server: The patch generated 2 new + 42 unchanged - 0 fixed = 44 total (was 42)  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-mapreduce: The patch generated 1 new + 27 unchanged - 0 fixed = 28 total (was 27)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  10m  7s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  4s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  37m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 1a0c5b583486 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 119885415c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/artifact/yetus-general-check/output/diff-checkstyle-hbase-mapreduce.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1839456898

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 41s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 51s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 38s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-server generated 1 new + 22 unchanged - 0 fixed = 23 total (was 22)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 254m  5s |  hbase-server in the patch failed.  |
   |  |   | 281m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a09fd9c6ca69 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 25e9228e2c |
   | Default Java | Temurin-1.8.0_352-b08 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/testReport/ |
   | Max. process+thread count | 4165 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416119851


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   It's possible to refactor this so DefaultCompactor becomes a dual file compactor that only compacts a single file. Can we try that instead? I do not think we need a new Store Engine for this. That introduces some configuration trouble for operators. Ideally operators do not need to change their store engine in order to take advantage of this and other improvements that are generally applicable and are in other respects backwards compatible. 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   StoreEngine and related interfaces have evolved organically and the current state is maybe not ideal.
   
   If we take the above approach, to refactor the default compactor to become a multi file compactor that only compacts to a single file, these called out compactors and engines can possibly be refactored to inherit from `DefaultStoreEngine` to take a cleaner approach (imho), although that would be follow up work. It would be worth taking a look at that. Later. Of course I am not suggesting that be done for this PR.
   Or if we look at the pattern that other recent refactors have taken there might be cause to consolidate common logic into an `AbstractStoreEngine` and have the store engines inherit from 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   ... and if we take the above approach, to refactor the default compactor to become a multi file compactor that only compacts to a single file, these called out compactors and engines can possibly be refactored to inherit from `DefaultStoreEngine` to take a cleaner approach (imho), although that would be follow up work. It would be worth taking a look at that. Later. Of course I am not suggesting that be done for 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837422307

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 23s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 32s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 38s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 36s |  the patch passed  |
   | -1 :x: |  compile  |   1m  9s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   1m  9s |  hbase-server in the patch failed.  |
   | -0 :warning: |  checkstyle  |   0m 48s |  hbase-server: The patch generated 7 new + 46 unchanged - 0 fixed = 53 total (was 46)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m  4s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 37s |  patch has 64 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   1m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  9s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 6eefa407c645 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 78 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/4/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1890541222

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 56s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | -0 :warning: |  patch  |   6m  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 _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 59s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 235m 52s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 48s |  hbase-mapreduce in the patch passed.  |
   |  |   | 274m 48s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 81b9375536c0 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e3a0174e20 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/testReport/ |
   | Max. process+thread count | 5309 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1930157639

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 23s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 43s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 55s |  master passed  |
   | -0 :warning: |  patch  |   0m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 53s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 53s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  0s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 79a37a532502 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 16de74c194 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | Max. process+thread count | 80 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1931165453

   This is on my list. It is kinda big so I need to spend some time to take a whole review, to prevent that I comment some of the problems, you update the PR, and then I comment with something else which conflicts with your new efforts...


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1962840902

   For me, I think whether to introduce a new store engine here is clear. If this is a new layout way, which can not be worked together with other store engines, we should introduce a new store engine. If not, we should consider this as an optimization and apply it to all store engines if they can be optimized.
   
   Here, I think at least for DateTieredStoreEngine, we could still use this optimization, as we split tiers by timestamp, generating two files does not break the rule. For StripeStoreEngine, for range based tier, the optimization should be OK, but for size based tier, I'm not sure but it may cause problems. For MobStoreEngine, I think at least it can be applied to the index file, I'm not sure if it is OK to also apply to mob data file.
   
   So in general I think the previous reviewers's opinion is correct, this should be considered an optimization, not a fresh new layout way.
   
   Back to the StoreFileWriter problem, as I said above, other store engine could also apply this optimization, so we should add the support for writing the latest cells to a separated file to StoreFileWriter directly, not introducing a DualFileWriter and only use it in DefaultStoreEngine.
   
   Technically, I agree that we could have a 'StoreFileWriter' to always write a single store file, and use combination to support writing multiple store files at once. The problem here is still about naming, I do not think it is a good idea to use 'DualFileWriter' in DefaultStoreEngine, we should give it another name. And also, for other store engines, we should use the new store file writer which support writing latest cells to a separated file, instead of using the single file store file writer, although in the first version, we can add a check to not enable it for the store engines other than default store engine, but theoretically it should also work with other store engine.
   
   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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1962862133

   Yes, for other store engines, we could add a check that you should not enable the 'write latest cells to separated file' flag, and open follow on issues to support this feature for other store engines.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1501398567


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   Does it actually throw any exceptions? Especially for insertNewFiles and addCompactionResults, I think we should hava already loaded the related metadata in memory already?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545925927


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   I have done that in the last commit.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545467547


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -235,10 +210,10 @@ private byte[] toCompactionEventTrackerBytes(Collection<HStoreFile> storeFiles)
    */
   public void appendMetadata(final long maxSequenceId, final boolean majorCompaction,
     final long mobCellsCount) throws IOException {
-    writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
-    writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
-    writer.appendFileInfo(MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
-    appendTrackedTimestampsToMetadata();
+    liveFileWriter.appendMetadata(maxSequenceId, majorCompaction, mobCellsCount);

Review Comment:
   I did not change the behavior. This change is due to refactoring the code. We do not support dual file writing for mob yet as mentioned in the previous comment.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558933042


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java:
##########
@@ -138,6 +140,12 @@ public class HStoreFile implements StoreFile {
   // Indicates if the file got compacted
   private volatile boolean compactedAway = false;
 
+  // Indicate if the file contains historical cell versions. This is used when
+  // hbase.enable.historical.compaction.files is set to true. In that case, compactions
+  // can generate two files, one with the live cell versions and the other with the remaining
+  // (historical) cell versions.
+  private volatile boolean isHistorical = false;

Review Comment:
   The new name seems good.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558933426


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   Ping.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558928060


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java:
##########
@@ -464,7 +464,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
             lastCleanCell = null;
             lastCleanCellSeqId = 0;
           }
-          writer.append(c);

Review Comment:
   Let's not mix different things up? I'm not sure if there are other side effects about this change...



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2074180292

   > There is only one blocker, about the concurrency control. We'd better still hold the consistent view while getting storefile list.
   > 
   > The other is not a blocker but still better to make it clear that whether we will throw any exception for some storefile management operations.
   > 
   > Thanks.
   
   @Apache9, I made the requested changes for having consistent view while getting store files. I also took the liberty to rename storefiles to storeFiles and getStorefiles() to getStoreFiles().  The hfile reader throws IOException. DefaultStoreFileManager reads the file metadata during loading the files to see which of the loaded files are live (since DefaultStoreFileManager maintains a separate list for live files). Therefore, I had to change the method signature. Thank you for review again!


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1602560727


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -48,17 +52,38 @@ class DefaultStoreFileManager implements StoreFileManager {
   private final CompactionConfiguration comConf;
   private final int blockingFileCount;
   private final Comparator<HStoreFile> storeFileComparator;
-  /**
-   * List of store files inside this store. This is an immutable list that is atomically replaced
-   * when its contents change.
-   */
-  private volatile ImmutableList<HStoreFile> storefiles = ImmutableList.of();
+
+  static class StoreFileList {
+    /**
+     * List of store files inside this store. This is an immutable list that is atomically replaced
+     * when its contents change.
+     */
+    final ImmutableList<HStoreFile> all;
+    /**
+     * List of store files that include the latest cells inside this store. This is an immutable
+     * list that is atomically replaced when its contents change.
+     */
+    @Nullable
+    final ImmutableList<HStoreFile> live;
+
+    StoreFileList(ImmutableList<HStoreFile> storeFiles, ImmutableList<HStoreFile> liveStoreFiles) {
+      this.all = storeFiles;
+      this.live = liveStoreFiles;
+    }
+  }
+
+  private static final StoreFileList EMPTY_STORE_FILE_LIST =
+    new StoreFileList(ImmutableList.of(), null);
+
+  private volatile StoreFileList storeFiles = EMPTY_STORE_FILE_LIST;

Review Comment:
   Got it. I changed the code such that null is passed only when live tracking is disabled.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1601087841


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -48,17 +52,38 @@ class DefaultStoreFileManager implements StoreFileManager {
   private final CompactionConfiguration comConf;
   private final int blockingFileCount;
   private final Comparator<HStoreFile> storeFileComparator;
-  /**
-   * List of store files inside this store. This is an immutable list that is atomically replaced
-   * when its contents change.
-   */
-  private volatile ImmutableList<HStoreFile> storefiles = ImmutableList.of();
+
+  static class StoreFileList {
+    /**
+     * List of store files inside this store. This is an immutable list that is atomically replaced
+     * when its contents change.
+     */
+    final ImmutableList<HStoreFile> all;
+    /**
+     * List of store files that include the latest cells inside this store. This is an immutable
+     * list that is atomically replaced when its contents change.
+     */
+    @Nullable
+    final ImmutableList<HStoreFile> live;
+
+    StoreFileList(ImmutableList<HStoreFile> storeFiles, ImmutableList<HStoreFile> liveStoreFiles) {
+      this.all = storeFiles;
+      this.live = liveStoreFiles;
+    }
+  }
+
+  private static final StoreFileList EMPTY_STORE_FILE_LIST =
+    new StoreFileList(ImmutableList.of(), null);
+
+  private volatile StoreFileList storeFiles = EMPTY_STORE_FILE_LIST;

Review Comment:
   When initializating we do not need to test whether live file tracking is enabled? I see in later code, if live file tracking is enabled we will pass ImmutableList.of() instead of null here.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   I know the there are throws declarations, what I mean here is whether it will actually throw any exceptions out. For example, in loadFiles, maybe we have already called initReader for all files so there is no problem? Just want to confirm 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443685793


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
 
   public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-    new CellSinkFactory<StoreFileWriter>() {
+  private final CellSinkFactory<DualFileWriter> writerFactory =

Review Comment:
   DualFileWriter is composed of 1 or 2 store file writes. If the dual file compaction is disabled, DualFileWriter will have a single store file writer and becomes just a simple wrapper for that store file writer. When the dual file compaction is enabled then, it splits the cells between two store file writers. So, StoreFileWriter is used as before.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1879650845

   > I think the PR still needs a lot of polishing...
   > 
   > A lot of changes seems unrelated, and the new flag does not work for mob and stripe compaction? At least we should discuss this in the design doc first.
   
   @Apache9, I appreciate your review feedback and questions. I have answered your questions.  Let me know if you have further questions or suggestions. I will be happy to improve the PR further.
   
   Yes, the dual file compaction is an improvement over the default compaction only currently. It can be extended to the stripe and mob compactions too. Please note that the dual file compaction is an optional improvement and it will be turned off by default. I think it is too early to make further changes in the code base for this improvement at this moment. I am looking forward to enabling this improvement in production at my company. Based on our experience and learning, we can decide to extend it for other compactions on mutable data. I hope this makes sense to you.


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

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

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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1866187868

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 10s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 24s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | -0 :warning: |  patch  |   6m 16s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 22s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 22s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 17s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 244m 17s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  18m 18s |  hbase-mapreduce in the patch passed.  |
   |  |   | 291m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux fe90fca4cca2 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2c07847656 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/testReport/ |
   | Max. process+thread count | 4746 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1850841141

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m  8s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 28s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 41s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 25s |  master passed  |
   | -0 :warning: |  patch  |   1m 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  |   3m 20s |  the patch passed  |
   | -1 :x: |  compile  |   1m 14s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   1m 14s |  hbase-server in the patch failed.  |
   | -0 :warning: |  checkstyle  |   0m 46s |  hbase-server: The patch generated 3 new + 56 unchanged - 0 fixed = 59 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m  5s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 34s |  patch has 32 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   1m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  34m  3s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 08c6f4e04e74 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 78c5ac3725 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 81 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1890430662

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  2s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 53s |  master passed  |
   | -0 :warning: |  patch  |   0m 36s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 41s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 51s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 51s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-server: The patch generated 2 new + 42 unchanged - 0 fixed = 44 total (was 42)  |
   | -0 :warning: |  checkstyle  |   0m  9s |  hbase-mapreduce: The patch generated 1 new + 27 unchanged - 0 fixed = 28 total (was 27)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  3s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  31m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 98434b12cdc8 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e3a0174e20 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/artifact/yetus-general-check/output/diff-checkstyle-hbase-mapreduce.txt |
   | Max. process+thread count | 80 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1412493239


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
-      sortedFiles.add(file);
+      if (onlyLatestVersion) {
+        if (file.hasLatestVersion()) {
+          sortedFiles.add(file);
+        }
+      } else {
+        sortedFiles.add(file);
+      }
     }
     boolean succ = false;
     try {
-      for (int i = 0, n = files.size(); i < n; i++) {
+      for (int i = 0, n = files.size(); i < n && !sortedFiles.isEmpty(); i++) {

Review Comment:
   I did not want to iterate over over sorted files for the same reason the existing code does not iterate over sorted files since sorted files are modified within the loop. I do not see the NPE issue here.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1871226766

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 20s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  master passed  |
   | -0 :warning: |  patch  |   6m 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 _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 15s |  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  |   5m 32s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 247m 53s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  18m  8s |  hbase-mapreduce in the patch passed.  |
   |  |   | 293m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0e7874e44ca5 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 75801a5291 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/testReport/ |
   | Max. process+thread count | 4468 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/12/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1873799206

   @apurtell, @bbeaudreault, @wchevreuil, @virajjasani, the PR is ready for another round of reviews. I did some perf testing and posted the results at https://issues.apache.org/jira/browse/HBASE-25972.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1849249665

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  5s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  8s |  master passed  |
   | -0 :warning: |  patch  |   1m 35s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 38s |  the patch passed  |
   | -1 :x: |  compile  |   1m  3s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   1m  3s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  checkstyle  |   0m 13s |  hbase-client: The patch generated 0 new + 0 unchanged - 1 fixed = 0 total (was 1)  |
   | -0 :warning: |  checkstyle  |   0m 31s |  hbase-server: The patch generated 15 new + 55 unchanged - 1 fixed = 70 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   9m 51s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 34s |  patch has 64 errors when running spotless:check, run spotless:apply to fix.  |
   | -1 :x: |  spotbugs  |   1m 35s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m 27s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Dead store to enableDualFileWriter in org.apache.hadoop.hbase.mob.MobStoreEngine.createComponents(Configuration, HStore, CellComparator)  At MobStoreEngine.java:org.apache.hadoop.hbase.mob.MobStoreEngine.createComponents(Configuration, HStore, CellComparator)  At MobStoreEngine.java:[line 73] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 63ce13402b7b 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 82a2ce10f2 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-general-check/output/patch-spotless.txt |
   | spotbugs | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 82 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998490854

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 17s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  0s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 45s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 42s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 56s |  master passed  |
   | -0 :warning: |  patch  |   1m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 57s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 36s |  hbase-server: The patch generated 1 new + 40 unchanged - 0 fixed = 41 total (was 40)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  7s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 46s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 19s |  The patch does not generate ASF License warnings.  |
   |  |   |  34m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux bc68c1e49735 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 80 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998613522

   :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  1s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 44s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 55s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 48s |  master passed  |
   | -0 :warning: |  patch  |   0m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 44s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 52s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 52s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 35s |  hbase-server: The patch generated 1 new + 37 unchanged - 0 fixed = 38 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  5s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  30m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 8cf606b7f57a 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 78 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000243609

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 17s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 51s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 43s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 58s |  master passed  |
   | -0 :warning: |  patch  |   0m 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 _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 54s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 54s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-server: The patch generated 1 new + 37 unchanged - 0 fixed = 38 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  3s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 44e74913fe23 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1931112957

   @Apache9 @apurtell @bbeaudreault @wchevreuil please let me know if you have any final feedback on this.


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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "d-c-manning (via GitHub)" <gi...@apache.org>.
d-c-manning commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1485004519


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java:
##########
@@ -70,36 +72,17 @@ protected void createComponents(Configuration conf, HStore store, CellComparator
   }
 
   protected void createCompactor(Configuration conf, HStore store) throws IOException {
-    String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
-    try {
-      compactor = ReflectionUtils.instantiateWithCustomCtor(className,
-        new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
-    } catch (Exception e) {
-      throw new IOException("Unable to load configured compactor '" + className + "'", e);
-    }
+    createCompactor(conf, store, DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
   }
 
   protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException {
-    String className =
-      conf.get(DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
-    try {
-      compactionPolicy = ReflectionUtils.instantiateWithCustomCtor(className,
-        new Class[] { Configuration.class, StoreConfigInformation.class },
-        new Object[] { conf, store });
-    } catch (Exception e) {
-      throw new IOException("Unable to load configured compaction policy '" + className + "'", e);
-    }
+    createCompactionPolicy(conf, store, DEFAULT_COMPACTION_POLICY_CLASS_KEY,

Review Comment:
   The default `CompactionConfiguration` would have `hbase.hstore.compaction.min == 3` for `HBASE_HSTORE_COMPACTION_MIN_KEY`. The hbase book (https://hbase.apache.org/book.html#compaction.parameters) has this comment:
   >The goal of tuning hbase.hstore.compaction.min is to avoid ending up with too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction each time you have two StoreFiles in a Store, and this is probably not appropriate.
   
   I confess I don't know why 2 is (probably) inappropriate. I assume it's because compacting immediately after every flush is perhaps "too much" compaction, and it is only a tradeoff between read performance and compaction load.
   
   But if that is at all true, then will we now be in that state by default with the dual file writer? If a compaction often leaves us with 2 files, then we will potentially compact on every flush (instead of after every 2 flushes, by default?)
   
   If someone was "inappropriate" and set `hbase.hstore.compaction.min` = `2`, would enabling DualFile writing cause us to always select the same 2 output files for compaction, and we would re-compact them infinitely?
   
   Do we need to add 1 to this `min` value if dual writing is enabled? Or otherwise change the compaction selection algorithm to be aware of live + historical files?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998576425

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 55s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 24s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 16s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 44s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  11m 45s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  12m 58s |  hbase-mapreduce in the patch passed.  |
   |  |   |  47m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4622541371ff 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-jdk17-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/testReport/ |
   | Max. process+thread count | 2950 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1999349199

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 51s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 15s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 10s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 46s |  master passed  |
   | -0 :warning: |  patch  |   7m 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 _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 30s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 268m  9s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 59s |  hbase-mapreduce in the patch passed.  |
   |  |   | 312m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 274c1f96c797 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/testReport/ |
   | Max. process+thread count | 5955 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998580631

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 54s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 59s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 51s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  13m 23s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m  3s |  hbase-mapreduce in the patch passed.  |
   |  |   |  51m 20s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 046e54d32a98 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/testReport/ |
   | Max. process+thread count | 2914 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/21/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2067809359

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 37s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  master passed  |
   | -0 :warning: |  patch  |   6m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 58s |  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  |   5m 47s |  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  | 211m 46s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  13m 30s |  hbase-mapreduce in the patch passed.  |
   |  |   | 252m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 074e7e460374 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3539581268 |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/testReport/ |
   | Max. process+thread count | 5135 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/28/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2074177864

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  4s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/29/console |
   | versions | git=2.25.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2115125977

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | -0 :warning: |  patch  |   6m 36s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 48s |  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  |   5m 34s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 222m  5s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  15m  1s |  hbase-mapreduce in the patch passed.  |
   |  |   | 263m 48s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 836ac80252d7 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/testReport/ |
   | Max. process+thread count | 5363 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/33/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1602580409


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   Anyway, not a blocker. Can debug later.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2114088089

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/32/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2113965547

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/31/console |
   | versions | git=2.25.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1890516333

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 40s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  master passed  |
   | -0 :warning: |  patch  |   6m 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 _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 49s |  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  |   5m 35s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 216m 42s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m  0s |  hbase-mapreduce in the patch passed.  |
   |  |   | 256m 24s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8df8d9fa0bb6 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e3a0174e20 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/testReport/ |
   | Max. process+thread count | 5394 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/17/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442652833


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java:
##########
@@ -138,6 +140,12 @@ public class HStoreFile implements StoreFile {
   // Indicates if the file got compacted
   private volatile boolean compactedAway = false;
 
+  // Indicate if the file contains live cell versions for a given column
+  // in a row. MemStore flushes generate files with all cell versions. However,
+  // compactions can generate two files, one with the live cell versions and the other
+  // with the remaining (historical) cell versions.
+  private volatile boolean hasLiveVersions = true;

Review Comment:
   Even when `hbase.hstore.defaultengine.enable.dualfilewriter` is enabled, the files generated by memstore flushes do not include the HAS_LIVE_VERSIONS HFile metadata key since memstore does not use compaction writers.  However, these files will include live version cells. When the HAS_LIVE_VERSIONS HFile metadata key is not included in the HFile, we still want to return true for HStoreFile#hasLiveVersion(). That is why the default is true. I will add more comments for 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1879002974

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 29s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 46s |  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  |   5m 28s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 218m 44s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m  6s |  hbase-mapreduce in the patch passed.  |
   |  |   | 258m 15s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 050ccab316ed 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 119885415c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/testReport/ |
   | Max. process+thread count | 4684 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/15/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443689986


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java:
##########
@@ -300,7 +300,7 @@ private double getMidStripeSplitRatio(long smallerSize, long largerSize, long la
 
   @Override
   public Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
-    byte[] stopRow, boolean includeStopRow) {
+    byte[] stopRow, boolean includeStopRow, boolean onlyLatestVersion) {

Review Comment:
   StoreFileManager is an interface implemented by DefaultStoreFileManager and StripeStoreFileManager. DefaultStoreFileManager needs this new parameter but StripeStoreFileManager does not currently.  I changed the signature of StoreFileManager#getFilesForScan() to add this new parameter. I am open to suggestions for better handling 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837280134

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 15s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 58s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 26s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 57s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 41s |  hbase-server: The patch generated 7 new + 37 unchanged - 0 fixed = 44 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  10m 21s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 36s |  patch has 64 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   1m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 10s |  The patch does not generate ASF License warnings.  |
   |  |   |  36m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 9497c498a0d4 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 82 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837394688

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 20s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 41s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-server generated 1 new + 22 unchanged - 0 fixed = 23 total (was 22)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 231m 47s |  hbase-server in the patch passed.  |
   |  |   | 253m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0451ca37a273 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/testReport/ |
   | Max. process+thread count | 4345 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1412892078


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   In order to reduce code duplication, I refactored StoreEngine instead.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "wchevreuil (via GitHub)" <gi...@apache.org>.
wchevreuil commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1411894932


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
-      sortedFiles.add(file);
+      if (onlyLatestVersion) {
+        if (file.hasLatestVersion()) {
+          sortedFiles.add(file);
+        }
+      } else {
+        sortedFiles.add(file);
+      }
     }
     boolean succ = false;
     try {
-      for (int i = 0, n = files.size(); i < n; i++) {
+      for (int i = 0, n = files.size(); i < n && !sortedFiles.isEmpty(); i++) {

Review Comment:
   Then iterate the sortedFiles directly. What happens if sortedFiles is not empty, but files > sortedFiles? Won't you get a NPE on lines #153/155?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   Like @bbeaudreault explained. We can avoid code duplication with a little refactoring on DefaultStoreEngine. As we would already need to change it a bit to avoid code duplication anyways.
   
   > Or perhaps its fine as is because it would be great to get this ported till 2.5 release line?
   
   We may allow that on the brackport for 2.5 only, but I don't think this would justify adding such technical debit in master 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   StoreEngine and related interfaces have evolved organically and the current state is maybe not ideal.
   
   If we take the above approach, to refactor the default compactor to become a multi file compactor that only compacts to a single file, these called out compactors and engines can possibly be refactored to inherit from `DefaultStoreEngine` to take a cleaner approach (imho), although that would be follow up work. It would be worth taking a look at that. Later. Of course I am not suggesting that be done for 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1857399557

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   | -0 :warning: |  patch  |   5m 21s |  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  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 39s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 232m 49s |  hbase-server in the patch passed.  |
   |  |   | 254m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b503db13a73e 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e8762dfea4 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/testReport/ |
   | Max. process+thread count | 4374 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1874822039

   Sure, i have gone through updated design doc, will start reviewing the 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1501399522


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
 
   public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-    new CellSinkFactory<StoreFileWriter>() {
+  private final CellSinkFactory<DualFileWriter> writerFactory =

Review Comment:
   Then we should not name it DualFileWriter? Just add the logic to StoreFileWriter? And I do not see we remove the unused StoreFileWriter?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1930674143

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 54s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 32s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  master passed  |
   | -0 :warning: |  patch  |   6m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 54s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 54s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 30s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 235m 33s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 28s |  hbase-mapreduce in the patch passed.  |
   |  |   | 274m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8fe3c4341bc7 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 16de74c194 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/testReport/ |
   | Max. process+thread count | 5495 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/18/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998829624

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m  9s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 18s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m  8s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 24s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 201m 55s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  13m  3s |  hbase-mapreduce in the patch passed.  |
   |  |   | 239m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 02e60f161e7d 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/testReport/ |
   | Max. process+thread count | 5446 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/22/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998979262

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 20s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 49s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 42s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 59s |  master passed  |
   | -0 :warning: |  patch  |   0m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m  6s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 57s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-server: The patch generated 1 new + 37 unchanged - 0 fixed = 38 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  4s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 39s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m  8s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 8f1e966c9c6f 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 83 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/23/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1998507271

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | -0 :warning: |  patch  |   6m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  11m 19s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 37s |  hbase-mapreduce in the patch passed.  |
   |  |   |  47m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1cc16ffdd389 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/testReport/ |
   | Max. process+thread count | 3120 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/20/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545362732


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -105,72 +132,44 @@ public class StoreFileWriter implements CellSink, ShipperListener {
    * @param fileContext            The HFile context
    * @param shouldDropCacheBehind  Drop pages written to page cache after writing the store file.
    * @param compactedFilesSupplier Returns the {@link HStore} compacted files which not archived
+   * @param comparator             Cell comparator
+   * @param maxVersions            max cell versions
+   * @param newVersionBehavior     enable new version behavior
    * @throws IOException problem writing to FS
    */
-  private StoreFileWriter(FileSystem fs, Path path, final Configuration conf, CacheConfig cacheConf,
-    BloomType bloomType, long maxKeys, InetSocketAddress[] favoredNodes, HFileContext fileContext,
-    boolean shouldDropCacheBehind, Supplier<Collection<HStoreFile>> compactedFilesSupplier)
-    throws IOException {
+  private StoreFileWriter(FileSystem fs, Path liveFilePath, Path historicalFilePath,
+    final Configuration conf, CacheConfig cacheConf, BloomType bloomType, long maxKeys,
+    InetSocketAddress[] favoredNodes, HFileContext fileContext, boolean shouldDropCacheBehind,
+    Supplier<Collection<HStoreFile>> compactedFilesSupplier, CellComparator comparator,
+    int maxVersions, boolean newVersionBehavior) throws IOException {
+    this.fs = fs;
+    this.historicalFilePath = historicalFilePath;
+    this.conf = conf;
+    this.cacheConf = cacheConf;
+    this.bloomType = bloomType;
+    this.maxKeys = maxKeys;
+    this.favoredNodes = favoredNodes;
+    this.fileContext = fileContext;
+    this.shouldDropCacheBehind = shouldDropCacheBehind;
     this.compactedFilesSupplier = compactedFilesSupplier;
-    this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC);
-    // TODO : Change all writers to be specifically created for compaction context
-    writer =
-      HFile.getWriterFactory(conf, cacheConf).withPath(fs, path).withFavoredNodes(favoredNodes)
-        .withFileContext(fileContext).withShouldDropCacheBehind(shouldDropCacheBehind).create();
-
-    generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf,
-      bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-
-    if (generalBloomFilterWriter != null) {
-      this.bloomType = bloomType;
-      this.bloomParam = BloomFilterUtil.getBloomFilterParam(bloomType, conf);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: "
-          + (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH
-            ? Bytes.toInt(bloomParam)
-            : Bytes.toStringBinary(bloomParam))
-          + ", " + generalBloomFilterWriter.getClass().getSimpleName());
-      }
-      // init bloom context
-      switch (bloomType) {
-        case ROW:
-          bloomContext =
-            new RowBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
-          break;
-        case ROWCOL:
-          bloomContext =
-            new RowColBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
-          break;
-        case ROWPREFIX_FIXED_LENGTH:
-          bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter,
-            fileContext.getCellComparator(), Bytes.toInt(bloomParam));
-          break;
-        default:
-          throw new IOException(
-            "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL or ROWPREFIX expected)");
-      }
-    } else {
-      // Not using Bloom filters.
-      this.bloomType = BloomType.NONE;
-    }
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.newVersionBehavior = newVersionBehavior;
+    liveFileWriter = new SingleStoreFileWriter(fs, liveFilePath, conf, cacheConf, bloomType,
+      maxKeys, favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier);
+  }
 
-    // initialize delete family Bloom filter when there is NO RowCol Bloom filter
-    if (this.bloomType != BloomType.ROWCOL) {
-      this.deleteFamilyBloomFilterWriter = BloomFilterFactory.createDeleteBloomAtWrite(conf,
-        cacheConf, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-      deleteFamilyBloomContext =
-        new RowBloomContext(deleteFamilyBloomFilterWriter, fileContext.getCellComparator());
-    } else {
-      deleteFamilyBloomFilterWriter = null;
-    }
-    if (deleteFamilyBloomFilterWriter != null && LOG.isTraceEnabled()) {
-      LOG.trace("Delete Family Bloom filter type for " + path + ": "
-        + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
-    }
+  public static boolean shouldEnableHistoricalCompactionFiles(Configuration conf) {
+    return conf.getBoolean(ENABLE_HISTORICAL_COMPACTION_FILES,
+      DEFAULT_ENABLE_HISTORICAL_COMPACTION_FILES)
+      && conf.get(STORE_ENGINE_CLASS_KEY, DefaultStoreEngine.class.getName())
+        .equals(DefaultStoreEngine.class.getName())
+      && conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DefaultCompactor.class.getName())
+        .equals(DefaultCompactor.class.getName());
   }
 
   public long getPos() throws IOException {

Review Comment:
   Is this the correct behavior after we support writing multiple files? Or do we still need this method?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java:
##########
@@ -138,6 +140,12 @@ public class HStoreFile implements StoreFile {
   // Indicates if the file got compacted
   private volatile boolean compactedAway = false;
 
+  // Indicate if the file contains historical cell versions. This is used when
+  // hbase.enable.historical.compaction.files is set to true. In that case, compactions
+  // can generate two files, one with the live cell versions and the other with the remaining
+  // (historical) cell versions.
+  private volatile boolean isHistorical = false;

Review Comment:
   I'm still a bit confusd here. What does it mean if we have false here? This file only contains live cell versions? What if we just disable live file tracking? The field will be true or false?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);
+    if (historicalFileWriter != null) {
+      historicalFileWriter.trackTimestamps(cell);
     }
-    timeRangeTracker.includeTimestamp(cell);
   }
 
-  private void appendGeneralBloomfilter(final Cell cell) throws IOException {
-    if (this.generalBloomFilterWriter != null) {
-      /*
-       * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
-       * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + Timestamp 3 Types of
-       * Filtering: 1. Row = Row 2. RowCol = Row + Qualifier 3. RowPrefixFixedLength = Fixed Length
-       * Row Prefix
-       */
-      bloomContext.writeBloom(cell);
+  @Override
+  public void beforeShipped() throws IOException {
+    liveFileWriter.beforeShipped();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.beforeShipped();
     }
   }
 
-  private void appendDeleteFamilyBloomFilter(final Cell cell) throws IOException {
-    if (!PrivateCellUtil.isDeleteFamily(cell) && !PrivateCellUtil.isDeleteFamilyVersion(cell)) {
-      return;
-    }
+  public Path getPath() {
+    return liveFileWriter.getPath();
+  }
 
-    // increase the number of delete family in the store file
-    deleteFamilyCnt++;
-    if (this.deleteFamilyBloomFilterWriter != null) {
-      deleteFamilyBloomContext.writeBloom(cell);
+  public List<Path> getPaths() {
+    if (historicalFileWriter == null) {
+      return Lists.newArrayList(liveFileWriter.getPath());
     }
+    return Lists.newArrayList(liveFileWriter.getPath(), historicalFileWriter.getPath());
   }
 
-  @Override
-  public void append(final Cell cell) throws IOException {
-    appendGeneralBloomfilter(cell);
-    appendDeleteFamilyBloomFilter(cell);
-    writer.append(cell);
-    trackTimestamps(cell);
+  public boolean hasGeneralBloom() {
+    return liveFileWriter.hasGeneralBloom();
   }
 
-  @Override
-  public void beforeShipped() throws IOException {
-    // For now these writer will always be of type ShipperListener true.
-    // TODO : Change all writers to be specifically created for compaction context
-    writer.beforeShipped();
-    if (generalBloomFilterWriter != null) {
-      generalBloomFilterWriter.beforeShipped();
-    }
-    if (deleteFamilyBloomFilterWriter != null) {
-      deleteFamilyBloomFilterWriter.beforeShipped();
+  /**
+   * For unit testing only.
+   * @return the Bloom filter used by this writer.
+   */
+  BloomFilterWriter getGeneralBloomWriter() {
+    return liveFileWriter.generalBloomFilterWriter;
+  }
+
+  public void close() throws IOException {
+    liveFileWriter.appendFileInfo(HISTORICAL_KEY, Bytes.toBytes(false));
+    liveFileWriter.close();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendFileInfo(HISTORICAL_KEY, Bytes.toBytes(true));
+      historicalFileWriter.close();
     }
   }
 
-  public Path getPath() {
-    return this.writer.getPath();
+  public void appendFileInfo(byte[] key, byte[] value) throws IOException {
+    liveFileWriter.appendFileInfo(key, value);
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendFileInfo(key, value);
+    }
   }
 
-  public boolean hasGeneralBloom() {
-    return this.generalBloomFilterWriter != null;
+  /**
+   * For use in testing.
+   */
+  HFile.Writer getHFileWriter() {

Review Comment:
   Name it getLiveFileWriter?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   Any updates here?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   So we update liveStoreFiles and storefiles with two statement, will this cause any consistent problems?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   Ditto.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();

Review Comment:
   OK, I think the metadata is a missing part in our previous designs...
   
   In general, we should support writing multiple files for the DefaultStoreFileWriter, but then, we need to track some metadatas separately, for the different files, like timestamp range, cells count, etc.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);

Review Comment:
   I do not think this is the correct behavior, we should only call the write's trackTimestamps method if the cell has been written to the writer...



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);
+    if (historicalFileWriter != null) {
+      historicalFileWriter.trackTimestamps(cell);
     }
-    timeRangeTracker.includeTimestamp(cell);
   }
 
-  private void appendGeneralBloomfilter(final Cell cell) throws IOException {
-    if (this.generalBloomFilterWriter != null) {
-      /*
-       * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
-       * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + Timestamp 3 Types of
-       * Filtering: 1. Row = Row 2. RowCol = Row + Qualifier 3. RowPrefixFixedLength = Fixed Length
-       * Row Prefix
-       */
-      bloomContext.writeBloom(cell);
+  @Override
+  public void beforeShipped() throws IOException {
+    liveFileWriter.beforeShipped();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.beforeShipped();
     }
   }
 
-  private void appendDeleteFamilyBloomFilter(final Cell cell) throws IOException {
-    if (!PrivateCellUtil.isDeleteFamily(cell) && !PrivateCellUtil.isDeleteFamilyVersion(cell)) {
-      return;
-    }
+  public Path getPath() {
+    return liveFileWriter.getPath();

Review Comment:
   Maybe we should change this to a List of Paths?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -235,10 +210,10 @@ private byte[] toCompactionEventTrackerBytes(Collection<HStoreFile> storeFiles)
    */
   public void appendMetadata(final long maxSequenceId, final boolean majorCompaction,
     final long mobCellsCount) throws IOException {
-    writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
-    writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
-    writer.appendFileInfo(MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
-    appendTrackedTimestampsToMetadata();
+    liveFileWriter.appendMetadata(maxSequenceId, majorCompaction, mobCellsCount);

Review Comment:
   Is this the correct behavior? We should track mobCellsCount for the two writers separately?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545755759


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);

Review Comment:
   The delete code here is moved to SingleStoreFileWriter#trackTimestamps(). So the behavior did not change. 
   `    private void trackTimestamps(final Cell cell) {
         if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
           earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
         }
         timeRangeTracker.includeTimestamp(cell);
       }`



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545464762


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java:
##########
@@ -138,6 +140,12 @@ public class HStoreFile implements StoreFile {
   // Indicates if the file got compacted
   private volatile boolean compactedAway = false;
 
+  // Indicate if the file contains historical cell versions. This is used when
+  // hbase.enable.historical.compaction.files is set to true. In that case, compactions
+  // can generate two files, one with the live cell versions and the other with the remaining
+  // (historical) cell versions.
+  private volatile boolean isHistorical = false;

Review Comment:
   It looks like I need to add more comments for this. If isHistorical is true then the hfile is historical. Historical files are skipped for regular (not raw) scans scanning latest row versions. When hbase.enable.historical.compaction.files is false,  the historical flag will be false for all files. This means all files will be treated as live files, and thus there is no need to track them. Historical files are generated  only when hbase.enable.historical.compaction.files is true. Only when hbase.enable.historical.compaction.files is true, we enable live file file tracking.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545925927


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -247,7 +222,10 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti
    * @throws IOException problem writing to FS
    */
   public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws IOException {
-    writer.appendFileInfo(MOB_FILE_REFS, MobUtils.serializeMobFileRefs(mobRefSet));
+    liveFileWriter.appendMobMetadata(mobRefSet);

Review Comment:
   I have done that the last commit.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028445194

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 40s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 55s |  master passed  |
   | -0 :warning: |  patch  |   0m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 56s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 56s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 34s |  hbase-server: The patch generated 1 new + 39 unchanged - 0 fixed = 40 total (was 39)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m  4s |  Patch does not cause any errors with Hadoop 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 40s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 30s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 73ace762e765 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 80 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000634670

   > @Apache9, @virajjasani, @bbeaudreault , This PR is not done yet. I just realized that I need to add testing for newVersionBehavior.
   > 
   > Also, I need to discuss the following case:
   > 
   > Assume that for a given cell, two versions inserted and max versions is set to 1. If memory compaction is not enabled then I expect that both versions will be written to a new hfile (hf1) during flush even though the second version is redundant (is that true? I need to verify this). Now during minor compaction, the latest version will be written to a new live file (hf2) and the redundant version to a new historical file (hf3). Assume that a delete version marker is inserted for the latest version. This delete marker will be written to a new hfile (hf4). This delete marker will mask mask the latest version, and regular scans for the latest versions will not return any of the versions of this cell as latest version is masked by the delete marker and the redundant version is in the historical file (hf3) will be omitted by these scans. When the major compaction happens I expect that the redundant version should be revived and will be written to a new live file (hf5). Now the redunda
 nt version would be visible to regular scans. Please let me know if any of these is incorrect.
   > 
   > Please note this should not happen with newVersionBehavior as the deleted versions are considered toward total version count. Should we enable this feature only when newVersionBehavior is enabled?
   
   
   @kadirozde i just verified with HBase 2.6 (branch-2) that the flush writes only maxVersions versions to the new HFile. Hence, if the max version is 1, and if we write 2 versions of the cell, only the latest cell is written to the HFile.
   So we should be good for the first case.
   
   At this point, we can add some tests for newVersionBehavior, and we should be good for the above concern.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2001894215

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 50s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 218m 57s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 19s |  hbase-mapreduce in the patch passed.  |
   |  |   | 259m 23s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5e8e264d8a26 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/testReport/ |
   | Max. process+thread count | 5229 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2001897341

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 10s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | -0 :warning: |  patch  |   6m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 232m 38s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  15m  2s |  hbase-mapreduce in the patch passed.  |
   |  |   | 272m  4s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2c78ec9342d4 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/testReport/ |
   | Max. process+thread count | 5695 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/25/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2074177990

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m  0s |  Docker mode activated.  |
   | -1 :x: |  patch  |   0m  3s |  https://github.com/apache/hbase/pull/5545 does not apply to master. Rebase required? Wrong Branch? See https://yetus.apache.org/documentation/in-progress/precommit-patchnames for help.  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/29/console |
   | versions | git=2.17.1 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1577104587


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   And also this one, could you please verify whether we actually throw any exceptions here? We'd better not declare throws if there is no actual exception thrown.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1577103908


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   I suggest that, we introduce a something like a Pair for storing all storefiles and live storefiles in a single field, and mark it as volatile, so the reader can get always get a consistent view. Something like
   
   ```
   class StoreFileList {
     @Nonnull
     final ImmutableList<HStoreFile> storefiles;
     @Nullable
     final ImmutableList<HStoreFile> liveStorefiles;
   
     StoreFileList(ImmutableList<HStoreFile> storefiles, ImmutableList<HStoreFile> liveStorefiles) {
       this.storefiles = storefiles;
       this.liveStorefiles = liveStorefiles;
     }
   }
   
   private static final StoreFileList EMPTY_STOREFILE_LIST = new StoreFileList(ImmutableList.of(), null);
   
   private volatile StoreFileList storefiles = EMPTY_STOREFILE_LIST;
   ```



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558961072


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -181,7 +197,10 @@ public long getPos() throws IOException {
    */
   public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
     throws IOException {
-    appendMetadata(maxSequenceId, majorCompaction, Collections.emptySet());
+    liveFileWriter.appendMetadata(maxSequenceId, majorCompaction);

Review Comment:
   OK, checked the other multi file writer implementation, found HBASE-15400, where we explained why we use the same maxSequenceId for multiple genedated store files. So at least this is not a critical problem.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2116939444

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 46s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 28s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  master passed  |
   | -0 :warning: |  patch  |   7m 38s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 28s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 45s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 268m 25s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  17m  5s |  hbase-mapreduce in the patch passed.  |
   |  |   | 316m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 44673d172319 5.4.0-172-generic #190-Ubuntu SMP Fri Feb 2 23:24:22 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/testReport/ |
   | Max. process+thread count | 4806 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443692228


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -41,7 +45,7 @@
  * Default implementation of StoreFileManager. Not thread-safe.
  */
 @InterfaceAudience.Private
-class DefaultStoreFileManager implements StoreFileManager {
+public class DefaultStoreFileManager implements StoreFileManager {

Review Comment:
   Adding dual file writing capability to default compactor but not mob compaction required MobStoreEngine not to extend DefaultStoreEngine anymore. Instead MobStoreEngine now extends StoreEngine<DefaultStoreFlusher, RatioBasedCompactionPolicy, DefaultMobStoreCompactor, DefaultStoreFileManager>. This required making DefaultStoreFileManager public.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443688028


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
 
   public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-    new CellSinkFactory<StoreFileWriter>() {
+  private final CellSinkFactory<DualFileWriter> writerFactory =
+    new CellSinkFactory<DualFileWriter>() {
       @Override
-      public StoreFileWriter createWriter(InternalScanner scanner, FileDetails fd,
+      public DualFileWriter createWriter(InternalScanner scanner, FileDetails fd,
         boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker)
         throws IOException {
-        return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major,
-          writerCreationTracker);
+        boolean enableDualFileWriter = conf.getBoolean(

Review Comment:
   StoreFileWriter is a single file writer. DualFileWriter can write to up to two files. It can behaves as a single file writer as well as a dual file writer. DualFileWriter extends AbstractMultiFileWriter as DateTieredMultiFileWriter and StripeMultiFileWriter do. The relationship type between StoreFileWriter and an AbstractMultiFileWriter is composition but not inheritance. Thus, they cannot be exchangeable. The store engine code is organized this way currently. Changing this organization would be very invasive.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442606527


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java:
##########
@@ -46,13 +45,16 @@ public class DefaultStoreEngine extends StoreEngine<DefaultStoreFlusher, RatioBa
     "hbase.hstore.defaultengine.compactor.class";
   public static final String DEFAULT_COMPACTION_POLICY_CLASS_KEY =
     "hbase.hstore.defaultengine.compactionpolicy.class";
+  public static final String DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY =
+    "hbase.hstore.defaultengine.enable.dualfilewriter";
 
   private static final Class<? extends DefaultStoreFlusher> DEFAULT_STORE_FLUSHER_CLASS =
     DefaultStoreFlusher.class;
   private static final Class<? extends DefaultCompactor> DEFAULT_COMPACTOR_CLASS =
     DefaultCompactor.class;
-  private static final Class<? extends RatioBasedCompactionPolicy> DEFAULT_COMPACTION_POLICY_CLASS =
+  public static final Class<? extends RatioBasedCompactionPolicy> DEFAULT_COMPACTION_POLICY_CLASS =
     ExploringCompactionPolicy.class;
+  public static final boolean DEFAULT_ENABLE_DUAL_FILE_WRITER = false;

Review Comment:
   I did that couple commits back and will do it again.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1879742301

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  4s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 46s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 35s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 55s |  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  |   5m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 220m  7s |  hbase-server in the patch passed.  |
   | -1 :x: |  unit  |  12m 10s |  hbase-mapreduce in the patch failed.  |
   |  |   | 258m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 76f1989e64ba 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4aeabdcc71 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-mapreduce.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/testReport/ |
   | Max. process+thread count | 4768 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1942868397

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  1s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 22s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  master passed  |
   | -0 :warning: |  patch  |   6m  9s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 22s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 217m 13s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m  8s |  hbase-mapreduce in the patch passed.  |
   |  |   | 256m  8s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 485546dc67c6 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9656006778 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/testReport/ |
   | Max. process+thread count | 5036 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1942885694

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | -0 :warning: |  patch  |   5m 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 _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  0s |  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  | 236m 47s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  15m 11s |  hbase-mapreduce in the patch passed.  |
   |  |   | 275m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux eb7f8a1dabe1 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9656006778 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/testReport/ |
   | Max. process+thread count | 5639 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/19/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1962788373

   > In general, I think this is a good improvement.
   > 
   > But for me, I think we should still keep the architecture clean, so I do not think we should introduce a new 'DualFileWriter' and replace StoreFileWriter in the 'Default'Compactor, and seems we still keep the StoreFileWriter? Where do we use it after this PR?
   > 
   > Thanks.
   
   Initially, I introduced a separate store engine for this, called DualFileStoreEngine that came with its own compactor (DualFileCompactor), writer (DualFileWriter) and store file manager (DualFileStoreManager). The reason I did that DefaultCompactor is a single file writer compactor while DualFileCompactor was a multi file writer like DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine and so I thought architecturally DualFileCompactor should have had its own store engine. It was a clean solution. However, the reviewers asked me to either make DualFileCompactor inherit from DefaultStoreCompactor or refactor DefaultStoreCompactor to embed the dual file writer functionality.  Inheritance did not work and so I went for refactoring DefaultStoreCompactor and I had to change DefaultStoreCompactor to use DualFileWriter instead of StoreFileWriter. You can see all that from the earlier commits. Are you suggesting that 
 we should have a separate store engine for dual file compaction as I did in the beginning?
   
   To answer your question on StoreFileWriter,  yes StoreFileWriter is used by all writers including DualFileWriter, StripeMultiFileWriter, DateTieredMultiFileWriter. Please see the implementation of WriterFactory interface defined in AbstractMultiFileWriter all these writers inherit. These implementations indirectly calls StoreEngine#createWriter which then indirectly calls StoreFileTrackerBase#createWriter where a StoreFileWriter is created using StoreFileWriter#Builder.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "bbeaudreault (via GitHub)" <gi...@apache.org>.
bbeaudreault commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1411474153


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   These are all IA.Private classes, you could do a small refactor to make it work?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1857226484

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 34s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 37s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 45s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 33s |  master passed  |
   | -0 :warning: |  patch  |   1m 39s |  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  |   2m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 34s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 34s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 36s |  hbase-server: The patch generated 3 new + 56 unchanged - 0 fixed = 59 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  10m 47s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 41s |  patch has no errors when running spotless:check.  |
   | -1 :x: |  spotbugs  |   1m 39s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 10s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m  3s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER isn't final but should be  At DefaultStoreEngine.java:be  At DefaultStoreEngine.java:[line 57] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 13e3f58e4d74 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e8762dfea4 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotbugs | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 77 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1857387341

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 35s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 10s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 15s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  master passed  |
   | -0 :warning: |  patch  |   6m 45s |  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  |   2m 58s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 52s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 52s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 21s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 217m  0s |  hbase-server in the patch passed.  |
   |  |   | 242m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 52bc66753193 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e8762dfea4 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/testReport/ |
   | Max. process+thread count | 4761 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/9/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1412892101


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
-      sortedFiles.add(file);
+      if (onlyLatestVersion) {
+        if (file.hasLatestVersion()) {
+          sortedFiles.add(file);
+        }
+      } else {
+        sortedFiles.add(file);
+      }
     }
     boolean succ = false;
     try {
-      for (int i = 0, n = files.size(); i < n; i++) {
+      for (int i = 0, n = files.size(); i < n && !sortedFiles.isEmpty(); i++) {

Review Comment:
   I eliminated the extra check.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1839040475

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 31s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 43s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 34s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 50s |  the patch passed  |
   | -1 :x: |  compile  |   1m  7s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   1m  7s |  hbase-server in the patch failed.  |
   | -0 :warning: |  checkstyle  |   0m 35s |  hbase-server: The patch generated 13 new + 46 unchanged - 0 fixed = 59 total (was 46)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  10m  2s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 33s |  patch has 64 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   1m 31s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  7s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux e6317fdbf31d 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 25e9228e2c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | compile | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 81 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/5/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837317526

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 20s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m  1s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 45s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-server generated 1 new + 95 unchanged - 0 fixed = 96 total (was 95)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 213m 50s |  hbase-server in the patch passed.  |
   |  |   | 238m 27s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 183e1e05e5f5 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/testReport/ |
   | Max. process+thread count | 4700 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837319685

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 41s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 49s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 42s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-server generated 1 new + 22 unchanged - 0 fixed = 23 total (was 22)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 232m 11s |  hbase-server in the patch failed.  |
   |  |   | 253m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0fe3f9c736f5 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/testReport/ |
   | Max. process+thread count | 4133 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/2/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "apurtell (via GitHub)" <gi...@apache.org>.
apurtell commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1416126845


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   > DualFileCompactor is a multi file writer as DateTierCompactor and StripeCompactor. Both DateTierStoreEngine and StripeStoreEngine do not inherit from DefaultStoreEngine
   
   ... and if we take the above approach, to refactor the default compactor to become a multi file compactor that only compacts to a single file, these called out compactors can be refactored to inherit from `DefaultStoreEngine`, although that would be follow up work. I am not suggesting that be done for 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000576912

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  9s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 23s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 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 _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 20s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 214m 52s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  13m 18s |  hbase-mapreduce in the patch passed.  |
   |  |   | 252m 55s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0a0e17d1ab46 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/testReport/ |
   | Max. process+thread count | 5202 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545462036


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   DefaultStoreFileManager is not thread safe, the caller is responsible for locking. In this case, StoreEngine acquires a write lock on the store before calling the methods of StoreFileManager. This prevents the consistency issues.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545467119


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -105,72 +132,44 @@ public class StoreFileWriter implements CellSink, ShipperListener {
    * @param fileContext            The HFile context
    * @param shouldDropCacheBehind  Drop pages written to page cache after writing the store file.
    * @param compactedFilesSupplier Returns the {@link HStore} compacted files which not archived
+   * @param comparator             Cell comparator
+   * @param maxVersions            max cell versions
+   * @param newVersionBehavior     enable new version behavior
    * @throws IOException problem writing to FS
    */
-  private StoreFileWriter(FileSystem fs, Path path, final Configuration conf, CacheConfig cacheConf,
-    BloomType bloomType, long maxKeys, InetSocketAddress[] favoredNodes, HFileContext fileContext,
-    boolean shouldDropCacheBehind, Supplier<Collection<HStoreFile>> compactedFilesSupplier)
-    throws IOException {
+  private StoreFileWriter(FileSystem fs, Path liveFilePath, Path historicalFilePath,
+    final Configuration conf, CacheConfig cacheConf, BloomType bloomType, long maxKeys,
+    InetSocketAddress[] favoredNodes, HFileContext fileContext, boolean shouldDropCacheBehind,
+    Supplier<Collection<HStoreFile>> compactedFilesSupplier, CellComparator comparator,
+    int maxVersions, boolean newVersionBehavior) throws IOException {
+    this.fs = fs;
+    this.historicalFilePath = historicalFilePath;
+    this.conf = conf;
+    this.cacheConf = cacheConf;
+    this.bloomType = bloomType;
+    this.maxKeys = maxKeys;
+    this.favoredNodes = favoredNodes;
+    this.fileContext = fileContext;
+    this.shouldDropCacheBehind = shouldDropCacheBehind;
     this.compactedFilesSupplier = compactedFilesSupplier;
-    this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC);
-    // TODO : Change all writers to be specifically created for compaction context
-    writer =
-      HFile.getWriterFactory(conf, cacheConf).withPath(fs, path).withFavoredNodes(favoredNodes)
-        .withFileContext(fileContext).withShouldDropCacheBehind(shouldDropCacheBehind).create();
-
-    generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf,
-      bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-
-    if (generalBloomFilterWriter != null) {
-      this.bloomType = bloomType;
-      this.bloomParam = BloomFilterUtil.getBloomFilterParam(bloomType, conf);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: "
-          + (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH
-            ? Bytes.toInt(bloomParam)
-            : Bytes.toStringBinary(bloomParam))
-          + ", " + generalBloomFilterWriter.getClass().getSimpleName());
-      }
-      // init bloom context
-      switch (bloomType) {
-        case ROW:
-          bloomContext =
-            new RowBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
-          break;
-        case ROWCOL:
-          bloomContext =
-            new RowColBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
-          break;
-        case ROWPREFIX_FIXED_LENGTH:
-          bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter,
-            fileContext.getCellComparator(), Bytes.toInt(bloomParam));
-          break;
-        default:
-          throw new IOException(
-            "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL or ROWPREFIX expected)");
-      }
-    } else {
-      // Not using Bloom filters.
-      this.bloomType = BloomType.NONE;
-    }
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.newVersionBehavior = newVersionBehavior;
+    liveFileWriter = new SingleStoreFileWriter(fs, liveFilePath, conf, cacheConf, bloomType,
+      maxKeys, favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier);
+  }
 
-    // initialize delete family Bloom filter when there is NO RowCol Bloom filter
-    if (this.bloomType != BloomType.ROWCOL) {
-      this.deleteFamilyBloomFilterWriter = BloomFilterFactory.createDeleteBloomAtWrite(conf,
-        cacheConf, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-      deleteFamilyBloomContext =
-        new RowBloomContext(deleteFamilyBloomFilterWriter, fileContext.getCellComparator());
-    } else {
-      deleteFamilyBloomFilterWriter = null;
-    }
-    if (deleteFamilyBloomFilterWriter != null && LOG.isTraceEnabled()) {
-      LOG.trace("Delete Family Bloom filter type for " + path + ": "
-        + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
-    }
+  public static boolean shouldEnableHistoricalCompactionFiles(Configuration conf) {
+    return conf.getBoolean(ENABLE_HISTORICAL_COMPACTION_FILES,
+      DEFAULT_ENABLE_HISTORICAL_COMPACTION_FILES)
+      && conf.get(STORE_ENGINE_CLASS_KEY, DefaultStoreEngine.class.getName())
+        .equals(DefaultStoreEngine.class.getName())
+      && conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DefaultCompactor.class.getName())
+        .equals(DefaultCompactor.class.getName());
   }
 
   public long getPos() throws IOException {

Review Comment:
   This is where we enable dual file writing (or in other words, we enable generating historical compaction files) only for default store engine with default compactor. As we decided, other compactions,  mob, stripe, and date tiered will be supported later. I will have separate jiras for them. Here we silently ignore historical file generation for these compactions.
   



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545755759


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -256,156 +234,571 @@ public void appendMobMetadata(SetMultimap<TableName, String> mobRefSet) throws I
   public void appendTrackedTimestampsToMetadata() throws IOException {
     // TODO: The StoreFileReader always converts the byte[] to TimeRange
     // via TimeRangeTracker, so we should write the serialization data of TimeRange directly.
-    appendFileInfo(TIMERANGE_KEY, TimeRangeTracker.toByteArray(timeRangeTracker));
-    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    liveFileWriter.appendTrackedTimestampsToMetadata();
+    if (historicalFileWriter != null) {
+      historicalFileWriter.appendTrackedTimestampsToMetadata();
+    }
   }
 
   /**
    * Record the earlest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker
    * to include the timestamp of this key
    */
   public void trackTimestamps(final Cell cell) {
-    if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-      earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
+    liveFileWriter.trackTimestamps(cell);

Review Comment:
   The delete code here is move to SingleStoreFileWriter#trackTimestamps(). So the behavior did not change. 
   `    private void trackTimestamps(final Cell cell) {
         if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
           earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
         }
         timeRangeTracker.includeTimestamp(cell);
       }`



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "bbeaudreault (via GitHub)" <gi...@apache.org>.
bbeaudreault commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545796726


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   Could you check if mob is enabled and, if so, log a warn and ignore the dual writer config value?
   



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028486055

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 46s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 20s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m  9s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  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  |   5m 18s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 202m 59s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  12m 56s |  hbase-mapreduce in the patch passed.  |
   |  |   | 241m 44s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2838688a6cf2 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/testReport/ |
   | Max. process+thread count | 5956 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/26/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "bbeaudreault (via GitHub)" <gi...@apache.org>.
bbeaudreault commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1999865811

   @Apache9 @kadirozde How close is this to ready? I think I'm reading to cut the first RC0 of 2.6.0 on Monday, but this might be a nice addition to the release. We've delayed 2.6.0 for a long time so I'd rather not delay it much longer though.


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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2045810903

   @Apache9, @bbeaudreault, @virajjasani, @apurtell, please let me know if there is any review item left to be addressed. From my perspective, I have addressed all review comments. 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1559993483


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   I just want to make sure that we are on the same page here. You think we need to make this method atomic even though
   (1) The class is not thread-safe as specified here 
   `Default implementation of StoreFileManager. Not thread-safe.`
   (2) There is only one caller which is StoreEngine and it takes a write lock whenever it calls a StoreFileManager methods, for example
   ```
   public void addStoreFiles(Collection<HStoreFile> storeFiles,
       IOExceptionRunnable actionAfterAdding) throws IOException {
       storeFileTracker.add(StoreUtils.toStoreFileInfo(storeFiles));
       writeLock();
       try {
         storeFileManager.insertNewFiles(storeFiles);
         actionAfterAdding.run();
       } finally {
         // We need the lock, as long as we are updating the storeFiles
         // or changing the memstore. Let us release it before calling
         // notifyChangeReadersObservers. See HBASE-4485 for a possible
         // deadlock scenario that could have happened if continue to hold
         // the lock.
         writeUnlock();
       }
     }
   ```
   
   I can easily add a write lock here and to the other methods but given the above points, is not it redundant? 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1559993483


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   I just want to make sure that we are on the same page here. You think we need to make this method atomic even though
   (1) The class is not thread-safe as specified here 
   `Default implementation of StoreFileManager. Not thread-safe.`
   (2) There is only one caller which is StoreEngine and it takes a write lock whenever it calls a StoreFileManager method, for example
   ```
   public void addStoreFiles(Collection<HStoreFile> storeFiles,
       IOExceptionRunnable actionAfterAdding) throws IOException {
       storeFileTracker.add(StoreUtils.toStoreFileInfo(storeFiles));
       writeLock();
       try {
         storeFileManager.insertNewFiles(storeFiles);
         actionAfterAdding.run();
       } finally {
         // We need the lock, as long as we are updating the storeFiles
         // or changing the memstore. Let us release it before calling
         // notifyChangeReadersObservers. See HBASE-4485 for a possible
         // deadlock scenario that could have happened if continue to hold
         // the lock.
         writeUnlock();
       }
     }
   ```
   
   I can easily add a write lock here and to the other methods but given the above points, is not it redundant? 



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "bbeaudreault (via GitHub)" <gi...@apache.org>.
bbeaudreault commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2073825447

   @Apache9 Can you respond here? I think we should move towards merging this, @kadirozde has been very accommodating. I also want to make sure you're happy with it, since your perspective is insightful and greatly appreciated. 


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2088988994

   @Apache9, would you please review the latest commit and let me know if you have more comments for this PR? I also updated the design doc to reflect the changes based on your review comments. Thank you!


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

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

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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1411603477


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   Or perhaps its fine as is because it would be great to get this ported till 2.5 release line?



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

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

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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "wchevreuil (via GitHub)" <gi...@apache.org>.
wchevreuil commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1410561791


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileStoreEngine.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.DualFileCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * HBASE-25972 This store engine allows us to store data in two files,
+ * one for the latest put cells and the other for the rest of the cells (i.e.,
+ * older put cells and delete markers).
+ */
+@InterfaceAudience.Private
+public class DualFileStoreEngine extends StoreEngine<DefaultStoreFlusher,

Review Comment:
   Can't we extend DefaultStoreEngine instead? There are few methods that are getting duplicated (entirely or partially) from there (needsCompaction, createComponents, createCompactor). 



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
-      sortedFiles.add(file);
+      if (onlyLatestVersion) {
+        if (file.hasLatestVersion()) {
+          sortedFiles.add(file);
+        }
+      } else {
+        sortedFiles.add(file);
+      }
     }
     boolean succ = false;
     try {
-      for (int i = 0, n = files.size(); i < n; i++) {
+      for (int i = 0, n = files.size(); i < n && !sortedFiles.isEmpty(); i++) {

Review Comment:
   Do we really need this extra check?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java:
##########
@@ -135,11 +136,17 @@ public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreF
     for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();

Review Comment:
   I wonder if we could have a DualStoreFileManager as well. That way, we could already keep separate list of store files there and it would contain the logic about which files should be returned for the scanner. It would also avoid us from having to open a reader on files we may not be interested.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1837391480

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 54s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 43s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 44s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 12s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-server generated 1 new + 95 unchanged - 0 fixed = 96 total (was 95)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 214m 35s |  hbase-server in the patch failed.  |
   |  |   | 237m  8s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e4e7f182a61e 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7dd4d0c532 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | javadoc | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/testReport/ |
   | Max. process+thread count | 4727 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/3/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1879744838

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 26s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  master passed  |
   | -0 :warning: |  patch  |   5m 57s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 59s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 231m 58s |  hbase-server in the patch passed.  |
   | -1 :x: |  unit  |  12m 54s |  hbase-mapreduce in the patch failed.  |
   |  |   | 268m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7ee718ee6359 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4aeabdcc71 |
   | Default Java | Temurin-1.8.0_352-b08 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-mapreduce.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/testReport/ |
   | Max. process+thread count | 5269 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/16/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1851096467

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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 _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   2m 25s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 47s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  master passed  |
   | -0 :warning: |  patch  |   5m 20s |  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  |   2m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 41s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 41s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 49s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 232m 29s |  hbase-server in the patch passed.  |
   |  |   | 254m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 028629dde5be 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 78c5ac3725 |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/testReport/ |
   | Max. process+thread count | 4134 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/7/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1849403537

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  0s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 18s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 38s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 14s |  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  |   1m 32s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 216m 48s |  hbase-server in the patch passed.  |
   |  |   | 243m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 33f36b4c9d37 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 82a2ce10f2 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/testReport/ |
   | Max. process+thread count | 4723 (vs. ulimit of 30000) |
   | modules | C: hbase-client hbase-server U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/6/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1851528694

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 24s |  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.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 28s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 39s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 25s |  master passed  |
   | -0 :warning: |  patch  |   1m 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  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 59s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 42s |  hbase-server: The patch generated 2 new + 56 unchanged - 0 fixed = 58 total (was 56)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 26s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | +1 :green_heart: |  spotless  |   0m 42s |  patch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   1m 33s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m  8s |  The patch does not generate ASF License warnings.  |
   |  |   |  36m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux ef3ad9810fc6 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 78c5ac3725 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 79 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/8/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443673538


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        deleteFamily = cell;
+        addLiveVersion(cell);
+      } else {
+        addHistoricalVersion(cell);
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (!isDeletedByDeleteFamily(cell)) {

Review Comment:
   Dual file compaction is an optimization and does not guarantee that live version files will include only live cell versions. Since minor compaction does not see all the cell versions, dual file compaction may include cells that are not actually live since the decision is done based on a subset of files in the case of minor compaction.  I just wanted to make sure we have the same understanding on this.
   
   If a DeleteFamily and DeleteFamilyVersion marker both have same timestamp and the DeleteFamilyVersion marker is not deleted/masked by the DeleteFamily marker then, the sequence id of the DeleteFamilyVersion marker must be higher than that of the DeleteFamily marker. In this case, we still need to track the DeleteFamilyVersion marker (i.e., need to add to the delete family version list). This tracking is necessary to make sure that the put cell with the same timestamp will be deleted by the DeleteFamilyVersion marker. And then we can append the DeleteFamilyVersion marker to the historical version file as you suggested. The same is also applicable to the Delete marker. I will make the changes for this in the next commit.
   



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443671033


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java:
##########
@@ -464,7 +464,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
             lastCleanCell = null;
             lastCleanCellSeqId = 0;
           }
-          writer.append(c);

Review Comment:
   Is this change related?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java:
##########
@@ -50,15 +50,15 @@ public interface StoreFileManager {
    */
   @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
       allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
-  void loadFiles(List<HStoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles) throws IOException;

Review Comment:
   Why we need to add the exception declaration here?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
 
   public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-    new CellSinkFactory<StoreFileWriter>() {
+  private final CellSinkFactory<DualFileWriter> writerFactory =

Review Comment:
   There is no store file writer any more?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
 
   public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-    new CellSinkFactory<StoreFileWriter>() {
+  private final CellSinkFactory<DualFileWriter> writerFactory =
+    new CellSinkFactory<DualFileWriter>() {
       @Override
-      public StoreFileWriter createWriter(InternalScanner scanner, FileDetails fd,
+      public DualFileWriter createWriter(InternalScanner scanner, FileDetails fd,
         boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker)
         throws IOException {
-        return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major,
-          writerCreationTracker);
+        boolean enableDualFileWriter = conf.getBoolean(

Review Comment:
   It seems a bit strange here, the flag is named enable dual file writer, but no matter it is true or false, we always create a DualFileWriter...
   
   I think here we should still return a StoreFileWriter? If dual file writer is enabled, we return a DualFileWriter, otherwise we return the default store file writer?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java:
##########
@@ -300,7 +300,7 @@ private double getMidStripeSplitRatio(long smallerSize, long largerSize, long la
 
   @Override
   public Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
-    byte[] stopRow, boolean includeStopRow) {
+    byte[] stopRow, boolean includeStopRow, boolean onlyLatestVersion) {

Review Comment:
   The new parameter is not used?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -41,7 +45,7 @@
  * Default implementation of StoreFileManager. Not thread-safe.
  */
 @InterfaceAudience.Private
-class DefaultStoreFileManager implements StoreFileManager {
+public class DefaultStoreFileManager implements StoreFileManager {

Review Comment:
   Why we need to make this public?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1443676077


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java:
##########
@@ -464,7 +464,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
             lastCleanCell = null;
             lastCleanCellSeqId = 0;
           }
-          writer.append(c);

Review Comment:
   DualFIleWriter works with both append() and the new appendAll(). I added the appendAll() for the compaction performance reasons as appendAll() eliminates the row key comparison for the cells of the same row (except the first cell).



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1442694151


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DualFileWriter.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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 static org.apache.hadoop.hbase.regionserver.HStoreFile.HAS_LIVE_VERSIONS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Separates the provided cells into two files, one file for the live cells and the other for the
+ * rest of the cells (historical cells). The live cells includes the live put cells, delete all and
+ * version delete markers that are not masked by other delete all markers.
+ */
+@InterfaceAudience.Private
+public class DualFileWriter extends AbstractMultiFileWriter {
+
+  private final CellComparator comparator;
+  private StoreFileWriter liveVersionWriter;
+  private StoreFileWriter historicalVersionWriter;
+
+  private final List<StoreFileWriter> writers;
+  // The last cell of the current row
+  private Cell lastCell;
+  // The first (latest) delete family marker of the current row
+  private Cell deleteFamily;
+  // The list of delete family version markers of the current row
+  private List<Cell> deleteFamilyVersionList = new ArrayList<>();
+  // The first (latest) delete column marker of the current column
+  private Cell deleteColumn;
+  // The list of delete column version markers of the current column
+  private List<Cell> deleteColumnVersionList = new ArrayList<>();
+  // The live put cell count for the current column
+  private int livePutCellCount;
+  private final boolean dualWriterEnabled;
+  private final int maxVersions;
+  private final boolean newVersionBehavior;
+
+  public DualFileWriter(CellComparator comparator, int maxVersions, boolean dualWriterEnabled,
+    boolean newVersionBehavior) {
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.dualWriterEnabled = dualWriterEnabled;
+    this.newVersionBehavior = newVersionBehavior;
+    writers = new ArrayList<>(2);
+    initRowState();
+  }
+
+  private void initRowState() {
+    deleteFamily = null;
+    deleteFamilyVersionList.clear();
+    lastCell = null;
+  }
+
+  private void initColumnState() {
+    livePutCellCount = 0;
+    deleteColumn = null;
+    deleteColumnVersionList.clear();
+
+  }
+
+  private void addLiveVersion(Cell cell) throws IOException {
+    if (liveVersionWriter == null) {
+      liveVersionWriter = writerFactory.createWriter();
+      writers.add(liveVersionWriter);
+    }
+    liveVersionWriter.append(cell);
+  }
+
+  private void addHistoricalVersion(Cell cell) throws IOException {
+    if (historicalVersionWriter == null) {
+      historicalVersionWriter = writerFactory.createWriter();
+      writers.add(historicalVersionWriter);
+    }
+    historicalVersionWriter.append(cell);
+  }
+
+  private boolean isDeletedByDeleteFamily(Cell cell) {
+    return deleteFamily != null && (deleteFamily.getTimestamp() > cell.getTimestamp()
+      || (deleteFamily.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteFamily.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteFamilyVersion(Cell cell) {
+    for (Cell deleteFamilyVersion : deleteFamilyVersionList) {
+      if (
+        deleteFamilyVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteFamilyVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeletedByDeleteColumn(Cell cell) {
+    return deleteColumn != null && (deleteColumn.getTimestamp() > cell.getTimestamp()
+      || (deleteColumn.getTimestamp() == cell.getTimestamp()
+        && (!newVersionBehavior || cell.getSequenceId() < deleteColumn.getSequenceId())));
+  }
+
+  private boolean isDeletedByDeleteColumnVersion(Cell cell) {
+    for (Cell deleteColumnVersion : deleteColumnVersionList) {
+      if (
+        deleteColumnVersion.getTimestamp() == cell.getTimestamp()
+          && (!newVersionBehavior || cell.getSequenceId() < deleteColumnVersion.getSequenceId())
+      ) return true;
+    }
+    return false;
+  }
+
+  private boolean isDeleted(Cell cell) {
+    return isDeletedByDeleteFamily(cell) || isDeletedByDeleteColumn(cell)
+      || isDeletedByDeleteFamilyVersion(cell) || isDeletedByDeleteColumnVersion(cell);
+  }
+
+  private void appendCell(Cell cell) throws IOException {
+    if ((lastCell == null || !CellUtil.matchingColumn(lastCell, cell))) {
+      initColumnState();
+    }
+    if (cell.getType() == Cell.Type.DeleteFamily) {
+      if (deleteFamily == null) {
+        if (cell.getType() == Cell.Type.DeleteFamily) {
+          deleteFamily = cell;
+          addLiveVersion(cell);
+        } else {
+          addHistoricalVersion(cell);
+        }
+      }
+    } else if (cell.getType() == Cell.Type.DeleteFamilyVersion) {
+      if (deleteFamily == null) {

Review Comment:
   Good catch! Yes, we need to do 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


Re: [PR] HBASE-25972 Dual File Compactor [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1865873392

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  master passed  |
   | +1 :green_heart: |  compile  |   4m 52s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  7s |  master passed  |
   | +1 :green_heart: |  spotless  |   0m 59s |  branch has no errors when running spotless:check.  |
   | +1 :green_heart: |  spotbugs  |   2m 43s |  master passed  |
   | -0 :warning: |  patch  |   0m 48s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 31s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 31s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 41s |  hbase-server: The patch generated 3 new + 56 unchanged - 0 fixed = 59 total (was 56)  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-mapreduce: The patch generated 2 new + 27 unchanged - 0 fixed = 29 total (was 27)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 47s |  Patch does not cause any errors with Hadoop 3.2.4 3.3.6.  |
   | -1 :x: |  spotless  |   0m 52s |  patch has 33 errors when running spotless:check, run spotless:apply to fix.  |
   | +1 :green_heart: |  spotbugs  |   3m 18s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 21s |  The patch does not generate ASF License warnings.  |
   |  |   |  47m 38s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti spotless checkstyle compile |
   | uname | Linux 63833ca57ad4 5.4.0-163-generic #180-Ubuntu SMP Tue Sep 5 13:21:23 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2c07847656 |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-mapreduce.txt |
   | spotless | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/artifact/yetus-general-check/output/patch-spotless.txt |
   | Max. process+thread count | 77 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/10/console |
   | versions | git=2.34.1 maven=3.8.6 spotbugs=4.7.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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1545554451


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java:
##########
@@ -105,72 +132,44 @@ public class StoreFileWriter implements CellSink, ShipperListener {
    * @param fileContext            The HFile context
    * @param shouldDropCacheBehind  Drop pages written to page cache after writing the store file.
    * @param compactedFilesSupplier Returns the {@link HStore} compacted files which not archived
+   * @param comparator             Cell comparator
+   * @param maxVersions            max cell versions
+   * @param newVersionBehavior     enable new version behavior
    * @throws IOException problem writing to FS
    */
-  private StoreFileWriter(FileSystem fs, Path path, final Configuration conf, CacheConfig cacheConf,
-    BloomType bloomType, long maxKeys, InetSocketAddress[] favoredNodes, HFileContext fileContext,
-    boolean shouldDropCacheBehind, Supplier<Collection<HStoreFile>> compactedFilesSupplier)
-    throws IOException {
+  private StoreFileWriter(FileSystem fs, Path liveFilePath, Path historicalFilePath,
+    final Configuration conf, CacheConfig cacheConf, BloomType bloomType, long maxKeys,
+    InetSocketAddress[] favoredNodes, HFileContext fileContext, boolean shouldDropCacheBehind,
+    Supplier<Collection<HStoreFile>> compactedFilesSupplier, CellComparator comparator,
+    int maxVersions, boolean newVersionBehavior) throws IOException {
+    this.fs = fs;
+    this.historicalFilePath = historicalFilePath;
+    this.conf = conf;
+    this.cacheConf = cacheConf;
+    this.bloomType = bloomType;
+    this.maxKeys = maxKeys;
+    this.favoredNodes = favoredNodes;
+    this.fileContext = fileContext;
+    this.shouldDropCacheBehind = shouldDropCacheBehind;
     this.compactedFilesSupplier = compactedFilesSupplier;
-    this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC);
-    // TODO : Change all writers to be specifically created for compaction context
-    writer =
-      HFile.getWriterFactory(conf, cacheConf).withPath(fs, path).withFavoredNodes(favoredNodes)
-        .withFileContext(fileContext).withShouldDropCacheBehind(shouldDropCacheBehind).create();
-
-    generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf,
-      bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-
-    if (generalBloomFilterWriter != null) {
-      this.bloomType = bloomType;
-      this.bloomParam = BloomFilterUtil.getBloomFilterParam(bloomType, conf);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: "
-          + (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH
-            ? Bytes.toInt(bloomParam)
-            : Bytes.toStringBinary(bloomParam))
-          + ", " + generalBloomFilterWriter.getClass().getSimpleName());
-      }
-      // init bloom context
-      switch (bloomType) {
-        case ROW:
-          bloomContext =
-            new RowBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
-          break;
-        case ROWCOL:
-          bloomContext =
-            new RowColBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
-          break;
-        case ROWPREFIX_FIXED_LENGTH:
-          bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter,
-            fileContext.getCellComparator(), Bytes.toInt(bloomParam));
-          break;
-        default:
-          throw new IOException(
-            "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL or ROWPREFIX expected)");
-      }
-    } else {
-      // Not using Bloom filters.
-      this.bloomType = BloomType.NONE;
-    }
+    this.comparator = comparator;
+    this.maxVersions = maxVersions;
+    this.newVersionBehavior = newVersionBehavior;
+    liveFileWriter = new SingleStoreFileWriter(fs, liveFilePath, conf, cacheConf, bloomType,
+      maxKeys, favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier);
+  }
 
-    // initialize delete family Bloom filter when there is NO RowCol Bloom filter
-    if (this.bloomType != BloomType.ROWCOL) {
-      this.deleteFamilyBloomFilterWriter = BloomFilterFactory.createDeleteBloomAtWrite(conf,
-        cacheConf, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-      deleteFamilyBloomContext =
-        new RowBloomContext(deleteFamilyBloomFilterWriter, fileContext.getCellComparator());
-    } else {
-      deleteFamilyBloomFilterWriter = null;
-    }
-    if (deleteFamilyBloomFilterWriter != null && LOG.isTraceEnabled()) {
-      LOG.trace("Delete Family Bloom filter type for " + path + ": "
-        + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
-    }
+  public static boolean shouldEnableHistoricalCompactionFiles(Configuration conf) {
+    return conf.getBoolean(ENABLE_HISTORICAL_COMPACTION_FILES,
+      DEFAULT_ENABLE_HISTORICAL_COMPACTION_FILES)
+      && conf.get(STORE_ENGINE_CLASS_KEY, DefaultStoreEngine.class.getName())
+        .equals(DefaultStoreEngine.class.getName())
+      && conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DefaultCompactor.class.getName())
+        .equals(DefaultCompactor.class.getName());
   }
 
   public long getPos() throws IOException {

Review Comment:
   How to silently ignore? Could you please add more comments in the code base to describe this? And at least we should have some warnings logs to tell users that this configuration can not be used together with your configured store engine or compaction algorithm...



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028948735

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 10s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  master passed  |
   | -0 :warning: |  patch  |   6m 22s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 12s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 38s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 245m  4s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 52s |  hbase-mapreduce in the patch passed.  |
   |  |   | 285m 53s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 344c7def2d2c 5.4.0-174-generic #193-Ubuntu SMP Thu Mar 7 14:29:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/testReport/ |
   | Max. process+thread count | 5958 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2028942924

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 51s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | -0 :warning: |  patch  |   6m 42s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 51s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 221m  3s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 19s |  hbase-mapreduce in the patch passed.  |
   |  |   | 262m  3s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 94b30f1ef591 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9adca10e9c |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/testReport/ |
   | Max. process+thread count | 5713 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/27/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000013796

   I can also re-review the PR since my last review is now stale.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000602763

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 30s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | -0 :warning: |  patch  |   6m  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 _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  7s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 247m 49s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 52s |  hbase-mapreduce in the patch passed.  |
   |  |   | 287m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.44 ServerAPI=1.44 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a36f7c68bc4f 5.4.0-169-generic #187-Ubuntu SMP Thu Nov 23 14:52:28 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Temurin-1.8.0_352-b08 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/testReport/ |
   | Max. process+thread count | 6033 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000584582

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 58s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 52s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   | -0 :warning: |  patch  |   6m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 54s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  9s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  9s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 54s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 221m 16s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  14m 32s |  hbase-mapreduce in the patch passed.  |
   |  |   | 262m 25s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c8b0ec027e2b 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9361ae506a |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/testReport/ |
   | Max. process+thread count | 5261 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/24/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2000579921

   > Assume that for a given cell, two versions inserted and max versions is set to 1. If memory compaction is not enabled then I expect that both versions will be written to a new hfile (hf1) during flush even though the second version is redundant (is that true? I need to verify this).
   
   If table descriptor level max version is set to 1, during flush only latest version will be written to the HFile hf1.


-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1916091487

   Please note that the default behaviour still remains same with feature turned off.


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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1558932155


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   The point here is that, in the past, we only have a single list for storing the store files, so in the reader's view, there is no inconsitency problem. But now, we have two lists for storing store files, and the update is not atomic, what if the reader gets the list in the middle of the updates to these two fields?



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-1961828205

   > This is on my list. It is kinda big so I need to spend some time to take a whole review, to prevent that I comment some of the problems, you update the PR, and then I comment with something else which conflicts with your new efforts...
   
   @Apache9, Please let me know how I can help making faster progress in this review process. If you really want me to extend this dual compaction for mob and stripe compaction in this PR, I am willing to work on that too. Would you like to me to start a discussion thread in the dev mailing list to invite more reviewers for 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache9 (via GitHub)" <gi...@apache.org>.
Apache9 commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1501399800


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java:
##########
@@ -17,42 +17,45 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_ENABLE_DUAL_FILE_WRITER_KEY;
+import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_ENABLE_DUAL_FILE_WRITER;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.DualFileWriter;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Compact passed set of files. Create an instance and then call
  * {@link #compact(CompactionRequestImpl, ThroughputController, User)}
  */
 @InterfaceAudience.Private
-public class DefaultCompactor extends Compactor<StoreFileWriter> {
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultCompactor.class);
+public class DefaultCompactor extends AbstractMultiOutputCompactor<DualFileWriter> {
 
   public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-    new CellSinkFactory<StoreFileWriter>() {
+  private final CellSinkFactory<DualFileWriter> writerFactory =
+    new CellSinkFactory<DualFileWriter>() {
       @Override
-      public StoreFileWriter createWriter(InternalScanner scanner, FileDetails fd,
+      public DualFileWriter createWriter(InternalScanner scanner, FileDetails fd,
         boolean shouldDropBehind, boolean major, Consumer<Path> writerCreationTracker)
         throws IOException {
-        return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major,
-          writerCreationTracker);
+        boolean enableDualFileWriter = conf.getBoolean(

Review Comment:
   As said above, then we should just change the StoreFileWriter, not introducing a new DualFileWriter but can still write one file...



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "kadirozde (via GitHub)" <gi...@apache.org>.
kadirozde commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1577321980


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java:
##########
@@ -86,13 +111,20 @@ public Collection<HStoreFile> getCompactedfiles() {
   }
 
   @Override
-  public void insertNewFiles(Collection<HStoreFile> sfs) {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    if (enableLiveFileTracking) {
+      this.liveStoreFiles = ImmutableList.sortedCopyOf(getStoreFileComparator(),

Review Comment:
   Sounds good! I will make the changes as suggested.



-- 
This is an automated message from the Apache Git Service.
To 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2075973876

   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 37s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  6s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 57s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  master passed  |
   | -0 :warning: |  patch  |   8m  3s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 29s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 277m 55s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |  15m  9s |  hbase-mapreduce in the patch passed.  |
   |  |   | 325m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9e27f690482d 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6c6e776eea |
   | Default Java | Eclipse Adoptium-11.0.17+8 |
   | unit | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/testReport/ |
   | Max. process+thread count | 4705 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2075904068

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 15s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  master passed  |
   | -0 :warning: |  patch  |   6m 41s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 10s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 59s |  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  |   5m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 224m 58s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  14m 27s |  hbase-mapreduce in the patch passed.  |
   |  |   | 266m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 952f4e74a2a6 5.4.0-1103-aws #111~18.04.1-Ubuntu SMP Tue May 23 20:04:10 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6c6e776eea |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/testReport/ |
   | Max. process+thread count | 4468 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/30/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "virajjasani (via GitHub)" <gi...@apache.org>.
virajjasani commented on code in PR #5545:
URL: https://github.com/apache/hbase/pull/5545#discussion_r1604229161


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileWriter.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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 static org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.NEW_VERSION_BEHAVIOR;
+import static org.apache.hadoop.hbase.regionserver.StoreFileWriter.ENABLE_HISTORICAL_COMPACTION_FILES;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.MemoryCompactionPolicy;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Store file writer does not do any compaction. Each cell written to either the live or historical
+ * file. Regular (i.e., not-raw) scans that reads the latest put cells scans only live files. To
+ * ensure the correctness of store file writer, we need to verify that live files includes all live
+ * cells. This test indirectly verify this as follows. The test creates two tables, each with one
+ * region and one store. The dual file writing (live vs historical) is configured on only one of the
+ * tables. The test generates exact set of mutations on both tables. These mutations include all
+ * types of cells and these cells are written to multiple files using multiple memstore flushes.
+ * After writing all cells, the test first verify that both tables return the same set of cells for
+ * regular and raw scans. Then the same verification is done after tables are minor and finally
+ * major compacted. The test also verifies that flushes do not generate historical files and the
+ * historical files are generated only when historical file generation is enabled (by the config
+ * hbase.enable.historical.compaction.files).
+ */
+@Category({ MediumTests.class, RegionServerTests.class })
+@RunWith(Parameterized.class)
+public class TestStoreFileWriter {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStoreFileWriter.class);
+  private final int ROW_NUM = 100;
+  private final Random RANDOM = new Random(11);
+  private final HBaseTestingUtil testUtil = new HBaseTestingUtil();
+  private HRegion[] regions = new HRegion[2];
+  private final byte[][] qualifiers =
+    { Bytes.toBytes("0"), Bytes.toBytes("1"), Bytes.toBytes("2") };
+  // This keeps track of all cells. It is a list of rows, each row is a list of columns, each
+  // column is a list of CellInfo object
+  private ArrayList<ArrayList<ArrayList<CellInfo>>> insertedCells;
+  private TableName[] tableName = new TableName[2];
+  private final Configuration conf = testUtil.getConfiguration();
+  private int flushCount = 0;
+
+  @Parameterized.Parameter(0)
+  public KeepDeletedCells keepDeletedCells;
+  @Parameterized.Parameter(1)
+  public int maxVersions;
+  @Parameterized.Parameter(2)
+  public boolean newVersionBehavior;
+
+  @Parameterized.Parameters(name = "keepDeletedCells={0}, maxVersions={1}, newVersionBehavior={2}")
+  public static synchronized Collection<Object[]> data() {
+    return Arrays.asList(
+      new Object[][] { { KeepDeletedCells.FALSE, 1, true }, { KeepDeletedCells.FALSE, 2, false },
+        { KeepDeletedCells.FALSE, 3, true }, { KeepDeletedCells.TRUE, 1, false },
+        // { KeepDeletedCells.TRUE, 2, true }, see HBASE-28442
+        { KeepDeletedCells.TRUE, 3, false } });
+  }
+
+  // In memory representation of a cell. We only need to know timestamp and type field for our
+  // testing for cell. Please note the row for the cell is implicit in insertedCells.
+  private static class CellInfo {
+    long timestamp;
+    Cell.Type type;
+
+    CellInfo(long timestamp, Cell.Type type) {
+      this.timestamp = timestamp;
+      this.type = type;
+    }
+  }
+
+  private void createTable(int index, boolean enableDualFileWriter) throws IOException {
+    tableName[index] = TableName.valueOf(getClass().getSimpleName() + "_" + index);
+    ColumnFamilyDescriptor familyDescriptor =
+      ColumnFamilyDescriptorBuilder.newBuilder(HBaseTestingUtil.fam1).setMaxVersions(maxVersions)
+        .setKeepDeletedCells(keepDeletedCells)
+        .setValue(NEW_VERSION_BEHAVIOR, Boolean.toString(newVersionBehavior)).build();
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(tableName[index]).setColumnFamily(familyDescriptor)
+        .setValue(ENABLE_HISTORICAL_COMPACTION_FILES, Boolean.toString(enableDualFileWriter));
+    testUtil.createTable(builder.build(), null);
+    regions[index] = testUtil.getMiniHBaseCluster().getRegions(tableName[index]).get(0);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 6);
+    conf.set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
+      String.valueOf(MemoryCompactionPolicy.NONE));
+    testUtil.startMiniCluster();
+    createTable(0, false);
+    createTable(1, true);
+    insertedCells = new ArrayList<>(ROW_NUM);
+    for (int r = 0; r < ROW_NUM; r++) {
+      insertedCells.add(new ArrayList<>(qualifiers.length));
+      for (int q = 0; q < qualifiers.length; q++) {
+        insertedCells.get(r).add(new ArrayList<>(10));
+      }
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    this.testUtil.shutdownMiniCluster();
+    testUtil.cleanupTestDir();
+  }
+
+  @Test
+  public void testCompactedFiles() throws Exception {
+    for (int i = 0; i < 10; i++) {
+      insertRows(ROW_NUM * maxVersions);
+      deleteRows(ROW_NUM / 8);
+      deleteRowVersions(ROW_NUM / 8);
+      deleteColumns(ROW_NUM / 8);
+      deleteColumnVersions(ROW_NUM / 8);
+      flushRegion();
+    }
+
+    verifyCells();
+
+    HStore[] stores = new HStore[2];
+
+    stores[0] = regions[0].getStore(HBaseTestingUtil.fam1);
+    assertEquals(flushCount, stores[0].getStorefilesCount());
+
+    stores[1] = regions[1].getStore(HBaseTestingUtil.fam1);
+    assertEquals(flushCount, stores[1].getStorefilesCount());
+
+    regions[0].compact(false);
+    assertEquals(flushCount - stores[0].getCompactedFiles().size() + 1,
+      stores[0].getStorefilesCount());
+
+    regions[1].compact(false);
+    assertEquals(flushCount - stores[1].getCompactedFiles().size() + 2,
+      stores[1].getStorefilesCount());
+
+    verifyCells();
+
+    regions[0].compact(true);
+    assertEquals(1, stores[0].getStorefilesCount());
+
+    regions[1].compact(true);
+    assertEquals(keepDeletedCells == KeepDeletedCells.FALSE ? 1 : 2,
+      stores[1].getStorefilesCount());
+
+    verifyCells();
+  }
+
+  private void verifyCells() throws Exception {
+    scanAndCompare(false);
+    scanAndCompare(true);
+  }
+
+  private void flushRegion() throws Exception {
+    regions[0].flush(true);
+    regions[1].flush(true);
+    flushCount++;
+  }
+
+  private Long getRowTimestamp(int row) {
+    Long maxTimestamp = null;
+    for (int q = 0; q < qualifiers.length; q++) {
+      int size = insertedCells.get(row).get(q).size();
+      if (size > 0) {
+        CellInfo mostRecentCellInfo = insertedCells.get(row).get(q).get(size - 1);
+        if (mostRecentCellInfo.type == Cell.Type.Put) {
+          if (maxTimestamp == null || maxTimestamp < mostRecentCellInfo.timestamp) {
+            maxTimestamp = mostRecentCellInfo.timestamp;
+          }
+        }
+      }
+    }
+    return maxTimestamp;
+  }
+
+  private long getNewTimestamp(long timestamp) throws Exception {
+    long newTimestamp = System.currentTimeMillis();
+    if (timestamp == newTimestamp) {
+      Thread.sleep(1);
+      newTimestamp = System.currentTimeMillis();
+      assert (timestamp < newTimestamp);
+    }
+    return newTimestamp;
+  }
+
+  private void insertRows(int rowCount) throws Exception {
+    int row;
+    long timestamp = System.currentTimeMillis();
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Put put = new Put(Bytes.toBytes(String.valueOf(row)), timestamp);
+      for (int q = 0; q < qualifiers.length; q++) {
+        put.addColumn(HBaseTestingUtil.fam1, qualifiers[q],
+          Bytes.toBytes(String.valueOf(timestamp)));
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.Put));
+      }
+      regions[0].put(put);
+      regions[1].put(put);
+      timestamp = getNewTimestamp(timestamp);
+    }
+  }
+
+  private void deleteRows(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      long timestamp = System.currentTimeMillis();
+      row = RANDOM.nextInt(ROW_NUM);
+      Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+      regions[0].delete(delete);
+      regions[1].delete(delete);
+      // For simplicity, the family delete markers are inserted for all columns (instead of
+      // allocating a separate column for them) in the memory representation of the data stored
+      // to HBase
+      for (int q = 0; q < qualifiers.length; q++) {
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteFamily));
+      }
+    }
+  }
+
+  private void deleteSingleRowVersion(int row, long timestamp) throws IOException {
+    Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+    delete.addFamilyVersion(HBaseTestingUtil.fam1, timestamp);
+    regions[0].delete(delete);
+    regions[1].delete(delete);
+    // For simplicity, the family delete version markers are inserted for all columns (instead of
+    // allocating a separate column for them) in the memory representation of the data stored
+    // to HBase
+    for (int q = 0; q < qualifiers.length; q++) {
+      insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteFamilyVersion));
+    }
+  }
+
+  private void deleteRowVersions(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Long timestamp = getRowTimestamp(row);
+      if (timestamp != null) {
+        deleteSingleRowVersion(row, timestamp);
+      }
+    }
+    // Just insert one more delete marker possibly does not delete any row version
+    row = RANDOM.nextInt(ROW_NUM);
+    deleteSingleRowVersion(row, System.currentTimeMillis());
+  }
+
+  private void deleteColumns(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      long timestamp = System.currentTimeMillis();
+      row = RANDOM.nextInt(ROW_NUM);
+      int q = RANDOM.nextInt(qualifiers.length);
+      Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)), timestamp);
+      delete.addColumns(HBaseTestingUtil.fam1, qualifiers[q], timestamp);
+      regions[0].delete(delete);
+      regions[1].delete(delete);
+      insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.DeleteColumn));
+    }
+  }
+
+  private void deleteColumnVersions(int rowCount) throws Exception {
+    int row;
+    for (int r = 0; r < rowCount; r++) {
+      row = RANDOM.nextInt(ROW_NUM);
+      Long timestamp = getRowTimestamp(row);
+      if (timestamp != null) {
+        Delete delete = new Delete(Bytes.toBytes(String.valueOf(row)));
+        int q = RANDOM.nextInt(qualifiers.length);
+        delete.addColumn(HBaseTestingUtil.fam1, qualifiers[q], timestamp);
+        regions[0].delete(delete);
+        regions[1].delete(delete);
+        insertedCells.get(row).get(q).add(new CellInfo(timestamp, Cell.Type.Delete));
+      }
+    }
+  }
+
+  private Scan createScan(boolean raw) {
+    Scan scan = new Scan();
+    scan.readAllVersions();
+    scan.setRaw(raw);
+    return scan;
+  }
+
+  private void scanAndCompare(boolean raw) throws Exception {
+    try (RegionScanner firstRS = regions[0].getScanner(createScan(raw))) {
+      try (RegionScanner secondRS = regions[1].getScanner(createScan(raw))) {
+        boolean firstHasMore;
+        boolean secondHasMore;
+        do {
+          List<Cell> firstRowList = new ArrayList<>();
+          List<Cell> secondRowList = new ArrayList<>();
+          firstHasMore = firstRS.nextRaw(firstRowList);
+          secondHasMore = secondRS.nextRaw(secondRowList);
+          assertEquals(firstRowList.size(), secondRowList.size());
+          int size = firstRowList.size();
+          for (int i = 0; i < size; i++) {
+            Cell firstCell = firstRowList.get(i);
+            Cell secondCell = secondRowList.get(i);
+            assert (CellUtil.matchingRowColumn(firstCell, secondCell));

Review Comment:
   nit: These asserts could be assertTrue or assertEquals, but this is not blocker for merging the PR, can be done later in future.



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

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

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


Re: [PR] HBASE-25972 Dual File Compaction [hbase]

Posted by "Apache-HBase (via GitHub)" <gi...@apache.org>.
Apache-HBase commented on PR #5545:
URL: https://github.com/apache/hbase/pull/5545#issuecomment-2116873570

   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 37s |  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 _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   2m 54s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 19s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  master passed  |
   | -0 :warning: |  patch  |   6m 21s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   2m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 226m 57s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |  16m 15s |  hbase-mapreduce in the patch passed.  |
   |  |   | 269m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.45 ServerAPI=1.45 base: https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/artifact/yetus-jdk17-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/5545 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e01eef41cba4 5.4.0-172-generic #190-Ubuntu SMP Fri Feb 2 23:24:22 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 716adf50e9 |
   | Default Java | Eclipse Adoptium-17.0.10+7 |
   |  Test Results | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/testReport/ |
   | Max. process+thread count | 5420 (vs. ulimit of 30000) |
   | modules | C: hbase-server hbase-mapreduce U: . |
   | Console output | https://ci-hbase.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-5545/34/console |
   | versions | git=2.34.1 maven=3.8.6 |
   | 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