Skip to content

Commit 7e2e173

Browse files
committed
Improvements to ExternalCompactionsIT
1 parent 9fee599 commit 7e2e173

File tree

3 files changed

+30
-37
lines changed

3 files changed

+30
-37
lines changed

test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java

+6
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,12 @@ public static Stream<ExternalCompactionFinalState> getFinalStatesForTable(Accumu
9999
.filter(state -> state.getExtent().tableId().equals(tid));
100100
}
101101

102+
public static long getFinalStateForTableCount(AccumuloCluster cluster, TableId tid) {
103+
try (var finalStatesForTable = getFinalStatesForTable(cluster, tid)) {
104+
return finalStatesForTable.count();
105+
}
106+
}
107+
102108
public static void compact(final AccumuloClient client, String table1, int modulus,
103109
String expectedQueue, boolean wait)
104110
throws AccumuloSecurityException, TableNotFoundException, AccumuloException {

test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java

+14-25
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE8;
3232
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.compact;
3333
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.createTable;
34+
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.getFinalStateForTableCount;
3435
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.getFinalStatesForTable;
3536
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.row;
3637
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.verify;
@@ -56,7 +57,6 @@
5657
import java.util.concurrent.TimeUnit;
5758
import java.util.concurrent.atomic.AtomicReference;
5859
import java.util.stream.Collectors;
59-
import java.util.stream.Stream;
6060

6161
import org.apache.accumulo.compactor.Compactor;
6262
import org.apache.accumulo.compactor.ExtCEnv.CompactorIterEnv;
@@ -90,20 +90,19 @@
9090
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
9191
import org.apache.accumulo.core.metadata.AccumuloTable;
9292
import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState;
93-
import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState.FinalState;
9493
import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
9594
import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
9695
import org.apache.accumulo.core.metadata.schema.TabletMetadata;
9796
import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
9897
import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
9998
import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
100-
import org.apache.accumulo.core.util.UtilWaitThread;
10199
import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
102100
import org.apache.accumulo.harness.SharedMiniClusterBase;
103101
import org.apache.accumulo.minicluster.ServerType;
104102
import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo;
105103
import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
106104
import org.apache.accumulo.test.functional.SlowIterator;
105+
import org.apache.accumulo.test.util.Wait;
107106
import org.apache.hadoop.conf.Configuration;
108107
import org.apache.hadoop.io.Text;
109108
import org.junit.jupiter.api.AfterEach;
@@ -245,12 +244,7 @@ public void testCompactionAndCompactorDies() throws Exception {
245244
getCluster().getClusterControl().stop(ServerType.COMPACTOR);
246245

247246
// DeadCompactionDetector in the CompactionCoordinator should fail the compaction.
248-
long count = 0;
249-
while (count == 0) {
250-
count = getFinalStatesForTable(getCluster(), tid)
251-
.filter(state -> state.getFinalState().equals(FinalState.FAILED)).count();
252-
UtilWaitThread.sleep(250);
253-
}
247+
Wait.waitFor(() -> getFinalStateForTableCount(getCluster(), tid) > 0);
254248

255249
// We need to cancel the compaction or delete the table here because we initiate a user
256250
// compaction above in the test. Even though the external compaction was cancelled
@@ -411,24 +405,22 @@ public void testExternalCompactionDeadTServer() throws Exception {
411405
// metadata table entries to show up.
412406
LOG.info("Waiting for external compaction to complete.");
413407
TableId tid = getCluster().getServerContext().getTableId(table3);
414-
Stream<ExternalCompactionFinalState> fs = getFinalStatesForTable(getCluster(), tid);
415-
while (fs.findAny().isEmpty()) {
408+
Wait.waitFor(() -> {
416409
LOG.info("Waiting for compaction completed marker to appear");
417-
UtilWaitThread.sleep(250);
418-
fs = getFinalStatesForTable(getCluster(), tid);
419-
}
410+
return getFinalStateForTableCount(getCluster(), tid) > 0;
411+
}, 120_000, 250);
420412

421413
LOG.info("Validating metadata table contents.");
422414
try (TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets()
423415
.forTable(tid).fetch(ColumnType.ECOMP).build()) {
424416
TabletMetadata m = tm.stream().collect(onlyElement());
425417
Map<ExternalCompactionId,ExternalCompactionMetadata> em = m.getExternalCompactions();
426418
assertEquals(1, em.size());
427-
List<ExternalCompactionFinalState> finished = new ArrayList<>();
428-
getFinalStatesForTable(getCluster(), tid).forEach(f -> finished.add(f));
429-
assertEquals(1, finished.size());
430-
assertEquals(em.entrySet().iterator().next().getKey(),
431-
finished.get(0).getExternalCompactionId());
419+
try (var finalStates = getFinalStatesForTable(getCluster(), tid)
420+
.map(ExternalCompactionFinalState::getExternalCompactionId)) {
421+
ExternalCompactionId actual = finalStates.collect(onlyElement());
422+
assertEquals(em.entrySet().iterator().next().getKey(), actual);
423+
}
432424
}
433425

434426
// Force a flush on the metadata table before killing our tserver
@@ -445,13 +437,10 @@ public void testExternalCompactionDeadTServer() throws Exception {
445437
getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
446438

447439
// Wait for the compaction to be committed.
448-
LOG.info("Waiting for compaction completed marker to disappear");
449-
Stream<ExternalCompactionFinalState> fs2 = getFinalStatesForTable(getCluster(), tid);
450-
while (fs2.findAny().isPresent()) {
440+
Wait.waitFor(() -> {
451441
LOG.info("Waiting for compaction completed marker to disappear");
452-
UtilWaitThread.sleep(500);
453-
fs2 = getFinalStatesForTable(getCluster(), tid);
454-
}
442+
return getFinalStateForTableCount(getCluster(), tid) == 0;
443+
}, 120_000, 500);
455444
verify(client, table3, 2);
456445

457446
// We need to cancel the compaction or delete the table here because we initiate a user

test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java

+10-12
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@
2828
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.confirmCompactionCompleted;
2929
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.confirmCompactionRunning;
3030
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.createTable;
31-
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.getFinalStatesForTable;
31+
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.getFinalStateForTableCount;
3232
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.getRunningCompactions;
3333
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.row;
3434
import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.waitForCompactionStartAndReturnEcids;
@@ -60,6 +60,7 @@
6060
import org.apache.accumulo.harness.SharedMiniClusterBase;
6161
import org.apache.accumulo.minicluster.ServerType;
6262
import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
63+
import org.apache.accumulo.test.util.Wait;
6364
import org.apache.hadoop.conf.Configuration;
6465
import org.apache.hadoop.io.Text;
6566
import org.junit.jupiter.api.AfterEach;
@@ -165,8 +166,8 @@ public void testExternalCompactionsSucceedsRunWithTableOffline() throws Exceptio
165166
.startCoordinator(TestCompactionCoordinatorForOfflineTable.class);
166167

167168
TableId tid = getCluster().getServerContext().getTableId(table1);
168-
// Confirm that no final state is in the metadata table
169-
assertEquals(0, getFinalStatesForTable(getCluster(), tid).count());
169+
assertEquals(0, getFinalStateForTableCount(getCluster(), tid),
170+
"Expected no final state is in the metadata table");
170171

171172
// Offline the table when the compaction starts
172173
final AtomicBoolean succeededInTakingOffline = new AtomicBoolean(false);
@@ -209,20 +210,17 @@ public void testExternalCompactionsSucceedsRunWithTableOffline() throws Exceptio
209210
confirmCompactionCompleted(getCluster().getServerContext(), ecids,
210211
TCompactionState.SUCCEEDED);
211212

212-
// Confirm that final state is in the metadata table
213-
assertEquals(1, getFinalStatesForTable(getCluster(), tid).count());
213+
assertEquals(1, getFinalStateForTableCount(getCluster(), tid),
214+
"Expected 1 final state in the metadata table");
214215

215216
// Online the table
216217
client.tableOperations().online(table1);
217218

218219
// wait for compaction to be committed by tserver or test timeout
219-
long finalStateCount = getFinalStatesForTable(getCluster(), tid).count();
220-
while (finalStateCount > 0) {
221-
finalStateCount = getFinalStatesForTable(getCluster(), tid).count();
222-
if (finalStateCount > 0) {
223-
UtilWaitThread.sleep(50);
224-
}
225-
}
220+
Wait.waitFor(() -> {
221+
LOG.info("Waiting for compaction completed marker to disappear");
222+
return getFinalStateForTableCount(getCluster(), tid) == 0;
223+
}, 120_000, 500);
226224

227225
// We need to cancel the compaction or delete the table here because we initiate a user
228226
// compaction above in the test. Even though the external compaction was cancelled

0 commit comments

Comments
 (0)