Internal: Remove generics from LifecycleComponent

The only reason for LifecycleComponent taking a generic type was so that
it could return that type on its start and stop methods. However, this
chaining has no practical necessity. Instead, start and stop can be
void, and a whole bunch of confusing generics disappear.
This commit is contained in:
Ryan Ernst 2016-07-01 16:17:42 -07:00
parent 62397c0c3d
commit 822c995367
55 changed files with 72 additions and 99 deletions

View File

@ -46,7 +46,7 @@ import static org.elasticsearch.common.settings.Setting.positiveTimeSetting;
* to pings. This is done by {@link org.elasticsearch.discovery.zen.fd.NodesFaultDetection}. Master fault detection
* is done by {@link org.elasticsearch.discovery.zen.fd.MasterFaultDetection}.
*/
public class NodeConnectionsService extends AbstractLifecycleComponent<NodeConnectionsService> {
public class NodeConnectionsService extends AbstractLifecycleComponent {
public static final Setting<TimeValue> CLUSTER_NODE_RECONNECT_INTERVAL_SETTING =
positiveTimeSetting("cluster.nodes.reconnect_interval", TimeValue.timeValueSeconds(10), Property.NodeScope);

View File

@ -51,7 +51,7 @@ import java.util.concurrent.atomic.AtomicReference;
* {@link AllocationService#removeDelayMarkers(RoutingAllocation)}, triggering yet
* another cluster change event.
*/
public class DelayedAllocationService extends AbstractLifecycleComponent<DelayedAllocationService> implements ClusterStateListener {
public class DelayedAllocationService extends AbstractLifecycleComponent implements ClusterStateListener {
static final String CLUSTER_UPDATE_TASK_SOURCE = "delayed_allocation_reroute";

View File

@ -44,7 +44,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
* actions.
* </p>
*/
public class RoutingService extends AbstractLifecycleComponent<RoutingService> {
public class RoutingService extends AbstractLifecycleComponent {
private static final String CLUSTER_UPDATE_TASK_SOURCE = "cluster_reroute";

View File

@ -88,7 +88,7 @@ import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadF
/**
*
*/
public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
public class ClusterService extends AbstractLifecycleComponent {
public static final Setting<TimeValue> CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING =
Setting.positiveTimeSetting("cluster.service.slow_task_logging_threshold", TimeValue.timeValueSeconds(30),

View File

@ -27,7 +27,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
/**
*
*/
public abstract class AbstractLifecycleComponent<T> extends AbstractComponent implements LifecycleComponent<T> {
public abstract class AbstractLifecycleComponent extends AbstractComponent implements LifecycleComponent {
protected final Lifecycle lifecycle = new Lifecycle();
@ -58,9 +58,9 @@ public abstract class AbstractLifecycleComponent<T> extends AbstractComponent im
@SuppressWarnings({"unchecked"})
@Override
public T start() {
public void start() {
if (!lifecycle.canMoveToStarted()) {
return (T) this;
return;
}
for (LifecycleListener listener : listeners) {
listener.beforeStart();
@ -70,16 +70,15 @@ public abstract class AbstractLifecycleComponent<T> extends AbstractComponent im
for (LifecycleListener listener : listeners) {
listener.afterStart();
}
return (T) this;
}
protected abstract void doStart();
@SuppressWarnings({"unchecked"})
@Override
public T stop() {
public void stop() {
if (!lifecycle.canMoveToStopped()) {
return (T) this;
return;
}
for (LifecycleListener listener : listeners) {
listener.beforeStop();
@ -89,7 +88,6 @@ public abstract class AbstractLifecycleComponent<T> extends AbstractComponent im
for (LifecycleListener listener : listeners) {
listener.afterStop();
}
return (T) this;
}
protected abstract void doStop();

View File

@ -24,7 +24,7 @@ import org.elasticsearch.common.lease.Releasable;
/**
*
*/
public interface LifecycleComponent<T> extends Releasable {
public interface LifecycleComponent extends Releasable {
Lifecycle.State lifecycleState();
@ -32,7 +32,7 @@ public interface LifecycleComponent<T> extends Releasable {
void removeLifecycleListener(LifecycleListener listener);
T start();
void start();
T stop();
void stop();
}

View File

@ -34,7 +34,7 @@ import java.io.IOException;
* state to all nodes, electing a master of the cluster that raises cluster state change
* events.
*/
public interface Discovery extends LifecycleComponent<Discovery> {
public interface Discovery extends LifecycleComponent {
DiscoveryNode localNode();

View File

@ -56,7 +56,7 @@ import static org.elasticsearch.cluster.ClusterState.Builder;
/**
*
*/
public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implements Discovery {
public class LocalDiscovery extends AbstractLifecycleComponent implements Discovery {
private static final LocalDiscovery[] NO_MEMBERS = new LocalDiscovery[0];

View File

@ -86,7 +86,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
/**
*
*/
public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implements Discovery, PingContextProvider {
public class ZenDiscovery extends AbstractLifecycleComponent implements Discovery, PingContextProvider {
public static final Setting<TimeValue> PING_TIMEOUT_SETTING =
Setting.positiveTimeSetting("discovery.zen.ping_timeout", timeValueSeconds(3), Property.NodeScope);

View File

@ -32,7 +32,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
public interface ZenPing extends LifecycleComponent<ZenPing> {
public interface ZenPing extends LifecycleComponent {
void setPingContextProvider(PingContextProvider contextProvider);

View File

@ -33,7 +33,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
public class ZenPingService extends AbstractLifecycleComponent<ZenPing> implements ZenPing {
public class ZenPingService extends AbstractLifecycleComponent implements ZenPing {
private List<ZenPing> zenPings = Collections.emptyList();

View File

@ -86,7 +86,7 @@ import static org.elasticsearch.discovery.zen.ping.ZenPing.PingResponse.readPing
/**
*
*/
public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implements ZenPing {
public class UnicastZenPing extends AbstractLifecycleComponent implements ZenPing {
public static final String ACTION_NAME = "internal:discovery/zen/unicast";
public static final Setting<List<String>> DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING =

View File

@ -52,7 +52,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
/**
*
*/
public class GatewayService extends AbstractLifecycleComponent<GatewayService> implements ClusterStateListener {
public class GatewayService extends AbstractLifecycleComponent implements ClusterStateListener {
public static final Setting<Integer> EXPECTED_NODES_SETTING =
Setting.intSetting("gateway.expected_nodes", -1, -1, Property.NodeScope);

View File

@ -51,7 +51,7 @@ import static org.elasticsearch.rest.RestStatus.INTERNAL_SERVER_ERROR;
/**
* A component to serve http requests, backed by rest handlers.
*/
public class HttpServer extends AbstractLifecycleComponent<HttpServer> implements HttpServerAdapter {
public class HttpServer extends AbstractLifecycleComponent implements HttpServerAdapter {
private final HttpServerTransport transport;
private final RestController restController;

View File

@ -25,7 +25,7 @@ import org.elasticsearch.common.transport.BoundTransportAddress;
/**
*
*/
public interface HttpServerTransport extends LifecycleComponent<HttpServerTransport> {
public interface HttpServerTransport extends LifecycleComponent {
BoundTransportAddress boundAddress();

View File

@ -113,7 +113,7 @@ import static org.elasticsearch.http.netty.cors.CorsHandler.ANY_ORIGIN;
/**
*
*/
public class NettyHttpServerTransport extends AbstractLifecycleComponent<HttpServerTransport> implements HttpServerTransport {
public class NettyHttpServerTransport extends AbstractLifecycleComponent implements HttpServerTransport {
static {
NettyUtils.setup();

View File

@ -135,7 +135,7 @@ import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList;
/**
*
*/
public class IndicesService extends AbstractLifecycleComponent<IndicesService>
public class IndicesService extends AbstractLifecycleComponent
implements IndicesClusterStateService.AllocatedIndices<IndexShard, IndexService>, IndexService.ShardStoreDeleter {
public static final String INDICES_SHARDS_CLOSED_TIMEOUT = "indices.shards_closed_timeout";

View File

@ -27,7 +27,7 @@ import org.elasticsearch.common.settings.Settings;
* Interface for Circuit Breaker services, which provide breakers to classes
* that load field data.
*/
public abstract class CircuitBreakerService extends AbstractLifecycleComponent<CircuitBreakerService> {
public abstract class CircuitBreakerService extends AbstractLifecycleComponent {
protected CircuitBreakerService(Settings settings) {
super(settings);

View File

@ -82,7 +82,7 @@ import java.util.concurrent.TimeUnit;
/**
*
*/
public class IndicesClusterStateService extends AbstractLifecycleComponent<IndicesClusterStateService> implements ClusterStateListener {
public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateListener {
final AllocatedIndices<? extends Shard, ? extends AllocatedIndex<? extends Shard>> indicesService;
private final ClusterService clusterService;

View File

@ -66,7 +66,7 @@ import java.util.concurrent.locks.ReentrantLock;
/**
* A node level service that delete expired docs on node primary shards.
*/
public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLService> {
public class IndicesTTLService extends AbstractLifecycleComponent {
public static final Setting<TimeValue> INDICES_TTL_INTERVAL_SETTING =
Setting.positiveTimeSetting("indices.ttl.interval", TimeValue.timeValueSeconds(60),

View File

@ -31,7 +31,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
public class MonitorService extends AbstractLifecycleComponent<MonitorService> {
public class MonitorService extends AbstractLifecycleComponent {
private final JvmGcMonitorService jvmGcMonitorService;
private final OsService osService;

View File

@ -40,7 +40,7 @@ import java.util.function.BiFunction;
import static java.util.Collections.unmodifiableMap;
public class JvmGcMonitorService extends AbstractLifecycleComponent<JvmGcMonitorService> {
public class JvmGcMonitorService extends AbstractLifecycleComponent {
private final ThreadPool threadPool;
private final boolean enabled;

View File

@ -47,7 +47,7 @@ import java.util.function.Predicate;
* with possible list of failures</li>
* </ul>
*/
public interface Repository extends LifecycleComponent<Repository> {
public interface Repository extends LifecycleComponent {
/**
* Reads snapshot description from repository.

View File

@ -115,7 +115,7 @@ import java.util.stream.Collectors;
* }
* </pre>
*/
public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Repository> implements Repository, RateLimiterListener {
public abstract class BlobStoreRepository extends AbstractLifecycleComponent implements Repository, RateLimiterListener {
private BlobContainer snapshotsBlobContainer;

View File

@ -44,7 +44,7 @@ import static org.elasticsearch.rest.RestStatus.OK;
/**
*
*/
public class RestController extends AbstractLifecycleComponent<RestController> {
public class RestController extends AbstractLifecycleComponent {
private final PathTrie<RestHandler> getHandlers = new PathTrie<>(RestUtils.REST_DECODER);
private final PathTrie<RestHandler> postHandlers = new PathTrie<>(RestUtils.REST_DECODER);
private final PathTrie<RestHandler> putHandlers = new PathTrie<>(RestUtils.REST_DECODER);

View File

@ -105,7 +105,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes;
/**
*
*/
public class SearchService extends AbstractLifecycleComponent<SearchService> implements IndexEventListener {
public class SearchService extends AbstractLifecycleComponent implements IndexEventListener {
// we can have 5 minutes here, since we make sure to clean with search requests and when shard/index closes
public static final Setting<TimeValue> DEFAULT_KEEPALIVE_SETTING =

View File

@ -75,7 +75,7 @@ import static org.elasticsearch.cluster.SnapshotsInProgress.completed;
* This service runs on data and master nodes and controls currently snapshotted shards on these nodes. It is responsible for
* starting and stopping shard level snapshots
*/
public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotShardsService> implements ClusterStateListener {
public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateListener {
public static final String UPDATE_SNAPSHOT_ACTION_NAME = "internal:cluster/snapshot/update_snapshot";

View File

@ -98,7 +98,7 @@ import static org.elasticsearch.cluster.SnapshotsInProgress.completed;
* notifies all {@link #snapshotCompletionListeners} that snapshot is completed, and finally calls {@link #removeSnapshotFromClusterState(Snapshot, SnapshotInfo, Throwable)} to remove snapshot from cluster state</li>
* </ul>
*/
public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsService> implements ClusterStateListener {
public class SnapshotsService extends AbstractLifecycleComponent implements ClusterStateListener {
private final ClusterService clusterService;

View File

@ -105,7 +105,7 @@ import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.new
/**
*/
public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent<Transport> implements Transport {
public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent implements Transport {
public static final String HTTP_SERVER_WORKER_THREAD_NAME_PREFIX = "http_server_worker";
public static final String HTTP_SERVER_BOSS_THREAD_NAME_PREFIX = "http_server_boss";

View File

@ -35,7 +35,7 @@ import java.util.Map;
/**
*
*/
public interface Transport extends LifecycleComponent<Transport> {
public interface Transport extends LifecycleComponent {
Setting<Boolean> TRANSPORT_TCP_COMPRESS = Setting.boolSetting("transport.tcp.compress", false, Property.NodeScope);

View File

@ -69,7 +69,7 @@ import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
/**
*
*/
public class TransportService extends AbstractLifecycleComponent<TransportService> {
public class TransportService extends AbstractLifecycleComponent {
public static final String DIRECT_RESPONSE_PROFILE = ".direct";
private static final String HANDSHAKE_ACTION_NAME = "internal:transport/handshake";

View File

@ -73,7 +73,7 @@ import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.new
/**
*
*/
public class LocalTransport extends AbstractLifecycleComponent<Transport> implements Transport {
public class LocalTransport extends AbstractLifecycleComponent implements Transport {
public static final String LOCAL_TRANSPORT_THREAD_NAME_PREFIX = "local_transport";
final ThreadPool threadPool;

View File

@ -86,7 +86,7 @@ import static java.util.Collections.unmodifiableMap;
* in another cluster, the conflict one will be discarded. This happens because we need to have the correct index name
* to propagate to the relevant cluster.
*/
public class TribeService extends AbstractLifecycleComponent<TribeService> {
public class TribeService extends AbstractLifecycleComponent {
public static final ClusterBlock TRIBE_METADATA_BLOCK = new ClusterBlock(10, "tribe node, metadata not allowed", false, false,
RestStatus.BAD_REQUEST, EnumSet.of(ClusterBlockLevel.METADATA_READ, ClusterBlockLevel.METADATA_WRITE));

View File

@ -40,7 +40,7 @@ import java.util.concurrent.ScheduledFuture;
* registered watcher periodically. The frequency of checks can be specified using {@code resource.reload.interval} setting, which
* defaults to {@code 60s}. The service can be disabled by setting {@code resource.reload.enabled} setting to {@code false}.
*/
public class ResourceWatcherService extends AbstractLifecycleComponent<ResourceWatcherService> {
public class ResourceWatcherService extends AbstractLifecycleComponent {
public enum Frequency {

View File

@ -187,19 +187,13 @@ abstract class FailAndRetryMockTransport<Response extends TransportResponse> imp
}
@Override
public Transport start() {
return null;
}
public void start() {}
@Override
public Transport stop() {
return null;
}
public void stop() {}
@Override
public void close() {
}
public void close() {}
@Override
public Map<String, BoundTransportAddress> profileBoundAddresses() {

View File

@ -253,18 +253,12 @@ public class NodeConnectionsServiceTests extends ESTestCase {
}
@Override
public Transport start() {
return null;
}
public void start() {}
@Override
public Transport stop() {
return null;
}
public void stop() {}
@Override
public void close() {
}
public void close() {}
}
}

View File

@ -591,7 +591,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
}
@Singleton
public static class MasterAwareService extends AbstractLifecycleComponent<MasterAwareService> implements LocalNodeMasterListener {
public static class MasterAwareService extends AbstractLifecycleComponent implements LocalNodeMasterListener {
private final ClusterService clusterService;
private volatile boolean master;

View File

@ -59,7 +59,7 @@ public class NetworkModuleTests extends ModuleTestCase {
}
}
static class FakeHttpTransport extends AbstractLifecycleComponent<HttpServerTransport> implements HttpServerTransport {
static class FakeHttpTransport extends AbstractLifecycleComponent implements HttpServerTransport {
public FakeHttpTransport() {
super(null);
}

View File

@ -120,7 +120,7 @@ public class HttpServerTests extends ESTestCase {
assertEquals(0, inFlightRequestsBreaker.getUsed());
}
private static final class TestHttpServerTransport extends AbstractLifecycleComponent<HttpServerTransport> implements
private static final class TestHttpServerTransport extends AbstractLifecycleComponent implements
HttpServerTransport {
public TestHttpServerTransport() {

View File

@ -142,9 +142,9 @@ public class NettyHttpServerPipeliningTests extends ESTestCase {
}
@Override
public HttpServerTransport stop() {
public void stop() {
executorService.shutdownNow();
return super.stop();
super.stop();
}
}

View File

@ -86,17 +86,11 @@ public class NoopDiscovery implements Discovery {
}
@Override
public Discovery start() {
return null;
}
public void start() {}
@Override
public Discovery stop() {
return null;
}
public void stop() {}
@Override
public void close() {
}
public void close() {}
}

View File

@ -39,7 +39,7 @@ import java.util.ServiceLoader;
/**
*
*/
public class AzureComputeServiceImpl extends AbstractLifecycleComponent<AzureComputeServiceImpl>
public class AzureComputeServiceImpl extends AbstractLifecycleComponent
implements AzureComputeService {
private final ComputeManagementClient client;

View File

@ -26,7 +26,7 @@ import org.elasticsearch.common.settings.Settings;
/**
*
*/
public abstract class AzureComputeServiceAbstractMock extends AbstractLifecycleComponent<AzureComputeServiceAbstractMock>
public abstract class AzureComputeServiceAbstractMock extends AbstractLifecycleComponent
implements AzureComputeService {
protected AzureComputeServiceAbstractMock(Settings settings) {

View File

@ -49,7 +49,7 @@ import java.util.Random;
/**
*
*/
public class AwsEc2ServiceImpl extends AbstractLifecycleComponent<AwsEc2Service> implements AwsEc2Service {
public class AwsEc2ServiceImpl extends AbstractLifecycleComponent implements AwsEc2Service {
public static final String EC2_METADATA_URL = "http://169.254.169.254/latest/meta-data/";

View File

@ -27,7 +27,7 @@ import org.elasticsearch.common.settings.Settings;
import java.util.List;
public class AwsEc2ServiceMock extends AbstractLifecycleComponent<AwsEc2Service> implements AwsEc2Service {
public class AwsEc2ServiceMock extends AbstractLifecycleComponent implements AwsEc2Service {
private int nodes;
private List<List<Tag>> tagsList;

View File

@ -31,7 +31,7 @@ import java.util.Collections;
import java.util.List;
import java.util.function.Function;
public interface GceComputeService extends LifecycleComponent<GceComputeService> {
public interface GceComputeService extends LifecycleComponent {
/**
* GCE API Version: Elasticsearch/GceCloud/1.0

View File

@ -56,7 +56,7 @@ import java.util.Collections;
import java.util.List;
import java.util.function.Function;
public class GceComputeServiceImpl extends AbstractLifecycleComponent<GceComputeService>
public class GceComputeServiceImpl extends AbstractLifecycleComponent
implements GceComputeService {
// all settings just used for testing - not registered by default

View File

@ -56,7 +56,8 @@ public class AzureBlobStore extends AbstractComponent implements BlobStore {
public AzureBlobStore(RepositoryName name, Settings settings, RepositorySettings repositorySettings,
AzureStorageService client) throws URISyntaxException, StorageException {
super(settings);
this.client = client.start();
this.client = client;
client.start();
this.container = getValue(repositorySettings, Repository.CONTAINER_SETTING, Storage.CONTAINER_SETTING);
this.repositoryName = name.getName();
this.accountName = getValue(repositorySettings, Repository.ACCOUNT_SETTING, Storage.ACCOUNT_SETTING);

View File

@ -86,5 +86,5 @@ public interface AzureStorageService {
void moveBlob(String account, LocationMode mode, String container, String sourceBlob, String targetBlob)
throws URISyntaxException, StorageException;
AzureStorageService start();
void start();
}

View File

@ -45,7 +45,7 @@ import java.net.URISyntaxException;
import java.util.HashMap;
import java.util.Map;
public class AzureStorageServiceImpl extends AbstractLifecycleComponent<AzureStorageServiceImpl>
public class AzureStorageServiceImpl extends AbstractLifecycleComponent
implements AzureStorageService {
final AzureStorageSettings primaryStorageSettings;

View File

@ -41,7 +41,7 @@ import java.util.concurrent.ConcurrentHashMap;
/**
* In memory storage for unit tests
*/
public class AzureStorageServiceMock extends AbstractLifecycleComponent<AzureStorageServiceMock>
public class AzureStorageServiceMock extends AbstractLifecycleComponent
implements AzureStorageService {
protected Map<String, ByteArrayOutputStream> blobs = new ConcurrentHashMap<>();

View File

@ -31,7 +31,7 @@ import java.util.function.Function;
/**
*
*/
public interface AwsS3Service extends LifecycleComponent<AwsS3Service> {
public interface AwsS3Service extends LifecycleComponent {
// Global AWS settings (shared between discovery-ec2 and repository-s3)
// Each setting starting with `cloud.aws` also exists in discovery-ec2 project. Don't forget to update

View File

@ -44,7 +44,7 @@ import java.util.Map;
/**
*
*/
public class InternalAwsS3Service extends AbstractLifecycleComponent<AwsS3Service> implements AwsS3Service {
public class InternalAwsS3Service extends AbstractLifecycleComponent implements AwsS3Service {
/**
* (acceskey, endpoint) -&gt; client

View File

@ -242,19 +242,13 @@ public class CapturingTransport implements Transport {
}
@Override
public Transport start() {
return null;
}
public void start() {}
@Override
public Transport stop() {
return null;
}
public void stop() {}
@Override
public void close() {
}
public void close() {}
@Override
public List<String> getLocalAddresses() {

View File

@ -558,15 +558,13 @@ public class MockTransportService extends TransportService {
}
@Override
public Transport start() {
public void start() {
transport.start();
return this;
}
@Override
public Transport stop() {
public void stop() {
transport.stop();
return this;
}
@Override