31
31
import java .io .FileNotFoundException ;
32
32
import java .io .IOException ;
33
33
import java .io .UncheckedIOException ;
34
+ import java .lang .ref .SoftReference ;
34
35
import java .time .Duration ;
35
36
import java .util .ArrayList ;
36
37
import java .util .Collection ;
43
44
import java .util .Objects ;
44
45
import java .util .Optional ;
45
46
import java .util .Set ;
47
+ import java .util .concurrent .CompletableFuture ;
46
48
import java .util .concurrent .ConcurrentHashMap ;
47
49
import java .util .concurrent .CountDownLatch ;
50
+ import java .util .concurrent .ExecutorService ;
48
51
import java .util .concurrent .ScheduledFuture ;
49
52
import java .util .concurrent .ScheduledThreadPoolExecutor ;
53
+ import java .util .concurrent .ThreadPoolExecutor ;
50
54
import java .util .concurrent .TimeUnit ;
51
55
import java .util .concurrent .atomic .AtomicReference ;
52
56
import java .util .function .Consumer ;
57
+ import java .util .function .Supplier ;
53
58
import java .util .stream .Collectors ;
54
59
55
60
import org .apache .accumulo .core .Constants ;
@@ -193,6 +198,9 @@ public class CompactionCoordinator
193
198
194
199
private volatile long coordinatorStartTime ;
195
200
201
+ private final Map <DataLevel ,ThreadPoolExecutor > reservationPools ;
202
+ private final Set <String > activeCompactorReservationRequest = ConcurrentHashMap .newKeySet ();
203
+
196
204
public CompactionCoordinator (ServerContext ctx , SecurityOperation security ,
197
205
AtomicReference <Map <FateInstanceType ,Fate <Manager >>> fateInstances , Manager manager ) {
198
206
this .ctx = ctx ;
@@ -232,6 +240,18 @@ public CompactionCoordinator(ServerContext ctx, SecurityOperation security,
232
240
deadCompactionDetector =
233
241
new DeadCompactionDetector (this .ctx , this , schedExecutor , fateInstances );
234
242
243
+ var rootReservationPool = ThreadPools .getServerThreadPools ().createExecutorService (
244
+ ctx .getConfiguration (), Property .COMPACTION_COORDINATOR_RESERVATION_THREADS_ROOT , true );
245
+
246
+ var metaReservationPool = ThreadPools .getServerThreadPools ().createExecutorService (
247
+ ctx .getConfiguration (), Property .COMPACTION_COORDINATOR_RESERVATION_THREADS_META , true );
248
+
249
+ var userReservationPool = ThreadPools .getServerThreadPools ().createExecutorService (
250
+ ctx .getConfiguration (), Property .COMPACTION_COORDINATOR_RESERVATION_THREADS_USER , true );
251
+
252
+ reservationPools = Map .of (Ample .DataLevel .ROOT , rootReservationPool , Ample .DataLevel .METADATA ,
253
+ metaReservationPool , Ample .DataLevel .USER , userReservationPool );
254
+
235
255
compactorCounts = ctx .getCaches ().createNewBuilder (CacheName .COMPACTOR_COUNTS , false )
236
256
.expireAfterWrite (30 , TimeUnit .SECONDS ).build (this ::countCompactors );
237
257
// At this point the manager does not have its lock so no actions should be taken yet
@@ -250,6 +270,9 @@ public void start() {
250
270
251
271
public void shutdown () {
252
272
shutdown .countDown ();
273
+
274
+ reservationPools .values ().forEach (ExecutorService ::shutdownNow );
275
+
253
276
var localThread = serviceThread ;
254
277
if (localThread != null ) {
255
278
try {
@@ -528,82 +551,142 @@ protected CompactionMetadata createExternalCompactionMetadata(CompactionJob job,
528
551
529
552
}
530
553
531
- protected CompactionMetadata reserveCompaction (CompactionJobQueues .MetaJob metaJob ,
532
- String compactorAddress , ExternalCompactionId externalCompactionId ) {
554
+ private class ReserveCompactionTask implements Supplier <CompactionMetadata > {
555
+
556
+ // Use a soft reference for this in case free memory gets low while this is sitting in the queue
557
+ // waiting to process. This object can contain the tablets list of files and if there are lots
558
+ // of tablet with lots of files then that could start to cause memory problems. This hack could
559
+ // be removed if #5188 were implemented.
560
+ private final SoftReference <CompactionJobQueues .MetaJob > metaJobRef ;
561
+ private final String compactorAddress ;
562
+ private final ExternalCompactionId externalCompactionId ;
563
+
564
+ private ReserveCompactionTask (CompactionJobQueues .MetaJob metaJob , String compactorAddress ,
565
+ ExternalCompactionId externalCompactionId ) {
566
+ Preconditions .checkArgument (metaJob .getJob ().getKind () == CompactionKind .SYSTEM
567
+ || metaJob .getJob ().getKind () == CompactionKind .USER );
568
+ this .metaJobRef = new SoftReference <>(Objects .requireNonNull (metaJob ));
569
+ this .compactorAddress = Objects .requireNonNull (compactorAddress );
570
+ this .externalCompactionId = Objects .requireNonNull (externalCompactionId );
571
+ Preconditions .checkState (activeCompactorReservationRequest .add (compactorAddress ),
572
+ "compactor %s already on has a reservation in flight, cannot process %s" ,
573
+ compactorAddress , externalCompactionId );
574
+ }
533
575
534
- Preconditions .checkArgument (metaJob .getJob ().getKind () == CompactionKind .SYSTEM
535
- || metaJob .getJob ().getKind () == CompactionKind .USER );
576
+ @ Override
577
+ public CompactionMetadata get () {
578
+ try {
579
+ var metaJob = metaJobRef .get ();
580
+ if (metaJob == null ) {
581
+ LOG .warn ("Compaction reservation request for {} {} was garbage collected." ,
582
+ compactorAddress , externalCompactionId );
583
+ return null ;
584
+ }
536
585
537
- var tabletMetadata = metaJob .getTabletMetadata ();
586
+ var tabletMetadata = metaJob .getTabletMetadata ();
538
587
539
- var jobFiles = metaJob .getJob ().getFiles ().stream (). map ( CompactableFileImpl :: toStoredTabletFile )
540
- .collect (Collectors .toSet ());
588
+ var jobFiles = metaJob .getJob ().getFiles ().stream ()
589
+ . map ( CompactableFileImpl :: toStoredTabletFile ) .collect (Collectors .toSet ());
541
590
542
- Retry retry = Retry .builder ().maxRetries (5 ).retryAfter (Duration .ofMillis (100 ))
543
- .incrementBy (Duration .ofMillis (100 )).maxWait (Duration .ofSeconds (10 )).backOffFactor (1.5 )
544
- .logInterval (Duration .ofMinutes (3 )).createRetry ();
591
+ Retry retry = Retry .builder ().maxRetries (5 ).retryAfter (Duration .ofMillis (100 ))
592
+ .incrementBy (Duration .ofMillis (100 )).maxWait (Duration .ofSeconds (10 )).backOffFactor (1.5 )
593
+ .logInterval (Duration .ofMinutes (3 )).createRetry ();
545
594
546
- while (retry .canRetry ()) {
547
- try (var tabletsMutator = ctx .getAmple ().conditionallyMutateTablets ()) {
548
- var extent = metaJob .getTabletMetadata ().getExtent ();
595
+ while (retry .canRetry ()) {
596
+ try (var tabletsMutator = ctx .getAmple ().conditionallyMutateTablets ()) {
597
+ var extent = metaJob .getTabletMetadata ().getExtent ();
549
598
550
- if (!canReserveCompaction (tabletMetadata , metaJob .getJob ().getKind (), jobFiles , ctx ,
551
- manager .getSteadyTime ())) {
552
- return null ;
553
- }
599
+ if (!canReserveCompaction (tabletMetadata , metaJob .getJob ().getKind (), jobFiles , ctx ,
600
+ manager .getSteadyTime ())) {
601
+ return null ;
602
+ }
554
603
555
- var ecm = createExternalCompactionMetadata (metaJob .getJob (), jobFiles , tabletMetadata ,
556
- compactorAddress , externalCompactionId );
557
-
558
- // any data that is read from the tablet to make a decision about if it can compact or not
559
- // must be checked for changes in the conditional mutation.
560
- var tabletMutator = tabletsMutator .mutateTablet (extent ).requireAbsentOperation ()
561
- .requireFiles (jobFiles ).requireNotCompacting (jobFiles );
562
- if (metaJob .getJob ().getKind () == CompactionKind .SYSTEM ) {
563
- // For system compactions the user compaction requested column is examined when deciding
564
- // if a compaction can start so need to check for changes to this column.
565
- tabletMutator .requireSame (tabletMetadata , SELECTED , USER_COMPACTION_REQUESTED );
566
- } else {
567
- tabletMutator .requireSame (tabletMetadata , SELECTED );
568
- }
604
+ var ecm = createExternalCompactionMetadata (metaJob .getJob (), jobFiles , tabletMetadata ,
605
+ compactorAddress , externalCompactionId );
606
+
607
+ // any data that is read from the tablet to make a decision about if it can compact or
608
+ // not
609
+ // must be checked for changes in the conditional mutation.
610
+ var tabletMutator = tabletsMutator .mutateTablet (extent ).requireAbsentOperation ()
611
+ .requireFiles (jobFiles ).requireNotCompacting (jobFiles );
612
+ if (metaJob .getJob ().getKind () == CompactionKind .SYSTEM ) {
613
+ // For system compactions the user compaction requested column is examined when
614
+ // deciding
615
+ // if a compaction can start so need to check for changes to this column.
616
+ tabletMutator .requireSame (tabletMetadata , SELECTED , USER_COMPACTION_REQUESTED );
617
+ } else {
618
+ tabletMutator .requireSame (tabletMetadata , SELECTED );
619
+ }
569
620
570
- if (metaJob .getJob ().getKind () == CompactionKind .SYSTEM ) {
571
- var selectedFiles = tabletMetadata .getSelectedFiles ();
572
- var reserved = getFilesReservedBySelection (tabletMetadata , manager .getSteadyTime (), ctx );
573
-
574
- // If there is a selectedFiles column, and the reserved set is empty this means that
575
- // either no user jobs were completed yet or the selection expiration time has passed
576
- // so the column is eligible to be deleted so a system job can run instead
577
- if (selectedFiles != null && reserved .isEmpty ()
578
- && !Collections .disjoint (jobFiles , selectedFiles .getFiles ())) {
579
- LOG .debug ("Deleting user compaction selected files for {} {}" , extent ,
580
- externalCompactionId );
581
- tabletMutator .deleteSelectedFiles ();
582
- }
583
- }
621
+ if (metaJob .getJob ().getKind () == CompactionKind .SYSTEM ) {
622
+ var selectedFiles = tabletMetadata .getSelectedFiles ();
623
+ var reserved =
624
+ getFilesReservedBySelection (tabletMetadata , manager .getSteadyTime (), ctx );
625
+
626
+ // If there is a selectedFiles column, and the reserved set is empty this means that
627
+ // either no user jobs were completed yet or the selection expiration time has passed
628
+ // so the column is eligible to be deleted so a system job can run instead
629
+ if (selectedFiles != null && reserved .isEmpty ()
630
+ && !Collections .disjoint (jobFiles , selectedFiles .getFiles ())) {
631
+ LOG .debug ("Deleting user compaction selected files for {} {}" , extent ,
632
+ externalCompactionId );
633
+ tabletMutator .deleteSelectedFiles ();
634
+ }
635
+ }
584
636
585
- tabletMutator .putExternalCompaction (externalCompactionId , ecm );
586
- tabletMutator .submit (tm -> tm .getExternalCompactions ().containsKey (externalCompactionId ));
637
+ tabletMutator .putExternalCompaction (externalCompactionId , ecm );
638
+ tabletMutator
639
+ .submit (tm -> tm .getExternalCompactions ().containsKey (externalCompactionId ));
587
640
588
- var result = tabletsMutator .process ().get (extent );
641
+ var result = tabletsMutator .process ().get (extent );
589
642
590
- if (result .getStatus () == Ample .ConditionalResult .Status .ACCEPTED ) {
591
- return ecm ;
592
- } else {
593
- tabletMetadata = result .readMetadata ();
643
+ if (result .getStatus () == Ample .ConditionalResult .Status .ACCEPTED ) {
644
+ return ecm ;
645
+ } else {
646
+ tabletMetadata = result .readMetadata ();
647
+ }
648
+ }
649
+
650
+ retry .useRetry ();
651
+ try {
652
+ retry .waitForNextAttempt (LOG ,
653
+ "Reserved compaction for " + metaJob .getTabletMetadata ().getExtent ());
654
+ } catch (InterruptedException e ) {
655
+ throw new RuntimeException (e );
656
+ }
594
657
}
595
- }
596
658
597
- retry .useRetry ();
598
- try {
599
- retry .waitForNextAttempt (LOG ,
600
- "Reserved compaction for " + metaJob .getTabletMetadata ().getExtent ());
601
- } catch (InterruptedException e ) {
602
- throw new RuntimeException (e );
659
+ return null ;
660
+ } finally {
661
+ Preconditions .checkState (activeCompactorReservationRequest .remove (compactorAddress ),
662
+ "compactorAddress:%s" , compactorAddress );
603
663
}
604
664
}
665
+ }
666
+
667
+ protected CompactionMetadata reserveCompaction (CompactionJobQueues .MetaJob metaJob ,
668
+ String compactorAddress , ExternalCompactionId externalCompactionId ) {
669
+
670
+ if (activeCompactorReservationRequest .contains (compactorAddress )) {
671
+ // In this case the compactor has a previously submitted reservation request that is still
672
+ // processing. Do not want to let it queue up another reservation request. One possible cause
673
+ // of this is that compactor timed out waiting for its last request to process and is now
674
+ // making another request. The previously submitted request can not be used because the
675
+ // compactor generates a new uuid for each request it makes. So the best thing to do is to
676
+ // return null and wait for this situation to resolve. This will likely happen when some part
677
+ // of the distributed system is not working well, so at this point want to avoid making
678
+ // problems worse instead of trying to reserve a job.
679
+ LOG .warn (
680
+ "Ignoring request from {} to reserve compaction job because it has a reservation request in progress." ,
681
+ compactorAddress );
682
+ return null ;
683
+ }
605
684
606
- return null ;
685
+ var dataLevel = DataLevel .of (metaJob .getTabletMetadata ().getTableId ());
686
+ var future = CompletableFuture .supplyAsync (
687
+ new ReserveCompactionTask (metaJob , compactorAddress , externalCompactionId ),
688
+ reservationPools .get (dataLevel ));
689
+ return future .join ();
607
690
}
608
691
609
692
protected TExternalCompactionJob createThriftJob (String externalCompactionId ,
@@ -1123,6 +1206,14 @@ public void cleanUpInternalState() {
1123
1206
// 5. Log compactors with no groups
1124
1207
// 6. Log groups with compactors and queued jos that have not checked in
1125
1208
1209
+ var config = ctx .getConfiguration ();
1210
+ ThreadPools .resizePool (reservationPools .get (DataLevel .ROOT ), config ,
1211
+ Property .COMPACTION_COORDINATOR_RESERVATION_THREADS_ROOT );
1212
+ ThreadPools .resizePool (reservationPools .get (DataLevel .METADATA ), config ,
1213
+ Property .COMPACTION_COORDINATOR_RESERVATION_THREADS_META );
1214
+ ThreadPools .resizePool (reservationPools .get (DataLevel .USER ), config ,
1215
+ Property .COMPACTION_COORDINATOR_RESERVATION_THREADS_USER );
1216
+
1126
1217
// grab a snapshot of the ids in the set before reading the metadata table. This is done to
1127
1218
// avoid removing things that are added while reading the metadata.
1128
1219
final Set <ExternalCompactionId > idsSnapshot = Set .copyOf (RUNNING_CACHE .keySet ());
0 commit comments