Przeglądaj źródła

Remove the use of AbstracLifecycleComponent constructor #37488 (#37488)

The AbstracLifecycleComponent used to extend AbstractComponent, so it had to pass settings to the constractor of its supper class.
It no longer extends the AbstractComponent so there is no need for this constructor
There is also no need for AbstracLifecycleComponent subclasses to have Settings in their constructors if they were only passing it over to super constructor.
This is part 1. which will be backported to 6.x with a migration guide/deprecation log.
part 2 will have this constructor removed in 7
relates #35560

relates #34488
Przemyslaw Gomulka 6 lat temu
rodzic
commit
5e94f384c4
46 zmienionych plików z 16 dodań i 65 usunięć
  1. 0 1
      plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceImpl.java
  2. 0 1
      plugins/discovery-gce/src/main/java/org/elasticsearch/cloud/gce/GceMetadataService.java
  3. 0 1
      server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java
  4. 0 1
      server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java
  5. 1 3
      server/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java
  6. 1 3
      server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java
  7. 0 1
      server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java
  8. 0 1
      server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java
  9. 0 1
      server/src/main/java/org/elasticsearch/cluster/service/MasterService.java
  10. 3 0
      server/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java
  11. 0 1
      server/src/main/java/org/elasticsearch/discovery/UnicastConfiguredHostsResolver.java
  12. 0 1
      server/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java
  13. 0 1
      server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java
  14. 0 1
      server/src/main/java/org/elasticsearch/gateway/GatewayService.java
  15. 0 1
      server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java
  16. 0 1
      server/src/main/java/org/elasticsearch/indices/IndicesService.java
  17. 1 3
      server/src/main/java/org/elasticsearch/indices/breaker/CircuitBreakerService.java
  18. 1 1
      server/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java
  19. 1 2
      server/src/main/java/org/elasticsearch/indices/breaker/NoneCircuitBreakerService.java
  20. 0 1
      server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java
  21. 0 1
      server/src/main/java/org/elasticsearch/monitor/MonitorService.java
  22. 0 1
      server/src/main/java/org/elasticsearch/monitor/jvm/JvmGcMonitorService.java
  23. 0 1
      server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
  24. 0 1
      server/src/main/java/org/elasticsearch/search/SearchService.java
  25. 0 1
      server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java
  26. 0 1
      server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
  27. 0 1
      server/src/main/java/org/elasticsearch/transport/TcpTransport.java
  28. 0 1
      server/src/main/java/org/elasticsearch/transport/TransportService.java
  29. 0 1
      server/src/main/java/org/elasticsearch/watcher/ResourceWatcherService.java
  30. 4 4
      server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java
  31. 1 2
      server/src/test/java/org/elasticsearch/cluster/routing/RoutingServiceTests.java
  32. 0 3
      server/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java
  33. 0 1
      server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
  34. 0 1
      server/src/test/java/org/elasticsearch/rest/RestControllerTests.java
  35. 1 1
      server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java
  36. 0 5
      test/framework/src/main/java/org/elasticsearch/test/MockHttpTransport.java
  37. 1 1
      x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java
  38. 0 1
      x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRepositoryManager.java
  39. 0 1
      x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java
  40. 0 4
      x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java
  41. 1 1
      x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java
  42. 0 1
      x-pack/plugin/core/src/main/java/org/elasticsearch/license/LicenseService.java
  43. 0 1
      x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringService.java
  44. 0 1
      x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/cleaner/CleanerService.java
  45. 0 1
      x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/Exporters.java
  46. 0 1
      x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/MonitoringServiceTests.java

+ 0 - 1
plugins/discovery-azure-classic/src/main/java/org/elasticsearch/cloud/azure/classic/management/AzureComputeServiceImpl.java

@@ -52,7 +52,6 @@ public class AzureComputeServiceImpl extends AbstractLifecycleComponent
     private final String serviceName;
 
     public AzureComputeServiceImpl(Settings settings) {
-        super(settings);
         String subscriptionId = getRequiredSetting(settings, Management.SUBSCRIPTION_ID_SETTING);
 
         serviceName = getRequiredSetting(settings, Management.SERVICE_NAME_SETTING);

+ 0 - 1
plugins/discovery-gce/src/main/java/org/elasticsearch/cloud/gce/GceMetadataService.java

@@ -53,7 +53,6 @@ public class GceMetadataService extends AbstractLifecycleComponent {
     private HttpTransport gceHttpTransport;
 
     public GceMetadataService(Settings settings) {
-        super(settings);
         this.settings = settings;
     }
 

+ 0 - 1
server/src/main/java/org/elasticsearch/cluster/NodeConnectionsService.java

@@ -75,7 +75,6 @@ public class NodeConnectionsService extends AbstractLifecycleComponent {
 
     @Inject
     public NodeConnectionsService(Settings settings, ThreadPool threadPool, TransportService transportService) {
-        super(settings);
         this.threadPool = threadPool;
         this.transportService = transportService;
         this.reconnectInterval = NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING.get(settings);

+ 0 - 1
server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java

@@ -145,7 +145,6 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
                        NamedWriteableRegistry namedWriteableRegistry, AllocationService allocationService, MasterService masterService,
                        Supplier<CoordinationState.PersistedState> persistedStateSupplier, UnicastHostsProvider unicastHostsProvider,
                        ClusterApplier clusterApplier, Collection<BiConsumer<DiscoveryNode, ClusterState>> onJoinValidators, Random random) {
-        super(settings);
         this.settings = settings;
         this.transportService = transportService;
         this.masterService = masterService;

+ 1 - 3
server/src/main/java/org/elasticsearch/cluster/routing/DelayedAllocationService.java

@@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.AbstractRunnable;
 import org.elasticsearch.common.util.concurrent.FutureUtils;
@@ -130,9 +129,8 @@ public class DelayedAllocationService extends AbstractLifecycleComponent impleme
     }
 
     @Inject
-    public DelayedAllocationService(Settings settings, ThreadPool threadPool, ClusterService clusterService,
+    public DelayedAllocationService(ThreadPool threadPool, ClusterService clusterService,
                                     AllocationService allocationService) {
-        super(settings);
         this.threadPool = threadPool;
         this.clusterService = clusterService;
         this.allocationService = allocationService;

+ 1 - 3
server/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java

@@ -30,7 +30,6 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.settings.Settings;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -57,8 +56,7 @@ public class RoutingService extends AbstractLifecycleComponent {
     private AtomicBoolean rerouting = new AtomicBoolean();
 
     @Inject
-    public RoutingService(Settings settings, ClusterService clusterService, AllocationService allocationService) {
-        super(settings);
+    public RoutingService(ClusterService clusterService, AllocationService allocationService) {
         this.clusterService = clusterService;
         this.allocationService = allocationService;
     }

+ 0 - 1
server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java

@@ -102,7 +102,6 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
     private NodeConnectionsService nodeConnectionsService;
 
     public ClusterApplierService(String nodeName, Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) {
-        super(settings);
         this.clusterSettings = clusterSettings;
         this.threadPool = threadPool;
         this.state = new AtomicReference<>();

+ 0 - 1
server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java

@@ -78,7 +78,6 @@ public class ClusterService extends AbstractLifecycleComponent {
 
     public ClusterService(Settings settings, ClusterSettings clusterSettings, MasterService masterService,
         ClusterApplierService clusterApplierService) {
-        super(settings);
         this.settings = settings;
         this.nodeName = Node.NODE_NAME_SETTING.get(settings);
         this.masterService = masterService;

+ 0 - 1
server/src/main/java/org/elasticsearch/cluster/service/MasterService.java

@@ -87,7 +87,6 @@ public class MasterService extends AbstractLifecycleComponent {
     private volatile Batcher taskBatcher;
 
     public MasterService(String nodeName, Settings settings, ThreadPool threadPool) {
-        super(settings);
         this.nodeName = nodeName;
         // TODO: introduce a dedicated setting for master service
         this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);

+ 3 - 0
server/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java

@@ -34,6 +34,9 @@ public abstract class AbstractLifecycleComponent implements LifecycleComponent {
 
     private final List<LifecycleListener> listeners = new CopyOnWriteArrayList<>();
 
+    protected AbstractLifecycleComponent() {}
+
+    @Deprecated
     protected AbstractLifecycleComponent(Settings settings) {
         // TODO drop settings from ctor
     }

+ 0 - 1
server/src/main/java/org/elasticsearch/discovery/UnicastConfiguredHostsResolver.java

@@ -56,7 +56,6 @@ public class UnicastConfiguredHostsResolver extends AbstractLifecycleComponent i
 
     public UnicastConfiguredHostsResolver(String nodeName, Settings settings, TransportService transportService,
                                           UnicastHostsProvider hostsProvider) {
-        super(settings);
         this.settings = settings;
         this.nodeName = nodeName;
         this.transportService = transportService;

+ 0 - 1
server/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java

@@ -59,7 +59,6 @@ public class SingleNodeDiscovery extends AbstractLifecycleComponent implements D
     public SingleNodeDiscovery(final Settings settings, final TransportService transportService,
                                final MasterService masterService, final ClusterApplier clusterApplier,
                                final GatewayMetaState gatewayMetaState) {
-        super(Objects.requireNonNull(settings));
         this.clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);
         this.transportService = Objects.requireNonNull(transportService);
         masterService.setClusterStateSupplier(() -> clusterState);

+ 0 - 1
server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java

@@ -161,7 +161,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent implements Discover
                         NamedWriteableRegistry namedWriteableRegistry, MasterService masterService, ClusterApplier clusterApplier,
                         ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider, AllocationService allocationService,
                         Collection<BiConsumer<DiscoveryNode, ClusterState>> onJoinValidators, GatewayMetaState gatewayMetaState) {
-        super(settings);
         this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators);
         this.masterService = masterService;
         this.clusterApplier = clusterApplier;

+ 0 - 1
server/src/main/java/org/elasticsearch/gateway/GatewayService.java

@@ -95,7 +95,6 @@ public class GatewayService extends AbstractLifecycleComponent implements Cluste
                           final ThreadPool threadPool,
                           final TransportNodesListGatewayMetaState listGatewayMetaState,
                           final IndicesService indicesService, final Discovery discovery) {
-        super(settings);
         this.allocationService = allocationService;
         this.clusterService = clusterService;
         this.threadPool = threadPool;

+ 0 - 1
server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java

@@ -87,7 +87,6 @@ public abstract class AbstractHttpServerTransport extends AbstractLifecycleCompo
 
     protected AbstractHttpServerTransport(Settings settings, NetworkService networkService, BigArrays bigArrays, ThreadPool threadPool,
                                           NamedXContentRegistry xContentRegistry, Dispatcher dispatcher) {
-        super(settings);
         this.settings = settings;
         this.networkService = networkService;
         this.bigArrays = bigArrays;

+ 0 - 1
server/src/main/java/org/elasticsearch/indices/IndicesService.java

@@ -205,7 +205,6 @@ public class IndicesService extends AbstractLifecycleComponent
                           ScriptService scriptService, Client client, MetaStateService metaStateService,
                           Collection<Function<IndexSettings, Optional<EngineFactory>>> engineFactoryProviders,
                           Map<String, Function<IndexSettings, IndexStore>> indexStoreFactories) {
-        super(settings);
         this.settings = settings;
         this.threadPool = threadPool;
         this.pluginsService = pluginsService;

+ 1 - 3
server/src/main/java/org/elasticsearch/indices/breaker/CircuitBreakerService.java

@@ -23,7 +23,6 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.settings.Settings;
 
 /**
  * Interface for Circuit Breaker services, which provide breakers to classes
@@ -32,8 +31,7 @@ import org.elasticsearch.common.settings.Settings;
 public abstract class CircuitBreakerService extends AbstractLifecycleComponent {
     private static final Logger logger = LogManager.getLogger(CircuitBreakerService.class);
 
-    protected CircuitBreakerService(Settings settings) {
-        super(settings);
+    protected CircuitBreakerService() {
     }
 
     /**

+ 1 - 1
server/src/main/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerService.java

@@ -104,7 +104,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
     private final AtomicLong parentTripCount = new AtomicLong(0);
 
     public HierarchyCircuitBreakerService(Settings settings, ClusterSettings clusterSettings) {
-        super(settings);
+        super();
         this.fielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA,
                 FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
                 FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),

+ 1 - 2
server/src/main/java/org/elasticsearch/indices/breaker/NoneCircuitBreakerService.java

@@ -21,7 +21,6 @@ package org.elasticsearch.indices.breaker;
 
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.breaker.NoopCircuitBreaker;
-import org.elasticsearch.common.settings.Settings;
 
 /**
  * Class that returns a breaker that never breaks
@@ -31,7 +30,7 @@ public class NoneCircuitBreakerService extends CircuitBreakerService {
     private final CircuitBreaker breaker = new NoopCircuitBreaker(CircuitBreaker.FIELDDATA);
 
     public NoneCircuitBreakerService() {
-        super(Settings.EMPTY);
+        super();
     }
 
     @Override

+ 0 - 1
server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java

@@ -171,7 +171,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
             final SnapshotShardsService snapshotShardsService,
             final PrimaryReplicaSyncer primaryReplicaSyncer,
             final Consumer<ShardId> globalCheckpointSyncer) {
-        super(settings);
         this.settings = settings;
         this.buildInIndexListener =
                 Arrays.asList(

+ 0 - 1
server/src/main/java/org/elasticsearch/monitor/MonitorService.java

@@ -42,7 +42,6 @@ public class MonitorService extends AbstractLifecycleComponent {
 
     public MonitorService(Settings settings, NodeEnvironment nodeEnvironment, ThreadPool threadPool,
                           ClusterInfoService clusterInfoService) throws IOException {
-        super(settings);
         this.jvmGcMonitorService = new JvmGcMonitorService(settings, threadPool);
         this.osService = new OsService(settings);
         this.processService = new ProcessService(settings);

+ 0 - 1
server/src/main/java/org/elasticsearch/monitor/jvm/JvmGcMonitorService.java

@@ -107,7 +107,6 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent {
     }
 
     public JvmGcMonitorService(Settings settings, ThreadPool threadPool) {
-        super(settings);
         this.threadPool = threadPool;
 
         this.enabled = ENABLED_SETTING.get(settings);

+ 0 - 1
server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java

@@ -228,7 +228,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
      * @param settings Settings for the node this repository object is created on
      */
     protected BlobStoreRepository(RepositoryMetaData metadata, Settings settings, NamedXContentRegistry namedXContentRegistry) {
-        super(settings);
         this.settings = settings;
         this.metadata = metadata;
         this.namedXContentRegistry = namedXContentRegistry;

+ 0 - 1
server/src/main/java/org/elasticsearch/search/SearchService.java

@@ -193,7 +193,6 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
     public SearchService(ClusterService clusterService, IndicesService indicesService,
                          ThreadPool threadPool, ScriptService scriptService, BigArrays bigArrays, FetchPhase fetchPhase,
                          ResponseCollectorService responseCollectorService) {
-        super(clusterService.getSettings());
         Settings settings = clusterService.getSettings();
         this.threadPool = threadPool;
         this.clusterService = clusterService;

+ 0 - 1
server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java

@@ -119,7 +119,6 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
     public SnapshotShardsService(Settings settings, ClusterService clusterService, SnapshotsService snapshotsService,
                                  ThreadPool threadPool, TransportService transportService, IndicesService indicesService,
                                  ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
-        super(settings);
         this.indicesService = indicesService;
         this.snapshotsService = snapshotsService;
         this.transportService = transportService;

+ 0 - 1
server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java

@@ -124,7 +124,6 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
     @Inject
     public SnapshotsService(Settings settings, ClusterService clusterService, IndexNameExpressionResolver indexNameExpressionResolver,
                             RepositoriesService repositoriesService, ThreadPool threadPool) {
-        super(settings);
         this.clusterService = clusterService;
         this.indexNameExpressionResolver = indexNameExpressionResolver;
         this.repositoriesService = repositoriesService;

+ 0 - 1
server/src/main/java/org/elasticsearch/transport/TcpTransport.java

@@ -156,7 +156,6 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
     public TcpTransport(String transportName, Settings settings,  Version version, ThreadPool threadPool,
                         PageCacheRecycler pageCacheRecycler, CircuitBreakerService circuitBreakerService,
                         NamedWriteableRegistry namedWriteableRegistry, NetworkService networkService) {
-        super(settings);
         this.settings = settings;
         this.profileSettings = getProfileSettings(settings);
         this.version = version;

+ 0 - 1
server/src/main/java/org/elasticsearch/transport/TransportService.java

@@ -155,7 +155,6 @@ public class TransportService extends AbstractLifecycleComponent implements Tran
     public TransportService(Settings settings, Transport transport, ThreadPool threadPool, TransportInterceptor transportInterceptor,
                             Function<BoundTransportAddress, DiscoveryNode> localNodeFactory, @Nullable ClusterSettings clusterSettings,
                             Set<String> taskHeaders, ConnectionManager connectionManager) {
-        super(settings);
         // The only time we do not want to validate node connections is when this is a transport client using the simple node sampler
         this.validateConnections = TransportClient.CLIENT_TYPE.equals(settings.get(Client.CLIENT_TYPE_SETTING_S.getKey())) == false ||
             TransportClient.CLIENT_TRANSPORT_SNIFF.get(settings);

+ 0 - 1
server/src/main/java/org/elasticsearch/watcher/ResourceWatcherService.java

@@ -90,7 +90,6 @@ public class ResourceWatcherService extends AbstractLifecycleComponent {
 
     @Inject
     public ResourceWatcherService(Settings settings, ThreadPool threadPool) {
-        super(settings);
         this.enabled = ENABLED.get(settings);
         this.threadPool = threadPool;
 

+ 4 - 4
server/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java

@@ -69,7 +69,7 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase {
         threadPool = new TestThreadPool(getTestName());
         clusterService = mock(ClusterService.class);
         allocationService = createAllocationService(Settings.EMPTY, new DelayedShardsMockGatewayAllocator());
-        delayedAllocationService = new TestDelayAllocationService(Settings.EMPTY, threadPool, clusterService, allocationService);
+        delayedAllocationService = new TestDelayAllocationService(threadPool, clusterService, allocationService);
         verify(clusterService).addListener(delayedAllocationService);
     }
 
@@ -464,9 +464,9 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase {
     private static class TestDelayAllocationService extends DelayedAllocationService {
         private volatile long nanoTimeOverride = -1L;
 
-        TestDelayAllocationService(Settings settings, ThreadPool threadPool, ClusterService clusterService,
-                                   AllocationService allocationService) {
-            super(settings, threadPool, clusterService, allocationService);
+        private TestDelayAllocationService(ThreadPool threadPool, ClusterService clusterService,
+                                           AllocationService allocationService) {
+            super(threadPool, clusterService, allocationService);
         }
 
         @Override

+ 1 - 2
server/src/test/java/org/elasticsearch/cluster/routing/RoutingServiceTests.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.cluster.routing;
 
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.cluster.ESAllocationTestCase;
 import org.junit.Before;
 
@@ -47,7 +46,7 @@ public class RoutingServiceTests extends ESAllocationTestCase {
         private AtomicBoolean rerouted = new AtomicBoolean();
 
         TestRoutingService() {
-            super(Settings.EMPTY, null, null);
+            super(null, null);
         }
 
         public boolean hasReroutedAndClear() {

+ 0 - 3
server/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java

@@ -72,9 +72,6 @@ public class NetworkModuleTests extends ESTestCase {
     }
 
     static class FakeHttpTransport extends AbstractLifecycleComponent implements HttpServerTransport {
-        FakeHttpTransport() {
-            super(null);
-        }
         @Override
         protected void doStart() {}
         @Override

+ 0 - 1
server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java

@@ -3126,7 +3126,6 @@ public class IndexShardTests extends IndexShardTestCase {
         private final String indexName;
 
         RestoreOnlyRepository(String indexName) {
-            super(Settings.EMPTY);
             this.indexName = indexName;
         }
 

+ 0 - 1
server/src/test/java/org/elasticsearch/rest/RestControllerTests.java

@@ -474,7 +474,6 @@ public class RestControllerTests extends ESTestCase {
         HttpServerTransport {
 
         TestHttpServerTransport() {
-            super(Settings.EMPTY);
         }
 
         @Override

+ 1 - 1
server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java

@@ -463,7 +463,7 @@ public class SnapshotsServiceTests extends ESTestCase {
                 transportService, indicesService, actionFilters, indexNameExpressionResolver);
             final ShardStateAction shardStateAction = new ShardStateAction(
                 clusterService, transportService, allocationService,
-                new RoutingService(settings, clusterService, allocationService),
+                new RoutingService(clusterService, allocationService),
                 deterministicTaskQueue.getThreadPool()
             );
             indicesClusterStateService = new IndicesClusterStateService(

+ 0 - 5
test/framework/src/main/java/org/elasticsearch/test/MockHttpTransport.java

@@ -20,7 +20,6 @@
 package org.elasticsearch.test;
 
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.http.HttpInfo;
@@ -46,10 +45,6 @@ public class MockHttpTransport extends AbstractLifecycleComponent implements Htt
     private static final HttpInfo DUMMY_HTTP_INFO = new HttpInfo(DUMMY_BOUND_ADDRESS, 0);
     private static final HttpStats DUMMY_HTTP_STATS = new HttpStats(0, 0);
 
-    public MockHttpTransport() {
-        super(Settings.EMPTY);
-    }
-
     @Override
     protected void doStart() {}
 

+ 1 - 1
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java

@@ -157,7 +157,7 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E
             return emptyList();
         }
 
-        CcrRestoreSourceService restoreSourceService = new CcrRestoreSourceService(settings);
+        CcrRestoreSourceService restoreSourceService = new CcrRestoreSourceService();
         this.restoreSourceService.set(restoreSourceService);
         return Arrays.asList(
             ccrLicenseChecker,

+ 0 - 1
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRepositoryManager.java

@@ -29,7 +29,6 @@ class CcrRepositoryManager extends AbstractLifecycleComponent {
     private final RemoteSettingsUpdateListener updateListener;
 
     CcrRepositoryManager(Settings settings, ClusterService clusterService, Client client) {
-        super(settings);
         this.client = client;
         updateListener = new RemoteSettingsUpdateListener(settings);
         updateListener.listenForUpdates(clusterService.getClusterSettings());

+ 0 - 1
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java

@@ -84,7 +84,6 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
     private final CcrLicenseChecker ccrLicenseChecker;
 
     public CcrRepository(RepositoryMetaData metadata, Client client, CcrLicenseChecker ccrLicenseChecker, Settings settings) {
-        super(settings);
         this.metadata = metadata;
         assert metadata.name().startsWith(NAME_PREFIX) : "CcrRepository metadata.name() must start with: " + NAME_PREFIX;
         this.remoteClusterAlias = Strings.split(metadata.name(), NAME_PREFIX)[1];

+ 0 - 4
x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java

@@ -48,10 +48,6 @@ public class CcrRestoreSourceService extends AbstractLifecycleComponent implemen
     private final CopyOnWriteArrayList<Consumer<String>> openSessionListeners = new CopyOnWriteArrayList<>();
     private final CopyOnWriteArrayList<Consumer<String>> closeSessionListeners = new CopyOnWriteArrayList<>();
 
-    public CcrRestoreSourceService(Settings settings) {
-        super(settings);
-    }
-
     @Override
     public synchronized void afterIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) {
         if (indexShard != null) {

+ 1 - 1
x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java

@@ -29,7 +29,7 @@ public class CcrRestoreSourceServiceTests extends IndexShardTestCase {
     @Before
     public void setUp() throws Exception {
         super.setUp();
-        restoreSourceService = new CcrRestoreSourceService(Settings.EMPTY);
+        restoreSourceService = new CcrRestoreSourceService();
     }
 
     public void testOpenSession() throws IOException {

+ 0 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/license/LicenseService.java

@@ -121,7 +121,6 @@ public class LicenseService extends AbstractLifecycleComponent implements Cluste
 
     public LicenseService(Settings settings, ClusterService clusterService, Clock clock, Environment env,
                           ResourceWatcherService resourceWatcherService, XPackLicenseState licenseState) {
-        super(settings);
         this.settings = settings;
         this.clusterService = clusterService;
         this.clock = clock;

+ 0 - 1
x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringService.java

@@ -91,7 +91,6 @@ public class MonitoringService extends AbstractLifecycleComponent {
 
     MonitoringService(Settings settings, ClusterService clusterService, ThreadPool threadPool,
                       Set<Collector> collectors, Exporters exporters) {
-        super(settings);
         this.clusterService = Objects.requireNonNull(clusterService);
         this.threadPool = Objects.requireNonNull(threadPool);
         this.collectors = Objects.requireNonNull(collectors);

+ 0 - 1
x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/cleaner/CleanerService.java

@@ -40,7 +40,6 @@ public class CleanerService extends AbstractLifecycleComponent {
 
     CleanerService(Settings settings, ClusterSettings clusterSettings, XPackLicenseState licenseState, ThreadPool threadPool,
                    ExecutionScheduler executionScheduler) {
-        super(settings);
         this.licenseState = licenseState;
         this.threadPool = threadPool;
         this.executionScheduler = executionScheduler;

+ 0 - 1
x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/Exporters.java

@@ -51,7 +51,6 @@ public class Exporters extends AbstractLifecycleComponent {
     public Exporters(Settings settings, Map<String, Exporter.Factory> factories,
                      ClusterService clusterService, XPackLicenseState licenseState,
                      ThreadContext threadContext) {
-        super(settings);
         this.settings = settings;
         this.factories = factories;
         this.exporters = new AtomicReference<>(emptyMap());

+ 0 - 1
x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/MonitoringServiceTests.java

@@ -173,7 +173,6 @@ public class MonitoringServiceTests extends ESTestCase {
         private final CountDownLatch latch;
 
         BlockingExporter(CountDownLatch latch) {
-            super();
             this.latch = latch;
         }