Skip to content

Commit 1fda136

Browse files
committed
Support notification for host shutting down
We want to support custom notification mechanism for quicker drain signal recognition to minimize compute wastage. This mechanism is currently intended to be used for both the exchange client and supporting a faster graceful shutdown of worker.
1 parent 7edd2c2 commit 1fda136

File tree

13 files changed

+218
-4
lines changed

13 files changed

+218
-4
lines changed

presto-main/src/main/java/com/facebook/presto/server/GracefulShutdownHandler.java

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import com.facebook.presto.execution.TaskManager;
2121
import io.airlift.units.Duration;
2222

23+
import javax.annotation.PreDestroy;
2324
import javax.annotation.concurrent.GuardedBy;
2425
import javax.inject.Inject;
2526

@@ -32,6 +33,7 @@
3233
import java.util.concurrent.TimeoutException;
3334

3435
import static com.facebook.airlift.concurrent.Threads.threadsNamed;
36+
import static com.google.common.base.Preconditions.checkState;
3537
import static com.google.common.collect.ImmutableList.toImmutableList;
3638
import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
3739
import static java.lang.Thread.currentThread;
@@ -55,6 +57,8 @@ public class GracefulShutdownHandler
5557
private final boolean isResourceManager;
5658
private final ShutdownAction shutdownAction;
5759
private final Duration gracePeriod;
60+
private final NodeStatusNotificationManager nodeStatusNotificationManager;
61+
private boolean isLoadNodeStatusNotification;
5862

5963
@GuardedBy("this")
6064
private boolean shutdownRequested;
@@ -65,7 +69,8 @@ public GracefulShutdownHandler(
6569
ServerConfig serverConfig,
6670
ShutdownAction shutdownAction,
6771
LifeCycleManager lifeCycleManager,
68-
QueryManager queryManager)
72+
QueryManager queryManager,
73+
NodeStatusNotificationManager nodeStatusNotificationManager)
6974
{
7075
this.sqlTaskManager = requireNonNull(sqlTaskManager, "sqlTaskManager is null");
7176
this.shutdownAction = requireNonNull(shutdownAction, "shutdownAction is null");
@@ -74,6 +79,21 @@ public GracefulShutdownHandler(
7479
this.isResourceManager = serverConfig.isResourceManager();
7580
this.gracePeriod = serverConfig.getGracePeriod();
7681
this.queryManager = requireNonNull(queryManager, "queryManager is null");
82+
this.nodeStatusNotificationManager = requireNonNull(nodeStatusNotificationManager, "nodeStatusNotificationManager is null");
83+
}
84+
85+
public void loadNodeStatusNotification()
86+
{
87+
log.debug("Loading node status notification");
88+
checkState(!isLoadNodeStatusNotification, "Node status notification can be registered only once");
89+
this.nodeStatusNotificationManager.getNotificationProvider().registerGracefulShutdownEventListener(this::initiateShutdown);
90+
isLoadNodeStatusNotification = true;
91+
}
92+
93+
private void initiateShutdown()
94+
{
95+
log.info("Trigger shutdown from status notification");
96+
requestShutdown();
7797
}
7898

7999
public synchronized void requestShutdown()
@@ -85,6 +105,7 @@ public synchronized void requestShutdown()
85105
}
86106

87107
if (isShutdownRequested()) {
108+
log.info("Shutdown already requested");
88109
return;
89110
}
90111

@@ -202,4 +223,10 @@ public synchronized boolean isShutdownRequested()
202223
{
203224
return shutdownRequested;
204225
}
226+
227+
@PreDestroy
228+
public synchronized void destroy()
229+
{
230+
this.nodeStatusNotificationManager.getNotificationProvider().removeGracefulShutdownEventListener(this::initiateShutdown);
231+
}
205232
}
Lines changed: 65 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,65 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.server;
15+
16+
import com.facebook.presto.spi.nodestatus.NoOpNodeStatusNotificationProvider;
17+
import com.facebook.presto.spi.nodestatus.NodeStatusNotificationProvider;
18+
import com.facebook.presto.spi.nodestatus.NodeStatusNotificationProviderFactory;
19+
import com.google.common.collect.ImmutableMap;
20+
21+
import java.io.File;
22+
import java.io.IOException;
23+
import java.util.Map;
24+
25+
import static com.facebook.presto.util.PropertiesUtil.loadProperties;
26+
import static com.google.common.base.Preconditions.checkState;
27+
import static java.util.Objects.requireNonNull;
28+
29+
public class NodeStatusNotificationManager
30+
{
31+
private static final File NODE_STATUS_NOTIFICATION_CONFIG = new File("etc/node-status-notification.properties");
32+
private NodeStatusNotificationProviderFactory notificationProviderFactory;
33+
private NodeStatusNotificationProvider notificationProvider = new NoOpNodeStatusNotificationProvider();
34+
private boolean isNotificationProviderAdded;
35+
36+
public void addNodeStatusNotificationProviderFactory(NodeStatusNotificationProviderFactory notificationProviderFactory)
37+
{
38+
this.notificationProviderFactory = requireNonNull(notificationProviderFactory, "notificationProviderFactory is null");
39+
}
40+
41+
public void loadNodeStatusNotificationProvider()
42+
throws IOException
43+
{
44+
if (this.notificationProviderFactory == null) {
45+
return;
46+
}
47+
checkState(!isNotificationProviderAdded, "NotificationProvider can only be set once");
48+
this.notificationProvider = this.notificationProviderFactory.create(getConfig());
49+
this.isNotificationProviderAdded = true;
50+
}
51+
52+
private Map<String, String> getConfig()
53+
throws IOException
54+
{
55+
if (NODE_STATUS_NOTIFICATION_CONFIG.exists()) {
56+
return loadProperties(NODE_STATUS_NOTIFICATION_CONFIG);
57+
}
58+
return ImmutableMap.of();
59+
}
60+
61+
public NodeStatusNotificationProvider getNotificationProvider()
62+
{
63+
return this.notificationProvider;
64+
}
65+
}

presto-main/src/main/java/com/facebook/presto/server/PluginManager.java

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@
3333
import com.facebook.presto.spi.connector.ConnectorFactory;
3434
import com.facebook.presto.spi.eventlistener.EventListenerFactory;
3535
import com.facebook.presto.spi.function.FunctionNamespaceManagerFactory;
36+
import com.facebook.presto.spi.nodestatus.NodeStatusNotificationProviderFactory;
3637
import com.facebook.presto.spi.prerequisites.QueryPrerequisitesFactory;
3738
import com.facebook.presto.spi.resourceGroups.ResourceGroupConfigurationManagerFactory;
3839
import com.facebook.presto.spi.security.PasswordAuthenticatorFactory;
@@ -123,6 +124,7 @@ public class PluginManager
123124
private final HistoryBasedPlanStatisticsManager historyBasedPlanStatisticsManager;
124125
private final TracerProviderManager tracerProviderManager;
125126
private final AnalyzerProviderManager analyzerProviderManager;
127+
private final NodeStatusNotificationManager nodeStatusNotificationManager;
126128

127129
@Inject
128130
public PluginManager(
@@ -142,7 +144,8 @@ public PluginManager(
142144
NodeTtlFetcherManager nodeTtlFetcherManager,
143145
ClusterTtlProviderManager clusterTtlProviderManager,
144146
HistoryBasedPlanStatisticsManager historyBasedPlanStatisticsManager,
145-
TracerProviderManager tracerProviderManager)
147+
TracerProviderManager tracerProviderManager,
148+
NodeStatusNotificationManager nodeStatusNotificationManager)
146149
{
147150
requireNonNull(nodeInfo, "nodeInfo is null");
148151
requireNonNull(config, "config is null");
@@ -172,6 +175,7 @@ public PluginManager(
172175
this.historyBasedPlanStatisticsManager = requireNonNull(historyBasedPlanStatisticsManager, "historyBasedPlanStatisticsManager is null");
173176
this.tracerProviderManager = requireNonNull(tracerProviderManager, "tracerProviderManager is null");
174177
this.analyzerProviderManager = requireNonNull(analyzerProviderManager, "analyzerProviderManager is null");
178+
this.nodeStatusNotificationManager = requireNonNull(nodeStatusNotificationManager, "nodeStatusNotificationManager is null");
175179
}
176180

177181
public void loadPlugins()
@@ -317,6 +321,11 @@ public void installPlugin(Plugin plugin)
317321
log.info("Registering analyzer provider %s", analyzerProvider.getType());
318322
analyzerProviderManager.addAnalyzerProvider(analyzerProvider);
319323
}
324+
325+
for (NodeStatusNotificationProviderFactory nodeStatusNotificationProviderFactory : plugin.getNodeStatusNotificationProviderFactory()) {
326+
log.info("Registering node status notification provider %s", nodeStatusNotificationProviderFactory.getName());
327+
nodeStatusNotificationManager.addNodeStatusNotificationProviderFactory(nodeStatusNotificationProviderFactory);
328+
}
320329
}
321330

322331
private URLClassLoader buildClassLoader(String plugin)

presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -175,7 +175,8 @@ public void run()
175175
injector.getInstance(NodeTtlFetcherManager.class).loadNodeTtlFetcher();
176176
injector.getInstance(ClusterTtlProviderManager.class).loadClusterTtlProvider();
177177
injector.getInstance(TracerProviderManager.class).loadTracerProvider();
178-
178+
injector.getInstance(NodeStatusNotificationManager.class).loadNodeStatusNotificationProvider();
179+
injector.getInstance(GracefulShutdownHandler.class).loadNodeStatusNotification();
179180
startAssociatedProcesses(injector);
180181

181182
injector.getInstance(Announcer.class).start();

presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -769,6 +769,7 @@ public ListeningExecutorService createResourceManagerExecutor(ResourceManagerCon
769769

770770
//Optional Status Detector
771771
newOptionalBinder(binder, NodeStatusService.class);
772+
binder.bind(NodeStatusNotificationManager.class).in(Scopes.SINGLETON);
772773
}
773774

774775
@Provides

presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -119,6 +119,7 @@
119119
import com.facebook.presto.operator.TaskContext;
120120
import com.facebook.presto.operator.index.IndexJoinLookupStats;
121121
import com.facebook.presto.server.ConnectorMetadataUpdateHandleJsonSerde;
122+
import com.facebook.presto.server.NodeStatusNotificationManager;
122123
import com.facebook.presto.server.PluginManager;
123124
import com.facebook.presto.server.PluginManagerConfig;
124125
import com.facebook.presto.server.SessionPropertyDefaults;
@@ -504,7 +505,8 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
504505
new ThrowingNodeTtlFetcherManager(),
505506
new ThrowingClusterTtlProviderManager(),
506507
historyBasedPlanStatisticsManager,
507-
new TracerProviderManager(new TracingConfig()));
508+
new TracerProviderManager(new TracingConfig()),
509+
new NodeStatusNotificationManager());
508510

509511
connectorManager.addConnectorFactory(globalSystemConnectorFactory);
510512
connectorManager.createConnection(GlobalSystemConnector.NAME, GlobalSystemConnector.NAME, ImmutableMap.of());

presto-main/src/test/java/com/facebook/presto/server/TestThriftServerInfoIntegration.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -136,6 +136,7 @@ public void configure(Binder binder)
136136
configBinder(binder).bindConfig(ServerConfig.class);
137137
//Bind noop QueryManager similar to the binding done for TaskManager here
138138
binder.bind(QueryManager.class).to(NoOpQueryManager.class).in(Scopes.SINGLETON);
139+
binder.bind(NodeStatusNotificationManager.class).in(Scopes.SINGLETON);
139140
binder.bind(GracefulShutdownHandler.class).in(Scopes.SINGLETON);
140141
binder.bind(ShutdownAction.class).to(TestingPrestoServer.TestShutdownAction.class).in(Scopes.SINGLETON);
141142

presto-spark-base/src/main/java/com/facebook/presto/spark/PrestoSparkModule.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -96,6 +96,7 @@
9696
import com.facebook.presto.resourcemanager.ResourceGroupService;
9797
import com.facebook.presto.server.ConnectorMetadataUpdateHandleJsonSerde;
9898
import com.facebook.presto.server.ForJsonMetadataUpdateHandle;
99+
import com.facebook.presto.server.NodeStatusNotificationManager;
99100
import com.facebook.presto.server.PluginManager;
100101
import com.facebook.presto.server.PluginManagerConfig;
101102
import com.facebook.presto.server.QuerySessionSupplier;
@@ -501,6 +502,7 @@ protected void setup(Binder binder)
501502
binder.bind(ResourceGroupService.class).to(NoopResourceGroupService.class).in(Scopes.SINGLETON);
502503
binder.bind(NodeTtlFetcherManager.class).to(ThrowingNodeTtlFetcherManager.class).in(Scopes.SINGLETON);
503504
binder.bind(ClusterTtlProviderManager.class).to(ThrowingClusterTtlProviderManager.class).in(Scopes.SINGLETON);
505+
binder.bind(NodeStatusNotificationManager.class).in(Scopes.SINGLETON);
504506

505507
// TODO: Decouple and remove: required by SessionPropertyDefaults, PluginManager, InternalResourceGroupManager, ConnectorManager
506508
configBinder(binder).bindConfig(NodeConfig.class);

presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import com.facebook.presto.spi.connector.ConnectorFactory;
2121
import com.facebook.presto.spi.eventlistener.EventListenerFactory;
2222
import com.facebook.presto.spi.function.FunctionNamespaceManagerFactory;
23+
import com.facebook.presto.spi.nodestatus.NodeStatusNotificationProviderFactory;
2324
import com.facebook.presto.spi.prerequisites.QueryPrerequisitesFactory;
2425
import com.facebook.presto.spi.resourceGroups.ResourceGroupConfigurationManagerFactory;
2526
import com.facebook.presto.spi.security.PasswordAuthenticatorFactory;
@@ -130,4 +131,9 @@ default Iterable<AnalyzerProvider> getAnalyzerProviders()
130131
{
131132
return emptyList();
132133
}
134+
135+
default Iterable<NodeStatusNotificationProviderFactory> getNodeStatusNotificationProviderFactory()
136+
{
137+
return emptyList();
138+
}
133139
}
Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,20 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.spi.nodestatus;
15+
16+
@FunctionalInterface
17+
public interface GracefulShutdownEventListener
18+
{
19+
void onNodeShuttingDown();
20+
}
Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,28 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.spi.nodestatus;
15+
16+
public class NoOpNodeStatusNotificationProvider
17+
implements NodeStatusNotificationProvider
18+
{
19+
@Override
20+
public void registerGracefulShutdownEventListener(GracefulShutdownEventListener listener)
21+
{
22+
}
23+
24+
@Override
25+
public void removeGracefulShutdownEventListener(GracefulShutdownEventListener listener)
26+
{
27+
}
28+
}
Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,29 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.spi.nodestatus;
15+
16+
/**
17+
* The {@code NodeStatusNotificationProvider} interface provides a registry for node status listeners.
18+
* Implementations of this interface can listen to node status events and notify all registered listeners,
19+
* especially when a node goes down.
20+
*
21+
* <p>It is essential for implementations to ensure proper synchronization if the registry is accessed
22+
* by multiple threads.</p>
23+
*/
24+
public interface NodeStatusNotificationProvider
25+
{
26+
void registerGracefulShutdownEventListener(GracefulShutdownEventListener listener);
27+
28+
void removeGracefulShutdownEventListener(GracefulShutdownEventListener listener);
29+
}
Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,23 @@
1+
/*
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package com.facebook.presto.spi.nodestatus;
15+
16+
import java.util.Map;
17+
18+
public interface NodeStatusNotificationProviderFactory
19+
{
20+
String getName();
21+
22+
NodeStatusNotificationProvider create(Map<String, String> config);
23+
}

0 commit comments

Comments
 (0)