keith-turner commented on PR #4715:
URL: https://github.com/apache/accumulo/pull/4715#issuecomment-2221772887
I was experimenting with this locally and its really neat to see it in
action.
Made the following changes to dae30cf746c135b061e4de87f21542e98c001dda. The
logging changes were to get a bit more information about timing. The test
changes were made to cause a surge of bulk import files after the system was
sitting idle for a bit.
```diff
diff --git
a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java
b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java
index ef07b78d8a..838fab35a7 100644
---
a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java
+++
b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java
@@ -462,7 +462,7 @@ public class CompactionCoordinator
TIME_COMPACTOR_LAST_CHECKED.put(groupId, System.currentTimeMillis());
return jobQueues.getAsync(groupId).thenApply(metaJob -> {
- LOG.trace("Next metaJob is ready {}", metaJob.getJob());
+ LOG.debug("Next metaJob is ready {}", metaJob.getJob());
Optional<CompactionConfig> compactionConfig =
getCompactionConfig(metaJob);
// this method may reread the metadata, do not use the metadata in
metaJob for anything after
@@ -471,6 +471,7 @@ public class CompactionCoordinator
var kind = metaJob.getJob().getKind();
+ LOG.debug("Reserving compaction job {}", externalCompactionId);
// Only reserve user compactions when the config is present. When
compactions are canceled the
// config is deleted.
var cid = ExternalCompactionId.from(externalCompactionId);
diff --git
a/test/src/main/java/org/apache/accumulo/test/ComprehensiveBaseIT.java
b/test/src/main/java/org/apache/accumulo/test/ComprehensiveBaseIT.java
index 1dc1c2c0f9..a3d5a61856 100644
--- a/test/src/main/java/org/apache/accumulo/test/ComprehensiveBaseIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ComprehensiveBaseIT.java
@@ -96,6 +96,7 @@ import org.apache.accumulo.core.security.ColumnVisibility;
import org.apache.accumulo.core.security.NamespacePermission;
import org.apache.accumulo.core.security.SystemPermission;
import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.UtilWaitThread;
import org.apache.accumulo.harness.SharedMiniClusterBase;
import org.apache.accumulo.test.util.Wait;
import org.apache.hadoop.fs.FileUtil;
@@ -126,12 +127,24 @@ public abstract class ComprehensiveBaseIT extends
SharedMiniClusterBase {
try (AccumuloClient client =
Accumulo.newClient().from(getClientProps()).build()) {
client.tableOperations().create(table);
- bulkImport(client, table, List.of(generateKeys(0, 100),
generateKeys(100, 200)));
+ // sleep for a bit to let compactors idle
+ UtilWaitThread.sleep(60000);
+
+ // add 4 files to a single tablet, should cause tablet to need
compaction
+ bulkImport(client, table, List.of(generateKeys(0, 50),
generateKeys(50, 100),
+ generateKeys(100, 150), generateKeys(150, 200)));
verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 200));
+ UtilWaitThread.sleep(60000);
+
+ // add 4 more files to tablet
bulkImport(client, table,
- List.of(generateKeys(200, 300), generateKeys(300, 400),
generateKeys(400, 500)));
+ List.of(generateKeys(200, 300), generateKeys(300, 400),
+ generateKeys(400, 450), generateKeys(450, 500)));
+
+ UtilWaitThread.sleep(60000);
+
verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 500));
}
```
Ran the modified test ad then looked in the manager log and saw the
following.
```
2024-07-10T19:55:04,450 120 [manager.EventCoordinator] DEBUG: Bulk load
completed on tablet 1<<
2024-07-10T19:55:04,455 63 [compaction.CompactionJobGenerator] DEBUG:
Planning for 1<< [SYSTEM, USER] 1445803685
2024-07-10T19:55:04,456 63 [manager.TabletGroupWatcher] DEBUG: 1<< may need
compacting adding 1 jobs
2024-07-10T19:55:04,456 63 [coordinator.CompactionCoordinator] DEBUG:
Reserving compaction job ECID-db4c21aa-b615-4109-b3dd-3fc544f8c391
2024-07-10T19:55:04,459 63 [tablet.files] DEBUG: Compacting 1<<
id:ECID-db4c21aa-b615-4109-b3dd-3fc544f8c391 group:default compactor:slim:9133
priority:-32759 size:35 KB kind:SYSTEM files:[I000005p.rf, I000005o.rf,
I000005n.rf, I000005m.rf]
2024-07-10T19:55:04,459 63 [coordinator.CompactionCoordinator] INFO : Found
job ECID-db4c21aa-b615-4109-b3dd-3fc544f8c391
2024-07-10T19:55:04,460 63 [rpc.ThriftProtobufUtil] DEBUG:
PExternalCompactionJob: externalCompactionId:
"ECID-db4c21aa-b615-4109-b3dd-3fc544f8c391"
2024-07-10T19:55:04,508 87 [fate.Fate] INFO : Seeding
FATE:USER:914dd323-a684-38af-aaaf-7cb202a9e672 Commit compaction
ECID-db4c21aa-b615-4109-b3dd-3fc544f8c391
```
Summarizing the above log messages.
* A time +0ms the bulk import completed adding files to the tablet and
signaled the TGW (tablet group watcher).
* At time +6ms the TGW adds a job to the compaction queue
* At time +6ms an async rpc job request starts processing the job
* At time +9ms the async rpc job request finishes reserving the job
* At time +10ms the last observed lob message for the async rpc job is
seen, at this point its on its about to write the return value of the rpc
* At time +58ms the compactor has completed the compaction and has made
another RPC to the coordinator to start the fate operation that will commit the
compaction
So this is really neat, the async request starts working immediately after
the job is queued and within 3ms to 4ms has reserved the job and returned it to
the compactor.
Saw something else that was interesting besides timing in the logs. The
numbers after the times in the logs are thread ids. So thread id `63` is the
TGW thread id. Can see that currently the TGW thread ends up processing the
async RPC job reservation and response. Once that is changed to use another
thread we should see a different thread id in the logs for those steps.
--
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]