keith-turner commented on code in PR #4254:
URL: https://github.com/apache/accumulo/pull/4254#discussion_r1492878541
##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java:
##########
@@ -257,8 +257,13 @@ public int updateAndCheckTablets(Manager manager, FateId
fateId)
otherSelected++;
}
} else {
- // ELASTICITY_TODO if there are compactions preventing selection of
files, then add
+ // If there are compactions preventing selection of files, then add
// selecting marker that prevents new compactions from starting
+ var mutator =
tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation()
+ .requireSame(tablet, ECOMP).putUserCompactionRequested(fateId);
+ mutator.submit(tm ->
tm.getUserCompactionsRequested().contains(fateId));
+
+ // Add marker here
Review Comment:
This comment is not related to this code, just this class. At the end of
this class there is code to cleanup on failure that could cleanup this new
column.
##########
test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java:
##########
@@ -909,6 +909,82 @@ public void testDeleteCompactionService() throws Exception
{
}
}
+ @Test
+ public void testUserCompactionRequested() throws Exception {
+
+ String tableName = getUniqueNames(1)[0];
+ try (final AccumuloClient client =
Accumulo.newClient().from(getClientProps()).build()) {
+
+ // configure tablet compaction iterator that slows compaction down so we
can test
+ // that the USER_COMPACTION_REQUESTED column is set when a user
compaction is requested
+ // when a system compaction is running and blocking
+
+ var ntc = new NewTableConfiguration();
+ IteratorSetting iterSetting = new IteratorSetting(50,
SlowIterator.class);
+ SlowIterator.setSleepTime(iterSetting, 1);
+ ntc.attachIterator(iterSetting, EnumSet.of(IteratorScope.majc));
+ ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "20"));
+ client.tableOperations().create(tableName, ntc);
+
+ // Insert MAX_DATA rows
+ try (BatchWriter bw = client.createBatchWriter(tableName)) {
+ for (int i = 0; i < MAX_DATA; i++) {
+ Mutation m = new Mutation(String.format("r:%04d", i));
+ m.put("", "", "" + i);
+ bw.addMutation(m);
+
+ if (i % 75 == 0) {
+ // create many files as this will cause a system compaction
+ bw.flush();
+ client.tableOperations().flush(tableName, null, null, true);
+ }
+ }
+ }
+ client.tableOperations().flush(tableName, null, null, true);
+
+ // set the compaction ratio 1 to trigger a system compaction
+ client.tableOperations().setProperty(tableName,
Property.TABLE_MAJC_RATIO.getKey(), "1");
+
+ var tableId =
TableId.of(client.tableOperations().tableIdMap().get(tableName));
+ var extent = new KeyExtent(tableId, null, null);
+
+ // Wait for the system compaction to start
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var externalCompactions = tabletMeta.getExternalCompactions().size();
+ log.debug("Current external compactions {}", externalCompactions);
+ return externalCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
+ // Trigger a user compaction which should be blocked by the system
compaction
+ // and should result in the userRequestedCompactions column being set so
no more
+ // system compactions run
+ client.tableOperations().compact(tableName, new
CompactionConfig().setWait(false));
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var userRequestedCompactions =
tabletMeta.getUserCompactionsRequested().size();
+ log.debug("Current user requested compactions {}",
userRequestedCompactions);
+ return userRequestedCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
Review Comment:
Could add more files here, that would cause a system compaction to want to
start.
##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CleanUp.java:
##########
@@ -74,16 +75,21 @@ public long isReady(FateId fateId, Manager manager) throws
Exception {
try (
var tablets =
ample.readTablets().forTable(tableId).overlapping(startRow, endRow)
- .fetch(PREV_ROW, COMPACTED).checkConsistency().build();
+ .fetch(PREV_ROW, COMPACTED,
USER_COMPACTION_REQUESTED).checkConsistency().build();
var tabletsMutator = ample.conditionallyMutateTablets(resultConsumer))
{
t1 = System.nanoTime();
for (TabletMetadata tablet : tablets) {
total++;
if (tablet.getCompacted().contains(fateId)) {
Review Comment:
This is completely unrelated to this PR, but another comment made me realize
that certain compaction metadata is completely unexpected at this point.
```suggestion
Preconditions.checkState(ensure selected column does not exists)
if (tablet.getCompacted().contains(fateId)) {
```
##########
test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java:
##########
@@ -909,6 +909,82 @@ public void testDeleteCompactionService() throws Exception
{
}
}
+ @Test
+ public void testUserCompactionRequested() throws Exception {
+
+ String tableName = getUniqueNames(1)[0];
+ try (final AccumuloClient client =
Accumulo.newClient().from(getClientProps()).build()) {
+
+ // configure tablet compaction iterator that slows compaction down so we
can test
+ // that the USER_COMPACTION_REQUESTED column is set when a user
compaction is requested
+ // when a system compaction is running and blocking
+
+ var ntc = new NewTableConfiguration();
+ IteratorSetting iterSetting = new IteratorSetting(50,
SlowIterator.class);
+ SlowIterator.setSleepTime(iterSetting, 1);
+ ntc.attachIterator(iterSetting, EnumSet.of(IteratorScope.majc));
+ ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "20"));
+ client.tableOperations().create(tableName, ntc);
+
+ // Insert MAX_DATA rows
+ try (BatchWriter bw = client.createBatchWriter(tableName)) {
+ for (int i = 0; i < MAX_DATA; i++) {
+ Mutation m = new Mutation(String.format("r:%04d", i));
+ m.put("", "", "" + i);
+ bw.addMutation(m);
+
+ if (i % 75 == 0) {
+ // create many files as this will cause a system compaction
+ bw.flush();
+ client.tableOperations().flush(tableName, null, null, true);
+ }
+ }
+ }
+ client.tableOperations().flush(tableName, null, null, true);
+
+ // set the compaction ratio 1 to trigger a system compaction
+ client.tableOperations().setProperty(tableName,
Property.TABLE_MAJC_RATIO.getKey(), "1");
+
+ var tableId =
TableId.of(client.tableOperations().tableIdMap().get(tableName));
+ var extent = new KeyExtent(tableId, null, null);
+
+ // Wait for the system compaction to start
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var externalCompactions = tabletMeta.getExternalCompactions().size();
+ log.debug("Current external compactions {}", externalCompactions);
+ return externalCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
Review Comment:
could poll more frequently
```suggestion
}, Wait.MAX_WAIT_MILLIS, 100);
```
##########
test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java:
##########
@@ -909,6 +909,82 @@ public void testDeleteCompactionService() throws Exception
{
}
}
+ @Test
+ public void testUserCompactionRequested() throws Exception {
+
+ String tableName = getUniqueNames(1)[0];
+ try (final AccumuloClient client =
Accumulo.newClient().from(getClientProps()).build()) {
+
+ // configure tablet compaction iterator that slows compaction down so we
can test
+ // that the USER_COMPACTION_REQUESTED column is set when a user
compaction is requested
+ // when a system compaction is running and blocking
+
+ var ntc = new NewTableConfiguration();
+ IteratorSetting iterSetting = new IteratorSetting(50,
SlowIterator.class);
+ SlowIterator.setSleepTime(iterSetting, 1);
+ ntc.attachIterator(iterSetting, EnumSet.of(IteratorScope.majc));
+ ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "20"));
+ client.tableOperations().create(tableName, ntc);
+
+ // Insert MAX_DATA rows
+ try (BatchWriter bw = client.createBatchWriter(tableName)) {
+ for (int i = 0; i < MAX_DATA; i++) {
+ Mutation m = new Mutation(String.format("r:%04d", i));
+ m.put("", "", "" + i);
+ bw.addMutation(m);
+
+ if (i % 75 == 0) {
+ // create many files as this will cause a system compaction
+ bw.flush();
+ client.tableOperations().flush(tableName, null, null, true);
+ }
+ }
+ }
+ client.tableOperations().flush(tableName, null, null, true);
+
+ // set the compaction ratio 1 to trigger a system compaction
+ client.tableOperations().setProperty(tableName,
Property.TABLE_MAJC_RATIO.getKey(), "1");
+
+ var tableId =
TableId.of(client.tableOperations().tableIdMap().get(tableName));
+ var extent = new KeyExtent(tableId, null, null);
+
+ // Wait for the system compaction to start
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var externalCompactions = tabletMeta.getExternalCompactions().size();
+ log.debug("Current external compactions {}", externalCompactions);
+ return externalCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
+ // Trigger a user compaction which should be blocked by the system
compaction
+ // and should result in the userRequestedCompactions column being set so
no more
+ // system compactions run
+ client.tableOperations().compact(tableName, new
CompactionConfig().setWait(false));
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var userRequestedCompactions =
tabletMeta.getUserCompactionsRequested().size();
+ log.debug("Current user requested compactions {}",
userRequestedCompactions);
+ return userRequestedCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
+ // Wait and verify that the system compaction finishes
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
Review Comment:
If files were added before this waitFor then could do some validation in the
waitFor
```java
if(!tabletMeta.getUserCompactionsRequested().isEmpty()){
// if this column exists in the metadata table then it should prevent
system compactions from started so do no expect to see any in the metadata table
assertTrue(tm.getExternalCompactions().values().stream().noneMatch(cm->cm.getKind()
== CompactionKind.SYSTEM));
}
```
##########
test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java:
##########
@@ -909,6 +909,82 @@ public void testDeleteCompactionService() throws Exception
{
}
}
+ @Test
+ public void testUserCompactionRequested() throws Exception {
+
+ String tableName = getUniqueNames(1)[0];
+ try (final AccumuloClient client =
Accumulo.newClient().from(getClientProps()).build()) {
+
+ // configure tablet compaction iterator that slows compaction down so we
can test
+ // that the USER_COMPACTION_REQUESTED column is set when a user
compaction is requested
+ // when a system compaction is running and blocking
+
+ var ntc = new NewTableConfiguration();
+ IteratorSetting iterSetting = new IteratorSetting(50,
SlowIterator.class);
+ SlowIterator.setSleepTime(iterSetting, 1);
+ ntc.attachIterator(iterSetting, EnumSet.of(IteratorScope.majc));
+ ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "20"));
+ client.tableOperations().create(tableName, ntc);
+
+ // Insert MAX_DATA rows
+ try (BatchWriter bw = client.createBatchWriter(tableName)) {
+ for (int i = 0; i < MAX_DATA; i++) {
+ Mutation m = new Mutation(String.format("r:%04d", i));
+ m.put("", "", "" + i);
+ bw.addMutation(m);
+
+ if (i % 75 == 0) {
+ // create many files as this will cause a system compaction
+ bw.flush();
+ client.tableOperations().flush(tableName, null, null, true);
+ }
+ }
+ }
+ client.tableOperations().flush(tableName, null, null, true);
+
+ // set the compaction ratio 1 to trigger a system compaction
+ client.tableOperations().setProperty(tableName,
Property.TABLE_MAJC_RATIO.getKey(), "1");
+
+ var tableId =
TableId.of(client.tableOperations().tableIdMap().get(tableName));
+ var extent = new KeyExtent(tableId, null, null);
+
+ // Wait for the system compaction to start
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var externalCompactions = tabletMeta.getExternalCompactions().size();
+ log.debug("Current external compactions {}", externalCompactions);
+ return externalCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
+ // Trigger a user compaction which should be blocked by the system
compaction
+ // and should result in the userRequestedCompactions column being set so
no more
+ // system compactions run
+ client.tableOperations().compact(tableName, new
CompactionConfig().setWait(false));
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var userRequestedCompactions =
tabletMeta.getUserCompactionsRequested().size();
+ log.debug("Current user requested compactions {}",
userRequestedCompactions);
+ return userRequestedCompactions == 1;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
+ // Wait and verify that the system compaction finishes
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var externalCompactions = tabletMeta.getExternalCompactions().size();
+ log.debug("Current external compactions {}", externalCompactions);
+ return externalCompactions == 0;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+
+ // After the user compaction completes the compactions requested column
should be cleared
+ Wait.waitFor(() -> {
+ var tabletMeta = ((ClientContext)
client).getAmple().readTablet(extent);
+ var userRequestedCompactions =
tabletMeta.getUserCompactionsRequested().size();
+ log.debug("Current user requested compactions {}",
userRequestedCompactions);
+ return userRequestedCompactions == 0;
+ }, Wait.MAX_WAIT_MILLIS, 500);
+ }
+ }
Review Comment:
At the conclusion of the test could call
ExternalCompactionTestUtils.assertNoCompactionMetadata(). Could also update
this method to look for the new column added.
##########
server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CleanUp.java:
##########
@@ -74,16 +75,21 @@ public long isReady(FateId fateId, Manager manager) throws
Exception {
try (
var tablets =
ample.readTablets().forTable(tableId).overlapping(startRow, endRow)
- .fetch(PREV_ROW, COMPACTED).checkConsistency().build();
+ .fetch(PREV_ROW, COMPACTED,
USER_COMPACTION_REQUESTED).checkConsistency().build();
var tabletsMutator = ample.conditionallyMutateTablets(resultConsumer))
{
t1 = System.nanoTime();
for (TabletMetadata tablet : tablets) {
total++;
if (tablet.getCompacted().contains(fateId)) {
-
tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation()
- .requireSame(tablet, COMPACTED).deleteCompacted(fateId)
- .submit(tabletMetadata ->
!tabletMetadata.getCompacted().contains(fateId));
+ var mutator =
tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation()
+ .requireSame(tablet, COMPACTED).deleteCompacted(fateId);
+ if (tablet.getUserCompactionsRequested().contains(fateId)) {
+ mutator.deleteUserCompactionRequested(fateId);
+ }
+
+ mutator.submit(tabletMetadata ->
!tabletMetadata.getCompacted().contains(fateId)
+ &&
!tabletMetadata.getUserCompactionsRequested().contains(fateId));
submitted++;
}
Review Comment:
Was wondering about the case where user compaction requested column exists
but the compacted column does not. Does not seem like this should happen, was
wondering if we should throw an error.
```suggestion
} else if(tablet.getUserCompactionsRequested().contains(fateId)){
throw new IllegalStateException("saw user comp req but did not
see compacted for <tablet> <fateId>");
}
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]