|
@@ -22,6 +22,7 @@ package org.elasticsearch.node;
|
|
|
import org.apache.lucene.util.IOUtils;
|
|
|
import org.elasticsearch.Build;
|
|
|
import org.elasticsearch.ElasticsearchException;
|
|
|
+import org.elasticsearch.ElasticsearchTimeoutException;
|
|
|
import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.ActionModule;
|
|
|
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
|
@@ -30,9 +31,13 @@ import org.elasticsearch.client.node.NodeClientModule;
|
|
|
import org.elasticsearch.cluster.ClusterModule;
|
|
|
import org.elasticsearch.cluster.ClusterNameModule;
|
|
|
import org.elasticsearch.cluster.ClusterService;
|
|
|
+import org.elasticsearch.cluster.ClusterState;
|
|
|
+import org.elasticsearch.cluster.ClusterStateObserver;
|
|
|
+import org.elasticsearch.cluster.MasterNodeChangePredicate;
|
|
|
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
|
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
|
|
import org.elasticsearch.cluster.routing.RoutingService;
|
|
|
+import org.elasticsearch.cluster.service.InternalClusterService;
|
|
|
import org.elasticsearch.common.StopWatch;
|
|
|
import org.elasticsearch.common.component.Lifecycle;
|
|
|
import org.elasticsearch.common.component.LifecycleComponent;
|
|
@@ -52,9 +57,10 @@ import org.elasticsearch.common.settings.Settings;
|
|
|
import org.elasticsearch.common.settings.SettingsModule;
|
|
|
import org.elasticsearch.common.transport.BoundTransportAddress;
|
|
|
import org.elasticsearch.common.transport.TransportAddress;
|
|
|
+import org.elasticsearch.common.unit.TimeValue;
|
|
|
import org.elasticsearch.discovery.Discovery;
|
|
|
import org.elasticsearch.discovery.DiscoveryModule;
|
|
|
-import org.elasticsearch.discovery.DiscoveryService;
|
|
|
+import org.elasticsearch.discovery.DiscoverySettings;
|
|
|
import org.elasticsearch.env.Environment;
|
|
|
import org.elasticsearch.env.EnvironmentModule;
|
|
|
import org.elasticsearch.env.NodeEnvironment;
|
|
@@ -111,6 +117,7 @@ import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.CountDownLatch;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.function.Function;
|
|
|
|
|
@@ -154,7 +161,7 @@ public class Node implements Closeable {
|
|
|
|
|
|
protected Node(Environment tmpEnv, Version version, Collection<Class<? extends Plugin>> classpathPlugins) {
|
|
|
Settings tmpSettings = settingsBuilder().put(tmpEnv.settings())
|
|
|
- .put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build();
|
|
|
+ .put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build();
|
|
|
tmpSettings = TribeService.processSettings(tmpSettings);
|
|
|
|
|
|
ESLogger logger = Loggers.getLogger(Node.class, NODE_NAME_SETTING.get(tmpSettings));
|
|
@@ -164,7 +171,7 @@ public class Node implements Closeable {
|
|
|
|
|
|
if (logger.isDebugEnabled()) {
|
|
|
logger.debug("using config [{}], data [{}], logs [{}], plugins [{}]",
|
|
|
- tmpEnv.configFile(), Arrays.toString(tmpEnv.dataFiles()), tmpEnv.logsFile(), tmpEnv.pluginsFile());
|
|
|
+ tmpEnv.configFile(), Arrays.toString(tmpEnv.dataFiles()), tmpEnv.logsFile(), tmpEnv.pluginsFile());
|
|
|
}
|
|
|
|
|
|
this.pluginsService = new PluginsService(tmpSettings, tmpEnv.modulesFile(), tmpEnv.pluginsFile(), classpathPlugins);
|
|
@@ -282,28 +289,73 @@ public class Node implements Closeable {
|
|
|
injector.getInstance(MonitorService.class).start();
|
|
|
injector.getInstance(RestController.class).start();
|
|
|
|
|
|
+ assert injector.getInstance(ClusterService.class) instanceof InternalClusterService :
|
|
|
+ "node cluster service implementation must inherit from InternalClusterService";
|
|
|
+ final InternalClusterService clusterService = (InternalClusterService) injector.getInstance(ClusterService.class);
|
|
|
+
|
|
|
// TODO hack around circular dependencies problems
|
|
|
- injector.getInstance(GatewayAllocator.class).setReallocation(injector.getInstance(ClusterService.class), injector.getInstance(RoutingService.class));
|
|
|
+ injector.getInstance(GatewayAllocator.class).setReallocation(clusterService, injector.getInstance(RoutingService.class));
|
|
|
|
|
|
injector.getInstance(ResourceWatcherService.class).start();
|
|
|
injector.getInstance(GatewayService.class).start();
|
|
|
+ Discovery discovery = injector.getInstance(Discovery.class);
|
|
|
+ clusterService.addInitialStateBlock(discovery.getDiscoverySettings().getNoMasterBlock());
|
|
|
+ clusterService.setClusterStatePublisher(discovery::publish);
|
|
|
+
|
|
|
+ // start before the cluster service since it adds/removes initial Cluster state blocks
|
|
|
+ final TribeService tribeService = injector.getInstance(TribeService.class);
|
|
|
+ tribeService.start();
|
|
|
+
|
|
|
|
|
|
// Start the transport service now so the publish address will be added to the local disco node in ClusterService
|
|
|
TransportService transportService = injector.getInstance(TransportService.class);
|
|
|
transportService.start();
|
|
|
- injector.getInstance(ClusterService.class).start();
|
|
|
+ clusterService.start();
|
|
|
|
|
|
// start after cluster service so the local disco is known
|
|
|
- DiscoveryService discoService = injector.getInstance(DiscoveryService.class).start();
|
|
|
-
|
|
|
-
|
|
|
+ discovery.start();
|
|
|
transportService.acceptIncomingRequests();
|
|
|
- discoService.joinClusterAndWaitForInitialState();
|
|
|
+ discovery.startInitialJoin();
|
|
|
+
|
|
|
+ // tribe nodes don't have a master so we shouldn't register an observer
|
|
|
+ if (DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings).millis() > 0) {
|
|
|
+ final ThreadPool thread = injector.getInstance(ThreadPool.class);
|
|
|
+ ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, thread.getThreadContext());
|
|
|
+ final CountDownLatch latch = new CountDownLatch(1);
|
|
|
+ if (observer.observedState().nodes().masterNodeId() == null) {
|
|
|
+ observer.waitForNextChange(new ClusterStateObserver.Listener() {
|
|
|
+ @Override
|
|
|
+ public void onNewClusterState(ClusterState state) {
|
|
|
+ latch.countDown();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onClusterServiceClose() {
|
|
|
+ latch.countDown();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onTimeout(TimeValue timeout) {
|
|
|
+ assert false;
|
|
|
+ }
|
|
|
+ // use null timeout as we use timeout on the latchwait
|
|
|
+ }, MasterNodeChangePredicate.INSTANCE, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ latch.await(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings).millis(), TimeUnit.MILLISECONDS);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new ElasticsearchTimeoutException("Interrupted while waiting for initial discovery state");
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
if (settings.getAsBoolean("http.enabled", true)) {
|
|
|
injector.getInstance(HttpServer.class).start();
|
|
|
}
|
|
|
- injector.getInstance(TribeService.class).start();
|
|
|
+
|
|
|
+ // start nodes now, after the http server, because it may take some time
|
|
|
+ tribeService.startNodes();
|
|
|
+
|
|
|
|
|
|
if (WRITE_PORTS_FIELD_SETTING.get(settings)) {
|
|
|
if (settings.getAsBoolean("http.enabled", true)) {
|
|
@@ -340,7 +392,6 @@ public class Node implements Closeable {
|
|
|
injector.getInstance(IndicesTTLService.class).stop();
|
|
|
injector.getInstance(RoutingService.class).stop();
|
|
|
injector.getInstance(ClusterService.class).stop();
|
|
|
- injector.getInstance(DiscoveryService.class).stop();
|
|
|
injector.getInstance(MonitorService.class).stop();
|
|
|
injector.getInstance(GatewayService.class).stop();
|
|
|
injector.getInstance(SearchService.class).stop();
|
|
@@ -378,50 +429,50 @@ public class Node implements Closeable {
|
|
|
toClose.add(injector.getInstance(TribeService.class));
|
|
|
toClose.add(() -> stopWatch.stop().start("node_service"));
|
|
|
toClose.add(injector.getInstance(NodeService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("http"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("http"));
|
|
|
if (settings.getAsBoolean("http.enabled", true)) {
|
|
|
toClose.add(injector.getInstance(HttpServer.class));
|
|
|
}
|
|
|
- toClose.add(() ->stopWatch.stop().start("snapshot_service"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("snapshot_service"));
|
|
|
toClose.add(injector.getInstance(SnapshotsService.class));
|
|
|
toClose.add(injector.getInstance(SnapshotShardsService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("client"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("client"));
|
|
|
Releasables.close(injector.getInstance(Client.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("indices_cluster"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("indices_cluster"));
|
|
|
toClose.add(injector.getInstance(IndicesClusterStateService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("indices"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("indices"));
|
|
|
toClose.add(injector.getInstance(IndicesTTLService.class));
|
|
|
toClose.add(injector.getInstance(IndicesService.class));
|
|
|
// close filter/fielddata caches after indices
|
|
|
toClose.add(injector.getInstance(IndicesStore.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("routing"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("routing"));
|
|
|
toClose.add(injector.getInstance(RoutingService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("cluster"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("cluster"));
|
|
|
toClose.add(injector.getInstance(ClusterService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("discovery"));
|
|
|
- toClose.add(injector.getInstance(DiscoveryService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("monitor"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("discovery"));
|
|
|
+ toClose.add(injector.getInstance(Discovery.class));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("monitor"));
|
|
|
toClose.add(injector.getInstance(MonitorService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("gateway"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("gateway"));
|
|
|
toClose.add(injector.getInstance(GatewayService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("search"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("search"));
|
|
|
toClose.add(injector.getInstance(SearchService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("rest"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("rest"));
|
|
|
toClose.add(injector.getInstance(RestController.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("transport"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("transport"));
|
|
|
toClose.add(injector.getInstance(TransportService.class));
|
|
|
- toClose.add(() ->stopWatch.stop().start("percolator_service"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("percolator_service"));
|
|
|
toClose.add(injector.getInstance(PercolatorService.class));
|
|
|
|
|
|
for (Class<? extends LifecycleComponent> plugin : pluginsService.nodeServices()) {
|
|
|
- toClose.add(() ->stopWatch.stop().start("plugin(" + plugin.getName() + ")"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("plugin(" + plugin.getName() + ")"));
|
|
|
toClose.add(injector.getInstance(plugin));
|
|
|
}
|
|
|
|
|
|
- toClose.add(() ->stopWatch.stop().start("script"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("script"));
|
|
|
toClose.add(injector.getInstance(ScriptService.class));
|
|
|
|
|
|
- toClose.add(() ->stopWatch.stop().start("thread_pool"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("thread_pool"));
|
|
|
// TODO this should really use ThreadPool.terminate()
|
|
|
toClose.add(() -> injector.getInstance(ThreadPool.class).shutdown());
|
|
|
toClose.add(() -> {
|
|
@@ -432,7 +483,7 @@ public class Node implements Closeable {
|
|
|
}
|
|
|
});
|
|
|
|
|
|
- toClose.add(() ->stopWatch.stop().start("thread_pool_force_shutdown"));
|
|
|
+ toClose.add(() -> stopWatch.stop().start("thread_pool_force_shutdown"));
|
|
|
toClose.add(() -> injector.getInstance(ThreadPool.class).shutdownNow());
|
|
|
toClose.add(() -> stopWatch.stop());
|
|
|
|