Browse Source

Move tribe to a module (#25778)

This commit moves tribe to a module, stripping core from the tribe functionality.
Yannick Welsch 8 years ago
parent
commit
1a01514081
26 changed files with 452 additions and 267 deletions
  1. 1 1
      buildSrc/src/main/resources/checkstyle_suppressions.xml
  2. 6 0
      core/src/main/java/org/elasticsearch/cluster/ClusterModule.java
  3. 43 0
      core/src/main/java/org/elasticsearch/cluster/MergableCustomMetaData.java
  4. 0 9
      core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
  5. 1 20
      core/src/main/java/org/elasticsearch/common/settings/SettingsModule.java
  6. 0 1
      core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java
  7. 4 27
      core/src/main/java/org/elasticsearch/node/Node.java
  8. 7 0
      core/src/main/java/org/elasticsearch/plugins/ClusterPlugin.java
  9. 8 1
      core/src/main/java/org/elasticsearch/plugins/Plugin.java
  10. 7 2
      core/src/test/java/org/elasticsearch/cluster/metadata/TemplateUpgradeServiceIT.java
  11. 0 51
      core/src/test/java/org/elasticsearch/common/settings/SettingsModuleTests.java
  12. 0 40
      core/src/test/java/org/elasticsearch/threadpool/SimpleThreadPoolIT.java
  13. 28 0
      modules/tribe/build.gradle
  14. 3 2
      modules/tribe/src/main/java/org/elasticsearch/tribe/TribeDiscovery.java
  15. 180 0
      modules/tribe/src/main/java/org/elasticsearch/tribe/TribePlugin.java
  16. 22 78
      modules/tribe/src/main/java/org/elasticsearch/tribe/TribeService.java
  17. 54 3
      modules/tribe/src/test/java/org/elasticsearch/tribe/TribeIntegrationTests.java
  18. 24 8
      modules/tribe/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java
  19. 6 7
      plugins/discovery-file/src/main/java/org/elasticsearch/discovery/file/FileBasedDiscoveryPlugin.java
  20. 1 0
      qa/evil-tests/build.gradle
  21. 36 6
      qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java
  22. 1 0
      settings.gradle
  23. 6 7
      test/framework/src/main/java/org/elasticsearch/node/MockNode.java
  24. 12 2
      test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java
  25. 1 1
      test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
  26. 1 1
      test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java

+ 1 - 1
buildSrc/src/main/resources/checkstyle_suppressions.xml

@@ -417,7 +417,7 @@
   <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]SnapshotShardsService.java" checks="LineLength" />
   <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]snapshots[/\\]SnapshotsService.java" checks="LineLength" />
   <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]threadpool[/\\]ThreadPool.java" checks="LineLength" />
-  <suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]tribe[/\\]TribeService.java" checks="LineLength" />
+  <suppress files="modules[/\\]tribe[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]tribe[/\\]TribeService.java" checks="LineLength" />
   <suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]apache[/\\]lucene[/\\]queries[/\\]BlendedTermQueryTests.java" checks="LineLength" />
   <suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]VersionTests.java" checks="LineLength" />
   <suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]RejectionActionIT.java" checks="LineLength" />

+ 6 - 0
core/src/main/java/org/elasticsearch/cluster/ClusterModule.java

@@ -94,6 +94,7 @@ public class ClusterModule extends AbstractModule {
     private final IndexNameExpressionResolver indexNameExpressionResolver;
     private final AllocationDeciders allocationDeciders;
     private final AllocationService allocationService;
+    private final Runnable onStarted;
     // pkg private for tests
     final Collection<AllocationDecider> deciderList;
     final ShardsAllocator shardsAllocator;
@@ -106,6 +107,7 @@ public class ClusterModule extends AbstractModule {
         this.clusterService = clusterService;
         this.indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
         this.allocationService = new AllocationService(settings, allocationDeciders, shardsAllocator, clusterInfoService);
+        this.onStarted = () -> clusterPlugins.forEach(plugin -> plugin.onNodeStarted());
     }
 
 
@@ -241,4 +243,8 @@ public class ClusterModule extends AbstractModule {
         bind(AllocationDeciders.class).toInstance(allocationDeciders);
         bind(ShardsAllocator.class).toInstance(shardsAllocator);
     }
+
+    public Runnable onStarted() {
+        return onStarted;
+    }
 }

+ 43 - 0
core/src/main/java/org/elasticsearch/cluster/MergableCustomMetaData.java

@@ -0,0 +1,43 @@
+/*
+ * 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.cluster;
+
+import org.elasticsearch.cluster.metadata.MetaData;
+
+/**
+ * Interface to allow merging {@link org.elasticsearch.cluster.metadata.MetaData.Custom}.
+ * When multiple Mergable Custom metadata of the same type are found (from underlying clusters), the
+ * Custom metadata can be merged using {@link #merge(MetaData.Custom)}.
+ *
+ * @param <T> type of custom meta data
+ */
+public interface MergableCustomMetaData<T extends MetaData.Custom> {
+
+    /**
+     * Merges this custom metadata with other, returning either this or <code>other</code> custom metadata.
+     * This method should not mutate either <code>this</code> or the <code>other</code> custom metadata.
+     *
+     * @param other custom meta data
+     * @return the same instance or <code>other</code> custom metadata based on implementation
+     *         if both the instances are considered equal, implementations should return this
+     *         instance to avoid redundant cluster state changes.
+     */
+    T merge(T other);
+}

+ 0 - 9
core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

@@ -71,7 +71,6 @@ import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
 import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.indices.store.IndicesStore;
-import org.elasticsearch.ingest.IngestService;
 import org.elasticsearch.monitor.fs.FsService;
 import org.elasticsearch.monitor.jvm.JvmGcMonitorService;
 import org.elasticsearch.monitor.jvm.JvmService;
@@ -91,7 +90,6 @@ import org.elasticsearch.transport.RemoteClusterService;
 import org.elasticsearch.transport.TcpTransport;
 import org.elasticsearch.transport.Transport;
 import org.elasticsearch.transport.TransportService;
-import org.elasticsearch.tribe.TribeService;
 import org.elasticsearch.watcher.ResourceWatcherService;
 
 import java.util.Arrays;
@@ -369,13 +367,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
                     ThreadContext.DEFAULT_HEADERS_SETTING,
                     ESLoggerFactory.LOG_DEFAULT_LEVEL_SETTING,
                     ESLoggerFactory.LOG_LEVEL_SETTING,
-                    TribeService.BLOCKS_METADATA_SETTING,
-                    TribeService.BLOCKS_WRITE_SETTING,
-                    TribeService.BLOCKS_WRITE_INDICES_SETTING,
-                    TribeService.BLOCKS_READ_INDICES_SETTING,
-                    TribeService.BLOCKS_METADATA_INDICES_SETTING,
-                    TribeService.ON_CONFLICT_SETTING,
-                    TribeService.TRIBE_NAME_SETTING,
                     NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING,
                     NodeEnvironment.ENABLE_LUCENE_SEGMENT_INFOS_TRACE_SETTING,
                     OsService.REFRESH_INTERVAL_SETTING,

+ 1 - 20
core/src/main/java/org/elasticsearch/common/settings/SettingsModule.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.tribe.TribeService;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -49,8 +48,6 @@ public class SettingsModule implements Module {
     private final Set<String> settingsFilterPattern = new HashSet<>();
     private final Map<String, Setting<?>> nodeSettings = new HashMap<>();
     private final Map<String, Setting<?>> indexSettings = new HashMap<>();
-    private static final Predicate<String> TRIBE_CLIENT_NODE_SETTINGS_PREDICATE =  (s) -> s.startsWith("tribe.")
-        && TribeService.TRIBE_SETTING_KEYS.contains(s) == false;
     private final Logger logger;
     private final IndexScopedSettings indexScopedSettings;
     private final ClusterSettings clusterSettings;
@@ -135,9 +132,7 @@ public class SettingsModule implements Module {
             }
         }
         // by now we are fully configured, lets check node level settings for unregistered index settings
-        final Predicate<String> acceptOnlyClusterSettings = TRIBE_CLIENT_NODE_SETTINGS_PREDICATE.negate();
-        clusterSettings.validate(settings.filter(acceptOnlyClusterSettings));
-        validateTribeSettings(settings, clusterSettings);
+        clusterSettings.validate(settings);
         this.settingsFilter = new SettingsFilter(settings, settingsFilterPattern);
      }
 
@@ -195,20 +190,6 @@ public class SettingsModule implements Module {
         settingsFilterPattern.add(filter);
     }
 
-    private void validateTribeSettings(Settings settings, ClusterSettings clusterSettings) {
-        Map<String, Settings> groups = settings.filter(TRIBE_CLIENT_NODE_SETTINGS_PREDICATE).getGroups("tribe.", true);
-        for (Map.Entry<String, Settings>  tribeSettings : groups.entrySet()) {
-            Settings thisTribesSettings = tribeSettings.getValue();
-            for (Map.Entry<String, String> entry : thisTribesSettings.getAsMap().entrySet()) {
-                try {
-                    clusterSettings.validate(entry.getKey(), thisTribesSettings);
-                } catch (IllegalArgumentException ex) {
-                    throw new IllegalArgumentException("tribe." + tribeSettings.getKey() +" validation failed: "+ ex.getMessage(), ex);
-                }
-            }
-        }
-    }
-
     public Settings getSettings() {
         return settings;
     }

+ 0 - 1
core/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java

@@ -86,7 +86,6 @@ public class DiscoveryModule {
         discoveryTypes.put("zen",
             () -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier,
                 clusterSettings, hostsProvider, allocationService));
-        discoveryTypes.put("tribe", () -> new TribeDiscovery(settings, transportService, masterService, clusterApplier));
         discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, masterService, clusterApplier));
         for (DiscoveryPlugin plugin : plugins) {
             plugin.getDiscoveryTypes(threadPool, transportService, namedWriteableRegistry,

+ 4 - 27
core/src/main/java/org/elasticsearch/node/Node.java

@@ -133,7 +133,6 @@ import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.Transport;
 import org.elasticsearch.transport.TransportInterceptor;
 import org.elasticsearch.transport.TransportService;
-import org.elasticsearch.tribe.TribeService;
 import org.elasticsearch.usage.UsageService;
 import org.elasticsearch.watcher.ResourceWatcherService;
 
@@ -256,8 +255,6 @@ public class Node implements Closeable {
             Settings tmpSettings = Settings.builder().put(environment.settings())
                 .put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build();
 
-            tmpSettings = TribeService.processSettings(tmpSettings);
-
             // create the node environment as soon as possible, to recover the node id and enable logging
             try {
                 nodeEnvironment = new NodeEnvironment(tmpSettings, environment);
@@ -385,15 +382,6 @@ public class Node implements Closeable {
                     .flatMap(p -> p.getNamedXContent().stream()),
                 ClusterModule.getNamedXWriteables().stream())
                 .flatMap(Function.identity()).collect(toList()));
-            final TribeService tribeService =
-                    new TribeService(
-                            settings,
-                            environment.configFile(),
-                            clusterService,
-                            nodeId,
-                            namedWriteableRegistry,
-                            (s, p) -> newTribeClientNode(s, classpathPlugins, p));
-            resourcesToClose.add(tribeService);
             modules.add(new RepositoriesModule(this.environment, pluginsService.filterPlugins(RepositoryPlugin.class), xContentRegistry));
             final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment, xContentRegistry);
             final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry,
@@ -404,7 +392,8 @@ public class Node implements Closeable {
 
             Collection<Object> pluginComponents = pluginsService.filterPlugins(Plugin.class).stream()
                 .flatMap(p -> p.createComponents(client, clusterService, threadPool, resourceWatcherService,
-                                                 scriptModule.getScriptService(), xContentRegistry).stream())
+                                                 scriptModule.getScriptService(), xContentRegistry, environment, nodeEnvironment,
+                                                 namedWriteableRegistry).stream())
                 .collect(Collectors.toList());
             final RestController restController = actionModule.getRestController();
             final NetworkModule networkModule = new NetworkModule(settings, false, pluginsService.filterPlugins(NetworkPlugin.class),
@@ -458,7 +447,6 @@ public class Node implements Closeable {
                     b.bind(Environment.class).toInstance(this.environment);
                     b.bind(ThreadPool.class).toInstance(threadPool);
                     b.bind(NodeEnvironment.class).toInstance(nodeEnvironment);
-                    b.bind(TribeService.class).toInstance(tribeService);
                     b.bind(ResourceWatcherService.class).toInstance(resourceWatcherService);
                     b.bind(CircuitBreakerService.class).toInstance(circuitBreakerService);
                     b.bind(BigArrays.class).toInstance(bigArrays);
@@ -612,10 +600,6 @@ public class Node implements Closeable {
         Discovery discovery = injector.getInstance(Discovery.class);
         clusterService.getMasterService().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.getTaskManager().setTaskResultsService(injector.getInstance(TaskResultsService.class));
@@ -682,10 +666,10 @@ public class Node implements Closeable {
             writePortsFile("transport", transport.boundAddress());
         }
 
-        // start nodes now, after the http server, because it may take some time
-        tribeService.startNodes();
         logger.info("started");
 
+        pluginsService.filterPlugins(ClusterPlugin.class).forEach(ClusterPlugin::onNodeStarted);
+
         return this;
     }
 
@@ -696,7 +680,6 @@ public class Node implements Closeable {
         Logger logger = Loggers.getLogger(Node.class, NODE_NAME_SETTING.get(settings));
         logger.info("stopping ...");
 
-        injector.getInstance(TribeService.class).stop();
         injector.getInstance(ResourceWatcherService.class).stop();
         if (NetworkModule.HTTP_ENABLED.get(settings)) {
             injector.getInstance(HttpServerTransport.class).stop();
@@ -744,7 +727,6 @@ public class Node implements Closeable {
         List<Closeable> toClose = new ArrayList<>();
         StopWatch stopWatch = new StopWatch("node_close");
         toClose.add(() -> stopWatch.start("tribe"));
-        toClose.add(injector.getInstance(TribeService.class));
         toClose.add(() -> stopWatch.stop().start("node_service"));
         toClose.add(nodeService);
         toClose.add(() -> stopWatch.stop().start("http"));
@@ -920,11 +902,6 @@ public class Node implements Closeable {
         return customNameResolvers;
     }
 
-    /** Constructs an internal node used as a client into a cluster fronted by this tribe node. */
-    protected Node newTribeClientNode(Settings settings, Collection<Class<? extends Plugin>> classpathPlugins, Path configPath) {
-        return new Node(new Environment(settings, configPath), classpathPlugins);
-    }
-
     /** Constructs a ClusterInfoService which may be mocked for tests. */
     protected ClusterInfoService newClusterInfoService(Settings settings, ClusterService clusterService,
                                                        ThreadPool threadPool, NodeClient client, Consumer<ClusterInfo> listeners) {

+ 7 - 0
core/src/main/java/org/elasticsearch/plugins/ClusterPlugin.java

@@ -58,4 +58,11 @@ public interface ClusterPlugin {
     default Map<String, Supplier<ShardsAllocator>> getShardsAllocators(Settings settings, ClusterSettings clusterSettings) {
         return Collections.emptyMap();
     }
+
+    /**
+     * Called when the node is started
+     */
+    default void onNodeStarted() {
+
+    }
 }

+ 8 - 1
core/src/main/java/org/elasticsearch/plugins/Plugin.java

@@ -38,6 +38,8 @@ import org.elasticsearch.common.settings.SettingsModule;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.discovery.DiscoveryModule;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.index.IndexModule;
 import org.elasticsearch.indices.analysis.AnalysisModule;
 import org.elasticsearch.repositories.RepositoriesModule;
@@ -104,10 +106,15 @@ public abstract class Plugin implements Closeable {
      * @param threadPool A service to allow retrieving an executor to run an async action
      * @param resourceWatcherService A service to watch for changes to node local files
      * @param scriptService A service to allow running scripts on the local node
+     * @param xContentRegistry the registry for extensible xContent parsing
+     * @param environment the environment for path and setting configurations
+     * @param nodeEnvironment the node environment used coordinate access to the data paths
+     * @param namedWriteableRegistry the registry for {@link NamedWriteable} object parsing
      */
     public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
                                                ResourceWatcherService resourceWatcherService, ScriptService scriptService,
-                                               NamedXContentRegistry xContentRegistry) {
+                                               NamedXContentRegistry xContentRegistry, Environment environment,
+                                               NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
         return Collections.emptyList();
     }
 

+ 7 - 2
core/src/test/java/org/elasticsearch/cluster/metadata/TemplateUpgradeServiceIT.java

@@ -22,10 +22,13 @@ package org.elasticsearch.cluster.metadata;
 import org.apache.logging.log4j.Logger;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.test.ESIntegTestCase;
@@ -67,11 +70,13 @@ public class TemplateUpgradeServiceIT extends ESIntegTestCase {
         @Override
         public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
                                                    ResourceWatcherService resourceWatcherService, ScriptService scriptService,
-                                                   NamedXContentRegistry xContentRegistry) {
+                                                   NamedXContentRegistry xContentRegistry, Environment environment,
+                                                   NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
             clusterService.getClusterSettings().addSettingsUpdateConsumer(UPDATE_TEMPLATE_DUMMY_SETTING, integer -> {
                 logger.debug("the template dummy setting was updated to {}", integer);
             });
-            return super.createComponents(client, clusterService, threadPool, resourceWatcherService, scriptService, xContentRegistry);
+            return super.createComponents(client, clusterService, threadPool, resourceWatcherService, scriptService, xContentRegistry,
+                environment, nodeEnvironment, namedWriteableRegistry);
         }
 
         @Override

+ 0 - 51
core/src/test/java/org/elasticsearch/common/settings/SettingsModuleTests.java

@@ -21,13 +21,10 @@ package org.elasticsearch.common.settings;
 
 import org.elasticsearch.common.inject.ModuleTestCase;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.joda.time.MonthDay;
 
 import java.util.Arrays;
-import java.util.Collections;
 
 import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.is;
 
 public class SettingsModuleTests extends ModuleTestCase {
 
@@ -87,54 +84,6 @@ public class SettingsModuleTests extends ModuleTestCase {
         }
     }
 
-    public void testTribeSetting() {
-        {
-            Settings settings = Settings.builder().put("tribe.t1.cluster.routing.allocation.balance.shard", "2.0").build();
-            SettingsModule module = new SettingsModule(settings);
-            assertInstanceBinding(module, Settings.class, (s) -> s == settings);
-        }
-        {
-            Settings settings = Settings.builder().put("tribe.t1.cluster.routing.allocation.balance.shard", "[2.0]").build();
-            try {
-                new SettingsModule(settings);
-                fail();
-            } catch (IllegalArgumentException ex) {
-                assertEquals(
-                        "tribe.t1 validation failed: Failed to parse value [[2.0]] for setting [cluster.routing.allocation.balance.shard]",
-                        ex.getMessage());
-            }
-        }
-    }
-
-    public void testSpecialTribeSetting() {
-        {
-            Settings settings = Settings.builder().put("tribe.blocks.write", "false").build();
-            SettingsModule module = new SettingsModule(settings);
-            assertInstanceBinding(module, Settings.class, (s) -> s == settings);
-        }
-        {
-            Settings settings = Settings.builder().put("tribe.blocks.write", "BOOM").build();
-            try {
-                new SettingsModule(settings);
-                fail();
-            } catch (IllegalArgumentException ex) {
-                assertEquals("Failed to parse value [BOOM] as only [true] or [false] are allowed.",
-                        ex.getMessage());
-            }
-        }
-        {
-            Settings settings = Settings.builder().put("tribe.blocks.wtf", "BOOM").build();
-            try {
-                new SettingsModule(settings);
-                fail();
-            } catch (IllegalArgumentException ex) {
-                assertEquals("tribe.blocks validation failed: unknown setting [wtf] please check that any required plugins are" +
-                    " installed, or check the breaking changes documentation for removed settings", ex.getMessage());
-            }
-        }
-    }
-
-
     public void testLoggerSettings() {
         {
             Settings settings = Settings.builder().put("logger._root", "TRACE").put("logger.transport", "INFO").build();

+ 0 - 40
core/src/test/java/org/elasticsearch/threadpool/SimpleThreadPoolIT.java

@@ -21,32 +21,22 @@ package org.elasticsearch.threadpool;
 
 import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.common.xcontent.json.JsonXContent;
-import org.elasticsearch.env.Environment;
 import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.node.Node;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
 import org.elasticsearch.test.ESIntegTestCase.Scope;
 import org.elasticsearch.test.InternalTestCluster;
 import org.elasticsearch.test.hamcrest.RegexMatcher;
 
-import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 import java.util.HashSet;
-import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
 
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
-import static org.hamcrest.Matchers.containsString;
 
 @ClusterScope(scope = Scope.TEST, numDataNodes = 0, numClientNodes = 0)
 public class SimpleThreadPoolIT extends ESIntegTestCase {
@@ -111,34 +101,4 @@ public class SimpleThreadPoolIT extends ESIntegTestCase {
         }
     }
 
-    public void testThreadPoolLeakingThreadsWithTribeNode() {
-        Settings settings = Settings.builder()
-                .put("node.name", "thread_pool_leaking_threads_tribe_node")
-                .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
-                .put("tribe.t1.cluster.name", "non_existing_cluster")
-                        //trigger initialization failure of one of the tribes (doesn't require starting the node)
-                .put("tribe.t1.plugin.mandatory", "non_existing").build();
-
-        try {
-            new Node(settings);
-            fail("The node startup is supposed to fail");
-        } catch(Exception e) {
-            //all good
-            assertThat(e.getMessage(), containsString("mandatory plugins [non_existing]"));
-        }
-    }
-
-    private Map<String, Object> getPoolSettingsThroughJson(ThreadPoolInfo info, String poolName) throws IOException {
-        XContentBuilder builder = XContentFactory.jsonBuilder();
-        builder.startObject();
-        info.toXContent(builder, ToXContent.EMPTY_PARAMS);
-        builder.endObject();
-        builder.close();
-        Map<String, Object> poolsMap;
-        try (XContentParser parser = createParser(JsonXContent.jsonXContent, builder.string())) {
-            poolsMap = parser.map();
-        }
-        return (Map<String, Object>) ((Map<String, Object>) poolsMap.get("thread_pool")).get(poolName);
-    }
-
 }

+ 28 - 0
modules/tribe/build.gradle

@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+esplugin {
+    description 'Tribe module'
+    classname 'org.elasticsearch.tribe.TribePlugin'
+}
+
+compileJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-rawtypes,-try,-unchecked"
+compileTestJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-rawtypes,-try,-unchecked"
+
+integTestRunner.enabled = false

+ 3 - 2
core/src/main/java/org/elasticsearch/discovery/TribeDiscovery.java → modules/tribe/src/main/java/org/elasticsearch/tribe/TribeDiscovery.java

@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.elasticsearch.discovery;
+package org.elasticsearch.tribe;
 
 import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.ClusterName;
@@ -28,6 +28,7 @@ import org.elasticsearch.cluster.service.ClusterApplier;
 import org.elasticsearch.cluster.service.MasterService;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.discovery.single.SingleNodeDiscovery;
 import org.elasticsearch.transport.TransportService;
 
@@ -37,7 +38,7 @@ import static org.elasticsearch.tribe.TribeService.TRIBE_METADATA_BLOCK;
 import static org.elasticsearch.tribe.TribeService.TRIBE_WRITE_BLOCK;
 
 /**
- * A {@link Discovery} implementation that is used by {@link org.elasticsearch.tribe.TribeService}. This implementation
+ * A {@link Discovery} implementation that is used by {@link TribeService}. This implementation
  * doesn't support any clustering features. Most notably {@link #startInitialJoin()} does nothing and
  * {@link #publish(ClusterChangedEvent, AckListener)} delegates state updates directly to the
  * {@link org.elasticsearch.cluster.service.ClusterApplierService}.

+ 180 - 0
modules/tribe/src/main/java/org/elasticsearch/tribe/TribePlugin.java

@@ -0,0 +1,180 @@
+/*
+ * 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.tribe;
+
+import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.cluster.routing.allocation.AllocationService;
+import org.elasticsearch.cluster.service.ClusterApplier;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.cluster.service.MasterService;
+import org.elasticsearch.common.UUIDs;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.settings.ClusterSettings;
+import org.elasticsearch.common.settings.Setting;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.discovery.Discovery;
+import org.elasticsearch.discovery.DiscoveryModule;
+import org.elasticsearch.discovery.DiscoverySettings;
+import org.elasticsearch.discovery.zen.UnicastHostsProvider;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.plugins.ClusterPlugin;
+import org.elasticsearch.plugins.DiscoveryPlugin;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.script.ScriptService;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.watcher.ResourceWatcherService;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+public class TribePlugin extends Plugin implements DiscoveryPlugin, ClusterPlugin {
+
+    private final Settings settings;
+    private TribeService tribeService;
+
+    public TribePlugin(Settings settings) {
+        this.settings = settings;
+    }
+
+    @Override
+    public Map<String, Supplier<Discovery>> getDiscoveryTypes(ThreadPool threadPool, TransportService transportService,
+                                                              NamedWriteableRegistry namedWriteableRegistry,
+                                                              MasterService masterService, ClusterApplier clusterApplier,
+                                                              ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider,
+                                                              AllocationService allocationService) {
+        return Collections.singletonMap("tribe", () -> new TribeDiscovery(settings, transportService, masterService, clusterApplier));
+    }
+
+    @Override
+    public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
+                                               ResourceWatcherService resourceWatcherService, ScriptService scriptService,
+                                               NamedXContentRegistry xContentRegistry, Environment environment,
+                                               NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
+        tribeService = new TribeService(settings, nodeEnvironment, clusterService, namedWriteableRegistry,
+            nodeBuilder(environment.configFile()));
+        return Collections.singleton(tribeService);
+    }
+
+    protected Function<Settings, Node> nodeBuilder(Path configPath) {
+        return settings -> new Node(new Environment(settings, configPath));
+    }
+
+    @Override
+    public void onNodeStarted() {
+        tribeService.startNodes();
+    }
+
+    @Override
+    public Settings additionalSettings() {
+        if (TribeService.TRIBE_NAME_SETTING.exists(settings) == false) {
+            Map<String, Settings> nodesSettings = settings.getGroups("tribe", true);
+            if (nodesSettings.isEmpty()) {
+                return Settings.EMPTY;
+            }
+            Settings.Builder sb = Settings.builder();
+
+            if (Node.NODE_MASTER_SETTING.exists(settings)) {
+                if (Node.NODE_MASTER_SETTING.get(settings)) {
+                    throw new IllegalArgumentException("node cannot be tribe as well as master node");
+                }
+            } else {
+                sb.put(Node.NODE_MASTER_SETTING.getKey(), false);
+            }
+            if (Node.NODE_DATA_SETTING.exists(settings)) {
+                if (Node.NODE_DATA_SETTING.get(settings)) {
+                    throw new IllegalArgumentException("node cannot be tribe as well as data node");
+                }
+            } else {
+                sb.put(Node.NODE_DATA_SETTING.getKey(), false);
+            }
+            if (Node.NODE_INGEST_SETTING.exists(settings)) {
+                if (Node.NODE_INGEST_SETTING.get(settings)) {
+                    throw new IllegalArgumentException("node cannot be tribe as well as ingest node");
+                }
+            } else {
+                sb.put(Node.NODE_INGEST_SETTING.getKey(), false);
+            }
+
+            if (!NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.exists(settings)) {
+                sb.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), nodesSettings.size());
+            }
+            sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "tribe"); // there is a special discovery implementation for tribe
+            // nothing is going to be discovered, since no master will be elected
+            sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
+            if (sb.get("cluster.name") == null) {
+                sb.put("cluster.name", "tribe_" + UUIDs.randomBase64UUID()); // make sure it won't join other tribe nodes in the same JVM
+            }
+            sb.put(TransportMasterNodeReadAction.FORCE_LOCAL_SETTING.getKey(), true);
+
+            return sb.build();
+        } else {
+            for (String s : settings.getAsMap().keySet()) {
+                if (s.startsWith("tribe.") && !s.equals(TribeService.TRIBE_NAME_SETTING.getKey())) {
+                    throw new IllegalArgumentException("tribe cannot contain inner tribes: " + s);
+                }
+            }
+        }
+        return Settings.EMPTY;
+    }
+
+    @Override
+    public List<Setting<?>> getSettings() {
+        List<Setting<?>> defaults = Arrays.asList(
+            TribeService.BLOCKS_METADATA_SETTING,
+            TribeService.BLOCKS_WRITE_SETTING,
+            TribeService.BLOCKS_WRITE_INDICES_SETTING,
+            TribeService.BLOCKS_READ_INDICES_SETTING,
+            TribeService.BLOCKS_METADATA_INDICES_SETTING,
+            TribeService.ON_CONFLICT_SETTING,
+            TribeService.TRIBE_NAME_SETTING
+        );
+        Map<String, Settings> nodesSettings = settings.getGroups("tribe", true);
+        if (nodesSettings.isEmpty()) {
+            return defaults;
+        }
+        List<Setting<?>> allSettings = new ArrayList<>(defaults);
+        for (Map.Entry<String, Settings> entry : nodesSettings.entrySet()) {
+            String prefix = "tribe." + entry.getKey() + ".";
+            if (TribeService.TRIBE_SETTING_KEYS.stream().anyMatch(s -> s.startsWith(prefix))) {
+                continue;
+            }
+            // create dummy setting just so that setting validation does not complain, these settings are going to be validated
+            // again by the SettingsModule of the nested tribe node.
+            Setting<String> setting = Setting.prefixKeySetting(prefix, (key) -> new Setting<>(key, "", Function.identity(),
+                Setting.Property.NodeScope));
+            allSettings.add(setting);
+        }
+
+        return allSettings;
+    }
+
+}

+ 22 - 78
core/src/main/java/org/elasticsearch/tribe/TribeService.java → modules/tribe/src/main/java/org/elasticsearch/tribe/TribeService.java

@@ -22,13 +22,13 @@ package org.elasticsearch.tribe;
 import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.apache.logging.log4j.util.Supplier;
 import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
+import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateListener;
 import org.elasticsearch.cluster.ClusterStateTaskConfig;
 import org.elasticsearch.cluster.ClusterStateTaskExecutor;
+import org.elasticsearch.cluster.MergableCustomMetaData;
 import org.elasticsearch.cluster.block.ClusterBlock;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.block.ClusterBlocks;
@@ -41,7 +41,6 @@ import org.elasticsearch.cluster.routing.RoutingTable;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.component.Lifecycle;
 import org.elasticsearch.common.hash.MurmurHash3;
@@ -59,8 +58,6 @@ import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.common.util.set.Sets;
-import org.elasticsearch.discovery.DiscoveryModule;
-import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.node.Node;
@@ -68,7 +65,6 @@ import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.transport.TcpTransport;
 
 import java.io.IOException;
-import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -78,7 +74,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -88,10 +83,6 @@ import static java.util.Collections.unmodifiableMap;
  * The tribe service holds a list of node clients connected to a list of tribe members, and uses their
  * cluster state events to update this local node cluster state with the merged view of it.
  * <p>
- * The {@link #processSettings(org.elasticsearch.common.settings.Settings)} method should be called before
- * starting the node, so it will make sure to configure this current node properly with the relevant tribe node
- * settings.
- * <p>
  * The tribe node settings make sure the discovery used is "local", but with no master elected. This means no
  * write level master node operations will work ({@link org.elasticsearch.discovery.MasterNotDiscoveredException}
  * will be thrown), and state level metadata operations with automatically use the local flag.
@@ -109,63 +100,6 @@ public class TribeService extends AbstractLifecycleComponent {
     public static final ClusterBlock TRIBE_WRITE_BLOCK = new ClusterBlock(11, "tribe node, write not allowed", false, false,
         false, RestStatus.BAD_REQUEST, EnumSet.of(ClusterBlockLevel.WRITE));
 
-    public static Settings processSettings(Settings settings) {
-        if (TRIBE_NAME_SETTING.exists(settings)) {
-            // if its a node client started by this service as tribe, remove any tribe group setting
-            // to avoid recursive configuration
-            Settings.Builder sb = Settings.builder().put(settings);
-            for (String s : settings.getAsMap().keySet()) {
-                if (s.startsWith("tribe.") && !s.equals(TRIBE_NAME_SETTING.getKey())) {
-                    sb.remove(s);
-                }
-            }
-            return sb.build();
-        }
-        Map<String, Settings> nodesSettings = settings.getGroups("tribe", true);
-        if (nodesSettings.isEmpty()) {
-            return settings;
-        }
-        // its a tribe configured node..., force settings
-        Settings.Builder sb = Settings.builder().put(settings);
-        sb.put(Node.NODE_MASTER_SETTING.getKey(), false);
-        sb.put(Node.NODE_DATA_SETTING.getKey(), false);
-        sb.put(Node.NODE_INGEST_SETTING.getKey(), false);
-        if (!NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.exists(settings)) {
-            sb.put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), nodesSettings.size());
-        }
-        sb.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "tribe"); // there is a special discovery implementation for tribe
-        // nothing is going to be discovered, since no master will be elected
-        sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0);
-        if (sb.get("cluster.name") == null) {
-            sb.put("cluster.name", "tribe_" + UUIDs.randomBase64UUID()); // make sure it won't join other tribe nodes in the same JVM
-        }
-        sb.put(TransportMasterNodeReadAction.FORCE_LOCAL_SETTING.getKey(), true);
-        return sb.build();
-    }
-
-    /**
-     * Interface to allow merging {@link org.elasticsearch.cluster.metadata.MetaData.Custom} in tribe node
-     * When multiple Mergable Custom metadata of the same type is found (from underlying clusters), the
-     * Custom metadata will be merged using {@link #merge(MetaData.Custom)} and the result will be stored
-     * in the tribe cluster state
-     *
-     * @param <T> type of custom meta data
-     */
-    public interface MergableCustomMetaData<T extends MetaData.Custom> {
-
-        /**
-         * Merges this custom metadata with other, returning either this or <code>other</code> custom metadata
-         * for tribe cluster state. This method should not mutate either <code>this</code> or the
-         * <code>other</code> custom metadata.
-         *
-         * @param other custom meta data
-         * @return the same instance or <code>other</code> custom metadata based on implementation
-         *         if both the instances are considered equal, implementations should return this
-         *         instance to avoid redundant cluster state changes.
-         */
-        T merge(T other);
-    }
-
     // internal settings only
     public static final Setting<String> TRIBE_NAME_SETTING = Setting.simpleString("tribe.name", Property.NodeScope);
     private final ClusterService clusterService;
@@ -200,7 +134,8 @@ public class TribeService extends AbstractLifecycleComponent {
         Setting.listSetting("tribe.blocks.metadata.indices", Collections.emptyList(), Function.identity(), Property.NodeScope);
 
     public static final Set<String> TRIBE_SETTING_KEYS = Sets.newHashSet(TRIBE_NAME_SETTING.getKey(), ON_CONFLICT_SETTING.getKey(),
-            BLOCKS_METADATA_INDICES_SETTING.getKey(), BLOCKS_METADATA_SETTING.getKey(), BLOCKS_READ_INDICES_SETTING.getKey(), BLOCKS_WRITE_INDICES_SETTING.getKey(), BLOCKS_WRITE_SETTING.getKey());
+            BLOCKS_METADATA_INDICES_SETTING.getKey(), BLOCKS_METADATA_SETTING.getKey(), BLOCKS_READ_INDICES_SETTING.getKey(),
+            BLOCKS_WRITE_INDICES_SETTING.getKey(), BLOCKS_WRITE_SETTING.getKey());
 
     // these settings should be passed through to each tribe client, if they are not set explicitly
     private static final List<Setting<?>> PASS_THROUGH_SETTINGS = Arrays.asList(
@@ -218,8 +153,8 @@ public class TribeService extends AbstractLifecycleComponent {
 
     private final NamedWriteableRegistry namedWriteableRegistry;
 
-    public TribeService(Settings settings, Path configPath, ClusterService clusterService, final String tribeNodeId,
-                        NamedWriteableRegistry namedWriteableRegistry, BiFunction<Settings, Path, Node> clientNodeBuilder) {
+    public TribeService(Settings settings, NodeEnvironment nodeEnvironment, ClusterService clusterService,
+                        NamedWriteableRegistry namedWriteableRegistry, Function<Settings, Node> clientNodeBuilder) {
         super(settings);
         this.clusterService = clusterService;
         this.namedWriteableRegistry = namedWriteableRegistry;
@@ -227,8 +162,21 @@ public class TribeService extends AbstractLifecycleComponent {
         nodesSettings.remove("blocks"); // remove prefix settings that don't indicate a client
         nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client
         for (Map.Entry<String, Settings> entry : nodesSettings.entrySet()) {
-            Settings clientSettings = buildClientSettings(entry.getKey(), tribeNodeId, settings, entry.getValue());
-            nodes.add(clientNodeBuilder.apply(clientSettings, configPath));
+            Settings clientSettings = buildClientSettings(entry.getKey(), nodeEnvironment.nodeId(), settings, entry.getValue());
+            try {
+                nodes.add(clientNodeBuilder.apply(clientSettings));
+            } catch (Exception e) {
+                // calling close is safe for non started nodes, we can just iterate over all
+                for (Node otherNode : nodes) {
+                    try {
+                        otherNode.close();
+                    } catch (Exception inner) {
+                        inner.addSuppressed(e);
+                        logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to close node {} on failed start", otherNode), inner);
+                    }
+                }
+                throw ExceptionsHelper.convertToRuntime(e);
+            }
         }
 
         this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
@@ -279,7 +227,6 @@ public class TribeService extends AbstractLifecycleComponent {
         return sb.build();
     }
 
-
     @Override
     protected void doStart() {
 
@@ -300,10 +247,7 @@ public class TribeService extends AbstractLifecycleComponent {
                         logger.warn((Supplier<?>) () -> new ParameterizedMessage("failed to close node {} on failed start", otherNode), inner);
                     }
                 }
-                if (e instanceof RuntimeException) {
-                    throw (RuntimeException) e;
-                }
-                throw new ElasticsearchException(e);
+                throw ExceptionsHelper.convertToRuntime(e);
             }
         }
     }

+ 54 - 3
core/src/test/java/org/elasticsearch/tribe/TribeIT.java → modules/tribe/src/test/java/org/elasticsearch/tribe/TribeIntegrationTests.java

@@ -23,8 +23,8 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
 import org.elasticsearch.action.support.DestructiveOperations;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.ClusterName;
-import org.elasticsearch.cluster.ClusterStateUpdateTask;
 import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ClusterStateUpdateTask;
 import org.elasticsearch.cluster.NamedDiff;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.health.ClusterHealthStatus;
@@ -41,12 +41,15 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.discovery.MasterNotDiscoveredException;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.node.MockNode;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.InternalTestCluster;
 import org.elasticsearch.test.NodeConfigurationSource;
 import org.elasticsearch.test.TestCustomMetaData;
+import org.elasticsearch.test.discovery.TestZenDiscovery;
 import org.elasticsearch.transport.MockTcpTransportPlugin;
 import org.elasticsearch.tribe.TribeServiceTests.MergableCustomMetaData1;
 import org.elasticsearch.tribe.TribeServiceTests.MergableCustomMetaData2;
@@ -55,6 +58,7 @@ import org.junit.AfterClass;
 import org.junit.Before;
 
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -84,7 +88,7 @@ import static org.hamcrest.core.Is.is;
  * does it by default.
  */
 @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 0, numClientNodes = 0, transportClientRatio = 0.0)
-public class TribeIT extends ESIntegTestCase {
+public class TribeIntegrationTests extends ESIntegTestCase {
 
     private static final String TRIBE_NODE = "tribe_node";
 
@@ -145,14 +149,49 @@ public class TribeIT extends ESIntegTestCase {
         }
     }
 
+    public static class MockTribePlugin extends TribePlugin {
+
+        public MockTribePlugin(Settings settings) {
+            super(settings);
+        }
+
+        protected Function<Settings, Node> nodeBuilder(Path configPath) {
+            return settings -> new MockNode(new Environment(settings, configPath), internalCluster().getPlugins());
+        }
+
+    }
+
     @Override
     protected Collection<Class<? extends Plugin>> nodePlugins() {
         ArrayList<Class<? extends Plugin>> plugins = new ArrayList<>();
         plugins.addAll(getMockPlugins());
+        plugins.add(MockTribePlugin.class);
+        plugins.add(TribeAwareTestZenDiscoveryPlugin.class);
         plugins.add(TestCustomMetaDataPlugin.class);
         return plugins;
     }
 
+    @Override
+    protected boolean addTestZenDiscovery() {
+        return false;
+    }
+
+    public static class TribeAwareTestZenDiscoveryPlugin extends TestZenDiscovery.TestPlugin {
+
+        public TribeAwareTestZenDiscoveryPlugin(Settings settings) {
+            super(settings);
+        }
+
+        @Override
+        public Settings additionalSettings() {
+            if (settings.getGroups("tribe", true).isEmpty()) {
+                return super.additionalSettings();
+            } else {
+                return Settings.EMPTY;
+            }
+        }
+    }
+
     @Before
     public void startRemoteClusters() {
         final int minNumDataNodes = 2;
@@ -249,9 +288,12 @@ public class TribeIT extends ESIntegTestCase {
         final Settings.Builder settings = Settings.builder();
         settings.put(Node.NODE_NAME_SETTING.getKey(), TRIBE_NODE);
         settings.put(Node.NODE_DATA_SETTING.getKey(), false);
-        settings.put(Node.NODE_MASTER_SETTING.getKey(), true);
+        settings.put(Node.NODE_MASTER_SETTING.getKey(), false);
+        settings.put(Node.NODE_INGEST_SETTING.getKey(), false);
         settings.put(NetworkModule.HTTP_ENABLED.getKey(), false);
         settings.put(NetworkModule.TRANSPORT_TYPE_SETTING.getKey(), MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME);
+        // add dummy tribe setting so that node is always identifiable as tribe in this test even if the set of connecting cluster is empty
+        settings.put(TribeService.BLOCKS_WRITE_SETTING.getKey(), TribeService.BLOCKS_WRITE_SETTING.getDefault(Settings.EMPTY));
 
         doWithAllClusters(filter, c -> {
             String tribeSetting = "tribe." + c.getClusterName() + ".";
@@ -263,6 +305,15 @@ public class TribeIT extends ESIntegTestCase {
         return settings;
     }
 
+    public void testTribeNodeWithBadSettings() throws Exception {
+        Settings brokenSettings = Settings.builder()
+            .put("tribe.some.setting.that.does.not.exist", true)
+            .build();
+
+        IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> startTribeNode(ALL, brokenSettings));
+        assertThat(e.getMessage(), containsString("unknown setting [setting.that.does.not.exist]"));
+    }
+
     public void testGlobalReadWriteBlocks() throws Exception {
         Settings additionalSettings = Settings.builder()
                 .put("tribe.blocks.write", true)

+ 24 - 8
core/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java → modules/tribe/src/test/java/org/elasticsearch/tribe/TribeServiceTests.java

@@ -20,16 +20,17 @@
 package org.elasticsearch.tribe;
 
 import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.MergableCustomMetaData;
 import org.elasticsearch.cluster.NamedDiff;
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.network.NetworkModule;
-import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.node.MockNode;
 import org.elasticsearch.node.Node;
+import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.TestCustomMetaData;
 import org.elasticsearch.transport.MockTcpTransportPlugin;
@@ -43,6 +44,7 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
 
 import static org.hamcrest.Matchers.instanceOf;
 
@@ -137,7 +139,7 @@ public class TribeServiceTests extends ESTestCase {
     }
 
     public void testMergeMultipleCustomMetaData() {
-        Map<String, List<TribeService.MergableCustomMetaData>> inputMap = new HashMap<>();
+        Map<String, List<MergableCustomMetaData>> inputMap = new HashMap<>();
         inputMap.put(MergableCustomMetaData1.TYPE,
                 Arrays.asList(new MergableCustomMetaData1("data10"), new MergableCustomMetaData1("data11")));
         inputMap.put(MergableCustomMetaData2.TYPE,
@@ -155,15 +157,15 @@ public class TribeServiceTests extends ESTestCase {
     }
 
     public void testMergeCustomMetaDataFromMany() {
-        Map<String, List<TribeService.MergableCustomMetaData>> inputMap = new HashMap<>();
+        Map<String, List<MergableCustomMetaData>> inputMap = new HashMap<>();
         int n = randomIntBetween(3, 5);
-        List<TribeService.MergableCustomMetaData> customList1 = new ArrayList<>();
+        List<MergableCustomMetaData> customList1 = new ArrayList<>();
         for (int i = 0; i <= n; i++) {
             customList1.add(new MergableCustomMetaData1("data1"+String.valueOf(i)));
         }
         Collections.shuffle(customList1, random());
         inputMap.put(MergableCustomMetaData1.TYPE, customList1);
-        List<TribeService.MergableCustomMetaData> customList2 = new ArrayList<>();
+        List<MergableCustomMetaData> customList2 = new ArrayList<>();
         for (int i = 0; i <= n; i++) {
             customList2.add(new MergableCustomMetaData2("data2"+String.valueOf(i)));
         }
@@ -182,6 +184,20 @@ public class TribeServiceTests extends ESTestCase {
         assertEquals(mergedCustom.getData(), "data2"+String.valueOf(n));
     }
 
+    public static class MockTribePlugin extends TribePlugin {
+
+        static List<Class<? extends Plugin>> classpathPlugins = Arrays.asList(MockTribePlugin.class, MockTcpTransportPlugin.class);
+
+        public MockTribePlugin(Settings settings) {
+            super(settings);
+        }
+
+        protected Function<Settings, Node> nodeBuilder(Path configPath) {
+            return settings -> new MockNode(new Environment(settings, configPath), classpathPlugins);
+        }
+
+    }
+
     public void testTribeNodeDeprecation() throws IOException {
         final Path tempDir = createTempDir();
         Settings.Builder settings = Settings.builder()
@@ -197,7 +213,7 @@ public class TribeServiceTests extends ESTestCase {
             settings.put(tribeSetting + ClusterName.CLUSTER_NAME_SETTING.getKey(), clusterName)
                 .put(tribeSetting + NetworkModule.TRANSPORT_TYPE_SETTING.getKey(), "mock-socket-network");
         }
-        try (Node node = new MockNode(settings.build(),Collections.singleton(MockTcpTransportPlugin.class) )) {
+        try (Node node = new MockNode(settings.build(), MockTribePlugin.classpathPlugins)) {
             if (tribeServiceEnable) {
                 assertWarnings("tribe nodes are deprecated in favor of cross-cluster search and will be removed in Elasticsearch 7.0.0");
             }
@@ -205,7 +221,7 @@ public class TribeServiceTests extends ESTestCase {
     }
 
     static class MergableCustomMetaData1 extends TestCustomMetaData
-            implements TribeService.MergableCustomMetaData<MergableCustomMetaData1> {
+            implements MergableCustomMetaData<MergableCustomMetaData1> {
         public static final String TYPE = "custom_md_1";
 
         protected MergableCustomMetaData1(String data) {
@@ -237,7 +253,7 @@ public class TribeServiceTests extends ESTestCase {
     }
 
     static class MergableCustomMetaData2 extends TestCustomMetaData
-            implements TribeService.MergableCustomMetaData<MergableCustomMetaData2> {
+            implements MergableCustomMetaData<MergableCustomMetaData2> {
         public static final String TYPE = "custom_md_2";
 
         protected MergableCustomMetaData2(String data) {

+ 6 - 7
plugins/discovery-file/src/main/java/org/elasticsearch/discovery/file/FileBasedDiscoveryPlugin.java

@@ -22,6 +22,7 @@ package org.elasticsearch.discovery.file;
 import org.apache.logging.log4j.Logger;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.logging.DeprecationLogger;
 import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.network.NetworkService;
@@ -32,6 +33,7 @@ import org.elasticsearch.discovery.DiscoveryModule;
 import org.elasticsearch.discovery.zen.UnicastHostsProvider;
 import org.elasticsearch.discovery.zen.UnicastZenPing;
 import org.elasticsearch.env.Environment;
+import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.plugins.DiscoveryPlugin;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.script.ScriptService;
@@ -66,13 +68,10 @@ public class FileBasedDiscoveryPlugin extends Plugin implements DiscoveryPlugin
     }
 
     @Override
-    public Collection<Object> createComponents(
-            Client client,
-            ClusterService clusterService,
-            ThreadPool threadPool,
-            ResourceWatcherService resourceWatcherService,
-            ScriptService scriptService,
-            NamedXContentRegistry xContentRegistry) {
+    public Collection<Object> createComponents(Client client, ClusterService clusterService, ThreadPool threadPool,
+                                               ResourceWatcherService resourceWatcherService, ScriptService scriptService,
+                                               NamedXContentRegistry xContentRegistry, Environment environment,
+                                               NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
         final int concurrentConnects = UnicastZenPing.DISCOVERY_ZEN_PING_UNICAST_CONCURRENT_CONNECTS_SETTING.get(settings);
         final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(settings, "[file_based_discovery_resolve]");
         fileBasedDiscoveryExecutorService = EsExecutors.newScaling(

+ 1 - 0
qa/evil-tests/build.gradle

@@ -27,6 +27,7 @@ apply plugin: 'elasticsearch.standalone-test'
 
 dependencies {
   testCompile 'com.google.jimfs:jimfs:1.1'
+  testCompile project(path: ':modules:tribe', configuration: 'runtime')
 }
 
 // TODO: give each evil test its own fresh JVM for more isolation.

+ 36 - 6
qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.Function;
 
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.client.Client;
@@ -55,6 +56,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 @SuppressForbidden(reason = "modifies system properties intentionally")
 public class TribeUnitTests extends ESTestCase {
 
+    private static List<Class<? extends Plugin>> classpathPlugins;
     private static Node tribe1;
     private static Node tribe2;
 
@@ -67,30 +69,60 @@ public class TribeUnitTests extends ESTestCase {
             .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), 2)
             .build();
 
-        final List<Class<? extends Plugin>> mockPlugins = Arrays.asList(MockTcpTransportPlugin.class, TestZenDiscovery.TestPlugin.class);
+        classpathPlugins = Arrays.asList(TribeAwareTestZenDiscoveryPlugin.class, MockTribePlugin.class, MockTcpTransportPlugin.class);
+
         tribe1 = new MockNode(
             Settings.builder()
                 .put(baseSettings)
                 .put("cluster.name", "tribe1")
                 .put("node.name", "tribe1_node")
                     .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong())
-                .build(), mockPlugins).start();
+                .build(), classpathPlugins).start();
         tribe2 = new MockNode(
             Settings.builder()
                 .put(baseSettings)
                 .put("cluster.name", "tribe2")
                 .put("node.name", "tribe2_node")
                     .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), random().nextLong())
-                .build(), mockPlugins).start();
+                .build(), classpathPlugins).start();
     }
 
     @AfterClass
     public static void closeTribes() throws IOException {
         IOUtils.close(tribe1, tribe2);
+        classpathPlugins = null;
         tribe1 = null;
         tribe2 = null;
     }
 
+    public static class TribeAwareTestZenDiscoveryPlugin extends TestZenDiscovery.TestPlugin {
+
+        public TribeAwareTestZenDiscoveryPlugin(Settings settings) {
+            super(settings);
+        }
+
+        @Override
+        public Settings additionalSettings() {
+            if (settings.getGroups("tribe", true).isEmpty()) {
+                return super.additionalSettings();
+            } else {
+                return Settings.EMPTY;
+            }
+        }
+    }
+
+    public static class MockTribePlugin extends TribePlugin {
+
+        public MockTribePlugin(Settings settings) {
+            super(settings);
+        }
+
+        protected Function<Settings, Node> nodeBuilder(Path configPath) {
+            return settings -> new MockNode(new Environment(settings, configPath), classpathPlugins);
+        }
+
+    }
+
     public void testThatTribeClientsIgnoreGlobalConfig() throws Exception {
         assertTribeNodeSuccessfullyCreated(getDataPath("elasticsearch.yml").getParent());
         assertWarnings("tribe nodes are deprecated in favor of cross-cluster search and will be removed in Elasticsearch 7.0.0");
@@ -99,14 +131,12 @@ public class TribeUnitTests extends ESTestCase {
     private static void assertTribeNodeSuccessfullyCreated(Path configPath) throws Exception {
         // the tribe clients do need it to make sure they can find their corresponding tribes using the proper transport
         Settings settings = Settings.builder().put(NetworkModule.HTTP_ENABLED.getKey(), false).put("node.name", "tribe_node")
-                .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME).put("discovery.type", "local")
+                .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME)
                 .put("tribe.t1.transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME)
                 .put("tribe.t2.transport.type",MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME)
                 .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir())
                 .build();
 
-        final List<Class<? extends Plugin>> classpathPlugins =
-                Arrays.asList(MockTcpTransportPlugin.class, TestZenDiscovery.TestPlugin.class);
         try (Node node = new MockNode(settings, classpathPlugins, configPath).start()) {
             try (Client client = node.client()) {
                 assertBusy(() -> {

+ 1 - 0
settings.gradle

@@ -40,6 +40,7 @@ List projects = [
   'modules:reindex',
   'modules:repository-url',
   'modules:transport-netty4',
+  'modules:tribe',
   'plugins:analysis-icu',
   'plugins:analysis-kuromoji',
   'plugins:analysis-phonetic',

+ 6 - 7
test/framework/src/main/java/org/elasticsearch/node/MockNode.java

@@ -30,6 +30,7 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.MockBigArrays;
+import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.indices.breaker.CircuitBreakerService;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -38,7 +39,6 @@ import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.search.MockSearchService;
 import org.elasticsearch.search.SearchService;
 import org.elasticsearch.search.fetch.FetchPhase;
-import org.elasticsearch.test.InternalSettingsPlugin;
 import org.elasticsearch.test.transport.MockTransportService;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.Transport;
@@ -66,7 +66,11 @@ public class MockNode extends Node {
     }
 
     public MockNode(Settings settings, Collection<Class<? extends Plugin>> classpathPlugins, Path configPath) {
-        super(InternalSettingsPreparer.prepareEnvironment(settings, null, Collections.emptyMap(), configPath), classpathPlugins);
+        this(InternalSettingsPreparer.prepareEnvironment(settings, null, Collections.emptyMap(), configPath), classpathPlugins);
+    }
+
+    public MockNode(Environment environment, Collection<Class<? extends Plugin>> classpathPlugins) {
+        super(environment, classpathPlugins);
         this.classpathPlugins = classpathPlugins;
     }
 
@@ -113,11 +117,6 @@ public class MockNode extends Node {
         }
     }
 
-    @Override
-    protected Node newTribeClientNode(Settings settings, Collection<Class<? extends Plugin>> classpathPlugins, Path configPath) {
-        return new MockNode(settings, classpathPlugins, configPath);
-    }
-
     @Override
     protected void processRecoverySettings(ClusterSettings clusterSettings, RecoverySettings recoverySettings) {
         if (false == getPluginsService().filterPlugins(RecoverySettingsChunkSizePlugin.class).isEmpty()) {

+ 12 - 2
test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java

@@ -1863,12 +1863,20 @@ public abstract class ESIntegTestCase extends ESTestCase {
 
     /**
      * Iff this returns true mock transport implementations are used for the test runs. Otherwise not mock transport impls are used.
-     * The defautl is <tt>true</tt>
+     * The default is <tt>true</tt>
      */
     protected boolean addMockTransportService() {
         return true;
     }
 
+    /**
+     * Iff this returns true test zen discovery implementations is used for the test runs.
+     * The default is <tt>true</tt>
+     */
+    protected boolean addTestZenDiscovery() {
+        return true;
+    }
+
     /**
      * Returns a function that allows to wrap / filter all clients that are exposed by the test cluster. This is useful
      * for debugging or request / response pre and post processing. It also allows to intercept all calls done by the test
@@ -1906,7 +1914,9 @@ public abstract class ESIntegTestCase extends ESTestCase {
             mocks.add(MockTcpTransportPlugin.class);
         }
 
-        mocks.add(TestZenDiscovery.TestPlugin.class);
+        if (addTestZenDiscovery()) {
+            mocks.add(TestZenDiscovery.TestPlugin.class);
+        }
         mocks.add(TestSeedPlugin.class);
         return Collections.unmodifiableList(mocks);
     }

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

@@ -381,7 +381,7 @@ public final class InternalTestCluster extends TestCluster {
         return builder.build();
     }
 
-    private Collection<Class<? extends Plugin>> getPlugins() {
+    public Collection<Class<? extends Plugin>> getPlugins() {
         Set<Class<? extends Plugin>> plugins = new HashSet<>(nodeConfigurationSource.nodePlugins());
         plugins.addAll(mockPlugins);
         return plugins;

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java

@@ -52,7 +52,7 @@ public class TestZenDiscovery extends ZenDiscovery {
 
     /** A plugin which installs mock discovery and configures it to be used. */
     public static class TestPlugin extends Plugin implements DiscoveryPlugin {
-        private Settings settings;
+        protected final Settings settings;
         public TestPlugin(Settings settings) {
             this.settings = settings;
         }