1313import org .apache .logging .log4j .Logger ;
1414import org .apache .lucene .util .SetOnce ;
1515import org .elasticsearch .ElasticsearchTimeoutException ;
16- import org .elasticsearch .action .search .TransportSearchAction ;
17- import org .elasticsearch .action .support .PlainActionFuture ;
18- import org .elasticsearch .action .support .RefCountingListener ;
19- import org .elasticsearch .action .support .SubscribableListener ;
2016import org .elasticsearch .bootstrap .BootstrapCheck ;
2117import org .elasticsearch .bootstrap .BootstrapContext ;
2218import org .elasticsearch .client .internal .Client ;
8278import org .elasticsearch .snapshots .SnapshotShardsService ;
8379import org .elasticsearch .snapshots .SnapshotsService ;
8480import org .elasticsearch .tasks .TaskCancellationService ;
85- import org .elasticsearch .tasks .TaskManager ;
8681import org .elasticsearch .tasks .TaskResultsService ;
8782import org .elasticsearch .threadpool .ThreadPool ;
8883import org .elasticsearch .transport .RemoteClusterPortSettings ;
106101import java .util .List ;
107102import java .util .Map ;
108103import java .util .concurrent .CountDownLatch ;
109- import java .util .concurrent .ExecutionException ;
110104import java .util .concurrent .TimeUnit ;
111- import java .util .concurrent .TimeoutException ;
112105import java .util .function .BiConsumer ;
113106import java .util .function .Function ;
114- import java .util .function .Supplier ;
115- import java .util .stream .Collectors ;
116107
117108import javax .net .ssl .SNIHostName ;
118109
119- import static org .elasticsearch .core .Strings .format ;
120-
121110/**
122111 * A node represent a node within a cluster ({@code cluster.name}). The {@link #client()} can be used
123112 * in order to use a {@link Client} to perform actions/operations against the cluster.
@@ -161,12 +150,6 @@ public class Node implements Closeable {
161150 Property .NodeScope
162151 );
163152
164- public static final Setting <TimeValue > MAXIMUM_SHUTDOWN_TIMEOUT_SETTING = Setting .positiveTimeSetting (
165- "node.maximum_shutdown_grace_period" ,
166- TimeValue .ZERO ,
167- Setting .Property .NodeScope
168- );
169-
170153 private final Lifecycle lifecycle = new Lifecycle ();
171154
172155 /**
@@ -187,6 +170,7 @@ public class Node implements Closeable {
187170 private final LocalNodeFactory localNodeFactory ;
188171 private final NodeService nodeService ;
189172 private final TerminationHandler terminationHandler ;
173+
190174 // for testing
191175 final NamedWriteableRegistry namedWriteableRegistry ;
192176 final NamedXContentRegistry namedXContentRegistry ;
@@ -606,105 +590,8 @@ public synchronized void close() throws IOException {
606590 * logic should use Node Shutdown, see {@link org.elasticsearch.cluster.metadata.NodesShutdownMetadata}.
607591 */
608592 public void prepareForClose () {
609- final var maxTimeout = MAXIMUM_SHUTDOWN_TIMEOUT_SETTING .get (this .settings ());
610-
611- record Stopper (String name , SubscribableListener <Void > listener ) {
612- boolean isIncomplete () {
613- return listener ().isDone () == false ;
614- }
615- }
616-
617- final var stoppers = new ArrayList <Stopper >();
618- final var allStoppersFuture = new PlainActionFuture <Void >();
619- try (var listeners = new RefCountingListener (allStoppersFuture )) {
620- final BiConsumer <String , Runnable > stopperRunner = (name , action ) -> {
621- final var stopper = new Stopper (name , new SubscribableListener <>());
622- stoppers .add (stopper );
623- stopper .listener ().addListener (listeners .acquire ());
624- new Thread (() -> {
625- try {
626- action .run ();
627- } catch (Exception ex ) {
628- logger .warn ("unexpected exception in shutdown task [" + stopper .name () + "]" , ex );
629- } finally {
630- stopper .listener ().onResponse (null );
631- }
632- }, stopper .name ()).start ();
633- };
634-
635- stopperRunner .accept ("http-server-transport-stop" , injector .getInstance (HttpServerTransport .class )::close );
636- stopperRunner .accept ("async-search-stop" , () -> awaitSearchTasksComplete (maxTimeout ));
637- if (terminationHandler != null ) {
638- stopperRunner .accept ("termination-handler-stop" , terminationHandler ::handleTermination );
639- }
640- }
641-
642- final Supplier <String > incompleteStoppersDescriber = () -> stoppers .stream ()
643- .filter (Stopper ::isIncomplete )
644- .map (Stopper ::name )
645- .collect (Collectors .joining (", " , "[" , "]" ));
646-
647- try {
648- if (TimeValue .ZERO .equals (maxTimeout )) {
649- allStoppersFuture .get ();
650- } else {
651- allStoppersFuture .get (maxTimeout .millis (), TimeUnit .MILLISECONDS );
652- }
653- } catch (ExecutionException e ) {
654- assert false : e ; // listeners are never completed exceptionally
655- logger .warn ("failed during graceful shutdown tasks" , e );
656- } catch (InterruptedException e ) {
657- Thread .currentThread ().interrupt ();
658- logger .warn ("interrupted while waiting for graceful shutdown tasks: " + incompleteStoppersDescriber .get (), e );
659- } catch (TimeoutException e ) {
660- logger .warn ("timed out while waiting for graceful shutdown tasks: " + incompleteStoppersDescriber .get ());
661- }
662- }
663-
664- private void awaitSearchTasksComplete (TimeValue asyncSearchTimeout ) {
665- TaskManager taskManager = injector .getInstance (TransportService .class ).getTaskManager ();
666- long millisWaited = 0 ;
667- while (true ) {
668- long searchTasksRemaining = taskManager .getTasks ()
669- .values ()
670- .stream ()
671- .filter (task -> TransportSearchAction .TYPE .name ().equals (task .getAction ()))
672- .count ();
673- if (searchTasksRemaining == 0 ) {
674- logger .debug ("all search tasks complete" );
675- return ;
676- } else {
677- // Let the system work on those searches for a while. We're on a dedicated thread to manage app shutdown, so we
678- // literally just want to wait and not take up resources on this thread for now. Poll period chosen to allow short
679- // response times, but checking the tasks list is relatively expensive, and we don't want to waste CPU time we could
680- // be spending on finishing those searches.
681- final TimeValue pollPeriod = TimeValue .timeValueMillis (500 );
682- millisWaited += pollPeriod .millis ();
683- if (TimeValue .ZERO .equals (asyncSearchTimeout ) == false && millisWaited >= asyncSearchTimeout .millis ()) {
684- logger .warn (
685- format (
686- "timed out after waiting [%s] for [%d] search tasks to finish" ,
687- asyncSearchTimeout .toString (),
688- searchTasksRemaining
689- )
690- );
691- return ;
692- }
693- logger .debug (format ("waiting for [%s] search tasks to finish, next poll in [%s]" , searchTasksRemaining , pollPeriod ));
694- try {
695- Thread .sleep (pollPeriod .millis ());
696- } catch (InterruptedException ex ) {
697- logger .warn (
698- format (
699- "interrupted while waiting [%s] for [%d] search tasks to finish" ,
700- asyncSearchTimeout .toString (),
701- searchTasksRemaining
702- )
703- );
704- return ;
705- }
706- }
707- }
593+ injector .getInstance (ShutdownPrepareService .class )
594+ .prepareForShutdown (injector .getInstance (TransportService .class ).getTaskManager ());
708595 }
709596
710597 /**
0 commit comments