4343import java .util .concurrent .atomic .AtomicLong ;
4444import java .util .concurrent .atomic .AtomicReference ;
4545
46+ import static org .elasticsearch .index .engine .ThreadPoolMergeScheduler .Schedule .ABORT ;
4647import static org .elasticsearch .index .engine .ThreadPoolMergeScheduler .Schedule .BACKLOG ;
4748import static org .elasticsearch .index .engine .ThreadPoolMergeScheduler .Schedule .RUN ;
49+ import static org .hamcrest .Matchers .greaterThanOrEqualTo ;
4850import static org .hamcrest .Matchers .is ;
51+ import static org .mockito .Mockito .atLeastOnce ;
4952import static org .mockito .Mockito .doAnswer ;
5053import static org .mockito .Mockito .mock ;
5154import static org .mockito .Mockito .times ;
@@ -210,6 +213,7 @@ public void testAvailableDiskSpaceMonitorSingleUpdateWithDefaultSettings() throw
210213 public void testAvailableDiskSpaceMonitorSettingsUpdate () throws Exception {
211214 Settings settings = Settings .builder ()
212215 .put (this .settings )
216+ // the default of "5s" slows down testing
213217 .put (ThreadPoolMergeExecutorService .INDICES_MERGE_DISK_CHECK_INTERVAL_SETTING .getKey (), "50ms" )
214218 .build ();
215219 ClusterSettings clusterSettings = ClusterSettings .createBuiltInClusterSettings (settings );
@@ -327,6 +331,128 @@ public void testAvailableDiskSpaceMonitorSettingsUpdate() throws Exception {
327331 }
328332 }
329333
334+ public void testBackloggedMergeTasksDoNotHoldUpBudget () throws Exception {
335+ aFileStore .totalSpace = randomLongBetween (100L , 10_000_000L );
336+ bFileStore .totalSpace = randomLongBetween (100L , 10_000_000L );
337+ aFileStore .usableSpace = randomLongBetween (10L , aFileStore .totalSpace );
338+ bFileStore .usableSpace = randomLongBetween (10L , bFileStore .totalSpace );
339+ boolean aHasMoreSpace = aFileStore .usableSpace > bFileStore .usableSpace ;
340+ Settings settings = Settings .builder ()
341+ .put (this .settings )
342+ // the default of "5s" slows down testing
343+ .put (ThreadPoolMergeExecutorService .INDICES_MERGE_DISK_CHECK_INTERVAL_SETTING .getKey (), "30ms" )
344+ .build ();
345+ try (TestThreadPool testThreadPool = new TestThreadPool ("test" , settings )) {
346+ try (NodeEnvironment nodeEnv = new NodeEnvironment (settings , TestEnvironment .newEnvironment (settings ))) {
347+ try (
348+ ThreadPoolMergeExecutorService threadPoolMergeExecutorService = ThreadPoolMergeExecutorService
349+ .maybeCreateThreadPoolMergeExecutorService (
350+ testThreadPool ,
351+ ClusterSettings .createBuiltInClusterSettings (settings ),
352+ nodeEnv
353+ )
354+ ) {
355+ assert threadPoolMergeExecutorService != null ;
356+ assertThat (threadPoolMergeExecutorService .getMaxConcurrentMerges (), greaterThanOrEqualTo (1 ));
357+ // assumes the 5% default value for the remaining space watermark
358+ final AtomicLong expectedAvailableBudget = new AtomicLong (
359+ aHasMoreSpace
360+ ? aFileStore .usableSpace - aFileStore .totalSpace / 20
361+ : bFileStore .usableSpace - bFileStore .totalSpace / 20
362+ );
363+ assertBusy (
364+ () -> assertThat (
365+ threadPoolMergeExecutorService .getDiskSpaceAvailableForNewMergeTasks (),
366+ is (expectedAvailableBudget .get ())
367+ )
368+ );
369+ long backloggedMergeTaskDiskSpaceBudget = randomLongBetween (1L , expectedAvailableBudget .get ());
370+ CountDownLatch blockMergeTaskLatch = new CountDownLatch (1 );
371+ // take care that there's still at least one thread available to run merges
372+ int maxBlockingTasksToSubmit = threadPoolMergeExecutorService .getMaxConcurrentMerges () - 1 ;
373+ // first maybe submit some running or aborting merge tasks that hold up some budget while running or aborting
374+ while (expectedAvailableBudget .get () - backloggedMergeTaskDiskSpaceBudget > 0L
375+ && maxBlockingTasksToSubmit -- > 0
376+ && randomBoolean ()) {
377+ ThreadPoolMergeScheduler .MergeTask mergeTask = mock (ThreadPoolMergeScheduler .MergeTask .class );
378+ long taskBudget = randomLongBetween (1L , expectedAvailableBudget .get () - backloggedMergeTaskDiskSpaceBudget );
379+ when (mergeTask .estimatedRemainingMergeSize ()).thenReturn (taskBudget );
380+ when (mergeTask .schedule ()).thenReturn (randomFrom (RUN , ABORT ));
381+ expectedAvailableBudget .set (expectedAvailableBudget .get () - taskBudget );
382+ // this task will hold up budget because it blocks when it runs (to simulate it running for a long time)
383+ doAnswer (mock -> {
384+ // wait to be signalled before completing (this holds up budget)
385+ blockMergeTaskLatch .await ();
386+ return null ;
387+ }).when (mergeTask ).run ();
388+ doAnswer (mock -> {
389+ // wait to be signalled before completing (this holds up budget)
390+ blockMergeTaskLatch .await ();
391+ return null ;
392+ }).when (mergeTask ).abort ();
393+ threadPoolMergeExecutorService .submitMergeTask (mergeTask );
394+ }
395+ assertBusy (
396+ () -> assertThat (
397+ threadPoolMergeExecutorService .getDiskSpaceAvailableForNewMergeTasks (),
398+ is (expectedAvailableBudget .get ())
399+ )
400+ );
401+ // submit some backlogging merge tasks which should NOT hold up any budget
402+ List <ThreadPoolMergeScheduler .MergeTask > backloggingMergeTasks = new ArrayList <>();
403+ int backloggingTaskCount = randomIntBetween (1 , 5 );
404+ while (backloggingTaskCount -- > 0 ) {
405+ ThreadPoolMergeScheduler .MergeTask mergeTask = mock (ThreadPoolMergeScheduler .MergeTask .class );
406+ long taskBudget = randomLongBetween (1L , backloggedMergeTaskDiskSpaceBudget );
407+ when (mergeTask .estimatedRemainingMergeSize ()).thenReturn (taskBudget );
408+ doAnswer (mock -> {
409+ // maybe re-enqueue backlogged merge task
410+ if (randomBoolean ()) {
411+ testThreadPool .executor (ThreadPool .Names .GENERIC ).execute (() -> {
412+ threadPoolMergeExecutorService .reEnqueueBackloggedMergeTask (mergeTask );
413+ });
414+ }
415+ // always backlog these merge tasks
416+ return BACKLOG ;
417+ }).when (mergeTask ).schedule ();
418+ threadPoolMergeExecutorService .submitMergeTask (mergeTask );
419+ backloggingMergeTasks .add (mergeTask );
420+ }
421+ // assert all backlogging merge tasks have been scheduled and possibly re-enqueued, none run and none aborted,
422+ // AND the available budget is intact
423+ assertBusy (
424+ () -> {
425+ for (ThreadPoolMergeScheduler .MergeTask mergeTask : backloggingMergeTasks ) {
426+ verify (mergeTask , atLeastOnce ()).schedule ();
427+ }
428+ for (ThreadPoolMergeScheduler .MergeTask mergeTask : backloggingMergeTasks ) {
429+ verify (mergeTask , times (0 )).run ();
430+ verify (mergeTask , times (0 )).abort ();
431+ }
432+ // budget hasn't changed!
433+ assertThat (
434+ threadPoolMergeExecutorService .getDiskSpaceAvailableForNewMergeTasks (),
435+ is (expectedAvailableBudget .get ())
436+ );
437+ }
438+ );
439+ // double check that submitting a runnable merge task under budget works correctly
440+ ThreadPoolMergeScheduler .MergeTask mergeTask = mock (ThreadPoolMergeScheduler .MergeTask .class );
441+ long taskBudget = randomLongBetween (1L , backloggedMergeTaskDiskSpaceBudget );
442+ when (mergeTask .estimatedRemainingMergeSize ()).thenReturn (taskBudget );
443+ when (mergeTask .schedule ()).thenReturn (RUN );
444+ threadPoolMergeExecutorService .submitMergeTask (mergeTask );
445+ assertBusy (() -> {
446+ verify (mergeTask ).schedule ();
447+ verify (mergeTask ).run ();
448+ });
449+ // let the test finish
450+ blockMergeTaskLatch .countDown ();
451+ }
452+ }
453+ }
454+ }
455+
330456 public void testUnavailableBudgetBlocksNewMergeTasksFromStartingExecution () throws Exception {
331457 int mergeExecutorThreadCount = randomIntBetween (3 , 7 );
332458 // fewer merge tasks than pool threads so that there's always a free thread available
@@ -335,9 +461,9 @@ public void testUnavailableBudgetBlocksNewMergeTasksFromStartingExecution() thro
335461 .put (this .settings )
336462 .put (ThreadPoolMergeScheduler .USE_THREAD_POOL_MERGE_SCHEDULER_SETTING .getKey (), true )
337463 .put (EsExecutors .NODE_PROCESSORS_SETTING .getKey (), mergeExecutorThreadCount )
464+ // the default of "5s" slows down testing
338465 .put (ThreadPoolMergeExecutorService .INDICES_MERGE_DISK_CHECK_INTERVAL_SETTING .getKey (), "30ms" )
339466 .build ();
340- ClusterSettings clusterSettings = ClusterSettings .createBuiltInClusterSettings (settings );
341467 aFileStore .totalSpace = 150_000L ;
342468 bFileStore .totalSpace = 140_000L ;
343469 boolean aHasMoreSpace = randomBoolean ();
@@ -354,7 +480,11 @@ public void testUnavailableBudgetBlocksNewMergeTasksFromStartingExecution() thro
354480 try (NodeEnvironment nodeEnv = new NodeEnvironment (settings , TestEnvironment .newEnvironment (settings ))) {
355481 try (
356482 ThreadPoolMergeExecutorService threadPoolMergeExecutorService = ThreadPoolMergeExecutorService
357- .maybeCreateThreadPoolMergeExecutorService (testThreadPool , clusterSettings , nodeEnv )
483+ .maybeCreateThreadPoolMergeExecutorService (
484+ testThreadPool ,
485+ ClusterSettings .createBuiltInClusterSettings (settings ),
486+ nodeEnv
487+ )
358488 ) {
359489 assert threadPoolMergeExecutorService != null ;
360490 // wait for the budget to be updated from the available disk space
0 commit comments