This is an automated email from the ASF dual-hosted git repository. kturner pushed a commit to branch 2.1 in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/2.1 by this push: new e57df50bdd fixes bug with compaction preemption (#3961) e57df50bdd is described below commit e57df50bdd940e1115cf2f55032ca3e7c2421f2b Author: Keith Turner <ktur...@apache.org> AuthorDate: Mon Nov 20 18:08:33 2023 -0800 fixes bug with compaction preemption (#3961) User compactions can preempt selector compaction under certain conditions. There was a bug in the code where a user compaction could preempt a selection compaction that was in the middle of selecting its files. The way the code is structured doing this causes an exception. This change makes the preemption wait until file selection is complete. --- .../accumulo/tserver/tablet/CompactableImpl.java | 11 +++++-- .../tablet/CompactableImplFileManagerTest.java | 34 ++++++++++++++++++++++ 2 files changed, 42 insertions(+), 3 deletions(-) diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java index 1a820bee5b..11722c100f 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java @@ -81,6 +81,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; import com.google.common.collect.Collections2; @@ -231,6 +232,11 @@ public class CompactableImpl implements Compactable { return selectKind; } + @VisibleForTesting + Set<StoredTabletFile> getSelectedFiles() { + return Set.copyOf(selectedFiles); + } + SelectedInfo getReservedInfo() { Preconditions.checkState(selectStatus == FileSelectionStatus.RESERVED); return new SelectedInfo(initiallySelectedAll, selectedFiles, selectKind); @@ -245,7 +251,8 @@ public class CompactableImpl implements Compactable { Preconditions.checkArgument(kind == CompactionKind.SELECTOR || kind == CompactionKind.USER); if (selectStatus == FileSelectionStatus.NOT_ACTIVE || (kind == CompactionKind.USER - && selectKind == CompactionKind.SELECTOR && noneRunning(CompactionKind.SELECTOR))) { + && selectKind == CompactionKind.SELECTOR && noneRunning(CompactionKind.SELECTOR) + && selectStatus != FileSelectionStatus.SELECTING)) { selectStatus = FileSelectionStatus.NEW; selectKind = kind; selectedFiles.clear(); @@ -1065,7 +1072,6 @@ public class CompactableImpl implements Compactable { manager.compactableChanged(this); } - } catch (Exception e) { log.error("Failed to select user compaction files {}", getExtent(), e); } finally { @@ -1075,7 +1081,6 @@ public class CompactableImpl implements Compactable { } } } - } static Collection<String> asFileNames(Set<StoredTabletFile> files) { diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java index 971ee51e31..8214823be3 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java @@ -288,6 +288,40 @@ public class CompactableImplFileManagerTest { assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); } + @Test + public void testUserCompactionPreemptsSelectorCompaction() { + TestFileManager fileMgr = new TestFileManager(); + + assertTrue(fileMgr.initiateSelection(SELECTOR)); + assertEquals(SELECTOR, fileMgr.getSelectionKind()); + assertTrue(fileMgr.beginSelection()); + // USER compaction should not be able to preempt while in the middle of selecting files + assertFalse(fileMgr.initiateSelection(USER)); + assertEquals(SELECTOR, fileMgr.getSelectionKind()); + fileMgr.finishSelection(newFiles("F00000.rf", "F00001.rf", "F00002.rf"), false); + // check state is as expected after finishing selection + assertEquals(SELECTOR, fileMgr.getSelectionKind()); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + assertFalse(fileMgr.getSelectedFiles().isEmpty()); + + // USER compaction should not be able to preempt when there are running compactions. + fileMgr.running.add(SELECTOR); + assertFalse(fileMgr.initiateSelection(USER)); + // check state is as expected + assertEquals(SELECTOR, fileMgr.getSelectionKind()); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + assertFalse(fileMgr.getSelectedFiles().isEmpty()); + + // after file selection is complete and there are no running compactions, should be able to + // preempt + fileMgr.running.clear(); + assertTrue(fileMgr.initiateSelection(USER)); + // check that things were properly reset + assertEquals(USER, fileMgr.getSelectionKind()); + assertEquals(FileSelectionStatus.NEW, fileMgr.getSelectionStatus()); + assertTrue(fileMgr.getSelectedFiles().isEmpty()); + } + @Test public void testUserCompactionCanceled() { TestFileManager fileMgr = new TestFileManager();