Explorar o código

Remove InternalNode interface

Simon Willnauer %!s(int64=10) %!d(string=hai) anos
pai
achega
3e1c7b5a0c
Modificáronse 27 ficheiros con 447 adicións e 518 borrados
  1. 381 15
      src/main/java/org/elasticsearch/node/Node.java
  2. 1 2
      src/main/java/org/elasticsearch/node/NodeBuilder.java
  3. 0 422
      src/main/java/org/elasticsearch/node/internal/InternalNode.java
  4. 10 10
      src/main/java/org/elasticsearch/tribe/TribeService.java
  5. 3 3
      src/test/java/org/elasticsearch/benchmark/search/aggregations/GlobalOrdinalsBenchmark.java
  6. 3 4
      src/test/java/org/elasticsearch/benchmark/search/aggregations/SubAggregationSearchCollectModeBenchmark.java
  7. 5 5
      src/test/java/org/elasticsearch/benchmark/search/aggregations/TermsAggregationSearchAndIndexingBenchmark.java
  8. 3 4
      src/test/java/org/elasticsearch/benchmark/search/aggregations/TermsAggregationSearchBenchmark.java
  9. 3 3
      src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityTests.java
  10. 1 2
      src/test/java/org/elasticsearch/client/transport/TransportClientTests.java
  11. 2 3
      src/test/java/org/elasticsearch/cluster/routing/RoutingBackwardCompatibilityTests.java
  12. 2 2
      src/test/java/org/elasticsearch/cluster/routing/RoutingBackwardCompatibilityUponUpgradeTests.java
  13. 2 2
      src/test/java/org/elasticsearch/http/netty/HttpPublishPortTests.java
  14. 2 2
      src/test/java/org/elasticsearch/http/netty/NettyPipeliningDisabledIntegrationTest.java
  15. 2 2
      src/test/java/org/elasticsearch/http/netty/NettyPipeliningEnabledIntegrationTest.java
  16. 2 4
      src/test/java/org/elasticsearch/network/DirectBufferNetworkTests.java
  17. 3 3
      src/test/java/org/elasticsearch/options/jsonp/JsonpOptionDisabledTest.java
  18. 2 2
      src/test/java/org/elasticsearch/options/jsonp/JsonpOptionEnabledTest.java
  19. 2 2
      src/test/java/org/elasticsearch/rest/CorsRegexDefaultTests.java
  20. 2 2
      src/test/java/org/elasticsearch/rest/CorsRegexTests.java
  21. 2 2
      src/test/java/org/elasticsearch/rest/action/admin/indices/upgrade/UpgradeTest.java
  22. 2 2
      src/test/java/org/elasticsearch/stresstest/leaks/GenericStatsLeak.java
  23. 1 3
      src/test/java/org/elasticsearch/stresstest/rollingrestart/RollingRestartStressTest.java
  24. 2 6
      src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java
  25. 1 2
      src/test/java/org/elasticsearch/test/ElasticsearchSingleNodeTest.java
  26. 6 7
      src/test/java/org/elasticsearch/test/InternalTestCluster.java
  27. 2 2
      src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTests.java

+ 381 - 15
src/main/java/org/elasticsearch/node/Node.java

@@ -19,9 +19,92 @@
 
 package org.elasticsearch.node;
 
+import org.elasticsearch.Build;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ElasticsearchIllegalStateException;
+import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionModule;
+import org.elasticsearch.action.bench.BenchmarkModule;
+import org.elasticsearch.cache.recycler.PageCacheRecycler;
+import org.elasticsearch.cache.recycler.PageCacheRecyclerModule;
 import org.elasticsearch.client.Client;
+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.action.index.MappingUpdatedAction;
+import org.elasticsearch.cluster.routing.RoutingService;
+import org.elasticsearch.cluster.routing.allocation.AllocationService;
+import org.elasticsearch.common.StopWatch;
+import org.elasticsearch.common.collect.Tuple;
+import org.elasticsearch.common.component.Lifecycle;
+import org.elasticsearch.common.component.LifecycleComponent;
+import org.elasticsearch.common.compress.CompressorFactory;
+import org.elasticsearch.common.inject.Injector;
+import org.elasticsearch.common.inject.ModulesBuilder;
 import org.elasticsearch.common.lease.Releasable;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.logging.ESLogger;
+import org.elasticsearch.common.logging.Loggers;
+import org.elasticsearch.common.network.NetworkModule;
+import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.settings.SettingsModule;
+import org.elasticsearch.common.util.BigArraysModule;
+import org.elasticsearch.discovery.Discovery;
+import org.elasticsearch.discovery.DiscoveryModule;
+import org.elasticsearch.discovery.DiscoveryService;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.EnvironmentModule;
+import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.env.NodeEnvironmentModule;
+import org.elasticsearch.gateway.GatewayModule;
+import org.elasticsearch.gateway.GatewayService;
+import org.elasticsearch.http.HttpServer;
+import org.elasticsearch.http.HttpServerModule;
+import org.elasticsearch.index.search.shape.ShapeModule;
+import org.elasticsearch.indices.IndicesModule;
+import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.indices.breaker.CircuitBreakerModule;
+import org.elasticsearch.indices.cache.filter.IndicesFilterCache;
+import org.elasticsearch.indices.cluster.IndicesClusterStateService;
+import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
+import org.elasticsearch.indices.memory.IndexingMemoryController;
+import org.elasticsearch.indices.store.IndicesStore;
+import org.elasticsearch.indices.ttl.IndicesTTLService;
+import org.elasticsearch.monitor.MonitorModule;
+import org.elasticsearch.monitor.MonitorService;
+import org.elasticsearch.monitor.jvm.JvmInfo;
+import org.elasticsearch.node.internal.InternalSettingsPreparer;
+import org.elasticsearch.node.internal.NodeModule;
+import org.elasticsearch.percolator.PercolatorModule;
+import org.elasticsearch.percolator.PercolatorService;
+import org.elasticsearch.plugins.PluginsModule;
+import org.elasticsearch.plugins.PluginsService;
+import org.elasticsearch.repositories.RepositoriesModule;
+import org.elasticsearch.rest.RestController;
+import org.elasticsearch.rest.RestModule;
+import org.elasticsearch.river.RiversManager;
+import org.elasticsearch.river.RiversModule;
+import org.elasticsearch.script.ScriptModule;
+import org.elasticsearch.script.ScriptService;
+import org.elasticsearch.search.SearchModule;
+import org.elasticsearch.search.SearchService;
+import org.elasticsearch.snapshots.SnapshotsService;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.threadpool.ThreadPoolModule;
+import org.elasticsearch.transport.TransportModule;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.tribe.TribeModule;
+import org.elasticsearch.tribe.TribeService;
+import org.elasticsearch.watcher.ResourceWatcherModule;
+import org.elasticsearch.watcher.ResourceWatcherService;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
 
 /**
  * A node represent a node within a cluster (<tt>cluster.name</tt>). The {@link #client()} can be used
@@ -29,39 +112,322 @@ import org.elasticsearch.common.settings.Settings;
  * <p/>
  * <p>In order to create a node, the {@link NodeBuilder} can be used. When done with it, make sure to
  * call {@link #close()} on it.
- *
- *
  */
-public interface Node extends Releasable {
+public class Node implements Releasable {
+
+    private static final String CLIENT_TYPE = "node";
+    public static final String HTTP_ENABLED = "http.enabled";
+    private final Lifecycle lifecycle = new Lifecycle();
+    private final Injector injector;
+    private final Settings settings;
+    private final Environment environment;
+    private final PluginsService pluginsService;
+    private final Client client;
+
+    public Node() throws ElasticsearchException {
+        this(ImmutableSettings.Builder.EMPTY_SETTINGS, true);
+    }
+
+    public Node(Settings preparedSettings, boolean loadConfigSettings) throws ElasticsearchException {
+        final Settings pSettings = settingsBuilder().put(preparedSettings)
+                .put(Client.CLIENT_TYPE_SETTING, CLIENT_TYPE).build();
+        Tuple<Settings, Environment> tuple = InternalSettingsPreparer.prepareSettings(pSettings, loadConfigSettings);
+        tuple = new Tuple<>(TribeService.processSettings(tuple.v1()), tuple.v2());
+
+        // The only place we can actually fake the version a node is running on:
+        Version version = pSettings.getAsVersion("tests.mock.version", Version.CURRENT);
+
+        ESLogger logger = Loggers.getLogger(Node.class, tuple.v1().get("name"));
+        logger.info("version[{}], pid[{}], build[{}/{}]", version, JvmInfo.jvmInfo().pid(), Build.CURRENT.hashShort(), Build.CURRENT.timestamp());
+
+        logger.info("initializing ...");
+
+        if (logger.isDebugEnabled()) {
+            Environment env = tuple.v2();
+            logger.debug("using home [{}], config [{}], data [{}], logs [{}], work [{}], plugins [{}]",
+                    env.homeFile(), env.configFile(), Arrays.toString(env.dataFiles()), env.logsFile(),
+                    env.workFile(), env.pluginsFile());
+        }
+
+        this.pluginsService = new PluginsService(tuple.v1(), tuple.v2());
+        this.settings = pluginsService.updatedSettings();
+        // create the environment based on the finalized (processed) view of the settings
+        this.environment = new Environment(this.settings());
+
+        CompressorFactory.configure(settings);
+        final NodeEnvironment nodeEnvironment;
+        try {
+            nodeEnvironment = new NodeEnvironment(this.settings, this.environment);
+        } catch (IOException ex) {
+            throw new ElasticsearchIllegalStateException("Failed to created node environment", ex);
+        }
+
+        boolean success = false;
+        try {
+            ModulesBuilder modules = new ModulesBuilder();
+            modules.add(new Version.Module(version));
+            modules.add(new PageCacheRecyclerModule(settings));
+            modules.add(new CircuitBreakerModule(settings));
+            modules.add(new BigArraysModule(settings));
+            modules.add(new PluginsModule(settings, pluginsService));
+            modules.add(new SettingsModule(settings));
+            modules.add(new NodeModule(this));
+            modules.add(new NetworkModule());
+            modules.add(new ScriptModule(settings));
+            modules.add(new EnvironmentModule(environment));
+            modules.add(new NodeEnvironmentModule(nodeEnvironment));
+            modules.add(new ClusterNameModule(settings));
+            modules.add(new ThreadPoolModule(settings));
+            modules.add(new DiscoveryModule(settings));
+            modules.add(new ClusterModule(settings));
+            modules.add(new RestModule(settings));
+            modules.add(new TransportModule(settings));
+            if (settings.getAsBoolean(HTTP_ENABLED, true)) {
+                modules.add(new HttpServerModule(settings));
+            }
+            modules.add(new RiversModule(settings));
+            modules.add(new IndicesModule(settings));
+            modules.add(new SearchModule());
+            modules.add(new ActionModule(false));
+            modules.add(new MonitorModule(settings));
+            modules.add(new GatewayModule());
+            modules.add(new NodeClientModule());
+            modules.add(new ShapeModule());
+            modules.add(new PercolatorModule());
+            modules.add(new ResourceWatcherModule());
+            modules.add(new RepositoriesModule());
+            modules.add(new TribeModule());
+            modules.add(new BenchmarkModule(settings));
+
+            injector = modules.createInjector();
+
+            client = injector.getInstance(Client.class);
+            success = true;
+        } finally {
+            if (!success) {
+                nodeEnvironment.close();
+            }
+        }
+
+        logger.info("initialized");
+    }
 
     /**
      * The settings that were used to create the node.
      */
-    Settings settings();
+    public Settings settings() {
+        return this.settings;
+    }
 
     /**
      * A client that can be used to execute actions (operations) against the cluster.
      */
-    Client client();
+    public Client client() {
+        return client;
+    }
 
     /**
      * Start the node. If the node is already started, this method is no-op.
      */
-    Node start();
+    public Node start() {
+        if (!lifecycle.moveToStarted()) {
+            return this;
+        }
 
-    /**
-     * Stops the node. If the node is already stopped, this method is no-op.
-     */
-    Node stop();
+        ESLogger logger = Loggers.getLogger(Node.class, settings.get("name"));
+        logger.info("starting ...");
 
-    /**
-     * Closes the node (and {@link #stop}s if its running).
-     */
+        // hack around dependency injection problem (for now...)
+        injector.getInstance(Discovery.class).setAllocationService(injector.getInstance(AllocationService.class));
+
+        for (Class<? extends LifecycleComponent> plugin : pluginsService.services()) {
+            injector.getInstance(plugin).start();
+        }
+
+        injector.getInstance(MappingUpdatedAction.class).start();
+        injector.getInstance(IndicesService.class).start();
+        injector.getInstance(IndexingMemoryController.class).start();
+        injector.getInstance(IndicesClusterStateService.class).start();
+        injector.getInstance(IndicesTTLService.class).start();
+        injector.getInstance(RiversManager.class).start();
+        injector.getInstance(SnapshotsService.class).start();
+        injector.getInstance(TransportService.class).start();
+        injector.getInstance(ClusterService.class).start();
+        injector.getInstance(RoutingService.class).start();
+        injector.getInstance(SearchService.class).start();
+        injector.getInstance(MonitorService.class).start();
+        injector.getInstance(RestController.class).start();
+        DiscoveryService discoService = injector.getInstance(DiscoveryService.class).start();
+        discoService.waitForInitialState();
+
+        // gateway should start after disco, so it can try and recovery from gateway on "start"
+        injector.getInstance(GatewayService.class).start();
+
+        if (settings.getAsBoolean("http.enabled", true)) {
+            injector.getInstance(HttpServer.class).start();
+        }
+        injector.getInstance(ResourceWatcherService.class).start();
+        injector.getInstance(TribeService.class).start();
+
+        logger.info("started");
+
+        return this;
+    }
+
+    public Node stop() {
+        if (!lifecycle.moveToStopped()) {
+            return this;
+        }
+        ESLogger logger = Loggers.getLogger(Node.class, settings.get("name"));
+        logger.info("stopping ...");
+
+        injector.getInstance(TribeService.class).stop();
+        injector.getInstance(ResourceWatcherService.class).stop();
+        if (settings.getAsBoolean("http.enabled", true)) {
+            injector.getInstance(HttpServer.class).stop();
+        }
+
+        injector.getInstance(MappingUpdatedAction.class).stop();
+        injector.getInstance(RiversManager.class).stop();
+
+        injector.getInstance(SnapshotsService.class).stop();
+        // stop any changes happening as a result of cluster state changes
+        injector.getInstance(IndicesClusterStateService.class).stop();
+        // we close indices first, so operations won't be allowed on it
+        injector.getInstance(IndexingMemoryController.class).stop();
+        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();
+        injector.getInstance(RestController.class).stop();
+        injector.getInstance(TransportService.class).stop();
+
+        for (Class<? extends LifecycleComponent> plugin : pluginsService.services()) {
+            injector.getInstance(plugin).stop();
+        }
+        // we should stop this last since it waits for resources to get released
+        // if we had scroll searchers etc or recovery going on we wait for to finish.
+        injector.getInstance(IndicesService.class).stop();
+        logger.info("stopped");
+
+        return this;
+    }
+
+    // During concurrent close() calls we want to make sure that all of them return after the node has completed it's shutdown cycle.
+    // If not, the hook that is added in Bootstrap#setup() will be useless: close() might not be executed, in case another (for example api) call
+    // to close() has already set some lifecycles to stopped. In this case the process will be terminated even if the first call to close() has not finished yet.
     @Override
-    void close();
+    public synchronized void close() {
+        if (lifecycle.started()) {
+            stop();
+        }
+        if (!lifecycle.moveToClosed()) {
+            return;
+        }
+
+        ESLogger logger = Loggers.getLogger(Node.class, settings.get("name"));
+        logger.info("closing ...");
+
+        StopWatch stopWatch = new StopWatch("node_close");
+        stopWatch.start("tribe");
+        injector.getInstance(TribeService.class).close();
+        stopWatch.stop().start("http");
+        if (settings.getAsBoolean("http.enabled", true)) {
+            injector.getInstance(HttpServer.class).close();
+        }
+
+        stopWatch.stop().start("rivers");
+        injector.getInstance(RiversManager.class).close();
+
+        stopWatch.stop().start("snapshot_service");
+        injector.getInstance(SnapshotsService.class).close();
+        stopWatch.stop().start("client");
+        Releasables.close(injector.getInstance(Client.class));
+        stopWatch.stop().start("indices_cluster");
+        injector.getInstance(IndicesClusterStateService.class).close();
+        stopWatch.stop().start("indices");
+        injector.getInstance(IndicesFilterCache.class).close();
+        injector.getInstance(IndicesFieldDataCache.class).close();
+        injector.getInstance(IndexingMemoryController.class).close();
+        injector.getInstance(IndicesTTLService.class).close();
+        injector.getInstance(IndicesService.class).close();
+        injector.getInstance(IndicesStore.class).close();
+        stopWatch.stop().start("routing");
+        injector.getInstance(RoutingService.class).close();
+        stopWatch.stop().start("cluster");
+        injector.getInstance(ClusterService.class).close();
+        stopWatch.stop().start("discovery");
+        injector.getInstance(DiscoveryService.class).close();
+        stopWatch.stop().start("monitor");
+        injector.getInstance(MonitorService.class).close();
+        stopWatch.stop().start("gateway");
+        injector.getInstance(GatewayService.class).close();
+        stopWatch.stop().start("search");
+        injector.getInstance(SearchService.class).close();
+        stopWatch.stop().start("rest");
+        injector.getInstance(RestController.class).close();
+        stopWatch.stop().start("transport");
+        injector.getInstance(TransportService.class).close();
+        stopWatch.stop().start("percolator_service");
+        injector.getInstance(PercolatorService.class).close();
+
+        for (Class<? extends LifecycleComponent> plugin : pluginsService.services()) {
+            stopWatch.stop().start("plugin(" + plugin.getName() + ")");
+            injector.getInstance(plugin).close();
+        }
+
+        stopWatch.stop().start("script");
+        injector.getInstance(ScriptService.class).close();
+
+        stopWatch.stop().start("thread_pool");
+        // TODO this should really use ThreadPool.terminate()
+        injector.getInstance(ThreadPool.class).shutdown();
+        try {
+            injector.getInstance(ThreadPool.class).awaitTermination(10, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+            // ignore
+        }
+        stopWatch.stop().start("thread_pool_force_shutdown");
+        try {
+            injector.getInstance(ThreadPool.class).shutdownNow();
+        } catch (Exception e) {
+            // ignore
+        }
+        stopWatch.stop();
+
+        if (logger.isTraceEnabled()) {
+            logger.trace("Close times for each service:\n{}", stopWatch.prettyPrint());
+        }
+
+        injector.getInstance(NodeEnvironment.class).close();
+        injector.getInstance(PageCacheRecycler.class).close();
+
+        logger.info("closed");
+    }
+
 
     /**
      * Returns <tt>true</tt> if the node is closed.
      */
-    boolean isClosed();
+    public boolean isClosed() {
+        return lifecycle.closed();
+    }
+
+    public Injector injector() {
+        return this.injector;
+    }
+
+    public static void main(String[] args) throws Exception {
+        final Node node = new Node();
+        node.start();
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                node.close();
+            }
+        });
+    }
 }

+ 1 - 2
src/main/java/org/elasticsearch/node/NodeBuilder.java

@@ -21,7 +21,6 @@ package org.elasticsearch.node;
 
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.internal.InternalNode;
 
 /**
  * A node builder is used to construct a {@link Node} instance.
@@ -156,7 +155,7 @@ public class NodeBuilder {
      * Builds the node without starting it.
      */
     public Node build() {
-        return new InternalNode(settings.build(), loadConfigSettings);
+        return new Node(settings.build(), loadConfigSettings);
     }
 
     /**

+ 0 - 422
src/main/java/org/elasticsearch/node/internal/InternalNode.java

@@ -1,422 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.node.internal;
-
-import org.elasticsearch.Build;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.ElasticsearchIllegalStateException;
-import org.elasticsearch.Version;
-import org.elasticsearch.action.ActionModule;
-import org.elasticsearch.action.bench.BenchmarkModule;
-import org.elasticsearch.cache.recycler.PageCacheRecycler;
-import org.elasticsearch.cache.recycler.PageCacheRecyclerModule;
-import org.elasticsearch.client.Client;
-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.action.index.MappingUpdatedAction;
-import org.elasticsearch.cluster.routing.RoutingService;
-import org.elasticsearch.cluster.routing.allocation.AllocationService;
-import org.elasticsearch.common.StopWatch;
-import org.elasticsearch.common.collect.Tuple;
-import org.elasticsearch.common.component.Lifecycle;
-import org.elasticsearch.common.component.LifecycleComponent;
-import org.elasticsearch.common.compress.CompressorFactory;
-import org.elasticsearch.common.inject.Injector;
-import org.elasticsearch.common.inject.Injectors;
-import org.elasticsearch.common.inject.ModulesBuilder;
-import org.elasticsearch.common.lease.Releasables;
-import org.elasticsearch.common.logging.ESLogger;
-import org.elasticsearch.common.logging.Loggers;
-import org.elasticsearch.common.network.NetworkModule;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.settings.SettingsModule;
-import org.elasticsearch.common.util.BigArraysModule;
-import org.elasticsearch.discovery.Discovery;
-import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.DiscoveryService;
-import org.elasticsearch.env.Environment;
-import org.elasticsearch.env.EnvironmentModule;
-import org.elasticsearch.env.NodeEnvironment;
-import org.elasticsearch.env.NodeEnvironmentModule;
-import org.elasticsearch.gateway.GatewayModule;
-import org.elasticsearch.gateway.GatewayService;
-import org.elasticsearch.http.HttpServer;
-import org.elasticsearch.http.HttpServerModule;
-import org.elasticsearch.index.search.shape.ShapeModule;
-import org.elasticsearch.indices.IndicesModule;
-import org.elasticsearch.indices.IndicesService;
-import org.elasticsearch.indices.breaker.CircuitBreakerModule;
-import org.elasticsearch.indices.cache.filter.IndicesFilterCache;
-import org.elasticsearch.indices.cluster.IndicesClusterStateService;
-import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
-import org.elasticsearch.indices.memory.IndexingMemoryController;
-import org.elasticsearch.indices.store.IndicesStore;
-import org.elasticsearch.indices.ttl.IndicesTTLService;
-import org.elasticsearch.monitor.MonitorModule;
-import org.elasticsearch.monitor.MonitorService;
-import org.elasticsearch.monitor.jvm.JvmInfo;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.percolator.PercolatorModule;
-import org.elasticsearch.percolator.PercolatorService;
-import org.elasticsearch.plugins.PluginsModule;
-import org.elasticsearch.plugins.PluginsService;
-import org.elasticsearch.repositories.RepositoriesModule;
-import org.elasticsearch.rest.RestController;
-import org.elasticsearch.rest.RestModule;
-import org.elasticsearch.river.RiversManager;
-import org.elasticsearch.river.RiversModule;
-import org.elasticsearch.script.ScriptModule;
-import org.elasticsearch.script.ScriptService;
-import org.elasticsearch.search.SearchModule;
-import org.elasticsearch.search.SearchService;
-import org.elasticsearch.snapshots.SnapshotsService;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.threadpool.ThreadPoolModule;
-import org.elasticsearch.transport.TransportModule;
-import org.elasticsearch.transport.TransportService;
-import org.elasticsearch.tribe.TribeModule;
-import org.elasticsearch.tribe.TribeService;
-import org.elasticsearch.watcher.ResourceWatcherModule;
-import org.elasticsearch.watcher.ResourceWatcherService;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.concurrent.TimeUnit;
-
-import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
-
-/**
- *
- */
-public final class InternalNode implements Node {
-
-    private static final String CLIENT_TYPE = "node";
-    public static final String HTTP_ENABLED = "http.enabled";
-
-
-    private final Lifecycle lifecycle = new Lifecycle();
-    private final Injector injector;
-    private final Settings settings;
-    private final Environment environment;
-    private final PluginsService pluginsService;
-    private final Client client;
-
-    public InternalNode() throws ElasticsearchException {
-        this(ImmutableSettings.Builder.EMPTY_SETTINGS, true);
-    }
-
-    public InternalNode(Settings preparedSettings, boolean loadConfigSettings) throws ElasticsearchException {
-        final Settings pSettings = settingsBuilder().put(preparedSettings)
-                .put(Client.CLIENT_TYPE_SETTING, CLIENT_TYPE).build();
-        Tuple<Settings, Environment> tuple = InternalSettingsPreparer.prepareSettings(pSettings, loadConfigSettings);
-        tuple = new Tuple<>(TribeService.processSettings(tuple.v1()), tuple.v2());
-
-        // The only place we can actually fake the version a node is running on:
-        Version version = pSettings.getAsVersion("tests.mock.version", Version.CURRENT);
-
-        ESLogger logger = Loggers.getLogger(Node.class, tuple.v1().get("name"));
-        logger.info("version[{}], pid[{}], build[{}/{}]", version, JvmInfo.jvmInfo().pid(), Build.CURRENT.hashShort(), Build.CURRENT.timestamp());
-
-        logger.info("initializing ...");
-
-        if (logger.isDebugEnabled()) {
-            Environment env = tuple.v2();
-            logger.debug("using home [{}], config [{}], data [{}], logs [{}], work [{}], plugins [{}]",
-                    env.homeFile(), env.configFile(), Arrays.toString(env.dataFiles()), env.logsFile(),
-                    env.workFile(), env.pluginsFile());
-        }
-
-        this.pluginsService = new PluginsService(tuple.v1(), tuple.v2());
-        this.settings = pluginsService.updatedSettings();
-        // create the environment based on the finalized (processed) view of the settings
-        this.environment = new Environment(this.settings());
-
-        CompressorFactory.configure(settings);
-        final NodeEnvironment nodeEnvironment;
-        try {
-            nodeEnvironment = new NodeEnvironment(this.settings, this.environment);
-        } catch (IOException ex) {
-            throw new ElasticsearchIllegalStateException("Failed to created node environment", ex);
-        }
-
-        boolean success = false;
-        try {
-            ModulesBuilder modules = new ModulesBuilder();
-            modules.add(new Version.Module(version));
-            modules.add(new PageCacheRecyclerModule(settings));
-            modules.add(new CircuitBreakerModule(settings));
-            modules.add(new BigArraysModule(settings));
-            modules.add(new PluginsModule(settings, pluginsService));
-            modules.add(new SettingsModule(settings));
-            modules.add(new NodeModule(this));
-            modules.add(new NetworkModule());
-            modules.add(new ScriptModule(settings));
-            modules.add(new EnvironmentModule(environment));
-            modules.add(new NodeEnvironmentModule(nodeEnvironment));
-            modules.add(new ClusterNameModule(settings));
-            modules.add(new ThreadPoolModule(settings));
-            modules.add(new DiscoveryModule(settings));
-            modules.add(new ClusterModule(settings));
-            modules.add(new RestModule(settings));
-            modules.add(new TransportModule(settings));
-            if (settings.getAsBoolean(HTTP_ENABLED, true)) {
-                modules.add(new HttpServerModule(settings));
-            }
-            modules.add(new RiversModule(settings));
-            modules.add(new IndicesModule(settings));
-            modules.add(new SearchModule());
-            modules.add(new ActionModule(false));
-            modules.add(new MonitorModule(settings));
-            modules.add(new GatewayModule());
-            modules.add(new NodeClientModule());
-            modules.add(new ShapeModule());
-            modules.add(new PercolatorModule());
-            modules.add(new ResourceWatcherModule());
-            modules.add(new RepositoriesModule());
-            modules.add(new TribeModule());
-            modules.add(new BenchmarkModule(settings));
-
-            injector = modules.createInjector();
-
-            client = injector.getInstance(Client.class);
-            success = true;
-        } finally {
-            if (!success) {
-                nodeEnvironment.close();
-            }
-        }
-
-        logger.info("initialized");
-    }
-
-    @Override
-    public Settings settings() {
-        return this.settings;
-    }
-
-    @Override
-    public Client client() {
-        return client;
-    }
-
-    @Override
-    public Node start() {
-        if (!lifecycle.moveToStarted()) {
-            return this;
-        }
-
-        ESLogger logger = Loggers.getLogger(Node.class, settings.get("name"));
-        logger.info("starting ...");
-
-        // hack around dependency injection problem (for now...)
-        injector.getInstance(Discovery.class).setAllocationService(injector.getInstance(AllocationService.class));
-
-        for (Class<? extends LifecycleComponent> plugin : pluginsService.services()) {
-            injector.getInstance(plugin).start();
-        }
-
-        injector.getInstance(MappingUpdatedAction.class).start();
-        injector.getInstance(IndicesService.class).start();
-        injector.getInstance(IndexingMemoryController.class).start();
-        injector.getInstance(IndicesClusterStateService.class).start();
-        injector.getInstance(IndicesTTLService.class).start();
-        injector.getInstance(RiversManager.class).start();
-        injector.getInstance(SnapshotsService.class).start();
-        injector.getInstance(TransportService.class).start();
-        injector.getInstance(ClusterService.class).start();
-        injector.getInstance(RoutingService.class).start();
-        injector.getInstance(SearchService.class).start();
-        injector.getInstance(MonitorService.class).start();
-        injector.getInstance(RestController.class).start();
-        DiscoveryService discoService = injector.getInstance(DiscoveryService.class).start();
-        discoService.waitForInitialState();
-
-        // gateway should start after disco, so it can try and recovery from gateway on "start"
-        injector.getInstance(GatewayService.class).start();
-
-        if (settings.getAsBoolean("http.enabled", true)) {
-            injector.getInstance(HttpServer.class).start();
-        }
-        injector.getInstance(ResourceWatcherService.class).start();
-        injector.getInstance(TribeService.class).start();
-
-        logger.info("started");
-
-        return this;
-    }
-
-    @Override
-    public Node stop() {
-        if (!lifecycle.moveToStopped()) {
-            return this;
-        }
-        ESLogger logger = Loggers.getLogger(Node.class, settings.get("name"));
-        logger.info("stopping ...");
-
-        injector.getInstance(TribeService.class).stop();
-        injector.getInstance(ResourceWatcherService.class).stop();
-        if (settings.getAsBoolean("http.enabled", true)) {
-            injector.getInstance(HttpServer.class).stop();
-        }
-
-        injector.getInstance(MappingUpdatedAction.class).stop();
-        injector.getInstance(RiversManager.class).stop();
-
-        injector.getInstance(SnapshotsService.class).stop();
-        // stop any changes happening as a result of cluster state changes
-        injector.getInstance(IndicesClusterStateService.class).stop();
-        // we close indices first, so operations won't be allowed on it
-        injector.getInstance(IndexingMemoryController.class).stop();
-        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();
-        injector.getInstance(RestController.class).stop();
-        injector.getInstance(TransportService.class).stop();
-
-        for (Class<? extends LifecycleComponent> plugin : pluginsService.services()) {
-            injector.getInstance(plugin).stop();
-        }
-        // we should stop this last since it waits for resources to get released
-        // if we had scroll searchers etc or recovery going on we wait for to finish.
-        injector.getInstance(IndicesService.class).stop();
-        logger.info("stopped");
-
-        return this;
-    }
-
-    // During concurrent close() calls we want to make sure that all of them return after the node has completed it's shutdown cycle.
-    // If not, the hook that is added in Bootstrap#setup() will be useless: close() might not be executed, in case another (for example api) call
-    // to close() has already set some lifecycles to stopped. In this case the process will be terminated even if the first call to close() has not finished yet.
-    @Override
-    public synchronized void close() {
-        if (lifecycle.started()) {
-            stop();
-        }
-        if (!lifecycle.moveToClosed()) {
-            return;
-        }
-
-        ESLogger logger = Loggers.getLogger(Node.class, settings.get("name"));
-        logger.info("closing ...");
-
-        StopWatch stopWatch = new StopWatch("node_close");
-        stopWatch.start("tribe");
-        injector.getInstance(TribeService.class).close();
-        stopWatch.stop().start("http");
-        if (settings.getAsBoolean("http.enabled", true)) {
-            injector.getInstance(HttpServer.class).close();
-        }
-
-        stopWatch.stop().start("rivers");
-        injector.getInstance(RiversManager.class).close();
-
-        stopWatch.stop().start("snapshot_service");
-        injector.getInstance(SnapshotsService.class).close();
-        stopWatch.stop().start("client");
-        Releasables.close(injector.getInstance(Client.class));
-        stopWatch.stop().start("indices_cluster");
-        injector.getInstance(IndicesClusterStateService.class).close();
-        stopWatch.stop().start("indices");
-        injector.getInstance(IndicesFilterCache.class).close();
-        injector.getInstance(IndicesFieldDataCache.class).close();
-        injector.getInstance(IndexingMemoryController.class).close();
-        injector.getInstance(IndicesTTLService.class).close();
-        injector.getInstance(IndicesService.class).close();
-        injector.getInstance(IndicesStore.class).close();
-        stopWatch.stop().start("routing");
-        injector.getInstance(RoutingService.class).close();
-        stopWatch.stop().start("cluster");
-        injector.getInstance(ClusterService.class).close();
-        stopWatch.stop().start("discovery");
-        injector.getInstance(DiscoveryService.class).close();
-        stopWatch.stop().start("monitor");
-        injector.getInstance(MonitorService.class).close();
-        stopWatch.stop().start("gateway");
-        injector.getInstance(GatewayService.class).close();
-        stopWatch.stop().start("search");
-        injector.getInstance(SearchService.class).close();
-        stopWatch.stop().start("rest");
-        injector.getInstance(RestController.class).close();
-        stopWatch.stop().start("transport");
-        injector.getInstance(TransportService.class).close();
-        stopWatch.stop().start("percolator_service");
-        injector.getInstance(PercolatorService.class).close();
-
-        for (Class<? extends LifecycleComponent> plugin : pluginsService.services()) {
-            stopWatch.stop().start("plugin(" + plugin.getName() + ")");
-            injector.getInstance(plugin).close();
-        }
-
-        stopWatch.stop().start("script");
-        injector.getInstance(ScriptService.class).close();
-
-        stopWatch.stop().start("thread_pool");
-        // TODO this should really use ThreadPool.terminate()
-        injector.getInstance(ThreadPool.class).shutdown();
-        try {
-            injector.getInstance(ThreadPool.class).awaitTermination(10, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-            // ignore
-        }
-        stopWatch.stop().start("thread_pool_force_shutdown");
-        try {
-            injector.getInstance(ThreadPool.class).shutdownNow();
-        } catch (Exception e) {
-            // ignore
-        }
-        stopWatch.stop();
-
-        if (logger.isTraceEnabled()) {
-            logger.trace("Close times for each service:\n{}", stopWatch.prettyPrint());
-        }
-
-        injector.getInstance(NodeEnvironment.class).close();
-        injector.getInstance(PageCacheRecycler.class).close();
-
-        logger.info("closed");
-    }
-
-    @Override
-    public boolean isClosed() {
-        return lifecycle.closed();
-    }
-
-    public Injector injector() {
-        return this.injector;
-    }
-
-    public static void main(String[] args) throws Exception {
-        final InternalNode node = new InternalNode();
-        node.start();
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                node.close();
-            }
-        });
-    }
-}

+ 10 - 10
src/main/java/org/elasticsearch/tribe/TribeService.java

@@ -44,8 +44,8 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.gateway.GatewayService;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.node.NodeBuilder;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.rest.RestStatus;
 
 import java.util.EnumSet;
@@ -115,7 +115,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
     private final String onConflict;
     private final Set<String> droppedIndices = ConcurrentCollections.newConcurrentSet();
 
-    private final List<InternalNode> nodes = Lists.newCopyOnWriteArrayList();
+    private final List<Node> nodes = Lists.newCopyOnWriteArrayList();
 
     @Inject
     public TribeService(Settings settings, ClusterService clusterService, DiscoveryService discoveryService) {
@@ -131,7 +131,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
             if (sb.get("http.enabled") == null) {
                 sb.put("http.enabled", false);
             }
-            nodes.add((InternalNode) NodeBuilder.nodeBuilder().settings(sb).client(true).build());
+            nodes.add(NodeBuilder.nodeBuilder().settings(sb).client(true).build());
         }
 
         String[] blockIndicesWrite = Strings.EMPTY_ARRAY;
@@ -151,7 +151,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
             }
             blockIndicesMetadata = settings.getAsArray("tribe.blocks.metadata.indices", Strings.EMPTY_ARRAY);
             blockIndicesRead = settings.getAsArray("tribe.blocks.read.indices", Strings.EMPTY_ARRAY);
-            for (InternalNode node : nodes) {
+            for (Node node : nodes) {
                 node.injector().getInstance(ClusterService.class).add(new TribeClusterStateListener(node));
             }
         }
@@ -164,12 +164,12 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
 
     @Override
     protected void doStart() throws ElasticsearchException {
-        for (InternalNode node : nodes) {
+        for (Node node : nodes) {
             try {
                 node.start();
             } catch (Throwable e) {
                 // calling close is safe for non started nodes, we can just iterate over all
-                for (InternalNode otherNode : nodes) {
+                for (Node otherNode : nodes) {
                     try {
                         otherNode.close();
                     } catch (Throwable t) {
@@ -186,7 +186,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
 
     @Override
     protected void doStop() throws ElasticsearchException {
-        for (InternalNode node : nodes) {
+        for (Node node : nodes) {
             try {
                 node.stop();
             } catch (Throwable t) {
@@ -197,7 +197,7 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
 
     @Override
     protected void doClose() throws ElasticsearchException {
-        for (InternalNode node : nodes) {
+        for (Node node : nodes) {
             try {
                 node.close();
             } catch (Throwable t) {
@@ -208,10 +208,10 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
 
     class TribeClusterStateListener implements ClusterStateListener {
 
-        private final InternalNode tribeNode;
+        private final Node tribeNode;
         private final String tribeName;
 
-        TribeClusterStateListener(InternalNode tribeNode) {
+        TribeClusterStateListener(Node tribeNode) {
             this.tribeNode = tribeNode;
             this.tribeName = tribeNode.settings().get(TRIBE_NAME);
         }

+ 3 - 3
src/test/java/org/elasticsearch/benchmark/search/aggregations/GlobalOrdinalsBenchmark.java

@@ -36,7 +36,7 @@ import org.elasticsearch.common.unit.SizeValue;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.indices.IndexAlreadyExistsException;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
 import org.elasticsearch.transport.TransportModule;
 
@@ -63,7 +63,7 @@ public class GlobalOrdinalsBenchmark {
     private static final boolean USE_DOC_VALUES = false;
 
     static long COUNT = SizeValue.parseSizeValue("5m").singles();
-    static InternalNode node;
+    static Node node;
     static Client client;
 
     public static void main(String[] args) throws Exception {
@@ -79,7 +79,7 @@ public class GlobalOrdinalsBenchmark {
                 .build();
 
         String clusterName = GlobalOrdinalsBenchmark.class.getSimpleName();
-        node = (InternalNode) nodeBuilder().clusterName(clusterName)
+        node = nodeBuilder().clusterName(clusterName)
                     .settings(settingsBuilder().put(settings))
                     .node();
 

+ 3 - 4
src/test/java/org/elasticsearch/benchmark/search/aggregations/SubAggregationSearchCollectModeBenchmark.java

@@ -39,7 +39,6 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.node.Node;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 
@@ -70,7 +69,7 @@ public class SubAggregationSearchCollectModeBenchmark {
     static int STRING_TERM_SIZE = 5;
 
     static Client client;
-    static InternalNode[] nodes;
+    static Node[] nodes;
 
     public static void main(String[] args) throws Exception {
         Natives.tryMlockall();
@@ -83,9 +82,9 @@ public class SubAggregationSearchCollectModeBenchmark {
                 .build();
 
         String clusterName = SubAggregationSearchCollectModeBenchmark.class.getSimpleName();
-        nodes = new InternalNode[1];
+        nodes = new Node[1];
         for (int i = 0; i < nodes.length; i++) {
-            nodes[i] = (InternalNode) nodeBuilder().clusterName(clusterName)
+            nodes[i] = nodeBuilder().clusterName(clusterName)
                     .settings(settingsBuilder().put(settings).put("name", "node" + i))
                     .node();
         }

+ 5 - 5
src/test/java/org/elasticsearch/benchmark/search/aggregations/TermsAggregationSearchAndIndexingBenchmark.java

@@ -37,7 +37,7 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.indices.IndexAlreadyExistsException;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -69,7 +69,7 @@ public class TermsAggregationSearchAndIndexingBenchmark {
     static int NUMBER_OF_MULTI_VALUE_TERMS = 10;
     static int STRING_TERM_SIZE = 5;
 
-    static InternalNode[] nodes;
+    static Node[] nodes;
 
     public static void main(String[] args) throws Exception {
         Natives.tryMlockall();
@@ -80,9 +80,9 @@ public class TermsAggregationSearchAndIndexingBenchmark {
                 .build();
 
         String clusterName = TermsAggregationSearchAndIndexingBenchmark.class.getSimpleName();
-        nodes = new InternalNode[1];
+        nodes = new Node[1];
         for (int i = 0; i < nodes.length; i++) {
-            nodes[i] = (InternalNode) nodeBuilder().settings(settingsBuilder().put(settings).put("name", "node1"))
+            nodes[i] = nodeBuilder().settings(settingsBuilder().put(settings).put("name", "node1"))
                     .clusterName(clusterName)
                     .node();
         }
@@ -197,7 +197,7 @@ public class TermsAggregationSearchAndIndexingBenchmark {
         System.out.println("----------------------------------------- SUMMARY ----------------------------------------------");
 
         client.close();
-        for (InternalNode node : nodes) {
+        for (Node node : nodes) {
             node.close();
         }
     }

+ 3 - 4
src/test/java/org/elasticsearch/benchmark/search/aggregations/TermsAggregationSearchBenchmark.java

@@ -40,7 +40,6 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.node.Node;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
 import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
 
@@ -71,7 +70,7 @@ public class TermsAggregationSearchBenchmark {
     static int STRING_TERM_SIZE = 5;
 
     static Client client;
-    static InternalNode[] nodes;
+    static Node[] nodes;
 
     public enum Method {
         AGGREGATION {
@@ -111,9 +110,9 @@ public class TermsAggregationSearchBenchmark {
                 .build();
 
         String clusterName = TermsAggregationSearchBenchmark.class.getSimpleName();
-        nodes = new InternalNode[1];
+        nodes = new Node[1];
         for (int i = 0; i < nodes.length; i++) {
-            nodes[i] = (InternalNode) nodeBuilder().clusterName(clusterName)
+            nodes[i] = nodeBuilder().clusterName(clusterName)
                     .settings(settingsBuilder().put(settings).put("name", "node" + i))
                     .node();
         }

+ 3 - 3
src/test/java/org/elasticsearch/bwcompat/OldIndexBackwardsCompatibilityTests.java

@@ -28,7 +28,7 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.index.merge.policy.MergePolicyModule;
 import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.rest.action.admin.indices.upgrade.UpgradeTest;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.sort.SortOrder;
@@ -134,7 +134,7 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
 
     void assertOldIndexWorks(String index) throws Exception {
         Settings settings = ImmutableSettings.builder()
-            .put(InternalNode.HTTP_ENABLED, true) // for _upgrade
+            .put(Node.HTTP_ENABLED, true) // for _upgrade
                 .put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class) // disable merging so no segments will be upgraded
                 .build();
         loadIndex(index, settings);
@@ -192,7 +192,7 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
             internalCluster().startNode(ImmutableSettings.builder()
                 .put("data.node", true)
                 .put("master.node", false)
-                .put(InternalNode.HTTP_ENABLED, true) // for _upgrade
+                .put(Node.HTTP_ENABLED, true) // for _upgrade
                 .build());
         }
         client().admin().cluster().prepareHealth("test").setWaitForNodes("" + (numReplicas + 1));

+ 1 - 2
src/test/java/org/elasticsearch/client/transport/TransportClientTests.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.node.Node;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
 import org.elasticsearch.transport.TransportService;
@@ -62,7 +61,7 @@ public class TransportClientTests extends ElasticsearchIntegrationTest {
                 .build()).clusterName("foobar").build();
         node.start();
         try {
-            TransportAddress transportAddress = ((InternalNode) node).injector().getInstance(TransportService.class).boundAddress().publishAddress();
+            TransportAddress transportAddress = node.injector().getInstance(TransportService.class).boundAddress().publishAddress();
             client.addTransportAddress(transportAddress);
             assertThat(nodeService.connectedNodes().size(), greaterThanOrEqualTo(1)); // since we force transport clients there has to be one node started that we connect to.
             for (DiscoveryNode discoveryNode : nodeService.connectedNodes()) {  // connected nodes have updated version

+ 2 - 3
src/test/java/org/elasticsearch/cluster/routing/RoutingBackwardCompatibilityTests.java

@@ -25,9 +25,8 @@ import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
-import org.elasticsearch.cluster.routing.OperationRouting;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchTestCase;
 
 import java.io.BufferedReader;
@@ -37,7 +36,7 @@ import java.util.Arrays;
 public class RoutingBackwardCompatibilityTests extends ElasticsearchTestCase {
 
     public void testBackwardCompatibility() throws Exception {
-        InternalNode node = new InternalNode();
+        Node node = new Node();
         try {
             try (BufferedReader reader = new BufferedReader(new InputStreamReader(RoutingBackwardCompatibilityTests.class.getResourceAsStream("/org/elasticsearch/cluster/routing/shard_routes.txt"), "UTF-8"))) {
                 for (String line = reader.readLine(); line != null; line = reader.readLine()) {

+ 2 - 2
src/test/java/org/elasticsearch/cluster/routing/RoutingBackwardCompatibilityUponUpgradeTests.java

@@ -26,7 +26,7 @@ import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 
@@ -52,7 +52,7 @@ public class RoutingBackwardCompatibilityUponUpgradeTests extends ElasticsearchI
         Settings baseSettings = prepareBackwardsDataDir(zippedIndexDir);
         internalCluster().startNode(ImmutableSettings.builder()
                 .put(baseSettings)
-                .put(InternalNode.HTTP_ENABLED, true)
+                .put(Node.HTTP_ENABLED, true)
                 .build());
         ensureYellow("test");
         GetIndexResponse getIndexResponse = client().admin().indices().prepareGetIndex().get();

+ 2 - 2
src/test/java/org/elasticsearch/http/netty/HttpPublishPortTests.java

@@ -24,7 +24,7 @@ import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.junit.Test;
 
@@ -40,7 +40,7 @@ public class HttpPublishPortTests extends ElasticsearchIntegrationTest {
     protected Settings nodeSettings(int nodeOrdinal) {
         return ImmutableSettings.settingsBuilder()
                 .put(super.nodeSettings(nodeOrdinal))
-                .put(InternalNode.HTTP_ENABLED, true)
+                .put(Node.HTTP_ENABLED, true)
                 .put("http.publish_port", 9080)
                 .build();
     }

+ 2 - 2
src/test/java/org/elasticsearch/http/netty/NettyPipeliningDisabledIntegrationTest.java

@@ -22,7 +22,7 @@ import com.google.common.collect.Lists;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
 import org.elasticsearch.http.HttpServerTransport;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.jboss.netty.handler.codec.http.HttpResponse;
 import org.junit.Test;
@@ -46,7 +46,7 @@ public class NettyPipeliningDisabledIntegrationTest extends ElasticsearchIntegra
 
     @Override
     protected Settings nodeSettings(int nodeOrdinal) {
-        return settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put(InternalNode.HTTP_ENABLED, true).put("http.pipelining", false).build();
+        return settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put(Node.HTTP_ENABLED, true).put("http.pipelining", false).build();
     }
 
     @Test

+ 2 - 2
src/test/java/org/elasticsearch/http/netty/NettyPipeliningEnabledIntegrationTest.java

@@ -21,7 +21,7 @@ package org.elasticsearch.http.netty;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
 import org.elasticsearch.http.HttpServerTransport;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.jboss.netty.handler.codec.http.HttpResponse;
 import org.junit.Test;
@@ -44,7 +44,7 @@ public class NettyPipeliningEnabledIntegrationTest extends ElasticsearchIntegrat
 
     @Override
     protected Settings nodeSettings(int nodeOrdinal) {
-        return settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put(InternalNode.HTTP_ENABLED, true).put("http.pipelining", true).build();
+        return settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put(Node.HTTP_ENABLED, true).put("http.pipelining", true).build();
     }
 
     @Test

+ 2 - 4
src/test/java/org/elasticsearch/network/DirectBufferNetworkTests.java

@@ -20,9 +20,7 @@
 package org.elasticsearch.network;
 
 import org.apache.http.impl.client.HttpClients;
-import org.apache.lucene.util.TestUtil;
 import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
@@ -30,7 +28,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.http.HttpServerTransport;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
 import org.hamcrest.Matchers;
@@ -50,7 +48,7 @@ public class DirectBufferNetworkTests extends ElasticsearchIntegrationTest {
     @Override
     protected Settings nodeSettings(int nodeOrdinal) {
         return ImmutableSettings.builder()
-            .put(InternalNode.HTTP_ENABLED, true)
+            .put(Node.HTTP_ENABLED, true)
             .put(super.nodeSettings(nodeOrdinal)).build();
     }
 

+ 3 - 3
src/test/java/org/elasticsearch/options/jsonp/JsonpOptionDisabledTest.java

@@ -23,7 +23,7 @@ import org.apache.http.impl.client.HttpClients;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.http.HttpServerTransport;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
@@ -47,11 +47,11 @@ public class JsonpOptionDisabledTest extends ElasticsearchIntegrationTest {
             logger.info("using default jsonp settings (should be false)");
             return ImmutableSettings.settingsBuilder()
                     .put(super.nodeSettings(nodeOrdinal))
-                    .put(InternalNode.HTTP_ENABLED, true).build();
+                    .put(Node.HTTP_ENABLED, true).build();
         }
         return ImmutableSettings.settingsBuilder()
                 .put(super.nodeSettings(nodeOrdinal))
-                .put(InternalNode.HTTP_ENABLED, true)
+                .put(Node.HTTP_ENABLED, true)
                 .put(RestController.HTTP_JSON_ENABLE, false)
                 .build();
     }

+ 2 - 2
src/test/java/org/elasticsearch/options/jsonp/JsonpOptionEnabledTest.java

@@ -23,7 +23,7 @@ import org.apache.http.impl.client.HttpClients;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.http.HttpServerTransport;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.rest.RestController;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
@@ -45,7 +45,7 @@ public class JsonpOptionEnabledTest extends ElasticsearchIntegrationTest {
         return ImmutableSettings.settingsBuilder()
                 .put(super.nodeSettings(nodeOrdinal))
                 .put(RestController.HTTP_JSON_ENABLE, true)
-                .put(InternalNode.HTTP_ENABLED, true)
+                .put(Node.HTTP_ENABLED, true)
                 .build();
     }
 

+ 2 - 2
src/test/java/org/elasticsearch/rest/CorsRegexDefaultTests.java

@@ -20,7 +20,7 @@ package org.elasticsearch.rest;
 
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.rest.client.http.HttpResponse;
 import org.junit.Test;
@@ -36,7 +36,7 @@ public class CorsRegexDefaultTests extends ElasticsearchIntegrationTest {
     @Override
     protected Settings nodeSettings(int nodeOrdinal) {
         return ImmutableSettings.builder()
-            .put(InternalNode.HTTP_ENABLED, true)
+            .put(Node.HTTP_ENABLED, true)
             .put(super.nodeSettings(nodeOrdinal)).build();
     }
 

+ 2 - 2
src/test/java/org/elasticsearch/rest/CorsRegexTests.java

@@ -25,7 +25,7 @@ import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
 import org.elasticsearch.http.HttpServerTransport;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
 import org.elasticsearch.test.rest.client.http.HttpResponse;
@@ -56,7 +56,7 @@ public class CorsRegexTests extends ElasticsearchIntegrationTest {
                 .put(SETTING_CORS_ALLOW_ORIGIN, "/https?:\\/\\/localhost(:[0-9]+)?/")
                 .put(SETTING_CORS_ALLOW_CREDENTIALS, true)
                 .put(SETTING_CORS_ENABLED, true)
-                .put(InternalNode.HTTP_ENABLED, true)
+                .put(Node.HTTP_ENABLED, true)
                 .build();
     }
 

+ 2 - 2
src/test/java/org/elasticsearch/rest/action/admin/indices/upgrade/UpgradeTest.java

@@ -35,7 +35,7 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.engine.Segment;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
@@ -278,6 +278,6 @@ public class UpgradeTest extends ElasticsearchBackwardsCompatIntegrationTest {
     @Override
     protected Settings nodeSettings(int nodeOrdinal) {
         return ImmutableSettings.builder().put(super.nodeSettings(nodeOrdinal))
-            .put(InternalNode.HTTP_ENABLED, true).build();
+            .put(Node.HTTP_ENABLED, true).build();
     }
 }

+ 2 - 2
src/test/java/org/elasticsearch/stresstest/leaks/GenericStatsLeak.java

@@ -24,13 +24,13 @@ import org.elasticsearch.monitor.jvm.JvmService;
 import org.elasticsearch.monitor.network.NetworkService;
 import org.elasticsearch.monitor.os.OsService;
 import org.elasticsearch.monitor.process.ProcessService;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.node.NodeBuilder;
-import org.elasticsearch.node.internal.InternalNode;
 
 public class GenericStatsLeak {
 
     public static void main(String[] args) {
-        InternalNode node = (InternalNode) NodeBuilder.nodeBuilder().settings(ImmutableSettings.settingsBuilder()
+        Node node = NodeBuilder.nodeBuilder().settings(ImmutableSettings.settingsBuilder()
                 .put("monitor.os.refresh_interval", 0)
                 .put("monitor.process.refresh_interval", 0)
                 .put("monitor.network.refresh_interval", 0)

+ 1 - 3
src/test/java/org/elasticsearch/stresstest/rollingrestart/RollingRestartStressTest.java

@@ -26,7 +26,6 @@ import org.elasticsearch.action.get.GetResponse;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.io.FileSystemUtils;
 import org.elasticsearch.common.logging.ESLogger;
 import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.ImmutableSettings;
@@ -37,7 +36,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.node.NodeBuilder;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.search.SearchHit;
 
 import java.nio.file.Path;
@@ -168,7 +166,7 @@ public class RollingRestartStressTest {
         // start doing the rolling restart
         int nodeIndex = 0;
         while (true) {
-            Path[] nodeData = ((InternalNode) nodes[nodeIndex]).injector().getInstance(NodeEnvironment.class).nodeDataPaths();
+            Path[] nodeData = nodes[nodeIndex].injector().getInstance(NodeEnvironment.class).nodeDataPaths();
             nodes[nodeIndex].close();
             if (clearNodeData) {
                 try {

+ 2 - 6
src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java

@@ -20,7 +20,6 @@ package org.elasticsearch.test;
 
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.carrotsearch.randomizedtesting.Randomness;
-import com.carrotsearch.randomizedtesting.SeedUtils;
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import com.google.common.base.Joiner;
@@ -70,15 +69,12 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Priority;
-import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.io.FileSystemUtils;
 import org.elasticsearch.common.regex.Regex;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.TimeValue;
@@ -109,7 +105,7 @@ import org.elasticsearch.indices.cache.query.IndicesQueryCache;
 import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.indices.store.IndicesStore;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.search.SearchService;
@@ -1625,7 +1621,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
         settingsSource = new SettingsSource() {
             @Override
             public Settings node(int nodeOrdinal) {
-                return ImmutableSettings.builder().put(InternalNode.HTTP_ENABLED, false).
+                return ImmutableSettings.builder().put(Node.HTTP_ENABLED, false).
                         put(nodeSettings(nodeOrdinal)).build();
             }
             

+ 1 - 2
src/test/java/org/elasticsearch/test/ElasticsearchSingleNodeTest.java

@@ -40,7 +40,6 @@ import org.elasticsearch.index.IndexService;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.node.NodeBuilder;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.search.internal.SearchContext;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.junit.*;
@@ -160,7 +159,7 @@ public abstract class ElasticsearchSingleNodeTest extends ElasticsearchTestCase
      * Get an instance for a particular class using the injector of the singleton node.
      */
     protected static <T> T getInstanceFromNode(Class<T> clazz) {
-        return ((InternalNode) NODE).injector().getInstance(clazz);
+        return NODE.injector().getInstance(clazz);
     }
 
     /**

+ 6 - 7
src/test/java/org/elasticsearch/test/InternalTestCluster.java

@@ -83,7 +83,6 @@ import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
 import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.node.Node;
-import org.elasticsearch.node.internal.InternalNode;
 import org.elasticsearch.node.service.NodeService;
 import org.elasticsearch.plugins.PluginsService;
 import org.elasticsearch.search.SearchService;
@@ -731,14 +730,14 @@ public final class InternalTestCluster extends TestCluster {
     }
 
     private final class NodeAndClient implements Closeable {
-        private InternalNode node;
+        private Node node;
         private Client nodeClient;
         private Client transportClient;
         private final AtomicBoolean closed = new AtomicBoolean(false);
         private final String name;
 
         NodeAndClient(String name, Node node) {
-            this.node = (InternalNode) node;
+            this.node = node;
             this.name = name;
         }
 
@@ -825,7 +824,7 @@ public final class InternalTestCluster extends TestCluster {
                     IOUtils.rm(nodeEnv.nodeDataPaths());
                 }
             }
-            node = (InternalNode) nodeBuilder().settings(node.settings()).settings(newSettings).node();
+            node = nodeBuilder().settings(node.settings()).settings(newSettings).node();
         }
 
         void registerDataPath() {
@@ -872,7 +871,7 @@ public final class InternalTestCluster extends TestCluster {
         }
 
         public Client client(Node node, String clusterName) {
-            TransportAddress addr = ((InternalNode) node).injector().getInstance(TransportService.class).boundAddress().publishAddress();
+            TransportAddress addr = node.injector().getInstance(TransportService.class).boundAddress().publishAddress();
             Settings nodeSettings = node.settings();
             Builder builder = settingsBuilder()
                     .put("client.transport.nodes_sampler_interval", "1s")
@@ -1078,7 +1077,7 @@ public final class InternalTestCluster extends TestCluster {
         return getInstance(clazz, Predicates.<NodeAndClient>alwaysTrue());
     }
 
-    private synchronized <T> T getInstanceFromNode(Class<T> clazz, InternalNode node) {
+    private synchronized <T> T getInstanceFromNode(Class<T> clazz, Node node) {
         return node.injector().getInstance(clazz);
     }
 
@@ -1596,7 +1595,7 @@ public final class InternalTestCluster extends TestCluster {
         assertThat(shard, greaterThanOrEqualTo(0));
         assertThat(shard, greaterThanOrEqualTo(0));
         for (NodeAndClient n : nodes.values()) {
-            InternalNode node = (InternalNode) n.node;
+            Node node = n.node;
             IndicesService indicesService = getInstanceFromNode(IndicesService.class, node);
             ClusterService clusterService = getInstanceFromNode(ClusterService.class, node);
             IndexService indexService = indicesService.indexService(index);

+ 2 - 2
src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTests.java

@@ -30,7 +30,7 @@ import org.apache.lucene.util.TimeUnits;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.internal.InternalNode;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
 import org.elasticsearch.test.rest.client.RestException;
@@ -112,7 +112,7 @@ public class ElasticsearchRestTests extends ElasticsearchIntegrationTest {
     @Override
     protected Settings nodeSettings(int nodeOrdinal) {
         return ImmutableSettings.builder()
-            .put(InternalNode.HTTP_ENABLED, true)
+            .put(Node.HTTP_ENABLED, true)
             .put(super.nodeSettings(nodeOrdinal)).build();
     }