This is an automated email from the ASF dual-hosted git repository. kturner pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push: new a8c27d5 Adds unit test for compaction file tracking (#2289) a8c27d5 is described below commit a8c27d5286f43abc246e799b60d0ba1e1b3910f9 Author: Keith Turner <ktur...@apache.org> AuthorDate: Wed Sep 29 14:13:12 2021 -0400 Adds unit test for compaction file tracking (#2289) --- .../accumulo/tserver/tablet/CompactableImpl.java | 53 ++- .../tablet/CompactableImplFileManagerTest.java | 500 +++++++++++++++++++++ 2 files changed, 534 insertions(+), 19 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 5498ec9..228517e 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 @@ -22,6 +22,7 @@ import static org.apache.accumulo.tserver.TabletStatsKeeper.Operation.MAJOR; import java.io.IOException; import java.io.UncheckedIOException; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -179,11 +180,11 @@ public class CompactableImpl implements Compactable { * in a mutually exclusive manner, so synchronization at this level is unnecessary. * */ - private class FileManager { + static abstract class FileManager { FileSelectionStatus selectStatus = FileSelectionStatus.NOT_ACTIVE; - private long selectedTime; - private long selectedExpirationDurationMs; + private long selectedTimeNanos; + private Duration selectedExpirationDuration; private CompactionKind selectKind = null; // Tracks if when a set of files was selected, if at that time the set was all of the tablets @@ -193,7 +194,7 @@ public class CompactableImpl implements Compactable { private boolean initiallySelectedAll = false; private Set<StoredTabletFile> selectedFiles = new HashSet<>(); - private Set<StoredTabletFile> allCompactingFiles = new HashSet<>(); + protected Set<StoredTabletFile> allCompactingFiles = new HashSet<>(); // track files produced by compactions of this tablet, those are considered chopped private Set<StoredTabletFile> choppedFiles = new HashSet<>(); @@ -201,10 +202,10 @@ public class CompactableImpl implements Compactable { private Set<StoredTabletFile> allFilesWhenChopStarted = new HashSet<>(); private final KeyExtent extent; - private Deriver<Long> selectionExpirationDeriver; + private Deriver<Duration> selectionExpirationDeriver; public FileManager(KeyExtent extent, Collection<StoredTabletFile> extCompactingFiles, - Optional<SelectedInfo> extSelInfo, Deriver<Long> selectionExpirationDeriver) { + Optional<SelectedInfo> extSelInfo, Deriver<Duration> selectionExpirationDeriver) { this.extent = extent; this.selectionExpirationDeriver = selectionExpirationDeriver; @@ -233,8 +234,14 @@ public class CompactableImpl implements Compactable { return new SelectedInfo(initiallySelectedAll, selectedFiles, selectKind); } + protected abstract boolean noneRunning(CompactionKind kind); + + protected abstract long getNanoTime(); + boolean initiateSelection(CompactionKind kind) { + Preconditions.checkArgument(kind == CompactionKind.SELECTOR || kind == CompactionKind.USER); + if (selectStatus == FileSelectionStatus.NOT_ACTIVE || (kind == CompactionKind.USER && selectKind == CompactionKind.SELECTOR && noneRunning(CompactionKind.SELECTOR))) { selectStatus = FileSelectionStatus.NEW; @@ -262,9 +269,9 @@ public class CompactableImpl implements Compactable { Preconditions.checkArgument(!selected.isEmpty()); Preconditions.checkState(selectStatus == FileSelectionStatus.SELECTING); selectStatus = FileSelectionStatus.SELECTED; - selectedTime = System.currentTimeMillis(); + selectedTimeNanos = getNanoTime(); // take a snapshot of this from config and use it for the entire selection for consistency - selectedExpirationDurationMs = selectionExpirationDeriver.derive(); + selectedExpirationDuration = selectionExpirationDeriver.derive(); selectedFiles.clear(); selectedFiles.addAll(selected); initiallySelectedAll = allSelected; @@ -396,7 +403,7 @@ public class CompactableImpl implements Compactable { case SELECTED: { Set<StoredTabletFile> candidates = new HashSet<>(currFiles); candidates.removeAll(allCompactingFiles); - if (System.currentTimeMillis() - selectedTime < selectedExpirationDurationMs) { + if (getNanoTime() - selectedTimeNanos < selectedExpirationDuration.toNanos()) { candidates.removeAll(selectedFiles); } return Collections.unmodifiableSet(candidates); @@ -467,12 +474,12 @@ public class CompactableImpl implements Compactable { * * @return true if the files were reserved and false otherwise */ - private boolean reserveFiles(CompactionJob job, Set<StoredTabletFile> jobFiles) { + boolean reserveFiles(CompactionJob job, Set<StoredTabletFile> jobFiles) { Preconditions.checkArgument(!jobFiles.isEmpty()); if (selectStatus == FileSelectionStatus.SELECTED - && System.currentTimeMillis() - selectedTime > selectedExpirationDurationMs + && getNanoTime() - selectedTimeNanos > selectedExpirationDuration.toNanos() && job.getKind() != selectKind && !Collections.disjoint(selectedFiles, jobFiles)) { // If a selected compaction starts running, it should always changes the state to RESERVED. // So would never expect there to be any running when in the SELECTED state. @@ -494,8 +501,7 @@ public class CompactableImpl implements Compactable { if (job.getKind() == CompactionKind.USER || job.getKind() == CompactionKind.SELECTOR) { if (selectKind == job.getKind()) { if (!selectedFiles.containsAll(jobFiles)) { - // TODO diff log level? - log.error("Ignoring {} compaction that does not contain selected files {} {} {}", + log.trace("Ignoring {} compaction that does not contain selected files {} {} {}", job.getKind(), getExtent(), asFileNames(selectedFiles), asFileNames(jobFiles)); return false; } @@ -546,8 +552,7 @@ public class CompactableImpl implements Compactable { * @param newFile * The file produced by a compaction. If the compaction failed, this can be null. */ - private void completed(CompactionJob job, Set<StoredTabletFile> jobFiles, - StoredTabletFile newFile) { + void completed(CompactionJob job, Set<StoredTabletFile> jobFiles, StoredTabletFile newFile) { Preconditions.checkArgument(!jobFiles.isEmpty()); Preconditions.checkState(allCompactingFiles.removeAll(jobFiles)); if (newFile != null) { @@ -574,7 +579,7 @@ public class CompactableImpl implements Compactable { || (selectStatus == FileSelectionStatus.CANCELED && noneRunning(selectKind))) { selectStatus = FileSelectionStatus.NOT_ACTIVE; log.trace("Selected compaction status changed {} {}", getExtent(), selectStatus); - } else if (selectStatus == FileSelectionStatus.SELECTED) { + } else if (selectStatus == FileSelectionStatus.RESERVED) { selectedFiles.add(newFile); log.trace("Compacted subset of selected files {} {} -> {}", getExtent(), asFileNames(jobFiles), newFile.getFileName()); @@ -648,10 +653,20 @@ public class CompactableImpl implements Compactable { return Set.copyOf(servicesIds); }, 2, TimeUnit.SECONDS); - Deriver<Long> selectionExpirationDeriver = tablet.getTableConfiguration() - .newDeriver(conf -> conf.getTimeInMillis(Property.TABLE_COMPACTION_SELECTION_EXPIRATION)); + Deriver<Duration> selectionExpirationNanosDeriver = + tablet.getTableConfiguration().newDeriver(conf -> Duration + .ofMillis(conf.getTimeInMillis(Property.TABLE_COMPACTION_SELECTION_EXPIRATION))); this.fileMgr = new FileManager(tablet.getExtent(), extCompactingFiles, extSelInfo, - selectionExpirationDeriver); + selectionExpirationNanosDeriver) { + protected boolean noneRunning(CompactionKind kind) { + return CompactableImpl.this.noneRunning(kind); + } + + @Override + protected long getNanoTime() { + return System.nanoTime(); + } + }; } private void verifyExternalCompactions( 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 new file mode 100644 index 0000000..42e934f --- /dev/null +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java @@ -0,0 +1,500 @@ +/* + * 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.accumulo.tserver.tablet; + +import static org.apache.accumulo.core.spi.compaction.CompactionKind.CHOP; +import static org.apache.accumulo.core.spi.compaction.CompactionKind.SELECTOR; +import static org.apache.accumulo.core.spi.compaction.CompactionKind.SYSTEM; +import static org.apache.accumulo.core.spi.compaction.CompactionKind.USER; +import static org.apache.accumulo.tserver.tablet.CompactableImplFileManagerTest.TestFileManager.SELECTION_EXPIRATION; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.accumulo.core.client.admin.compaction.CompactableFile; +import org.apache.accumulo.core.conf.AccumuloConfiguration.Deriver; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.metadata.CompactableFileImpl; +import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.DataFileValue; +import org.apache.accumulo.core.spi.compaction.CompactionExecutorId; +import org.apache.accumulo.core.spi.compaction.CompactionJob; +import org.apache.accumulo.core.spi.compaction.CompactionKind; +import org.apache.accumulo.tserver.tablet.CompactableImpl.ChopSelectionStatus; +import org.apache.accumulo.tserver.tablet.CompactableImpl.FileManager.ChopSelector; +import org.apache.accumulo.tserver.tablet.CompactableImpl.FileSelectionStatus; +import org.junit.Test; + +import com.google.common.collect.Sets; + +public class CompactableImplFileManagerTest { + + @Test + public void testSystem() { + TestFileManager fileMgr = new TestFileManager(); + + var tabletFiles = newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"); + + assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR); + + var job1 = newJob(CompactionKind.SYSTEM, "F00000.rf", "F00001.rf"); + + assertTrue(fileMgr.reserveFiles(job1)); + assertEquals(job1.getSTFiles(), fileMgr.getCompactingFiles()); + + assertEquals(newFiles("F00002.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR); + + var job2 = newJob(CompactionKind.SYSTEM, "F00002.rf", "F00003.rf"); + + assertTrue(fileMgr.reserveFiles(job2)); + assertEquals(Sets.union(job1.getSTFiles(), job2.getSTFiles()), fileMgr.getCompactingFiles()); + + // try to reserve files reserved by other compactions, should fail + assertFalse(fileMgr.reserveFiles(newJob(CompactionKind.SYSTEM, "F00001.rf", "F00002.rf"))); + + assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CompactionKind.SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR); + + fileMgr.completed(job2, newFile("C00004.rf")); + fileMgr.completed(job1, newFile("C00005.rf")); + + tabletFiles = newFiles("C00004.rf", "C00005.rf"); + + assertEquals(newFiles("C00004.rf", "C00005.rf"), + fileMgr.getCandidates(tabletFiles, CompactionKind.SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR); + + var job3 = newJob(CompactionKind.SYSTEM, "C00004.rf", "C00005.rf"); + assertTrue(fileMgr.reserveFiles(job3)); + + assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR); + + fileMgr.completed(job3, newFile("A00006.rf")); + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + + } + + @Test + public void testSelection() { + TestFileManager fileMgr = new TestFileManager(); + + var tabletFiles = newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"); + + // This job will be used to attempt to reserve files that are not selected. This could happen if + // a job was queued for a bit and things changed. + var staleJob = newJob(USER, "F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"); + + assertFalse(fileMgr.reserveFiles(staleJob)); + + assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR); + + assertTrue(fileMgr.initiateSelection(USER)); + + assertFalse(fileMgr.reserveFiles(staleJob)); + + assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR); + + assertTrue(fileMgr.beginSelection()); + + assertFalse(fileMgr.reserveFiles(staleJob)); + + assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR); + + fileMgr.finishSelection(newFiles("F00000.rf", "F00001.rf", "F00002.rf"), false); + + assertFalse(fileMgr.reserveFiles(staleJob)); + + assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf"), + fileMgr.getCandidates(tabletFiles, USER, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR); + + assertFalse(fileMgr.reserveFiles(newJob(SELECTOR, "F00000.rf", "F00001.rf"))); + assertFalse(fileMgr.reserveFiles(newJob(SYSTEM, "F00000.rf", "F00001.rf"))); + + var job1 = newJob(USER, "F00000.rf", "F00001.rf"); + + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + assertTrue(fileMgr.reserveFiles(job1)); + assertEquals(FileSelectionStatus.RESERVED, fileMgr.getSelectionStatus()); + + assertFalse(fileMgr.reserveFiles(staleJob)); + assertFalse(fileMgr.reserveFiles(newJob(USER, "F00001.rf", "F00002.rf"))); + + // advance time past the expiration timeout, however this should not matter since the first + // reservation was successfully made + fileMgr.setNanoTime(2 * SELECTION_EXPIRATION.toNanos()); + + assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertEquals(newFiles("F00002.rf"), fileMgr.getCandidates(tabletFiles, USER, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR); + + fileMgr.completed(job1, newFile("C00004.rf")); + assertEquals(FileSelectionStatus.RESERVED, fileMgr.getSelectionStatus()); + + tabletFiles = newFiles("C00004.rf", "F00002.rf", "F00003.rf"); + + assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertEquals(newFiles("F00002.rf", "C00004.rf"), + fileMgr.getCandidates(tabletFiles, USER, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR); + + var job2 = newJob(USER, "F00002.rf", "C00004.rf"); + + assertTrue(fileMgr.reserveFiles(job2)); + assertEquals(FileSelectionStatus.RESERVED, fileMgr.getSelectionStatus()); + + fileMgr.completed(job2, newFile("C00005.rf")); + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + + assertEquals(FileSelectionStatus.NOT_ACTIVE, fileMgr.getSelectionStatus()); + + tabletFiles = newFiles("C00005.rf", "F00003.rf"); + + assertEquals(newFiles("C00005.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, USER, CHOP, SELECTOR); + } + + @Test + public void testSelectionExpiration() { + TestFileManager fileMgr = new TestFileManager(); + var tabletFiles = newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"); + + assertTrue(fileMgr.initiateSelection(USER)); + assertTrue(fileMgr.beginSelection()); + fileMgr.finishSelection(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"), false); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + + // a system compaction should not be able to reserved files that are selected + assertFalse(fileMgr.reserveFiles(newJob(SYSTEM, "F00001.rf", "F00002.rf"))); + + assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, USER, false)); + assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, SELECTOR); + + // advance time to a point where the selection is eligible to expire + fileMgr.setNanoTime(2 * SELECTION_EXPIRATION.toNanos()); + + // now that the selection is eligible to expire, the selected files should be available as + // system compaction candidates + assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, USER, false)); + assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"), + fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR); + + // a system compaction should be able to reserve selected files after expiration which should + // deactivate the selection + var job1 = newJob(SYSTEM, "F00000.rf", "F00001.rf"); + assertTrue(fileMgr.reserveFiles(job1)); + assertEquals(FileSelectionStatus.NOT_ACTIVE, fileMgr.getSelectionStatus()); + + // this should fail because the selection was deactivated + assertFalse(fileMgr.reserveFiles(newJob(USER, "F00002.rf", "F00003.rf"))); + + fileMgr.completed(job1, newFile("C00004.rf")); + + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + + } + + @Test + public void testSelectionExpirationDisjoint() { + TestFileManager fileMgr = new TestFileManager(); + + assertTrue(fileMgr.initiateSelection(USER)); + assertTrue(fileMgr.beginSelection()); + fileMgr.finishSelection(newFiles("F00000.rf", "F00001.rf"), false); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + + // advance time to a point where the selection is eligible to expire + fileMgr.setNanoTime(2 * SELECTION_EXPIRATION.toNanos()); + + // the following reservation for a system compaction should not cancel the selection because its + // not reserving files that are selected + var job1 = newJob(SYSTEM, "F00002.rf", "F00003.rf"); + assertTrue(fileMgr.reserveFiles(job1)); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + assertEquals(job1.getSTFiles(), fileMgr.getCompactingFiles()); + + var job2 = newJob(USER, "F00000.rf", "F00001.rf"); + assertTrue(fileMgr.reserveFiles(job2)); + assertEquals(FileSelectionStatus.RESERVED, fileMgr.getSelectionStatus()); + assertEquals(Sets.union(job1.getSTFiles(), job2.getSTFiles()), fileMgr.getCompactingFiles()); + + fileMgr.completed(job2, newFile("C00004.rf")); + assertEquals(job1.getSTFiles(), fileMgr.getCompactingFiles()); + assertEquals(FileSelectionStatus.NOT_ACTIVE, fileMgr.getSelectionStatus()); + + fileMgr.completed(job1, newFile("C00005.rf")); + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + } + + @Test + public void testSelectionWaitsForCompaction() { + TestFileManager fileMgr = new TestFileManager(); + + var job1 = newJob(SYSTEM, "F00000.rf", "F00001.rf"); + assertTrue(fileMgr.reserveFiles(job1)); + + assertTrue(fileMgr.initiateSelection(USER)); + + // selection was initiated, so a new system compaction should not be able to start + assertFalse(fileMgr.reserveFiles(newJob(SYSTEM, "F00002.rf", "F00003.rf"))); + + // selection can not begin because there is still a compaction running + assertFalse(fileMgr.beginSelection()); + + fileMgr.completed(job1, null); + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + + // now that no compactions are running, selection can begin + assertTrue(fileMgr.beginSelection()); + + assertFalse(fileMgr.reserveFiles(newJob(SYSTEM, "F00002.rf", "F00003.rf"))); + + fileMgr.finishSelection(newFiles("F00000.rf", "F00001.rf"), false); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + } + + @Test + public void testUserCompactionCanceled() { + TestFileManager fileMgr = new TestFileManager(); + var tabletFiles = newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf", "F00004.rf"); + + assertTrue(fileMgr.initiateSelection(USER)); + assertTrue(fileMgr.beginSelection()); + fileMgr.finishSelection( + newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf", "F00004.rf"), false); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + + // since no compactions are running, this should transition selection to not active + fileMgr.userCompactionCanceled(); + assertEquals(FileSelectionStatus.NOT_ACTIVE, fileMgr.getSelectionStatus()); + + assertTrue(fileMgr.initiateSelection(USER)); + assertTrue(fileMgr.beginSelection()); + fileMgr.finishSelection( + newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf", "F00004.rf"), false); + assertEquals(FileSelectionStatus.SELECTED, fileMgr.getSelectionStatus()); + + var job1 = newJob(USER, "F00000.rf", "F00001.rf"); + fileMgr.running.add(USER); + assertTrue(fileMgr.reserveFiles(job1)); + assertEquals(job1.getSTFiles(), fileMgr.getCompactingFiles()); + + var job2 = newJob(USER, "F00002.rf", "F00003.rf"); + assertTrue(fileMgr.reserveFiles(job2)); + assertEquals(Sets.union(job1.getSTFiles(), job2.getSTFiles()), fileMgr.getCompactingFiles()); + + // all tablet files are selected, so there are no candidates for system compaction + assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + + // this time when the user compaction is canceled jobs are running, so transition to canceled + fileMgr.userCompactionCanceled(); + assertEquals(FileSelectionStatus.CANCELED, fileMgr.getSelectionStatus()); + + // files that were selected should now be available as candidates after canceling + assertEquals(newFiles("F00004.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false)); + + // when this job completes it should not transition from CANCELED to NOT_ACTIVE because there is + // still another + fileMgr.completed(job1, newFile("C00005.rf")); + assertEquals(FileSelectionStatus.CANCELED, fileMgr.getSelectionStatus()); + assertEquals(job2.getSTFiles(), fileMgr.getCompactingFiles()); + + fileMgr.running.remove(USER); + // when this job completes it should transition from CANCELED to NOT_ACTIVE because its the last + // job. This transition should happen even though not all selected files were compacted (file + // F00004.rf was never compacted) because its in the canceled state. + fileMgr.completed(job2, newFile("C00006.rf")); + assertEquals(FileSelectionStatus.NOT_ACTIVE, fileMgr.getSelectionStatus()); + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + + } + + @Test + public void testChop() { + TestFileManager fileMgr = new TestFileManager(); + + // simulate a compaction because files that were created by compaction are remembered as not + // needing a chop + var job1 = newJob(SYSTEM, "F00000.rf", "F00001.rf"); + assertTrue(fileMgr.reserveFiles(job1)); + fileMgr.completed(job1, newFile("C00005.rf")); + + var tabletFiles = newFiles("C00005.rf", "F00002.rf", "F00003.rf", "F00004.rf"); + + ChopSelector chopSel = fileMgr.initiateChop(tabletFiles); + assertEquals(ChopSelectionStatus.SELECTING, fileMgr.getChopStatus()); + + assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CHOP, false)); + + // this should not include C00005.rf because it was created by a compaction observed by the file + // manager + assertEquals(newFiles("F00002.rf", "F00003.rf", "F00004.rf"), chopSel.getFilesToExamine()); + + chopSel.selectChopFiles(newFiles("F00002.rf", "F00004.rf")); + assertEquals(ChopSelectionStatus.SELECTED, fileMgr.getChopStatus()); + + assertEquals(newFiles("F00002.rf", "F00004.rf"), + fileMgr.getCandidates(tabletFiles, CHOP, false)); + + // simulate compacting one of the files that needs to be chopped, but this should not finish the + // chop because more files need to be chopped + var job2 = newJob(CHOP, "F00002.rf"); + assertTrue(fileMgr.reserveFiles(job2)); + fileMgr.completed(job2, newFile("C00006.rf")); + tabletFiles = newFiles("C00004.rf", "C00006.rf", "F00003.rf", "F00004.rf"); + assertFalse(fileMgr.finishChop(tabletFiles)); + assertEquals(ChopSelectionStatus.SELECTED, fileMgr.getChopStatus()); + + assertEquals(newFiles("F00004.rf"), fileMgr.getCandidates(tabletFiles, CHOP, false)); + + // simulate compacting the last file to chop.. should cause the chop finish + var job3 = newJob(CHOP, "F00004.rf"); + assertTrue(fileMgr.reserveFiles(job3)); + fileMgr.completed(job3, newFile("C00007.rf")); + tabletFiles = newFiles("C00004.rf", "C00006.rf", "F00003.rf", "C00007.rf"); + assertTrue(fileMgr.finishChop(tabletFiles)); + + assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CHOP, false)); + assertEquals(ChopSelectionStatus.NOT_ACTIVE, fileMgr.getChopStatus()); + assertEquals(Set.of(), fileMgr.getCompactingFiles()); + + } + + private void assertNoCandidates(TestFileManager fileMgr, Set<StoredTabletFile> tabletFiles, + CompactionKind... kinds) { + for (CompactionKind kind : kinds) { + assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, kind, false)); + } + + } + + static class TestFileManager extends CompactableImpl.FileManager { + + public static final Duration SELECTION_EXPIRATION = Duration.ofSeconds(120); + private long time = 0; + public Set<CompactionKind> running = new HashSet<>(); + + public TestFileManager() { + super(new KeyExtent(TableId.of("1"), null, null), Set.of(), Optional.empty(), + new Deriver<Duration>() { + + @Override + public Duration derive() { + return SELECTION_EXPIRATION; + } + }); + } + + @Override + protected boolean noneRunning(CompactionKind kind) { + return !running.contains(kind); + } + + boolean reserveFiles(TestCompactionJob job) { + return super.reserveFiles(job, job.getSTFiles()); + } + + void completed(TestCompactionJob job, StoredTabletFile newFile) { + super.completed(job, job.getSTFiles(), newFile); + } + + @Override + protected long getNanoTime() { + return time; + } + + void setNanoTime(long t) { + time = t; + } + + Set<StoredTabletFile> getCompactingFiles() { + return Set.copyOf(allCompactingFiles); + } + + } + + private static StoredTabletFile newFile(String f) { + return new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-0001/" + f); + } + + private static Set<StoredTabletFile> newFiles(String... strings) { + return Arrays.asList(strings).stream().map(s -> newFile(s)).collect(Collectors.toSet()); + } + + private static class TestCompactionJob implements CompactionJob { + + private final CompactionKind kind; + private final Set<StoredTabletFile> jobFiles; + + public TestCompactionJob(CompactionKind kind, Set<StoredTabletFile> jobFiles) { + super(); + this.kind = kind; + this.jobFiles = jobFiles; + } + + @Override + public short getPriority() { + return 0; + } + + @Override + public CompactionExecutorId getExecutor() { + throw new UnsupportedOperationException(); + } + + @Override + public Set<CompactableFile> getFiles() { + return jobFiles.stream().map(stf -> new CompactableFileImpl(stf, new DataFileValue(0, 0))) + .collect(Collectors.toSet()); + } + + public Set<StoredTabletFile> getSTFiles() { + return jobFiles; + } + + @Override + public CompactionKind getKind() { + return kind; + } + + } + + private TestCompactionJob newJob(CompactionKind kind, String... files) { + return new TestCompactionJob(kind, newFiles(files)); + } +}