Internal: Remove unused methods and classes

This change removes a handful of classes and methods that were simply
unused. Some of the classes were intermediate abstract classes that
added nothing to the base class they extended.
This commit is contained in:
Ryan Ernst 2016-07-02 11:33:16 -07:00
parent c7b9489be8
commit 5cf7583bde
24 changed files with 66 additions and 249 deletions

View File

@ -20,7 +20,7 @@
package org.elasticsearch.action;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportResponse;
@ -31,7 +31,7 @@ import java.util.function.Supplier;
* A simple base class for action response listeners, defaulting to using the SAME executor (as its
* very common on response handlers).
*/
public class ActionListenerResponseHandler<Response extends TransportResponse> extends BaseTransportResponseHandler<Response> {
public class ActionListenerResponseHandler<Response extends TransportResponse> implements TransportResponseHandler<Response> {
private final ActionListener<Response> listener;
private final Supplier<Response> responseSupplier;

View File

@ -44,9 +44,9 @@ import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.tasks.TaskPersistenceService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
@ -111,7 +111,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
GetTaskRequest nodeRequest = request.nodeRequest(clusterService.localNode().getId(), thisTask.getId());
taskManager.registerChildTask(thisTask, node.getId());
transportService.sendRequest(node, GetTaskAction.NAME, nodeRequest, builder.build(),
new BaseTransportResponseHandler<GetTaskResponse>() {
new TransportResponseHandler<GetTaskResponse>() {
@Override
public GetTaskResponse newInstance() {
return new GetTaskResponse();

View File

@ -37,7 +37,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequestHandler;
@ -177,7 +177,7 @@ public abstract class TransportBroadcastAction<Request extends BroadcastRequest<
onOperation(shard, shardIt, shardIndex, new NoShardAvailableActionException(shardIt.shardId()));
} else {
taskManager.registerChildTask(task, node.getId());
transportService.sendRequest(node, transportShardAction, shardRequest, new BaseTransportResponseHandler<ShardResponse>() {
transportService.sendRequest(node, transportShardAction, shardRequest, new TransportResponseHandler<ShardResponse>() {
@Override
public ShardResponse newInstance() {
return newShardResponse();

View File

@ -46,7 +46,7 @@ import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.NodeShouldNotConnectException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -318,7 +318,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId());
taskManager.registerChildTask(task, node.getId());
}
transportService.sendRequest(node, transportNodeBroadcastAction, nodeRequest, new BaseTransportResponseHandler<NodeResponse>() {
transportService.sendRequest(node, transportNodeBroadcastAction, nodeRequest, new TransportResponseHandler<NodeResponse>() {
@Override
public NodeResponse newInstance() {
return new NodeResponse();

View File

@ -31,7 +31,7 @@ import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.NodeShouldNotConnectException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -204,7 +204,7 @@ public abstract class TransportNodesAction<NodesRequest extends BaseNodesRequest
}
transportService.sendRequest(node, transportNodeAction, nodeRequest, builder.build(),
new BaseTransportResponseHandler<NodeResponse>() {
new TransportResponseHandler<NodeResponse>() {
@Override
public NodeResponse newInstance() {
return newNodeResponse();

View File

@ -55,7 +55,7 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportChannelResponseHandler;
@ -628,7 +628,7 @@ public abstract class TransportReplicationAction<
}
private void performAction(final DiscoveryNode node, final String action, final boolean isPrimaryAction) {
transportService.sendRequest(node, action, request, transportOptions, new BaseTransportResponseHandler<Response>() {
transportService.sendRequest(node, action, request, transportOptions, new TransportResponseHandler<Response>() {
@Override
public Response newInstance() {

View File

@ -39,7 +39,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -174,7 +174,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
request.shardId = shardIt.shardId();
DiscoveryNode node = nodes.get(shard.currentNodeId());
transportService.sendRequest(node, shardActionName, request, transportOptions(), new BaseTransportResponseHandler<Response>() {
transportService.sendRequest(node, shardActionName, request, transportOptions(), new TransportResponseHandler<Response>() {
@Override
public Response newInstance() {

View File

@ -39,7 +39,7 @@ import org.elasticsearch.common.logging.LoggerMessageFormat;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequestHandler;
@ -159,7 +159,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
public void start() {
if (shardIt == null) {
// just execute it on the local node
transportService.sendRequest(clusterService.localNode(), transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() {
transportService.sendRequest(clusterService.localNode(), transportShardAction, internalRequest.request(), new TransportResponseHandler<Response>() {
@Override
public Response newInstance() {
return newResponse();
@ -224,7 +224,7 @@ public abstract class TransportSingleShardAction<Request extends SingleShardRequ
node
);
}
transportService.sendRequest(node, transportShardAction, internalRequest.request(), new BaseTransportResponseHandler<Response>() {
transportService.sendRequest(node, transportShardAction, internalRequest.request(), new TransportResponseHandler<Response>() {
@Override
public Response newInstance() {

View File

@ -38,7 +38,7 @@ import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.NodeShouldNotConnectException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -237,7 +237,7 @@ public abstract class TransportTasksAction<
nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId());
taskManager.registerChildTask(task, node.getId());
transportService.sendRequest(node, transportNodeAction, nodeRequest, builder.build(),
new BaseTransportResponseHandler<NodeTasksResponse>() {
new TransportResponseHandler<NodeTasksResponse>() {
@Override
public NodeTasksResponse newInstance() {
return new NodeTasksResponse();

View File

@ -41,7 +41,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.FutureTransportResponseHandler;
import org.elasticsearch.transport.TransportException;
@ -459,7 +459,7 @@ public class TransportClientNodesService extends AbstractComponent implements Cl
Requests.clusterStateRequest().clear().nodes(true).local(true),
TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE)
.withTimeout(pingTimeout).build(),
new BaseTransportResponseHandler<ClusterStateResponse>() {
new TransportResponseHandler<ClusterStateResponse>() {
@Override
public ClusterStateResponse newInstance() {

View File

@ -1,28 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.io;
import java.io.Reader;
/**
*
*/
public abstract class CharSequenceReader extends Reader implements CharSequence {
}

View File

@ -28,7 +28,7 @@ import java.io.Reader;
* (shay.banon
* )
*/
public class FastStringReader extends CharSequenceReader {
public class FastStringReader extends Reader implements CharSequence {
private String str;
private int length;

View File

@ -19,16 +19,6 @@
package org.elasticsearch.common.util;
import com.carrotsearch.hppc.DoubleArrayList;
import com.carrotsearch.hppc.FloatArrayList;
import com.carrotsearch.hppc.LongArrayList;
import com.carrotsearch.hppc.ObjectArrayList;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefArray;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.IntroSorter;
import java.util.AbstractList;
import java.util.ArrayList;
import java.util.Arrays;
@ -40,11 +30,15 @@ import java.util.List;
import java.util.Objects;
import java.util.RandomAccess;
import com.carrotsearch.hppc.ObjectArrayList;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefArray;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.IntroSorter;
/** Collections-related utility methods. */
public class CollectionUtils {
public static void sort(LongArrayList list) {
sort(list.buffer, list.size());
}
public static void sort(final long[] array, int len) {
new IntroSorter() {
@ -76,29 +70,6 @@ public class CollectionUtils {
}.sort(0, len);
}
public static void sortAndDedup(LongArrayList list) {
list.elementsCount = sortAndDedup(list.buffer, list.elementsCount);
}
/** Sort and deduplicate values in-place, then return the unique element count. */
public static int sortAndDedup(long[] array, int len) {
if (len <= 1) {
return len;
}
sort(array, len);
int uniqueCount = 1;
for (int i = 1; i < len; ++i) {
if (array[i] != array[i - 1]) {
array[uniqueCount++] = array[i];
}
}
return uniqueCount;
}
public static void sort(FloatArrayList list) {
sort(list.buffer, list.size());
}
public static void sort(final float[] array, int len) {
new IntroSorter() {
@ -129,29 +100,6 @@ public class CollectionUtils {
}.sort(0, len);
}
public static void sortAndDedup(FloatArrayList list) {
list.elementsCount = sortAndDedup(list.buffer, list.elementsCount);
}
/** Sort and deduplicate values in-place, then return the unique element count. */
public static int sortAndDedup(float[] array, int len) {
if (len <= 1) {
return len;
}
sort(array, len);
int uniqueCount = 1;
for (int i = 1; i < len; ++i) {
if (Float.compare(array[i], array[i - 1]) != 0) {
array[uniqueCount++] = array[i];
}
}
return uniqueCount;
}
public static void sort(DoubleArrayList list) {
sort(list.buffer, list.size());
}
public static void sort(final double[] array, int len) {
new IntroSorter() {
@ -182,25 +130,6 @@ public class CollectionUtils {
}.sort(0, len);
}
public static void sortAndDedup(DoubleArrayList list) {
list.elementsCount = sortAndDedup(list.buffer, list.elementsCount);
}
/** Sort and deduplicate values in-place, then return the unique element count. */
public static int sortAndDedup(double[] array, int len) {
if (len <= 1) {
return len;
}
sort(array, len);
int uniqueCount = 1;
for (int i = 1; i < len; ++i) {
if (Double.compare(array[i], array[i - 1]) != 0) {
array[uniqueCount++] = array[i];
}
}
return uniqueCount;
}
/**
* Checks if the given array contains any elements.
*

View File

@ -34,7 +34,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -232,7 +232,7 @@ public class MasterFaultDetection extends FaultDetection {
}
final MasterPingRequest request = new MasterPingRequest(clusterService.localNode().getId(), masterToPing.getId(), clusterName);
final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build();
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, new BaseTransportResponseHandler<MasterPingResponseResponse>() {
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, new TransportResponseHandler<MasterPingResponseResponse>() {
@Override
public MasterPingResponseResponse newInstance() {

View File

@ -28,7 +28,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -203,7 +203,7 @@ public class NodesFaultDetection extends FaultDetection {
}
final PingRequest pingRequest = new PingRequest(node.getId(), clusterName, localNode, clusterStateVersion);
final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build();
transportService.sendRequest(node, PING_ACTION_NAME, pingRequest, options, new BaseTransportResponseHandler<PingResponse>() {
transportService.sendRequest(node, PING_ACTION_NAME, pingRequest, options, new TransportResponseHandler<PingResponse>() {
@Override
public PingResponse newInstance() {
return new PingResponse();

View File

@ -46,7 +46,7 @@ import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.discovery.zen.ping.PingContextProvider;
import org.elasticsearch.discovery.zen.ping.ZenPing;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.RemoteTransportException;
import org.elasticsearch.transport.TransportChannel;
@ -444,7 +444,7 @@ public class UnicastZenPing extends AbstractLifecycleComponent implements ZenPin
final CountDownLatch latch, final DiscoveryNode node, final DiscoveryNode nodeToSend) {
logger.trace("[{}] sending to {}", id, nodeToSend);
transportService.sendRequest(nodeToSend, ACTION_NAME, pingRequest, TransportRequestOptions.builder()
.withTimeout((long) (timeout.millis() * 1.25)).build(), new BaseTransportResponseHandler<UnicastPingResponse>() {
.withTimeout((long) (timeout.millis() * 1.25)).build(), new TransportResponseHandler<UnicastPingResponse>() {
@Override
public UnicastPingResponse newInstance() {

View File

@ -50,7 +50,7 @@ import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.indices.IndexClosedException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
@ -270,7 +270,7 @@ public class SyncedFlushService extends AbstractComponent implements IndexEventL
}
logger.trace("{} retrieving in flight operation count", shardId);
transportService.sendRequest(primaryNode, IN_FLIGHT_OPS_ACTION_NAME, new InFlightOpsRequest(shardId),
new BaseTransportResponseHandler<InFlightOpsResponse>() {
new TransportResponseHandler<InFlightOpsResponse>() {
@Override
public InFlightOpsResponse newInstance() {
return new InFlightOpsResponse();
@ -319,7 +319,7 @@ public class SyncedFlushService extends AbstractComponent implements IndexEventL
}
logger.trace("{} sending synced flush request to {}. sync id [{}].", shardId, shard, syncId);
transportService.sendRequest(node, SYNCED_FLUSH_ACTION_NAME, new ShardSyncedFlushRequest(shard.shardId(), syncId, expectedCommitId),
new BaseTransportResponseHandler<ShardSyncedFlushResponse>() {
new TransportResponseHandler<ShardSyncedFlushResponse>() {
@Override
public ShardSyncedFlushResponse newInstance() {
return new ShardSyncedFlushResponse();
@ -373,7 +373,7 @@ public class SyncedFlushService extends AbstractComponent implements IndexEventL
}
continue;
}
transportService.sendRequest(node, PRE_SYNCED_FLUSH_ACTION_NAME, new PreShardSyncedFlushRequest(shard.shardId()), new BaseTransportResponseHandler<PreSyncedFlushResponse>() {
transportService.sendRequest(node, PRE_SYNCED_FLUSH_ACTION_NAME, new PreShardSyncedFlushRequest(shard.shardId()), new TransportResponseHandler<PreSyncedFlushResponse>() {
@Override
public PreSyncedFlushResponse newInstance() {
return new PreSyncedFlushResponse();

View File

@ -61,15 +61,16 @@ public class NodeService extends AbstractComponent implements Closeable {
private ScriptService scriptService;
@Nullable
private HttpServer httpServer;
private final HttpServer httpServer;
private final Discovery discovery;
@Inject
public NodeService(Settings settings, ThreadPool threadPool, MonitorService monitorService,
Discovery discovery, TransportService transportService, IndicesService indicesService,
PluginsService pluginService, CircuitBreakerService circuitBreakerService, @Nullable HttpServer httpServer,
ProcessorsRegistry.Builder processorsRegistryBuilder, ClusterService clusterService, SettingsFilter settingsFilter) {
public NodeService(Settings settings, ThreadPool threadPool, MonitorService monitorService, Discovery discovery,
TransportService transportService, IndicesService indicesService, PluginsService pluginService,
CircuitBreakerService circuitBreakerService, @Nullable HttpServer httpServer,
ProcessorsRegistry.Builder processorsRegistryBuilder, ClusterService clusterService,
SettingsFilter settingsFilter) {
super(settings);
this.threadPool = threadPool;
this.monitorService = monitorService;
@ -93,21 +94,6 @@ public class NodeService extends AbstractComponent implements Closeable {
this.ingestService.buildProcessorsFactoryRegistry(scriptService, clusterService);
}
public NodeInfo info() {
return new NodeInfo(Version.CURRENT, Build.CURRENT, discovery.localNode(),
settings,
monitorService.osService().info(),
monitorService.processService().info(),
monitorService.jvmService().info(),
threadPool.info(),
transportService.info(),
httpServer == null ? null : httpServer.info(),
pluginService == null ? null : pluginService.info(),
ingestService == null ? null : ingestService.info(),
indicesService.getTotalIndexingBufferBytes()
);
}
public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool,
boolean transport, boolean http, boolean plugin, boolean ingest, boolean indices) {
return new NodeInfo(Version.CURRENT, Build.CURRENT, discovery.localNode(),
@ -124,25 +110,6 @@ public class NodeService extends AbstractComponent implements Closeable {
);
}
public NodeStats stats() throws IOException {
// for indices stats we want to include previous allocated shards stats as well (it will
// only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
return new NodeStats(discovery.localNode(), System.currentTimeMillis(),
indicesService.stats(true),
monitorService.osService().stats(),
monitorService.processService().stats(),
monitorService.jvmService().stats(),
threadPool.stats(),
monitorService.fsService().stats(),
transportService.stats(),
httpServer == null ? null : httpServer.stats(),
circuitBreakerService.stats(),
scriptService.stats(),
discovery.stats(),
ingestService.getPipelineExecutionService().stats()
);
}
public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, boolean jvm, boolean threadPool,
boolean fs, boolean transport, boolean http, boolean circuitBreaker,
boolean script, boolean discoveryStats, boolean ingest) {

View File

@ -1,27 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
/**
* A simple based class that always spawns.
*/
public abstract class BaseTransportResponseHandler<T extends TransportResponse> implements TransportResponseHandler<T> {
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.threadpool.ThreadPool;
/**
* A response handler to be used when all interaction will be done through the {@link TransportFuture}.
*/
public abstract class FutureTransportResponseHandler<T extends TransportResponse> extends BaseTransportResponseHandler<T> {
public abstract class FutureTransportResponseHandler<T extends TransportResponse> implements TransportResponseHandler<T> {
@Override
public void handleResponse(T response) {

View File

@ -30,7 +30,7 @@ import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestOptions;
@ -188,7 +188,7 @@ public class TransportClientNodesServiceTests extends ESTestCase {
}
iteration.transportService.sendRequest(node, "action", new TestRequest(),
TransportRequestOptions.EMPTY, new BaseTransportResponseHandler<TestResponse>() {
TransportRequestOptions.EMPTY, new TransportResponseHandler<TestResponse>() {
@Override
public TestResponse newInstance() {
return new TestResponse();

View File

@ -1,24 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.settings.bar;
//used in SettingsTest
public class BarTestClass {
}

View File

@ -153,7 +153,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
});
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
new StringMessageRequest("moshe"), new BaseTransportResponseHandler<StringMessageResponse>() {
new StringMessageRequest("moshe"), new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -184,7 +184,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
}
res = serviceB.submitRequest(nodeA, "sayHello", new StringMessageRequest("moshe"),
TransportRequestOptions.builder().withCompress(true).build(), new BaseTransportResponseHandler<StringMessageResponse>() {
TransportRequestOptions.builder().withCompress(true).build(), new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -233,7 +233,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
});
final Object context = new Object();
final String executor = randomFrom(ThreadPool.THREAD_POOL_TYPES.keySet().toArray(new String[0]));
TransportResponseHandler<StringMessageResponse> responseHandler = new BaseTransportResponseHandler<StringMessageResponse>() {
TransportResponseHandler<StringMessageResponse> responseHandler = new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -336,7 +336,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportFuture<TransportResponse.Empty> res = serviceB.submitRequest(nodeA, "sayHello",
TransportRequest.Empty.INSTANCE, TransportRequestOptions.builder().withCompress(true).build(),
new BaseTransportResponseHandler<TransportResponse.Empty>() {
new TransportResponseHandler<TransportResponse.Empty>() {
@Override
public TransportResponse.Empty newInstance() {
return TransportResponse.Empty.INSTANCE;
@ -386,7 +386,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
new StringMessageRequest("moshe"), TransportRequestOptions.builder().withCompress(true).build(),
new BaseTransportResponseHandler<StringMessageResponse>() {
new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -430,7 +430,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
});
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHelloException",
new StringMessageRequest("moshe"), new BaseTransportResponseHandler<StringMessageResponse>() {
new StringMessageRequest("moshe"), new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -520,7 +520,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHelloTimeoutNoResponse",
new StringMessageRequest("moshe"), TransportRequestOptions.builder().withTimeout(100).build(),
new BaseTransportResponseHandler<StringMessageResponse>() {
new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -575,7 +575,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
final CountDownLatch latch = new CountDownLatch(1);
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHelloTimeoutDelayedResponse",
new StringMessageRequest("2m"), TransportRequestOptions.builder().withTimeout(100).build(),
new BaseTransportResponseHandler<StringMessageResponse>() {
new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -612,7 +612,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
// now, try and send another request, this times, with a short timeout
res = serviceB.submitRequest(nodeA, "sayHelloTimeoutDelayedResponse",
new StringMessageRequest(counter + "ms"), TransportRequestOptions.builder().withTimeout(3000).build(),
new BaseTransportResponseHandler<StringMessageResponse>() {
new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -664,7 +664,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
};
final Semaphore requestCompleted = new Semaphore(0);
TransportResponseHandler noopResponseHandler = new BaseTransportResponseHandler<StringMessageResponse>() {
TransportResponseHandler noopResponseHandler = new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
@ -967,7 +967,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
Version0Request version0Request = new Version0Request();
version0Request.value1 = 1;
Version0Response version0Response = serviceA.submitRequest(nodeB, "/version", version0Request,
new BaseTransportResponseHandler<Version0Response>() {
new TransportResponseHandler<Version0Response>() {
@Override
public Version0Response newInstance() {
return new Version0Response();
@ -1009,7 +1009,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
version1Request.value1 = 1;
version1Request.value2 = 2;
Version1Response version1Response = serviceB.submitRequest(nodeA, "/version", version1Request,
new BaseTransportResponseHandler<Version1Response>() {
new TransportResponseHandler<Version1Response>() {
@Override
public Version1Response newInstance() {
return new Version1Response();
@ -1055,7 +1055,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
version1Request.value1 = 1;
version1Request.value2 = 2;
Version1Response version1Response = serviceB.submitRequest(nodeB, "/version", version1Request,
new BaseTransportResponseHandler<Version1Response>() {
new TransportResponseHandler<Version1Response>() {
@Override
public Version1Response newInstance() {
return new Version1Response();
@ -1098,7 +1098,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
Version0Request version0Request = new Version0Request();
version0Request.value1 = 1;
Version0Response version0Response = serviceA.submitRequest(nodeA, "/version", version0Request,
new BaseTransportResponseHandler<Version0Response>() {
new TransportResponseHandler<Version0Response>() {
@Override
public Version0Response newInstance() {
return new Version0Response();
@ -1137,7 +1137,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
serviceB.addFailToSendNoConnectRule(serviceA);
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
new StringMessageRequest("moshe"), new BaseTransportResponseHandler<StringMessageResponse>() {
new StringMessageRequest("moshe"), new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();
@ -1197,7 +1197,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
new StringMessageRequest("moshe"), TransportRequestOptions.builder().withTimeout(100).build(),
new BaseTransportResponseHandler<StringMessageResponse>() {
new TransportResponseHandler<StringMessageResponse>() {
@Override
public StringMessageResponse newInstance() {
return new StringMessageResponse();

View File

@ -31,7 +31,7 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TcpTransport;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -114,7 +114,7 @@ public class NettyScheduledPingTests extends ESTestCase {
for (int i = 0; i < rounds; i++) {
serviceB.submitRequest(nodeA, "sayHello",
TransportRequest.Empty.INSTANCE, TransportRequestOptions.builder().withCompress(randomBoolean()).build(),
new BaseTransportResponseHandler<TransportResponse.Empty>() {
new TransportResponseHandler<TransportResponse.Empty>() {
@Override
public TransportResponse.Empty newInstance() {
return TransportResponse.Empty.INSTANCE;