Detach Transport from TransportService (#31727)
Today TransportService is tightly coupled with Transport since it requires an instance of TransportService in order to receive responses and send requests. This is mainly due to the Request and Response handlers being maintained in TransportService but also because of the lack of a proper callback interface. This change moves request handler registry and response handler registration into Transport and adds all necessary methods to `TransportConnectionListener` in order to remove the `TransportService` dependency from `Transport` Transport now accepts one or more `TransportConnectionListener` instances that are executed sequentially in a blocking fashion.
This commit is contained in:
parent
896317fe36
commit
3f2a241b7f
|
@ -89,7 +89,7 @@ public class Netty4ScheduledPingTests extends ESTestCase {
|
|||
assertThat(nettyA.getPing().getFailedPings(), equalTo(0L));
|
||||
assertThat(nettyB.getPing().getFailedPings(), equalTo(0L));
|
||||
|
||||
serviceA.registerRequestHandler("sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
|
||||
new TransportRequestHandler<TransportRequest.Empty>() {
|
||||
@Override
|
||||
public void messageReceived(TransportRequest.Empty request, TransportChannel channel, Task task) {
|
||||
|
@ -104,7 +104,7 @@ public class Netty4ScheduledPingTests extends ESTestCase {
|
|||
|
||||
int rounds = scaledRandomIntBetween(100, 5000);
|
||||
for (int i = 0; i < rounds; i++) {
|
||||
serviceB.submitRequest(nodeA, "sayHello",
|
||||
serviceB.submitRequest(nodeA, "internal:sayHello",
|
||||
TransportRequest.Empty.INSTANCE, TransportRequestOptions.builder().withCompress(randomBoolean()).build(),
|
||||
new TransportResponseHandler<TransportResponse.Empty>() {
|
||||
@Override
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.elasticsearch.common.settings.Setting;
|
|||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
@ -93,13 +94,20 @@ public abstract class FaultDetection extends AbstractComponent implements Closea
|
|||
abstract void handleTransportDisconnect(DiscoveryNode node);
|
||||
|
||||
private class FDConnectionListener implements TransportConnectionListener {
|
||||
@Override
|
||||
public void onNodeConnected(DiscoveryNode node) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode node) {
|
||||
handleTransportDisconnect(node);
|
||||
AbstractRunnable runnable = new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
logger.warn("failed to handle transport disconnect for node: {}", node);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doRun() {
|
||||
handleTransportDisconnect(node);
|
||||
}
|
||||
};
|
||||
threadPool.generic().execute(runnable);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.elasticsearch.common.breaker.CircuitBreaker;
|
|||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.bytes.CompositeBytesReference;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
import org.elasticsearch.common.component.Lifecycle;
|
||||
import org.elasticsearch.common.compress.Compressor;
|
||||
|
@ -98,10 +99,10 @@ import java.util.Set;
|
|||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
@ -205,7 +206,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
protected final NetworkService networkService;
|
||||
protected final Set<ProfileSettings> profileSettings;
|
||||
|
||||
private volatile TransportService transportService;
|
||||
private final DelegatingTransportConnectionListener transportListener = new DelegatingTransportConnectionListener();
|
||||
|
||||
private final ConcurrentMap<String, BoundTransportAddress> profileBoundAddresses = newConcurrentMap();
|
||||
// node id to actual channel
|
||||
|
@ -225,12 +226,13 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
protected final ConnectionProfile defaultConnectionProfile;
|
||||
|
||||
private final ConcurrentMap<Long, HandshakeResponseHandler> pendingHandshakes = new ConcurrentHashMap<>();
|
||||
private final AtomicLong requestIdGenerator = new AtomicLong();
|
||||
private final CounterMetric numHandshakes = new CounterMetric();
|
||||
private static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
|
||||
|
||||
private final MeanMetric readBytesMetric = new MeanMetric();
|
||||
private final MeanMetric transmittedBytesMetric = new MeanMetric();
|
||||
private volatile Map<String, RequestHandlerRegistry> requestHandlers = Collections.emptyMap();
|
||||
private final ResponseHandlers responseHandlers = new ResponseHandlers();
|
||||
|
||||
public TcpTransport(String transportName, Settings settings, ThreadPool threadPool, BigArrays bigArrays,
|
||||
CircuitBreakerService circuitBreakerService, NamedWriteableRegistry namedWriteableRegistry,
|
||||
|
@ -287,6 +289,16 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addConnectionListener(TransportConnectionListener listener) {
|
||||
transportListener.listeners.add(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeConnectionListener(TransportConnectionListener listener) {
|
||||
return transportListener.listeners.remove(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CircuitBreaker getInFlightRequestBreaker() {
|
||||
// We always obtain a fresh breaker to reflect changes to the breaker configuration.
|
||||
|
@ -294,11 +306,11 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setTransportService(TransportService service) {
|
||||
if (service.getRequestHandler(HANDSHAKE_ACTION_NAME) != null) {
|
||||
throw new IllegalStateException(HANDSHAKE_ACTION_NAME + " is a reserved request handler and must not be registered");
|
||||
public synchronized <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
|
||||
if (requestHandlers.containsKey(reg.getAction())) {
|
||||
throw new IllegalArgumentException("transport handlers for action " + reg.getAction() + " is already registered");
|
||||
}
|
||||
this.transportService = service;
|
||||
requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
|
||||
}
|
||||
|
||||
private static class HandshakeResponseHandler implements TransportResponseHandler<VersionHandshakeResponse> {
|
||||
|
@ -482,7 +494,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
boolean block = lifecycle.stopped() && Transports.isTransportThread(Thread.currentThread()) == false;
|
||||
CloseableChannel.closeChannels(channels, block);
|
||||
} finally {
|
||||
transportService.onConnectionClosed(this);
|
||||
transportListener.onConnectionClosed(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -538,7 +550,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
logger.debug("connected to node [{}]", node);
|
||||
}
|
||||
try {
|
||||
transportService.onNodeConnected(node);
|
||||
transportListener.onNodeConnected(node);
|
||||
} finally {
|
||||
if (nodeChannels.isClosed()) {
|
||||
// we got closed concurrently due to a disconnect or some other event on the channel.
|
||||
|
@ -550,7 +562,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
// try to remove it first either way one of the two wins even if the callback has run before we even added the
|
||||
// tuple to the map since in that case we remove it here again
|
||||
if (connectedNodes.remove(node, nodeChannels)) {
|
||||
transportService.onNodeDisconnected(node);
|
||||
transportListener.onNodeDisconnected(node);
|
||||
}
|
||||
throw new NodeNotConnectedException(node, "connection concurrently closed");
|
||||
}
|
||||
|
@ -652,7 +664,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
// At this point we should construct the connection, notify the transport service, and attach close listeners to the
|
||||
// underlying channels.
|
||||
nodeChannels = new NodeChannels(node, channels, connectionProfile, version);
|
||||
transportService.onConnectionOpened(nodeChannels);
|
||||
transportListener.onConnectionOpened(nodeChannels);
|
||||
final NodeChannels finalNodeChannels = nodeChannels;
|
||||
final AtomicBoolean runOnce = new AtomicBoolean(false);
|
||||
Consumer<TcpChannel> onClose = c -> {
|
||||
|
@ -695,7 +707,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
if (closeLock.readLock().tryLock()) {
|
||||
try {
|
||||
if (connectedNodes.remove(node, nodeChannels)) {
|
||||
transportService.onNodeDisconnected(node);
|
||||
transportListener.onNodeDisconnected(node);
|
||||
}
|
||||
} finally {
|
||||
closeLock.readLock().unlock();
|
||||
|
@ -722,7 +734,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
} finally {
|
||||
closeLock.readLock().unlock();
|
||||
if (nodeChannels != null) { // if we found it and removed it we close and notify
|
||||
IOUtils.closeWhileHandlingException(nodeChannels, () -> transportService.onNodeDisconnected(node));
|
||||
IOUtils.closeWhileHandlingException(nodeChannels, () -> transportListener.onNodeDisconnected(node));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -979,7 +991,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
Map.Entry<DiscoveryNode, NodeChannels> next = iterator.next();
|
||||
try {
|
||||
IOUtils.closeWhileHandlingException(next.getValue());
|
||||
transportService.onNodeDisconnected(next.getKey());
|
||||
transportListener.onNodeDisconnected(next.getKey());
|
||||
} finally {
|
||||
iterator.remove();
|
||||
}
|
||||
|
@ -1133,7 +1145,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
final TransportRequestOptions finalOptions = options;
|
||||
// this might be called in a different thread
|
||||
SendListener onRequestSent = new SendListener(channel, stream,
|
||||
() -> transportService.onRequestSent(node, requestId, action, request, finalOptions), message.length());
|
||||
() -> transportListener.onRequestSent(node, requestId, action, request, finalOptions), message.length());
|
||||
internalSendMessage(channel, message, onRequestSent);
|
||||
addedReleaseListener = true;
|
||||
} finally {
|
||||
|
@ -1187,7 +1199,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
final BytesReference header = buildHeader(requestId, status, nodeVersion, bytes.length());
|
||||
CompositeBytesReference message = new CompositeBytesReference(header, bytes);
|
||||
SendListener onResponseSent = new SendListener(channel, null,
|
||||
() -> transportService.onResponseSent(requestId, action, error), message.length());
|
||||
() -> transportListener.onResponseSent(requestId, action, error), message.length());
|
||||
internalSendMessage(channel, message, onResponseSent);
|
||||
}
|
||||
}
|
||||
|
@ -1236,7 +1248,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
final TransportResponseOptions finalOptions = options;
|
||||
// this might be called in a different thread
|
||||
SendListener listener = new SendListener(channel, stream,
|
||||
() -> transportService.onResponseSent(requestId, action, response, finalOptions), message.length());
|
||||
() -> transportListener.onResponseSent(requestId, action, response, finalOptions), message.length());
|
||||
internalSendMessage(channel, message, listener);
|
||||
addedReleaseListener = true;
|
||||
} finally {
|
||||
|
@ -1492,7 +1504,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
if (isHandshake) {
|
||||
handler = pendingHandshakes.remove(requestId);
|
||||
} else {
|
||||
TransportResponseHandler theHandler = transportService.onResponseReceived(requestId);
|
||||
TransportResponseHandler theHandler = responseHandlers.onResponseReceived(requestId, transportListener);
|
||||
if (theHandler == null && TransportStatus.isError(status)) {
|
||||
handler = pendingHandshakes.remove(requestId);
|
||||
} else {
|
||||
|
@ -1599,7 +1611,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
features = Collections.emptySet();
|
||||
}
|
||||
final String action = stream.readString();
|
||||
transportService.onRequestReceived(requestId, action);
|
||||
transportListener.onRequestReceived(requestId, action);
|
||||
TransportChannel transportChannel = null;
|
||||
try {
|
||||
if (TransportStatus.isHandshake(status)) {
|
||||
|
@ -1607,7 +1619,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
sendResponse(version, features, channel, response, requestId, HANDSHAKE_ACTION_NAME, TransportResponseOptions.EMPTY,
|
||||
TransportStatus.setHandshake((byte) 0));
|
||||
} else {
|
||||
final RequestHandlerRegistry reg = transportService.getRequestHandler(action);
|
||||
final RequestHandlerRegistry reg = getRequestHandler(action);
|
||||
if (reg == null) {
|
||||
throw new ActionNotFoundTransportException(action);
|
||||
}
|
||||
|
@ -1714,7 +1726,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
protected Version executeHandshake(DiscoveryNode node, TcpChannel channel, TimeValue timeout)
|
||||
throws IOException, InterruptedException {
|
||||
numHandshakes.inc();
|
||||
final long requestId = newRequestId();
|
||||
final long requestId = responseHandlers.newRequestId();
|
||||
final HandshakeResponseHandler handler = new HandshakeResponseHandler(channel);
|
||||
AtomicReference<Version> versionRef = handler.versionRef;
|
||||
AtomicReference<Exception> exceptionRef = handler.exceptionRef;
|
||||
|
@ -1764,11 +1776,6 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
return numHandshakes.count(); // for testing
|
||||
}
|
||||
|
||||
@Override
|
||||
public long newRequestId() {
|
||||
return requestIdGenerator.incrementAndGet();
|
||||
}
|
||||
|
||||
/**
|
||||
* Called once the channel is closed for instance due to a disconnect or a closed socket etc.
|
||||
*/
|
||||
|
@ -1912,4 +1919,82 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements
|
|||
PUBLISH_PORT_PROFILE.getConcreteSettingForNamespace(profileName).get(settings);
|
||||
}
|
||||
}
|
||||
|
||||
private static final class DelegatingTransportConnectionListener implements TransportConnectionListener {
|
||||
private final List<TransportConnectionListener> listeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
@Override
|
||||
public void onRequestReceived(long requestId, String action) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onRequestReceived(requestId, action);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponseSent(long requestId, String action, TransportResponse response, TransportResponseOptions finalOptions) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onResponseSent(requestId, action, response, finalOptions);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponseSent(long requestId, String action, Exception error) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onResponseSent(requestId, action, error);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRequestSent(DiscoveryNode node, long requestId, String action, TransportRequest request,
|
||||
TransportRequestOptions finalOptions) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onRequestSent(node, requestId, action, request, finalOptions);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeDisconnected(DiscoveryNode key) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onNodeDisconnected(key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onConnectionOpened(Connection nodeChannels) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onConnectionOpened(nodeChannels);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNodeConnected(DiscoveryNode node) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onNodeConnected(node);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onConnectionClosed(Connection nodeChannels) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onConnectionClosed(nodeChannels);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponseReceived(long requestId, ResponseContext holder) {
|
||||
for (TransportConnectionListener listener : listeners) {
|
||||
listener.onResponseReceived(requestId, holder);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final ResponseHandlers getResponseHandlers() {
|
||||
return responseHandlers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final RequestHandlerRegistry getRequestHandler(String action) {
|
||||
return requestHandlers.get(action);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,18 +29,45 @@ import org.elasticsearch.common.settings.Setting;
|
|||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.transport.BoundTransportAddress;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public interface Transport extends LifecycleComponent {
|
||||
|
||||
Setting<Boolean> TRANSPORT_TCP_COMPRESS = Setting.boolSetting("transport.tcp.compress", false, Property.NodeScope);
|
||||
|
||||
void setTransportService(TransportService service);
|
||||
/**
|
||||
* Registers a new request handler
|
||||
*/
|
||||
<Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg);
|
||||
|
||||
/**
|
||||
* Returns the registered request handler registry for the given action or <code>null</code> if it's not registered
|
||||
* @param action the action to look up
|
||||
*/
|
||||
RequestHandlerRegistry getRequestHandler(String action);
|
||||
|
||||
/**
|
||||
* Adds a new event listener
|
||||
* @param listener the listener to add
|
||||
*/
|
||||
void addConnectionListener(TransportConnectionListener listener);
|
||||
|
||||
/**
|
||||
* Removes an event listener
|
||||
* @param listener the listener to remove
|
||||
* @return <code>true</code> iff the listener was removed otherwise <code>false</code>
|
||||
*/
|
||||
boolean removeConnectionListener(TransportConnectionListener listener);
|
||||
|
||||
/**
|
||||
* The address the transport is bound on.
|
||||
|
@ -75,17 +102,15 @@ public interface Transport extends LifecycleComponent {
|
|||
*/
|
||||
void disconnectFromNode(DiscoveryNode node);
|
||||
|
||||
/**
|
||||
* Returns a list of all local adresses for this transport
|
||||
*/
|
||||
List<String> getLocalAddresses();
|
||||
|
||||
default CircuitBreaker getInFlightRequestBreaker() {
|
||||
return new NoopCircuitBreaker("in-flight-noop");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new request ID to use when sending a message via {@link Connection#sendRequest(long, String,
|
||||
* TransportRequest, TransportRequestOptions)}
|
||||
*/
|
||||
long newRequestId();
|
||||
/**
|
||||
* Returns a connection for the given node if the node is connected.
|
||||
* Connections returned from this method must not be closed. The lifecycle of this connection is maintained by the Transport
|
||||
|
@ -107,6 +132,8 @@ public interface Transport extends LifecycleComponent {
|
|||
|
||||
TransportStats getStats();
|
||||
|
||||
ResponseHandlers getResponseHandlers();
|
||||
|
||||
/**
|
||||
* A unidirectional connection to a {@link DiscoveryNode}
|
||||
*/
|
||||
|
@ -118,6 +145,10 @@ public interface Transport extends LifecycleComponent {
|
|||
|
||||
/**
|
||||
* Sends the request to the node this connection is associated with
|
||||
* @param requestId see {@link ResponseHandlers#add(ResponseContext)} for details
|
||||
* @param action the action to execute
|
||||
* @param request the request to send
|
||||
* @param options request options to apply
|
||||
* @throws NodeNotConnectedException if the given node is not connected
|
||||
*/
|
||||
void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws
|
||||
|
@ -138,4 +169,111 @@ public interface Transport extends LifecycleComponent {
|
|||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This class represents a response context that encapsulates the actual response handler, the action and the conneciton it was
|
||||
* executed on.
|
||||
*/
|
||||
final class ResponseContext<T extends TransportResponse> {
|
||||
|
||||
private final TransportResponseHandler<T> handler;
|
||||
|
||||
private final Connection connection;
|
||||
|
||||
private final String action;
|
||||
|
||||
ResponseContext(TransportResponseHandler<T> handler, Connection connection, String action) {
|
||||
this.handler = handler;
|
||||
this.connection = connection;
|
||||
this.action = action;
|
||||
}
|
||||
|
||||
public TransportResponseHandler<T> handler() {
|
||||
return handler;
|
||||
}
|
||||
|
||||
public Connection connection() {
|
||||
return this.connection;
|
||||
}
|
||||
|
||||
public String action() {
|
||||
return this.action;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This class is a registry that allows
|
||||
*/
|
||||
final class ResponseHandlers {
|
||||
private final ConcurrentMapLong<ResponseContext> handlers = ConcurrentCollections
|
||||
.newConcurrentMapLongWithAggressiveConcurrency();
|
||||
private final AtomicLong requestIdGenerator = new AtomicLong();
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if the give request ID has a context associated with it.
|
||||
*/
|
||||
public boolean contains(long requestId) {
|
||||
return handlers.containsKey(requestId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes and return the {@link ResponseContext} for the given request ID or returns
|
||||
* <code>null</code> if no context is associated with this request ID.
|
||||
*/
|
||||
public ResponseContext remove(long requestId) {
|
||||
return handlers.remove(requestId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new response context and associates it with a new request ID.
|
||||
* @return the new request ID
|
||||
* @see Connection#sendRequest(long, String, TransportRequest, TransportRequestOptions)
|
||||
*/
|
||||
public long add(ResponseContext holder) {
|
||||
long requestId = newRequestId();
|
||||
ResponseContext existing = handlers.put(requestId, holder);
|
||||
assert existing == null : "request ID already in use: " + requestId;
|
||||
return requestId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new request ID to use when sending a message via {@link Connection#sendRequest(long, String,
|
||||
* TransportRequest, TransportRequestOptions)}
|
||||
*/
|
||||
long newRequestId() {
|
||||
return requestIdGenerator.incrementAndGet();
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes and returns all {@link ResponseContext} instances that match the predicate
|
||||
*/
|
||||
public List<ResponseContext> prune(Predicate<ResponseContext> predicate) {
|
||||
final List<ResponseContext> holders = new ArrayList<>();
|
||||
for (Map.Entry<Long, ResponseContext> entry : handlers.entrySet()) {
|
||||
ResponseContext holder = entry.getValue();
|
||||
if (predicate.test(holder)) {
|
||||
ResponseContext remove = handlers.remove(entry.getKey());
|
||||
if (remove != null) {
|
||||
holders.add(holder);
|
||||
}
|
||||
}
|
||||
}
|
||||
return holders;
|
||||
}
|
||||
|
||||
/**
|
||||
* called by the {@link Transport} implementation when a response or an exception has been received for a previously
|
||||
* sent request (before any processing or deserialization was done). Returns the appropriate response handler or null if not
|
||||
* found.
|
||||
*/
|
||||
public TransportResponseHandler onResponseReceived(final long requestId, TransportConnectionListener listener) {
|
||||
ResponseContext context = handlers.remove(requestId);
|
||||
listener.onResponseReceived(requestId, context);
|
||||
if (context == null) {
|
||||
return null;
|
||||
} else {
|
||||
return context.handler();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,8 +21,68 @@ package org.elasticsearch.transport;
|
|||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
|
||||
/**
|
||||
* A listener interface that allows to react on transport events. All methods may be
|
||||
* executed on network threads. Consumers must fork in the case of long running or blocking
|
||||
* operations.
|
||||
*/
|
||||
public interface TransportConnectionListener {
|
||||
|
||||
/**
|
||||
* Called once a request is received
|
||||
* @param requestId the internal request ID
|
||||
* @param action the request action
|
||||
*
|
||||
*/
|
||||
default void onRequestReceived(long requestId, String action) {}
|
||||
|
||||
/**
|
||||
* Called for every action response sent after the response has been passed to the underlying network implementation.
|
||||
* @param requestId the request ID (unique per client)
|
||||
* @param action the request action
|
||||
* @param response the response send
|
||||
* @param finalOptions the response options
|
||||
*/
|
||||
default void onResponseSent(long requestId, String action, TransportResponse response, TransportResponseOptions finalOptions) {}
|
||||
|
||||
/***
|
||||
* Called for every failed action response after the response has been passed to the underlying network implementation.
|
||||
* @param requestId the request ID (unique per client)
|
||||
* @param action the request action
|
||||
* @param error the error sent back to the caller
|
||||
*/
|
||||
default void onResponseSent(long requestId, String action, Exception error) {}
|
||||
|
||||
/**
|
||||
* Called for every request sent to a server after the request has been passed to the underlying network implementation
|
||||
* @param node the node the request was sent to
|
||||
* @param requestId the internal request id
|
||||
* @param action the action name
|
||||
* @param request the actual request
|
||||
* @param finalOptions the request options
|
||||
*/
|
||||
default void onRequestSent(DiscoveryNode node, long requestId, String action, TransportRequest request,
|
||||
TransportRequestOptions finalOptions) {}
|
||||
|
||||
/**
|
||||
* Called once a connection was opened
|
||||
* @param connection the connection
|
||||
*/
|
||||
default void onConnectionOpened(Transport.Connection connection) {}
|
||||
|
||||
/**
|
||||
* Called once a connection ws closed.
|
||||
* @param connection the closed connection
|
||||
*/
|
||||
default void onConnectionClosed(Transport.Connection connection) {}
|
||||
|
||||
/**
|
||||
* Called for every response received
|
||||
* @param requestId the request id for this reponse
|
||||
* @param context the response context or null if the context was already processed ie. due to a timeout.
|
||||
*/
|
||||
default void onResponseReceived(long requestId, Transport.ResponseContext context) {}
|
||||
|
||||
/**
|
||||
* Called once a node connection is opened and registered.
|
||||
*/
|
||||
|
@ -32,15 +92,4 @@ public interface TransportConnectionListener {
|
|||
* Called once a node connection is closed and unregistered.
|
||||
*/
|
||||
default void onNodeDisconnected(DiscoveryNode node) {}
|
||||
|
||||
/**
|
||||
* Called once a node connection is closed. The connection might not have been registered in the
|
||||
* transport as a shared connection to a specific node
|
||||
*/
|
||||
default void onConnectionClosed(Transport.Connection connection) {}
|
||||
|
||||
/**
|
||||
* Called once a node connection is opened.
|
||||
*/
|
||||
default void onConnectionOpened(Transport.Connection connection) {}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.elasticsearch.cluster.ClusterName;
|
|||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -45,8 +44,6 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.transport.BoundTransportAddress;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
|
@ -59,24 +56,23 @@ import java.io.IOException;
|
|||
import java.net.UnknownHostException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
import static org.elasticsearch.common.settings.Setting.listSetting;
|
||||
|
||||
public class TransportService extends AbstractLifecycleComponent {
|
||||
public class TransportService extends AbstractLifecycleComponent implements TransportConnectionListener {
|
||||
|
||||
public static final String DIRECT_RESPONSE_PROFILE = ".direct";
|
||||
public static final String HANDSHAKE_ACTION_NAME = "internal:transport/handshake";
|
||||
|
@ -89,14 +85,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
private final TransportInterceptor.AsyncSender asyncSender;
|
||||
private final Function<BoundTransportAddress, DiscoveryNode> localNodeFactory;
|
||||
private final boolean connectToRemoteCluster;
|
||||
|
||||
volatile Map<String, RequestHandlerRegistry> requestHandlers = Collections.emptyMap();
|
||||
final Object requestHandlerMutex = new Object();
|
||||
|
||||
final ConcurrentMapLong<RequestHolder> clientHandlers = ConcurrentCollections.newConcurrentMapLongWithAggressiveConcurrency();
|
||||
|
||||
final CopyOnWriteArrayList<TransportConnectionListener> connectionListeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
private final Transport.ResponseHandlers responseHandlers;
|
||||
private final TransportInterceptor interceptor;
|
||||
|
||||
// An LRU (don't really care about concurrency here) that holds the latest timed out requests so if they
|
||||
|
@ -138,12 +127,12 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
|
||||
@Override
|
||||
public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options)
|
||||
throws IOException, TransportException {
|
||||
throws TransportException {
|
||||
sendLocalRequest(requestId, action, request, options);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
public void close() {
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -172,6 +161,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
this.asyncSender = interceptor.interceptSender(this::sendRequestInternal);
|
||||
this.connectToRemoteCluster = RemoteClusterService.ENABLE_REMOTE_CLUSTERS.get(settings);
|
||||
remoteClusterService = new RemoteClusterService(settings, this);
|
||||
responseHandlers = transport.getResponseHandlers();
|
||||
if (clusterSettings != null) {
|
||||
clusterSettings.addSettingsUpdateConsumer(TRACE_LOG_INCLUDE_SETTING, this::setTracerLogInclude);
|
||||
clusterSettings.addSettingsUpdateConsumer(TRACE_LOG_EXCLUDE_SETTING, this::setTracerLogExclude);
|
||||
|
@ -179,6 +169,13 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
remoteClusterService.listenForUpdates(clusterSettings);
|
||||
}
|
||||
}
|
||||
registerRequestHandler(
|
||||
HANDSHAKE_ACTION_NAME,
|
||||
() -> HandshakeRequest.INSTANCE,
|
||||
ThreadPool.Names.SAME,
|
||||
false, false,
|
||||
(request, channel, task) -> channel.sendResponse(
|
||||
new HandshakeResponse(localNode, clusterName, localNode.getVersion())));
|
||||
}
|
||||
|
||||
public RemoteClusterService getRemoteClusterService() {
|
||||
|
@ -202,7 +199,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
*
|
||||
* @return the executor service
|
||||
*/
|
||||
protected ExecutorService getExecutorService() {
|
||||
private ExecutorService getExecutorService() {
|
||||
return threadPool.generic();
|
||||
}
|
||||
|
||||
|
@ -216,9 +213,8 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
|
||||
@Override
|
||||
protected void doStart() {
|
||||
transport.setTransportService(this);
|
||||
transport.addConnectionListener(this);
|
||||
transport.start();
|
||||
|
||||
if (transport.boundAddress() != null && logger.isInfoEnabled()) {
|
||||
logger.info("{}", transport.boundAddress());
|
||||
for (Map.Entry<String, BoundTransportAddress> entry : transport.profileBoundAddresses().entrySet()) {
|
||||
|
@ -226,13 +222,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
}
|
||||
localNode = localNodeFactory.apply(transport.boundAddress());
|
||||
registerRequestHandler(
|
||||
HANDSHAKE_ACTION_NAME,
|
||||
() -> HandshakeRequest.INSTANCE,
|
||||
ThreadPool.Names.SAME,
|
||||
false, false,
|
||||
(request, channel, task) -> channel.sendResponse(
|
||||
new HandshakeResponse(localNode, clusterName, localNode.getVersion())));
|
||||
|
||||
if (connectToRemoteCluster) {
|
||||
// here we start to connect to the remote clusters
|
||||
remoteClusterService.initializeRemoteClusters();
|
||||
|
@ -246,36 +236,33 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
} finally {
|
||||
// in case the transport is not connected to our local node (thus cleaned on node disconnect)
|
||||
// make sure to clean any leftover on going handles
|
||||
for (Map.Entry<Long, RequestHolder> entry : clientHandlers.entrySet()) {
|
||||
final RequestHolder holderToNotify = clientHandlers.remove(entry.getKey());
|
||||
if (holderToNotify != null) {
|
||||
// callback that an exception happened, but on a different thread since we don't
|
||||
// want handlers to worry about stack overflows
|
||||
getExecutorService().execute(new AbstractRunnable() {
|
||||
@Override
|
||||
public void onRejection(Exception e) {
|
||||
// if we get rejected during node shutdown we don't wanna bubble it up
|
||||
logger.debug(
|
||||
() -> new ParameterizedMessage(
|
||||
"failed to notify response handler on rejection, action: {}",
|
||||
holderToNotify.action()),
|
||||
e);
|
||||
}
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
logger.warn(
|
||||
() -> new ParameterizedMessage(
|
||||
"failed to notify response handler on exception, action: {}",
|
||||
holderToNotify.action()),
|
||||
e);
|
||||
}
|
||||
@Override
|
||||
public void doRun() {
|
||||
TransportException ex = new TransportException("transport stopped, action: " + holderToNotify.action());
|
||||
holderToNotify.handler().handleException(ex);
|
||||
}
|
||||
});
|
||||
}
|
||||
for (final Transport.ResponseContext holderToNotify : responseHandlers.prune(h -> true)) {
|
||||
// callback that an exception happened, but on a different thread since we don't
|
||||
// want handlers to worry about stack overflows
|
||||
getExecutorService().execute(new AbstractRunnable() {
|
||||
@Override
|
||||
public void onRejection(Exception e) {
|
||||
// if we get rejected during node shutdown we don't wanna bubble it up
|
||||
logger.debug(
|
||||
() -> new ParameterizedMessage(
|
||||
"failed to notify response handler on rejection, action: {}",
|
||||
holderToNotify.action()),
|
||||
e);
|
||||
}
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
logger.warn(
|
||||
() -> new ParameterizedMessage(
|
||||
"failed to notify response handler on exception, action: {}",
|
||||
holderToNotify.action()),
|
||||
e);
|
||||
}
|
||||
@Override
|
||||
public void doRun() {
|
||||
TransportException ex = new TransportException("transport stopped, action: " + holderToNotify.action());
|
||||
holderToNotify.handler().handleException(ex);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -479,11 +466,11 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
|
||||
public void addConnectionListener(TransportConnectionListener listener) {
|
||||
connectionListeners.add(listener);
|
||||
transport.addConnectionListener(listener);
|
||||
}
|
||||
|
||||
public void removeConnectionListener(TransportConnectionListener listener) {
|
||||
connectionListeners.remove(listener);
|
||||
transport.removeConnectionListener(listener);
|
||||
}
|
||||
|
||||
public <T extends TransportResponse> TransportFuture<T> submitRequest(DiscoveryNode node, String action, TransportRequest request,
|
||||
|
@ -594,18 +581,19 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
throw new IllegalStateException("can't send request to a null connection");
|
||||
}
|
||||
DiscoveryNode node = connection.getNode();
|
||||
final long requestId = transport.newRequestId();
|
||||
final TimeoutHandler timeoutHandler;
|
||||
try {
|
||||
|
||||
if (options.timeout() == null) {
|
||||
timeoutHandler = null;
|
||||
} else {
|
||||
timeoutHandler = new TimeoutHandler(requestId);
|
||||
}
|
||||
Supplier<ThreadContext.StoredContext> storedContextSupplier = threadPool.getThreadContext().newRestorableContext(true);
|
||||
TransportResponseHandler<T> responseHandler = new ContextRestoreResponseHandler<>(storedContextSupplier, handler);
|
||||
clientHandlers.put(requestId, new RequestHolder<>(responseHandler, connection, action, timeoutHandler));
|
||||
Supplier<ThreadContext.StoredContext> storedContextSupplier = threadPool.getThreadContext().newRestorableContext(true);
|
||||
ContextRestoreResponseHandler<T> responseHandler = new ContextRestoreResponseHandler<>(storedContextSupplier, handler);
|
||||
// TODO we can probably fold this entire request ID dance into connection.sendReqeust but it will be a bigger refactoring
|
||||
final long requestId = responseHandlers.add(new Transport.ResponseContext<>(responseHandler, connection, action));
|
||||
final TimeoutHandler timeoutHandler;
|
||||
if (options.timeout() != null) {
|
||||
timeoutHandler = new TimeoutHandler(requestId, connection.getNode(), action);
|
||||
responseHandler.setTimeoutHandler(timeoutHandler);
|
||||
} else {
|
||||
timeoutHandler = null;
|
||||
}
|
||||
try {
|
||||
if (lifecycle.stoppedOrClosed()) {
|
||||
// if we are not started the exception handling will remove the RequestHolder again and calls the handler to notify
|
||||
// the caller. It will only notify if the toStop code hasn't done the work yet.
|
||||
|
@ -619,10 +607,12 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
} catch (final Exception e) {
|
||||
// usually happen either because we failed to connect to the node
|
||||
// or because we failed serializing the message
|
||||
final RequestHolder holderToNotify = clientHandlers.remove(requestId);
|
||||
final Transport.ResponseContext contextToNotify = responseHandlers.remove(requestId);
|
||||
// If holderToNotify == null then handler has already been taken care of.
|
||||
if (holderToNotify != null) {
|
||||
holderToNotify.cancelTimeout();
|
||||
if (contextToNotify != null) {
|
||||
if (timeoutHandler != null) {
|
||||
timeoutHandler.cancel();
|
||||
}
|
||||
// callback that an exception happened, but on a different thread since we don't
|
||||
// want handlers to worry about stack overflows
|
||||
final SendRequestTransportException sendRequestException = new SendRequestTransportException(node, action, e);
|
||||
|
@ -633,7 +623,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
logger.debug(
|
||||
() -> new ParameterizedMessage(
|
||||
"failed to notify response handler on rejection, action: {}",
|
||||
holderToNotify.action()),
|
||||
contextToNotify.action()),
|
||||
e);
|
||||
}
|
||||
@Override
|
||||
|
@ -641,12 +631,12 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
logger.warn(
|
||||
() -> new ParameterizedMessage(
|
||||
"failed to notify response handler on exception, action: {}",
|
||||
holderToNotify.action()),
|
||||
contextToNotify.action()),
|
||||
e);
|
||||
}
|
||||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
holderToNotify.handler().handleException(sendRequestException);
|
||||
contextToNotify.handler().handleException(sendRequestException);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
|
@ -722,6 +712,44 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
return transport.addressesFromString(address, perAddressLimit);
|
||||
}
|
||||
|
||||
/**
|
||||
* A set of all valid action prefixes.
|
||||
*/
|
||||
public static final Set<String> VALID_ACTION_PREFIXES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
|
||||
"indices:admin",
|
||||
"indices:monitor",
|
||||
"indices:data/write",
|
||||
"indices:data/read",
|
||||
"indices:internal",
|
||||
"cluster:admin",
|
||||
"cluster:monitor",
|
||||
"cluster:internal",
|
||||
"internal:"
|
||||
)));
|
||||
|
||||
private void validateActionName(String actionName) {
|
||||
// TODO we should makes this a hard validation and throw an exception but we need a good way to add backwards layer
|
||||
// for it. Maybe start with a deprecation layer
|
||||
if (isValidActionName(actionName) == false) {
|
||||
logger.warn("invalid action name [" + actionName + "] must start with one of: " +
|
||||
TransportService.VALID_ACTION_PREFIXES );
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the action name starts with a valid prefix.
|
||||
*
|
||||
* @see #VALID_ACTION_PREFIXES
|
||||
*/
|
||||
public static boolean isValidActionName(String actionName) {
|
||||
for (String prefix : VALID_ACTION_PREFIXES) {
|
||||
if (actionName.startsWith(prefix)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Registers a new request handler
|
||||
*
|
||||
|
@ -732,10 +760,11 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
*/
|
||||
public <Request extends TransportRequest> void registerRequestHandler(String action, Supplier<Request> requestFactory,
|
||||
String executor, TransportRequestHandler<Request> handler) {
|
||||
validateActionName(action);
|
||||
handler = interceptor.interceptHandler(action, executor, false, handler);
|
||||
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(
|
||||
action, Streamable.newWriteableReader(requestFactory), taskManager, handler, executor, false, true);
|
||||
registerRequestHandler(reg);
|
||||
transport.registerRequestHandler(reg);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -749,10 +778,11 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
public <Request extends TransportRequest> void registerRequestHandler(String action, String executor,
|
||||
Writeable.Reader<Request> requestReader,
|
||||
TransportRequestHandler<Request> handler) {
|
||||
validateActionName(action);
|
||||
handler = interceptor.interceptHandler(action, executor, false, handler);
|
||||
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(
|
||||
action, requestReader, taskManager, handler, executor, false, true);
|
||||
registerRequestHandler(reg);
|
||||
transport.registerRequestHandler(reg);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -769,10 +799,11 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
String executor, boolean forceExecution,
|
||||
boolean canTripCircuitBreaker,
|
||||
TransportRequestHandler<Request> handler) {
|
||||
validateActionName(action);
|
||||
handler = interceptor.interceptHandler(action, executor, forceExecution, handler);
|
||||
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(
|
||||
action, Streamable.newWriteableReader(request), taskManager, handler, executor, forceExecution, canTripCircuitBreaker);
|
||||
registerRequestHandler(reg);
|
||||
transport.registerRequestHandler(reg);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -790,24 +821,16 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
boolean canTripCircuitBreaker,
|
||||
Writeable.Reader<Request> requestReader,
|
||||
TransportRequestHandler<Request> handler) {
|
||||
validateActionName(action);
|
||||
handler = interceptor.interceptHandler(action, executor, forceExecution, handler);
|
||||
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(
|
||||
action, requestReader, taskManager, handler, executor, forceExecution, canTripCircuitBreaker);
|
||||
registerRequestHandler(reg);
|
||||
}
|
||||
|
||||
private <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
|
||||
synchronized (requestHandlerMutex) {
|
||||
if (requestHandlers.containsKey(reg.getAction())) {
|
||||
throw new IllegalArgumentException("transport handlers for action " + reg.getAction() + " is already registered");
|
||||
}
|
||||
requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
|
||||
}
|
||||
transport.registerRequestHandler(reg);
|
||||
}
|
||||
|
||||
/** called by the {@link Transport} implementation once a request has been sent */
|
||||
void onRequestSent(DiscoveryNode node, long requestId, String action, TransportRequest request,
|
||||
TransportRequestOptions options) {
|
||||
public void onRequestSent(DiscoveryNode node, long requestId, String action, TransportRequest request,
|
||||
TransportRequestOptions options) {
|
||||
if (traceEnabled() && shouldTraceAction(action)) {
|
||||
traceRequestSent(node, requestId, action, options);
|
||||
}
|
||||
|
@ -818,14 +841,14 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
|
||||
/** called by the {@link Transport} implementation once a response was sent to calling node */
|
||||
void onResponseSent(long requestId, String action, TransportResponse response, TransportResponseOptions options) {
|
||||
public void onResponseSent(long requestId, String action, TransportResponse response, TransportResponseOptions options) {
|
||||
if (traceEnabled() && shouldTraceAction(action)) {
|
||||
traceResponseSent(requestId, action);
|
||||
}
|
||||
}
|
||||
|
||||
/** called by the {@link Transport} implementation after an exception was sent as a response to an incoming request */
|
||||
void onResponseSent(long requestId, String action, Exception e) {
|
||||
public void onResponseSent(long requestId, String action, Exception e) {
|
||||
if (traceEnabled() && shouldTraceAction(action)) {
|
||||
traceResponseSent(requestId, action, e);
|
||||
}
|
||||
|
@ -839,7 +862,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
* called by the {@link Transport} implementation when an incoming request arrives but before
|
||||
* any parsing of it has happened (with the exception of the requestId and action)
|
||||
*/
|
||||
void onRequestReceived(long requestId, String action) {
|
||||
public void onRequestReceived(long requestId, String action) {
|
||||
try {
|
||||
blockIncomingRequestsLatch.await();
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -851,33 +874,24 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
|
||||
public RequestHandlerRegistry getRequestHandler(String action) {
|
||||
return requestHandlers.get(action);
|
||||
return transport.getRequestHandler(action);
|
||||
}
|
||||
|
||||
/**
|
||||
* called by the {@link Transport} implementation when a response or an exception has been received for a previously
|
||||
* sent request (before any processing or deserialization was done). Returns the appropriate response handler or null if not
|
||||
* found.
|
||||
*/
|
||||
public TransportResponseHandler onResponseReceived(final long requestId) {
|
||||
RequestHolder holder = clientHandlers.remove(requestId);
|
||||
|
||||
@Override
|
||||
public void onResponseReceived(long requestId, Transport.ResponseContext holder) {
|
||||
if (holder == null) {
|
||||
checkForTimeout(requestId);
|
||||
return null;
|
||||
}
|
||||
holder.cancelTimeout();
|
||||
if (traceEnabled() && shouldTraceAction(holder.action())) {
|
||||
} else if (traceEnabled() && shouldTraceAction(holder.action())) {
|
||||
traceReceivedResponse(requestId, holder.connection().getNode(), holder.action());
|
||||
}
|
||||
return holder.handler();
|
||||
}
|
||||
|
||||
private void checkForTimeout(long requestId) {
|
||||
// lets see if its in the timeout holder, but sync on mutex to make sure any ongoing timeout handling has finished
|
||||
final DiscoveryNode sourceNode;
|
||||
final String action;
|
||||
assert clientHandlers.get(requestId) == null;
|
||||
assert responseHandlers.contains(requestId) == false;
|
||||
TimeoutInfoHolder timeoutInfoHolder = timeoutInfoHandlers.remove(requestId);
|
||||
if (timeoutInfoHolder != null) {
|
||||
long time = System.currentTimeMillis();
|
||||
|
@ -903,48 +917,18 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
}
|
||||
|
||||
void onNodeConnected(final DiscoveryNode node) {
|
||||
// capture listeners before spawning the background callback so the following pattern won't trigger a call
|
||||
// connectToNode(); connection is completed successfully
|
||||
// addConnectionListener(); this listener shouldn't be called
|
||||
final Stream<TransportConnectionListener> listenersToNotify = TransportService.this.connectionListeners.stream();
|
||||
getExecutorService().execute(() -> listenersToNotify.forEach(listener -> listener.onNodeConnected(node)));
|
||||
}
|
||||
|
||||
void onConnectionOpened(Transport.Connection connection) {
|
||||
// capture listeners before spawning the background callback so the following pattern won't trigger a call
|
||||
// connectToNode(); connection is completed successfully
|
||||
// addConnectionListener(); this listener shouldn't be called
|
||||
final Stream<TransportConnectionListener> listenersToNotify = TransportService.this.connectionListeners.stream();
|
||||
getExecutorService().execute(() -> listenersToNotify.forEach(listener -> listener.onConnectionOpened(connection)));
|
||||
}
|
||||
|
||||
public void onNodeDisconnected(final DiscoveryNode node) {
|
||||
@Override
|
||||
public void onConnectionClosed(Transport.Connection connection) {
|
||||
try {
|
||||
getExecutorService().execute( () -> {
|
||||
for (final TransportConnectionListener connectionListener : connectionListeners) {
|
||||
connectionListener.onNodeDisconnected(node);
|
||||
List<Transport.ResponseContext> pruned = responseHandlers.prune(h -> h.connection().getCacheKey().equals(connection
|
||||
.getCacheKey()));
|
||||
// callback that an exception happened, but on a different thread since we don't
|
||||
// want handlers to worry about stack overflows
|
||||
getExecutorService().execute(() -> {
|
||||
for (Transport.ResponseContext holderToNotify : pruned) {
|
||||
holderToNotify.handler().handleException(new NodeDisconnectedException(connection.getNode(), holderToNotify.action()));
|
||||
}
|
||||
});
|
||||
} catch (EsRejectedExecutionException ex) {
|
||||
logger.debug("Rejected execution on NodeDisconnected", ex);
|
||||
}
|
||||
}
|
||||
|
||||
void onConnectionClosed(Transport.Connection connection) {
|
||||
try {
|
||||
for (Map.Entry<Long, RequestHolder> entry : clientHandlers.entrySet()) {
|
||||
RequestHolder holder = entry.getValue();
|
||||
if (holder.connection().getCacheKey().equals(connection.getCacheKey())) {
|
||||
final RequestHolder holderToNotify = clientHandlers.remove(entry.getKey());
|
||||
if (holderToNotify != null) {
|
||||
// callback that an exception happened, but on a different thread since we don't
|
||||
// want handlers to worry about stack overflows
|
||||
getExecutorService().execute(() -> holderToNotify.handler().handleException(new NodeDisconnectedException(
|
||||
connection.getNode(), holderToNotify.action())));
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (EsRejectedExecutionException ex) {
|
||||
logger.debug("Rejected execution on onConnectionClosed", ex);
|
||||
}
|
||||
|
@ -970,32 +954,31 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
tracerLog.trace("[{}][{}] sent to [{}] (timeout: [{}])", requestId, action, node, options.timeout());
|
||||
}
|
||||
|
||||
class TimeoutHandler implements Runnable {
|
||||
final class TimeoutHandler implements Runnable {
|
||||
|
||||
private final long requestId;
|
||||
|
||||
private final long sentTime = System.currentTimeMillis();
|
||||
|
||||
private final String action;
|
||||
private final DiscoveryNode node;
|
||||
volatile ScheduledFuture future;
|
||||
|
||||
TimeoutHandler(long requestId) {
|
||||
TimeoutHandler(long requestId, DiscoveryNode node, String action) {
|
||||
this.requestId = requestId;
|
||||
this.node = node;
|
||||
this.action = action;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
// we get first to make sure we only add the TimeoutInfoHandler if needed.
|
||||
final RequestHolder holder = clientHandlers.get(requestId);
|
||||
if (holder != null) {
|
||||
// add it to the timeout information holder, in case we are going to get a response later
|
||||
if (responseHandlers.contains(requestId)) {
|
||||
long timeoutTime = System.currentTimeMillis();
|
||||
timeoutInfoHandlers.put(requestId, new TimeoutInfoHolder(holder.connection().getNode(), holder.action(), sentTime,
|
||||
timeoutTime));
|
||||
timeoutInfoHandlers.put(requestId, new TimeoutInfoHolder(node, action, sentTime, timeoutTime));
|
||||
// now that we have the information visible via timeoutInfoHandlers, we try to remove the request id
|
||||
final RequestHolder removedHolder = clientHandlers.remove(requestId);
|
||||
if (removedHolder != null) {
|
||||
assert removedHolder == holder : "two different holder instances for request [" + requestId + "]";
|
||||
removedHolder.handler().handleException(
|
||||
final Transport.ResponseContext holder = responseHandlers.remove(requestId);
|
||||
if (holder != null) {
|
||||
assert holder.action().equals(action);
|
||||
assert holder.connection().getNode().equals(node);
|
||||
holder.handler().handleException(
|
||||
new ReceiveTimeoutTransportException(holder.connection().getNode(), holder.action(),
|
||||
"request_id [" + requestId + "] timed out after [" + (timeoutTime - sentTime) + "ms]"));
|
||||
} else {
|
||||
|
@ -1006,11 +989,11 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
|
||||
/**
|
||||
* cancels timeout handling. this is a best effort only to avoid running it. remove the requestId from {@link #clientHandlers}
|
||||
* cancels timeout handling. this is a best effort only to avoid running it. remove the requestId from {@link #responseHandlers}
|
||||
* to make sure this doesn't run.
|
||||
*/
|
||||
public void cancel() {
|
||||
assert clientHandlers.get(requestId) == null :
|
||||
assert responseHandlers.contains(requestId) == false :
|
||||
"cancel must be called after the requestId [" + requestId + "] has been removed from clientHandlers";
|
||||
FutureUtils.cancel(future);
|
||||
}
|
||||
|
@ -1047,42 +1030,6 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
}
|
||||
|
||||
static class RequestHolder<T extends TransportResponse> {
|
||||
|
||||
private final TransportResponseHandler<T> handler;
|
||||
|
||||
private final Transport.Connection connection;
|
||||
|
||||
private final String action;
|
||||
|
||||
private final TimeoutHandler timeoutHandler;
|
||||
|
||||
RequestHolder(TransportResponseHandler<T> handler, Transport.Connection connection, String action, TimeoutHandler timeoutHandler) {
|
||||
this.handler = handler;
|
||||
this.connection = connection;
|
||||
this.action = action;
|
||||
this.timeoutHandler = timeoutHandler;
|
||||
}
|
||||
|
||||
public TransportResponseHandler<T> handler() {
|
||||
return handler;
|
||||
}
|
||||
|
||||
public Transport.Connection connection() {
|
||||
return this.connection;
|
||||
}
|
||||
|
||||
public String action() {
|
||||
return this.action;
|
||||
}
|
||||
|
||||
public void cancelTimeout() {
|
||||
if (timeoutHandler != null) {
|
||||
timeoutHandler.cancel();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This handler wrapper ensures that the response thread executes with the correct thread context. Before any of the handle methods
|
||||
* are invoked we restore the context.
|
||||
|
@ -1091,6 +1038,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
|
||||
private final TransportResponseHandler<T> delegate;
|
||||
private final Supplier<ThreadContext.StoredContext> contextSupplier;
|
||||
private volatile TimeoutHandler handler;
|
||||
|
||||
public ContextRestoreResponseHandler(Supplier<ThreadContext.StoredContext> contextSupplier, TransportResponseHandler<T> delegate) {
|
||||
this.delegate = delegate;
|
||||
|
@ -1104,6 +1052,9 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
|
||||
@Override
|
||||
public void handleResponse(T response) {
|
||||
if(handler != null) {
|
||||
handler.cancel();
|
||||
}
|
||||
try (ThreadContext.StoredContext ignore = contextSupplier.get()) {
|
||||
delegate.handleResponse(response);
|
||||
}
|
||||
|
@ -1111,6 +1062,9 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
|
||||
@Override
|
||||
public void handleException(TransportException exp) {
|
||||
if(handler != null) {
|
||||
handler.cancel();
|
||||
}
|
||||
try (ThreadContext.StoredContext ignore = contextSupplier.get()) {
|
||||
delegate.handleException(exp);
|
||||
}
|
||||
|
@ -1126,6 +1080,10 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
return getClass().getName() + "/" + delegate.toString();
|
||||
}
|
||||
|
||||
void setTimeoutHandler(TimeoutHandler handler) {
|
||||
this.handler = handler;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class DirectResponseChannel implements TransportChannel {
|
||||
|
@ -1159,7 +1117,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
@Override
|
||||
public void sendResponse(final TransportResponse response, TransportResponseOptions options) throws IOException {
|
||||
service.onResponseSent(requestId, action, response, options);
|
||||
final TransportResponseHandler handler = service.onResponseReceived(requestId);
|
||||
final TransportResponseHandler handler = service.responseHandlers.onResponseReceived(requestId, service);
|
||||
// ignore if its null, the service logs it
|
||||
if (handler != null) {
|
||||
final String executor = handler.executor();
|
||||
|
@ -1183,7 +1141,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
@Override
|
||||
public void sendResponse(Exception exception) throws IOException {
|
||||
service.onResponseSent(requestId, action, exception);
|
||||
final TransportResponseHandler handler = service.onResponseReceived(requestId);
|
||||
final TransportResponseHandler handler = service.responseHandlers.onResponseReceived(requestId, service);
|
||||
// ignore if its null, the service logs it
|
||||
if (handler != null) {
|
||||
final RemoteTransportException rtx = wrapInRemote(exception);
|
||||
|
@ -1224,6 +1182,7 @@ public class TransportService extends AbstractLifecycleComponent {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns the internal thread pool
|
||||
*/
|
||||
|
|
|
@ -214,7 +214,7 @@ public class CancellableTasksTests extends TaskManagerTestCase {
|
|||
for (int i = 0; i < testNodes.length; i++) {
|
||||
boolean shouldBlock = blockOnNodes.contains(testNodes[i]);
|
||||
logger.info("The action in the node [{}] should block: [{}]", testNodes[i].getNodeId(), shouldBlock);
|
||||
actions[i] = new CancellableTestNodesAction(CLUSTER_SETTINGS, "testAction", threadPool, testNodes[i]
|
||||
actions[i] = new CancellableTestNodesAction(CLUSTER_SETTINGS, "internal:testAction", threadPool, testNodes[i]
|
||||
.clusterService, testNodes[i].transportService, shouldBlock, actionLatch);
|
||||
}
|
||||
Task task = actions[0].execute(request, listener);
|
||||
|
|
|
@ -276,7 +276,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
TestNodesAction[] actions = new TestNodesAction[nodesCount];
|
||||
for (int i = 0; i < testNodes.length; i++) {
|
||||
final int node = i;
|
||||
actions[i] = new TestNodesAction(CLUSTER_SETTINGS, "testAction", threadPool, testNodes[i].clusterService,
|
||||
actions[i] = new TestNodesAction(CLUSTER_SETTINGS, "internal:testAction", threadPool, testNodes[i].clusterService,
|
||||
testNodes[i].transportService) {
|
||||
@Override
|
||||
protected NodeResponse nodeOperation(NodeRequest request) {
|
||||
|
@ -341,7 +341,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
int testNodeNum = randomIntBetween(0, testNodes.length - 1);
|
||||
TestNode testNode = testNodes[testNodeNum];
|
||||
ListTasksRequest listTasksRequest = new ListTasksRequest();
|
||||
listTasksRequest.setActions("testAction*"); // pick all test actions
|
||||
listTasksRequest.setActions("internal:testAction*"); // pick all test actions
|
||||
logger.info("Listing currently running tasks using node [{}]", testNodeNum);
|
||||
ListTasksResponse response = ActionTestUtils.executeBlocking(testNode.transportListTasksAction, listTasksRequest);
|
||||
logger.info("Checking currently running tasks");
|
||||
|
@ -361,7 +361,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
// Check task counts using transport with filtering
|
||||
testNode = testNodes[randomIntBetween(0, testNodes.length - 1)];
|
||||
listTasksRequest = new ListTasksRequest();
|
||||
listTasksRequest.setActions("testAction[n]"); // only pick node actions
|
||||
listTasksRequest.setActions("internal:testAction[n]"); // only pick node actions
|
||||
response = ActionTestUtils.executeBlocking(testNode.transportListTasksAction, listTasksRequest);
|
||||
assertEquals(testNodes.length, response.getPerNodeTasks().size());
|
||||
for (Map.Entry<String, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
|
||||
|
@ -384,7 +384,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
}
|
||||
|
||||
// Make sure that the main task on coordinating node is the task that was returned to us by execute()
|
||||
listTasksRequest.setActions("testAction"); // only pick the main task
|
||||
listTasksRequest.setActions("internal:testAction"); // only pick the main task
|
||||
response = ActionTestUtils.executeBlocking(testNode.transportListTasksAction, listTasksRequest);
|
||||
assertEquals(1, response.getTasks().size());
|
||||
assertEquals(mainTask.getId(), response.getTasks().get(0).getId());
|
||||
|
@ -412,7 +412,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
|
||||
// Get the parent task
|
||||
ListTasksRequest listTasksRequest = new ListTasksRequest();
|
||||
listTasksRequest.setActions("testAction");
|
||||
listTasksRequest.setActions("internal:testAction");
|
||||
ListTasksResponse response = ActionTestUtils.executeBlocking(testNode.transportListTasksAction, listTasksRequest);
|
||||
assertEquals(1, response.getTasks().size());
|
||||
String parentNode = response.getTasks().get(0).getTaskId().getNodeId();
|
||||
|
@ -424,7 +424,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
response = ActionTestUtils.executeBlocking(testNode.transportListTasksAction, listTasksRequest);
|
||||
assertEquals(testNodes.length, response.getTasks().size());
|
||||
for (TaskInfo task : response.getTasks()) {
|
||||
assertEquals("testAction[n]", task.getAction());
|
||||
assertEquals("internal:testAction[n]", task.getAction());
|
||||
assertEquals(parentNode, task.getParentTaskId().getNodeId());
|
||||
assertEquals(parentTaskId, task.getParentTaskId().getId());
|
||||
}
|
||||
|
@ -446,7 +446,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
// Check task counts using transport with filtering
|
||||
TestNode testNode = testNodes[randomIntBetween(0, testNodes.length - 1)];
|
||||
ListTasksRequest listTasksRequest = new ListTasksRequest();
|
||||
listTasksRequest.setActions("testAction[n]"); // only pick node actions
|
||||
listTasksRequest.setActions("internal:testAction[n]"); // only pick node actions
|
||||
ListTasksResponse response = ActionTestUtils.executeBlocking(testNode.transportListTasksAction, listTasksRequest);
|
||||
assertEquals(testNodes.length, response.getPerNodeTasks().size());
|
||||
for (Map.Entry<String, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
|
||||
|
@ -488,7 +488,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
responseLatch.countDown();
|
||||
}
|
||||
});
|
||||
String actionName = "testAction"; // only pick the main action
|
||||
String actionName = "internal:testAction"; // only pick the main action
|
||||
|
||||
// Try to cancel main task using action name
|
||||
CancelTasksRequest request = new CancelTasksRequest();
|
||||
|
@ -538,10 +538,10 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
setupTestNodes(settings);
|
||||
connectNodes(testNodes);
|
||||
TestNodesAction[] actions = new TestNodesAction[nodesCount];
|
||||
RecordingTaskManagerListener[] listeners = setupListeners(testNodes, "testAction*");
|
||||
RecordingTaskManagerListener[] listeners = setupListeners(testNodes, "internal:testAction*");
|
||||
for (int i = 0; i < testNodes.length; i++) {
|
||||
final int node = i;
|
||||
actions[i] = new TestNodesAction(CLUSTER_SETTINGS, "testAction", threadPool, testNodes[i].clusterService,
|
||||
actions[i] = new TestNodesAction(CLUSTER_SETTINGS, "internal:testAction", threadPool, testNodes[i].clusterService,
|
||||
testNodes[i].transportService) {
|
||||
@Override
|
||||
protected NodeResponse nodeOperation(NodeRequest request) {
|
||||
|
@ -581,7 +581,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
for (int i = 0; i < testNodes.length; i++) {
|
||||
final int node = i;
|
||||
// Simulate task action that fails on one of the tasks on one of the nodes
|
||||
tasksActions[i] = new TestTasksAction(CLUSTER_SETTINGS, "testTasksAction", testNodes[i].clusterService,
|
||||
tasksActions[i] = new TestTasksAction(CLUSTER_SETTINGS, "internal:testTasksAction", testNodes[i].clusterService,
|
||||
testNodes[i].transportService) {
|
||||
@Override
|
||||
protected void taskOperation(TestTasksRequest request, Task task, ActionListener<TestTaskResponse> listener) {
|
||||
|
@ -619,7 +619,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
// Run task action on node tasks that are currently running
|
||||
// should be successful on all nodes except one
|
||||
TestTasksRequest testTasksRequest = new TestTasksRequest();
|
||||
testTasksRequest.setActions("testAction[n]"); // pick all test actions
|
||||
testTasksRequest.setActions("internal:testAction[n]"); // pick all test actions
|
||||
TestTasksResponse response = ActionTestUtils.executeBlocking(tasksActions[0], testTasksRequest);
|
||||
assertThat(response.getTaskFailures(), hasSize(1)); // one task failed
|
||||
assertThat(response.getTaskFailures().get(0).getReason(), containsString("Task level failure"));
|
||||
|
@ -660,7 +660,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
final int node = i;
|
||||
// Simulate a task action that works on all nodes except nodes listed in filterNodes.
|
||||
// We are testing that it works.
|
||||
tasksActions[i] = new TestTasksAction(CLUSTER_SETTINGS, "testTasksAction",
|
||||
tasksActions[i] = new TestTasksAction(CLUSTER_SETTINGS, "internal:testTasksAction",
|
||||
testNodes[i].clusterService, testNodes[i].transportService) {
|
||||
|
||||
@Override
|
||||
|
@ -689,7 +689,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
|
|||
// Run task action on node tasks that are currently running
|
||||
// should be successful on all nodes except nodes that we filtered out
|
||||
TestTasksRequest testTasksRequest = new TestTasksRequest();
|
||||
testTasksRequest.setActions("testAction[n]"); // pick all test actions
|
||||
testTasksRequest.setActions("internal:testAction[n]"); // pick all test actions
|
||||
TestTasksResponse response = ActionTestUtils.executeBlocking(tasksActions[randomIntBetween(0, nodesCount - 1)], testTasksRequest);
|
||||
|
||||
// Get successful responses from all nodes except nodes that we filtered out
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.Collections;
|
||||
|
@ -66,8 +67,8 @@ public class MainActionTests extends ESTestCase {
|
|||
ClusterState state = ClusterState.builder(clusterName).blocks(blocks).build();
|
||||
when(clusterService.state()).thenReturn(state);
|
||||
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService
|
||||
.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportMainAction action = new TransportMainAction(settings, transportService, mock(ActionFilters.class), clusterService);
|
||||
AtomicReference<MainResponse> responseRef = new AtomicReference<>();
|
||||
action.doExecute(mock(Task.class), new MainRequest(), new ActionListener<MainResponse>() {
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.tasks.TaskManager;
|
|||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -128,10 +129,9 @@ public class MultiSearchActionTookTests extends ESTestCase {
|
|||
|
||||
private TransportMultiSearchAction createTransportMultiSearchAction(boolean controlledClock, AtomicLong expected) {
|
||||
Settings settings = Settings.builder().put("node.name", TransportMultiSearchActionTests.class.getSimpleName()).build();
|
||||
TaskManager taskManager = mock(TaskManager.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
boundAddress -> DiscoveryNode.createLocal(settings, boundAddress.publishAddress(), UUIDs.randomBase64UUID()), null,
|
||||
Collections.emptySet()) {
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundAddress -> DiscoveryNode.createLocal(settings, boundAddress.publishAddress(),
|
||||
UUIDs.randomBase64UUID()), null, Collections.emptySet()) {
|
||||
@Override
|
||||
public TaskManager getTaskManager() {
|
||||
return taskManager;
|
||||
|
@ -140,7 +140,6 @@ public class MultiSearchActionTookTests extends ESTestCase {
|
|||
ActionFilters actionFilters = new ActionFilters(new HashSet<>());
|
||||
ClusterService clusterService = mock(ClusterService.class);
|
||||
when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("test")).build());
|
||||
IndexNameExpressionResolver resolver = new Resolver(Settings.EMPTY);
|
||||
|
||||
final int availableProcessors = Runtime.getRuntime().availableProcessors();
|
||||
AtomicInteger counter = new AtomicInteger();
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.elasticsearch.action.support.ActionTestUtils;
|
|||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
|
@ -38,6 +37,7 @@ import org.elasticsearch.tasks.TaskManager;
|
|||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -82,8 +82,7 @@ public class TransportMultiSearchActionTests extends ESTestCase {
|
|||
ActionFilters actionFilters = mock(ActionFilters.class);
|
||||
when(actionFilters.filters()).thenReturn(new ActionFilter[0]);
|
||||
ThreadPool threadPool = new ThreadPool(settings);
|
||||
TaskManager taskManager = mock(TaskManager.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, threadPool,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), threadPool,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
boundAddress -> DiscoveryNode.createLocal(settings, boundAddress.publishAddress(), UUIDs.randomBase64UUID()), null,
|
||||
Collections.emptySet()) {
|
||||
|
@ -94,7 +93,6 @@ public class TransportMultiSearchActionTests extends ESTestCase {
|
|||
};
|
||||
ClusterService clusterService = mock(ClusterService.class);
|
||||
when(clusterService.state()).thenReturn(ClusterState.builder(new ClusterName("test")).build());
|
||||
IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(Settings.EMPTY);
|
||||
|
||||
// Keep track of the number of concurrent searches started by multi search api,
|
||||
// and if there are more searches than is allowed create an error and remember that.
|
||||
|
|
|
@ -174,7 +174,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, localNode, allNodes));
|
||||
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool) {
|
||||
@Override
|
||||
protected void masterOperation(Task task, Request request, ClusterState state, ActionListener<Response> listener) throws Exception {
|
||||
if (masterOperationFailure) {
|
||||
|
@ -211,7 +211,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
.blocks(ClusterBlocks.builder().addGlobalBlock(block)).build();
|
||||
setState(clusterService, stateWithBlock);
|
||||
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool) {
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
|
||||
Set<ClusterBlock> blocks = state.blocks().global();
|
||||
|
@ -253,7 +253,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
.blocks(ClusterBlocks.builder().addGlobalBlock(block)).build();
|
||||
setState(clusterService, stateWithBlock);
|
||||
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool) {
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(Request request, ClusterState state) {
|
||||
Set<ClusterBlock> blocks = state.blocks().global();
|
||||
|
@ -281,7 +281,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, randomFrom(localNode, remoteNode, null), allNodes));
|
||||
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool) {
|
||||
@Override
|
||||
protected boolean localExecute(Request request) {
|
||||
return true;
|
||||
|
@ -296,7 +296,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
Request request = new Request().masterNodeTimeout(TimeValue.timeValueSeconds(0));
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, null, allNodes));
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
assertTrue(listener.isDone());
|
||||
assertListenerThrows("MasterNotDiscoveredException should be thrown", listener, MasterNotDiscoveredException.class);
|
||||
}
|
||||
|
@ -305,7 +305,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
Request request = new Request();
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, null, allNodes));
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
assertFalse(listener.isDone());
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, localNode, allNodes));
|
||||
assertTrue(listener.isDone());
|
||||
|
@ -317,13 +317,13 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
setState(clusterService, ClusterStateCreationUtils.state(localNode, remoteNode, allNodes));
|
||||
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
|
||||
assertThat(transport.capturedRequests().length, equalTo(1));
|
||||
CapturingTransport.CapturedRequest capturedRequest = transport.capturedRequests()[0];
|
||||
assertTrue(capturedRequest.node.isMasterNode());
|
||||
assertThat(capturedRequest.request, equalTo(request));
|
||||
assertThat(capturedRequest.action, equalTo("testAction"));
|
||||
assertThat(capturedRequest.action, equalTo("internal:testAction"));
|
||||
|
||||
Response response = new Response();
|
||||
transport.handleResponse(capturedRequest.requestId, response);
|
||||
|
@ -340,14 +340,14 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
.version(randomIntBetween(0, 10))); // use a random base version so it can go down when simulating a restart.
|
||||
|
||||
PlainActionFuture<Response> listener = new PlainActionFuture<>();
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool).execute(request, listener);
|
||||
|
||||
CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
|
||||
assertThat(capturedRequests.length, equalTo(1));
|
||||
CapturingTransport.CapturedRequest capturedRequest = capturedRequests[0];
|
||||
assertTrue(capturedRequest.node.isMasterNode());
|
||||
assertThat(capturedRequest.request, equalTo(request));
|
||||
assertThat(capturedRequest.action, equalTo("testAction"));
|
||||
assertThat(capturedRequest.action, equalTo("internal:testAction"));
|
||||
|
||||
if (rejoinSameMaster) {
|
||||
transport.handleRemoteError(capturedRequest.requestId, new ConnectTransportException(masterNode, "Fake error"));
|
||||
|
@ -380,7 +380,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
capturedRequest = capturedRequests[0];
|
||||
assertTrue(capturedRequest.node.isMasterNode());
|
||||
assertThat(capturedRequest.request, equalTo(request));
|
||||
assertThat(capturedRequest.action, equalTo("testAction"));
|
||||
assertThat(capturedRequest.action, equalTo("internal:testAction"));
|
||||
} else if (failsWithConnectTransportException) {
|
||||
transport.handleRemoteError(capturedRequest.requestId, new ConnectTransportException(masterNode, "Fake error"));
|
||||
assertFalse(listener.isDone());
|
||||
|
@ -413,7 +413,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
|
||||
setState(clusterService, ClusterStateCreationUtils.state(localNode, localNode, allNodes));
|
||||
|
||||
new Action(Settings.EMPTY, "testAction", transportService, clusterService, threadPool) {
|
||||
new Action(Settings.EMPTY, "internal:testAction", transportService, clusterService, threadPool) {
|
||||
@Override
|
||||
protected void masterOperation(Request request, ClusterState state, ActionListener<Response> listener) throws Exception {
|
||||
// The other node has become master, simulate failures of this node while publishing cluster state through ZenDiscovery
|
||||
|
@ -429,7 +429,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
|
|||
CapturingTransport.CapturedRequest capturedRequest = transport.capturedRequests()[0];
|
||||
assertTrue(capturedRequest.node.isMasterNode());
|
||||
assertThat(capturedRequest.request, equalTo(request));
|
||||
assertThat(capturedRequest.action, equalTo("testAction"));
|
||||
assertThat(capturedRequest.action, equalTo("internal:testAction"));
|
||||
|
||||
transport.handleResponse(capturedRequest.requestId, response);
|
||||
assertTrue(listener.isDone());
|
||||
|
|
|
@ -210,7 +210,7 @@ public class BroadcastReplicationTests extends ESTestCase {
|
|||
TestBroadcastReplicationAction(Settings settings, ClusterService clusterService, TransportService transportService,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
TransportReplicationAction replicatedBroadcastShardAction) {
|
||||
super("test-broadcast-replication-action", DummyBroadcastRequest::new, settings, clusterService, transportService,
|
||||
super("internal:test-broadcast-replication-action", DummyBroadcastRequest::new, settings, clusterService, transportService,
|
||||
actionFilters, indexNameExpressionResolver, replicatedBroadcastShardAction);
|
||||
}
|
||||
|
||||
|
|
|
@ -168,7 +168,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
transportService.start();
|
||||
transportService.acceptIncomingRequests();
|
||||
shardStateAction = new ShardStateAction(Settings.EMPTY, clusterService, transportService, null, null, threadPool);
|
||||
action = new TestAction(Settings.EMPTY, "testAction", transportService, clusterService, shardStateAction, threadPool);
|
||||
action = new TestAction(Settings.EMPTY, "internal:testAction", transportService, clusterService, shardStateAction, threadPool);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -196,7 +196,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
Request request = new Request();
|
||||
PlainActionFuture<TestResponse> listener = new PlainActionFuture<>();
|
||||
ReplicationTask task = maybeTask();
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testActionWithBlocks",
|
||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithBlocks",
|
||||
transportService, clusterService, shardStateAction, threadPool) {
|
||||
@Override
|
||||
protected ClusterBlockLevel globalBlockLevel() {
|
||||
|
@ -236,7 +236,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
ClusterBlockException.class);
|
||||
assertIndexShardUninitialized();
|
||||
|
||||
action = new TestAction(Settings.EMPTY, "testActionWithNoBlocks", transportService, clusterService, shardStateAction, threadPool) {
|
||||
action = new TestAction(Settings.EMPTY, "internal:testActionWithNoBlocks", transportService, clusterService, shardStateAction,
|
||||
threadPool) {
|
||||
@Override
|
||||
protected ClusterBlockLevel globalBlockLevel() {
|
||||
return null;
|
||||
|
@ -287,7 +288,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||
assertThat(capturedRequests, notNullValue());
|
||||
assertThat(capturedRequests.size(), equalTo(1));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("internal:testAction[p]"));
|
||||
assertIndexShardCounter(0);
|
||||
}
|
||||
|
||||
|
@ -339,7 +340,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||
assertThat(capturedRequests, notNullValue());
|
||||
assertThat(capturedRequests.size(), equalTo(1));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("internal:testAction[p]"));
|
||||
assertIndexShardCounter(0);
|
||||
}
|
||||
|
||||
|
@ -378,7 +379,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
ReplicationTask task = maybeTask();
|
||||
|
||||
ClusterBlockLevel indexBlockLevel = randomBoolean() ? ClusterBlockLevel.WRITE : null;
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testActionWithBlocks", transportService,
|
||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithBlocks", transportService,
|
||||
clusterService, shardStateAction, threadPool) {
|
||||
@Override
|
||||
protected ClusterBlockLevel indexBlockLevel() {
|
||||
|
@ -416,7 +417,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
reroutePhase.run();
|
||||
CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
|
||||
assertThat(capturedRequests, arrayWithSize(1));
|
||||
assertThat(capturedRequests[0].action, equalTo("testAction[p]"));
|
||||
assertThat(capturedRequests[0].action, equalTo("internal:testAction[p]"));
|
||||
assertPhase(task, "waiting_on_primary");
|
||||
assertFalse(request.isRetrySet.get());
|
||||
transport.handleRemoteError(capturedRequests[0].requestId, randomRetryPrimaryException(shardId));
|
||||
|
@ -427,7 +428,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
assertThat(listener.isDone(), equalTo(false));
|
||||
capturedRequests = transport.getCapturedRequestsAndClear();
|
||||
assertThat(capturedRequests, arrayWithSize(1));
|
||||
assertThat(capturedRequests[0].action, equalTo("testAction[p]"));
|
||||
assertThat(capturedRequests[0].action, equalTo("internal:testAction[p]"));
|
||||
assertPhase(task, "waiting_on_primary");
|
||||
transport.handleRemoteError(capturedRequests[0].requestId, randomRetryPrimaryException(shardId));
|
||||
assertListenerThrows("must throw index not found exception", listener, ElasticsearchException.class);
|
||||
|
@ -438,7 +439,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
setState(clusterService, clusterService.state());
|
||||
capturedRequests = transport.getCapturedRequestsAndClear();
|
||||
assertThat(capturedRequests, arrayWithSize(1));
|
||||
assertThat(capturedRequests[0].action, equalTo("testAction[p]"));
|
||||
assertThat(capturedRequests[0].action, equalTo("internal:testAction[p]"));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -474,10 +475,10 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
assertThat(capturedRequests, notNullValue());
|
||||
assertThat(capturedRequests.size(), equalTo(1));
|
||||
if (clusterService.state().nodes().getLocalNodeId().equals(primaryNodeId)) {
|
||||
assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("internal:testAction[p]"));
|
||||
assertPhase(task, "waiting_on_primary");
|
||||
} else {
|
||||
assertThat(capturedRequests.get(0).action, equalTo("testAction"));
|
||||
assertThat(capturedRequests.get(0).action, equalTo("internal:testAction"));
|
||||
assertPhase(task, "rerouted");
|
||||
}
|
||||
assertFalse(request.isRetrySet.get());
|
||||
|
@ -531,7 +532,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
transport.capturedRequestsByTargetNode().get(primaryShard.relocatingNodeId());
|
||||
assertThat(requests, notNullValue());
|
||||
assertThat(requests.size(), equalTo(1));
|
||||
assertThat("primary request was not delegated to relocation target", requests.get(0).action, equalTo("testAction[p]"));
|
||||
assertThat("primary request was not delegated to relocation target", requests.get(0).action, equalTo("internal:testAction[p]"));
|
||||
assertThat("primary term not properly set on primary delegation",
|
||||
((TransportReplicationAction.ConcreteShardRequest<Request>)requests.get(0).request).getPrimaryTerm(), equalTo(primaryTerm));
|
||||
assertPhase(task, "primary_delegation");
|
||||
|
@ -705,7 +706,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
};
|
||||
|
||||
TestAction action =
|
||||
new TestAction(Settings.EMPTY, "testSeqNoIsSetOnPrimary", transportService, clusterService, shardStateAction, threadPool) {
|
||||
new TestAction(Settings.EMPTY, "internal:testSeqNoIsSetOnPrimary", transportService, clusterService, shardStateAction,
|
||||
threadPool) {
|
||||
@Override
|
||||
protected IndexShard getIndexShard(ShardId shardId) {
|
||||
return shard;
|
||||
|
@ -788,8 +790,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0);
|
||||
boolean throwException = randomBoolean();
|
||||
final ReplicationTask task = maybeTask();
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, shardStateAction,
|
||||
threadPool) {
|
||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
||||
shardStateAction, threadPool) {
|
||||
@Override
|
||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
||||
assertIndexShardCounter(1);
|
||||
|
@ -924,8 +926,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
setState(clusterService, state);
|
||||
AtomicBoolean throwException = new AtomicBoolean(true);
|
||||
final ReplicationTask task = maybeTask();
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, shardStateAction,
|
||||
threadPool) {
|
||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
||||
shardStateAction, threadPool) {
|
||||
@Override
|
||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
||||
assertPhase(task, "replica");
|
||||
|
@ -960,7 +962,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
assertThat(capturedRequests, notNullValue());
|
||||
assertThat(capturedRequests.size(), equalTo(1));
|
||||
final CapturingTransport.CapturedRequest capturedRequest = capturedRequests.get(0);
|
||||
assertThat(capturedRequest.action, equalTo("testActionWithExceptions[r]"));
|
||||
assertThat(capturedRequest.action, equalTo("internal:testActionWithExceptions[r]"));
|
||||
assertThat(capturedRequest.request, instanceOf(TransportReplicationAction.ConcreteReplicaRequest.class));
|
||||
assertThat(((TransportReplicationAction.ConcreteReplicaRequest) capturedRequest.request).getGlobalCheckpoint(),
|
||||
equalTo(checkpoint));
|
||||
|
@ -988,8 +990,8 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
transportService.acceptIncomingRequests();
|
||||
|
||||
AtomicBoolean calledSuccessfully = new AtomicBoolean(false);
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, shardStateAction,
|
||||
threadPool) {
|
||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService,
|
||||
shardStateAction, threadPool) {
|
||||
@Override
|
||||
protected ReplicaResult shardOperationOnReplica(Request request, IndexShard replica) {
|
||||
assertPhase(task, "replica");
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.elasticsearch.test.ESTestCase;
|
|||
import org.elasticsearch.test.transport.CapturingTransport;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
@ -259,7 +260,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
|||
transportService.start();
|
||||
transportService.acceptIncomingRequests();
|
||||
ShardStateAction shardStateAction = new ShardStateAction(Settings.EMPTY, clusterService, transportService, null, null, threadPool);
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testAction", transportService,
|
||||
TestAction action = new TestAction(Settings.EMPTY, "internal:testAction", transportService,
|
||||
clusterService, shardStateAction, threadPool);
|
||||
final String index = "test";
|
||||
final ShardId shardId = new ShardId(index, "_na_", 0);
|
||||
|
@ -355,10 +356,10 @@ public class TransportWriteActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
protected TestAction(boolean withDocumentFailureOnPrimary, boolean withDocumentFailureOnReplica) {
|
||||
super(Settings.EMPTY, "test",
|
||||
new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null,
|
||||
Collections.emptySet()), null,
|
||||
null, null, null, new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new,
|
||||
super(Settings.EMPTY, "internal:test",
|
||||
new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet()), null, null, null, null,
|
||||
new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new,
|
||||
TestRequest::new, ThreadPool.Names.SAME);
|
||||
this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary;
|
||||
this.withDocumentFailureOnReplica = withDocumentFailureOnReplica;
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterName;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.CheckedBiConsumer;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.component.Lifecycle;
|
||||
import org.elasticsearch.common.component.LifecycleListener;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -34,13 +35,14 @@ import org.elasticsearch.common.transport.BoundTransportAddress;
|
|||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
import org.elasticsearch.transport.ConnectionProfile;
|
||||
import org.elasticsearch.transport.RequestHandlerRegistry;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportResponseHandler;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.transport.TransportStats;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -51,22 +53,22 @@ import java.util.Random;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArraySet;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
abstract class FailAndRetryMockTransport<Response extends TransportResponse> implements Transport {
|
||||
|
||||
private final Random random;
|
||||
private final ClusterName clusterName;
|
||||
private volatile Map<String, RequestHandlerRegistry> requestHandlers = Collections.emptyMap();
|
||||
final Object requestHandlerMutex = new Object();
|
||||
private final ResponseHandlers responseHandlers = new ResponseHandlers();
|
||||
private TransportConnectionListener listener;
|
||||
|
||||
private boolean connectMode = true;
|
||||
|
||||
private TransportService transportService;
|
||||
|
||||
private final AtomicInteger connectTransportExceptions = new AtomicInteger();
|
||||
private final AtomicInteger failures = new AtomicInteger();
|
||||
private final AtomicInteger successes = new AtomicInteger();
|
||||
private final Set<DiscoveryNode> triedNodes = new CopyOnWriteArraySet<>();
|
||||
private final AtomicLong requestId = new AtomicLong();
|
||||
|
||||
FailAndRetryMockTransport(Random random, ClusterName clusterName) {
|
||||
this.random = new Random(random.nextLong());
|
||||
|
@ -90,12 +92,12 @@ abstract class FailAndRetryMockTransport<Response extends TransportResponse> imp
|
|||
//we make sure that nodes get added to the connected ones when calling addTransportAddress, by returning proper nodes info
|
||||
if (connectMode) {
|
||||
if (TransportLivenessAction.NAME.equals(action)) {
|
||||
TransportResponseHandler transportResponseHandler = transportService.onResponseReceived(requestId);
|
||||
TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener);
|
||||
transportResponseHandler.handleResponse(new LivenessResponse(ClusterName.CLUSTER_NAME_SETTING.
|
||||
getDefault(Settings.EMPTY),
|
||||
node));
|
||||
} else if (ClusterStateAction.NAME.equals(action)) {
|
||||
TransportResponseHandler transportResponseHandler = transportService.onResponseReceived(requestId);
|
||||
TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener);
|
||||
ClusterState clusterState = getMockClusterState(node);
|
||||
transportResponseHandler.handleResponse(new ClusterStateResponse(clusterName, clusterState, 0L));
|
||||
} else {
|
||||
|
@ -116,7 +118,7 @@ abstract class FailAndRetryMockTransport<Response extends TransportResponse> imp
|
|||
//throw whatever exception that is not a subclass of ConnectTransportException
|
||||
throw new IllegalStateException();
|
||||
} else {
|
||||
TransportResponseHandler transportResponseHandler = transportService.onResponseReceived(requestId);
|
||||
TransportResponseHandler transportResponseHandler = responseHandlers.onResponseReceived(requestId, listener);
|
||||
if (random.nextBoolean()) {
|
||||
successes.incrementAndGet();
|
||||
transportResponseHandler.handleResponse(newResponse());
|
||||
|
@ -162,10 +164,6 @@ abstract class FailAndRetryMockTransport<Response extends TransportResponse> imp
|
|||
return triedNodes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTransportService(TransportService transportServiceAdapter) {
|
||||
this.transportService = transportServiceAdapter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BoundTransportAddress boundAddress() {
|
||||
|
@ -223,13 +221,37 @@ abstract class FailAndRetryMockTransport<Response extends TransportResponse> imp
|
|||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long newRequestId() {
|
||||
return requestId.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TransportStats getStats() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
|
||||
synchronized (requestHandlerMutex) {
|
||||
if (requestHandlers.containsKey(reg.getAction())) {
|
||||
throw new IllegalArgumentException("transport handlers for action " + reg.getAction() + " is already registered");
|
||||
}
|
||||
requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public ResponseHandlers getResponseHandlers() {
|
||||
return responseHandlers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RequestHandlerRegistry getRequestHandler(String action) {
|
||||
return requestHandlers.get(action);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addConnectionListener(TransportConnectionListener listener) {
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeConnectionListener(TransportConnectionListener listener) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,9 @@ import org.elasticsearch.threadpool.TestThreadPool;
|
|||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
import org.elasticsearch.transport.ConnectionProfile;
|
||||
import org.elasticsearch.transport.RequestHandlerRegistry;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
|
@ -54,7 +56,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
|
@ -171,12 +172,28 @@ public class NodeConnectionsServiceTests extends ESTestCase {
|
|||
}
|
||||
|
||||
final class MockTransport implements Transport {
|
||||
private final AtomicLong requestId = new AtomicLong();
|
||||
Set<DiscoveryNode> connectedNodes = ConcurrentCollections.newConcurrentSet();
|
||||
volatile boolean randomConnectionExceptions = false;
|
||||
private ResponseHandlers responseHandlers = new ResponseHandlers();
|
||||
private TransportConnectionListener listener = new TransportConnectionListener() {};
|
||||
|
||||
@Override
|
||||
public void setTransportService(TransportService service) {
|
||||
public <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public RequestHandlerRegistry getRequestHandler(String action) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addConnectionListener(TransportConnectionListener listener) {
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeConnectionListener(TransportConnectionListener listener) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -208,12 +225,14 @@ public class NodeConnectionsServiceTests extends ESTestCase {
|
|||
throw new ConnectTransportException(node, "simulated");
|
||||
}
|
||||
connectedNodes.add(node);
|
||||
listener.onNodeConnected(node);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void disconnectFromNode(DiscoveryNode node) {
|
||||
connectedNodes.remove(node);
|
||||
listener.onNodeDisconnected(node);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -226,20 +245,22 @@ public class NodeConnectionsServiceTests extends ESTestCase {
|
|||
|
||||
@Override
|
||||
public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options)
|
||||
throws IOException, TransportException {
|
||||
throws TransportException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
public void close() {
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException {
|
||||
return getConnection(node);
|
||||
public Connection openConnection(DiscoveryNode node, ConnectionProfile profile) {
|
||||
Connection connection = getConnection(node);
|
||||
listener.onConnectionOpened(connection);
|
||||
return connection;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -247,11 +268,6 @@ public class NodeConnectionsServiceTests extends ESTestCase {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long newRequestId() {
|
||||
return requestId.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Lifecycle.State lifecycleState() {
|
||||
return null;
|
||||
|
@ -278,5 +294,10 @@ public class NodeConnectionsServiceTests extends ESTestCase {
|
|||
public TransportStats getStats() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResponseHandlers getResponseHandlers() {
|
||||
return responseHandlers;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -139,7 +139,7 @@ public class ClusterStateChanges extends AbstractComponent {
|
|||
IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
|
||||
DestructiveOperations destructiveOperations = new DestructiveOperations(settings, clusterSettings);
|
||||
Environment environment = TestEnvironment.newEnvironment(settings);
|
||||
Transport transport = null; // it's not used
|
||||
Transport transport = mock(Transport.class); // it's not used
|
||||
|
||||
// mocks
|
||||
clusterService = mock(ClusterService.class);
|
||||
|
|
|
@ -53,6 +53,7 @@ import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
|||
import org.elasticsearch.repositories.RepositoriesService;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -399,7 +400,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice
|
|||
when(threadPool.generic()).thenReturn(mock(ExecutorService.class));
|
||||
final MockIndicesService indicesService = indicesServiceSupplier.get();
|
||||
final Settings settings = Settings.builder().put("node.name", discoveryNode.getName()).build();
|
||||
final TransportService transportService = new TransportService(settings, null, threadPool,
|
||||
final TransportService transportService = new TransportService(settings, mock(Transport.class), threadPool,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
boundAddress -> DiscoveryNode.createLocal(settings, boundAddress.publishAddress(), UUIDs.randomBase64UUID()), null,
|
||||
Collections.emptySet());
|
||||
|
|
|
@ -1,54 +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;
|
||||
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.either;
|
||||
import static org.hamcrest.CoreMatchers.startsWith;
|
||||
|
||||
/**
|
||||
* This test verifies that all of the action names follow our defined naming conventions.
|
||||
* The identified categories are:
|
||||
* - indices:admin: apis that allow to perform administration tasks against indices
|
||||
* - indices:data: apis that are about data
|
||||
* - indices:read: apis that read data
|
||||
* - indices:write: apis that write data
|
||||
* - cluster:admin: cluster apis that allow to perform administration tasks
|
||||
* - cluster:monitor: cluster apis that allow to monitor the system
|
||||
* - internal: internal actions that are used from node to node but not directly exposed to users
|
||||
*
|
||||
* Any transport action belongs to one of the above categories and its name starts with its category, followed by a '/'
|
||||
* and the name of the api itself (e.g. cluster:admin/nodes/restart).
|
||||
* When an api exposes multiple transport handlers, some of which are invoked internally during the execution of the api,
|
||||
* we use the `[n]` suffix to identify node actions and the `[s]` suffix to identify shard actions.
|
||||
*/
|
||||
public class ActionNamesIT extends ESIntegTestCase {
|
||||
public void testActionNamesCategories() throws NoSuchFieldException, IllegalAccessException {
|
||||
TransportService transportService = internalCluster().getInstance(TransportService.class);
|
||||
for (String action : transportService.requestHandlers.keySet()) {
|
||||
assertThat("action doesn't belong to known category", action,
|
||||
either(startsWith("indices:admin")).or(startsWith("indices:monitor"))
|
||||
.or(startsWith("indices:data/read")).or(startsWith("indices:data/write"))
|
||||
.or(startsWith("cluster:admin")).or(startsWith("cluster:monitor"))
|
||||
.or(startsWith("internal:")));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -87,36 +87,36 @@ public class TransportActionProxyTests extends ESTestCase {
|
|||
|
||||
|
||||
public void testSendMessage() throws InterruptedException {
|
||||
serviceA.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceA.registerRequestHandler("internal:test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertEquals(request.sourceNode, "TS_A");
|
||||
SimpleTestResponse response = new SimpleTestResponse();
|
||||
response.targetNode = "TS_A";
|
||||
channel.sendResponse(response);
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(serviceA, "/test", SimpleTestResponse::new);
|
||||
TransportActionProxy.registerProxyAction(serviceA, "internal:test", SimpleTestResponse::new);
|
||||
serviceA.connectToNode(nodeB);
|
||||
|
||||
serviceB.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceB.registerRequestHandler("internal:test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertEquals(request.sourceNode, "TS_A");
|
||||
SimpleTestResponse response = new SimpleTestResponse();
|
||||
response.targetNode = "TS_B";
|
||||
channel.sendResponse(response);
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(serviceB, "/test", SimpleTestResponse::new);
|
||||
TransportActionProxy.registerProxyAction(serviceB, "internal:test", SimpleTestResponse::new);
|
||||
serviceB.connectToNode(nodeC);
|
||||
serviceC.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceC.registerRequestHandler("internal:test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertEquals(request.sourceNode, "TS_A");
|
||||
SimpleTestResponse response = new SimpleTestResponse();
|
||||
response.targetNode = "TS_C";
|
||||
channel.sendResponse(response);
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(serviceC, "/test", SimpleTestResponse::new);
|
||||
TransportActionProxy.registerProxyAction(serviceC, "internal:test", SimpleTestResponse::new);
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
serviceA.sendRequest(nodeB, TransportActionProxy.getProxyAction("/test"), TransportActionProxy.wrapRequest(nodeC,
|
||||
serviceA.sendRequest(nodeB, TransportActionProxy.getProxyAction("internal:test"), TransportActionProxy.wrapRequest(nodeC,
|
||||
new SimpleTestRequest("TS_A")), new TransportResponseHandler<SimpleTestResponse>() {
|
||||
@Override
|
||||
public SimpleTestResponse newInstance() {
|
||||
|
@ -150,33 +150,33 @@ public class TransportActionProxyTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testException() throws InterruptedException {
|
||||
serviceA.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceA.registerRequestHandler("internal:test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertEquals(request.sourceNode, "TS_A");
|
||||
SimpleTestResponse response = new SimpleTestResponse();
|
||||
response.targetNode = "TS_A";
|
||||
channel.sendResponse(response);
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(serviceA, "/test", SimpleTestResponse::new);
|
||||
TransportActionProxy.registerProxyAction(serviceA, "internal:test", SimpleTestResponse::new);
|
||||
serviceA.connectToNode(nodeB);
|
||||
|
||||
serviceB.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceB.registerRequestHandler("internal:test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertEquals(request.sourceNode, "TS_A");
|
||||
SimpleTestResponse response = new SimpleTestResponse();
|
||||
response.targetNode = "TS_B";
|
||||
channel.sendResponse(response);
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(serviceB, "/test", SimpleTestResponse::new);
|
||||
TransportActionProxy.registerProxyAction(serviceB, "internal:test", SimpleTestResponse::new);
|
||||
serviceB.connectToNode(nodeC);
|
||||
serviceC.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceC.registerRequestHandler("internal:test", SimpleTestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
throw new ElasticsearchException("greetings from TS_C");
|
||||
});
|
||||
TransportActionProxy.registerProxyAction(serviceC, "/test", SimpleTestResponse::new);
|
||||
TransportActionProxy.registerProxyAction(serviceC, "internal:test", SimpleTestResponse::new);
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
serviceA.sendRequest(nodeB, TransportActionProxy.getProxyAction("/test"), TransportActionProxy.wrapRequest(nodeC,
|
||||
serviceA.sendRequest(nodeB, TransportActionProxy.getProxyAction("internal:test"), TransportActionProxy.wrapRequest(nodeC,
|
||||
new SimpleTestRequest("TS_A")), new TransportResponseHandler<SimpleTestResponse>() {
|
||||
@Override
|
||||
public SimpleTestResponse newInstance() {
|
||||
|
|
|
@ -104,6 +104,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit;
|
|||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -139,6 +140,7 @@ import org.elasticsearch.indices.IndicesService;
|
|||
import org.elasticsearch.indices.store.IndicesStore;
|
||||
import org.elasticsearch.ingest.IngestMetadata;
|
||||
import org.elasticsearch.node.NodeMocksPlugin;
|
||||
import org.elasticsearch.plugins.NetworkPlugin;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.script.ScriptMetaData;
|
||||
|
@ -152,6 +154,10 @@ import org.elasticsearch.test.disruption.NetworkDisruption;
|
|||
import org.elasticsearch.test.disruption.ServiceDisruptionScheme;
|
||||
import org.elasticsearch.test.store.MockFSIndexStore;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.transport.TransportInterceptor;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportRequestHandler;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -2011,6 +2017,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
mocks.add(MockHttpTransport.TestPlugin.class);
|
||||
}
|
||||
mocks.add(TestSeedPlugin.class);
|
||||
mocks.add(AssertActionNamePlugin.class);
|
||||
return Collections.unmodifiableList(mocks);
|
||||
}
|
||||
|
||||
|
@ -2021,6 +2028,25 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public static final class AssertActionNamePlugin extends Plugin implements NetworkPlugin {
|
||||
@Override
|
||||
public List<TransportInterceptor> getTransportInterceptors(NamedWriteableRegistry namedWriteableRegistry,
|
||||
ThreadContext threadContext) {
|
||||
return Arrays.asList(new TransportInterceptor() {
|
||||
@Override
|
||||
public <T extends TransportRequest> TransportRequestHandler<T> interceptHandler(String action, String executor,
|
||||
boolean forceExecution,
|
||||
TransportRequestHandler<T> actualHandler) {
|
||||
if (TransportService.isValidActionName(action) == false) {
|
||||
throw new IllegalArgumentException("invalid action name [" + action + "] must start with one of: " +
|
||||
TransportService.VALID_ACTION_PREFIXES );
|
||||
}
|
||||
return actualHandler;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the client ratio configured via
|
||||
*/
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.CheckedBiConsumer;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.component.Lifecycle;
|
||||
import org.elasticsearch.common.component.LifecycleListener;
|
||||
|
@ -33,13 +34,14 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
|||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
import org.elasticsearch.transport.ConnectionProfile;
|
||||
import org.elasticsearch.transport.RemoteTransportException;
|
||||
import org.elasticsearch.transport.RequestHandlerRegistry;
|
||||
import org.elasticsearch.transport.SendRequestTransportException;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
import org.elasticsearch.transport.TransportResponse;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.transport.TransportStats;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -54,14 +56,16 @@ import java.util.Map;
|
|||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static org.apache.lucene.util.LuceneTestCase.rarely;
|
||||
|
||||
/** A transport class that doesn't send anything but rather captures all requests for inspection from tests */
|
||||
public class CapturingTransport implements Transport {
|
||||
|
||||
private TransportService transportService;
|
||||
private volatile Map<String, RequestHandlerRegistry> requestHandlers = Collections.emptyMap();
|
||||
final Object requestHandlerMutex = new Object();
|
||||
private final ResponseHandlers responseHandlers = new ResponseHandlers();
|
||||
private TransportConnectionListener listener;
|
||||
|
||||
public static class CapturedRequest {
|
||||
public final DiscoveryNode node;
|
||||
|
@ -79,8 +83,6 @@ public class CapturingTransport implements Transport {
|
|||
|
||||
private ConcurrentMap<Long, Tuple<DiscoveryNode, String>> requests = new ConcurrentHashMap<>();
|
||||
private BlockingQueue<CapturedRequest> capturedRequests = ConcurrentCollections.newBlockingQueue();
|
||||
private final AtomicLong requestId = new AtomicLong();
|
||||
|
||||
|
||||
/** returns all requests captured so far. Doesn't clear the captured request list. See {@link #clear()} */
|
||||
public CapturedRequest[] capturedRequests() {
|
||||
|
@ -137,7 +139,7 @@ public class CapturingTransport implements Transport {
|
|||
|
||||
/** simulate a response for the given requestId */
|
||||
public void handleResponse(final long requestId, final TransportResponse response) {
|
||||
transportService.onResponseReceived(requestId).handleResponse(response);
|
||||
responseHandlers.onResponseReceived(requestId, listener).handleResponse(response);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -189,7 +191,7 @@ public class CapturingTransport implements Transport {
|
|||
* @param e the failure
|
||||
*/
|
||||
public void handleError(final long requestId, final TransportException e) {
|
||||
transportService.onResponseReceived(requestId).handleException(e);
|
||||
responseHandlers.onResponseReceived(requestId, listener).handleException(e);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -219,11 +221,6 @@ public class CapturingTransport implements Transport {
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTransportService(TransportService transportService) {
|
||||
this.transportService = transportService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BoundTransportAddress boundAddress() {
|
||||
return null;
|
||||
|
@ -285,11 +282,6 @@ public class CapturingTransport implements Transport {
|
|||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long newRequestId() {
|
||||
return requestId.incrementAndGet();
|
||||
}
|
||||
|
||||
public Connection getConnection(DiscoveryNode node) {
|
||||
try {
|
||||
return openConnection(node, null);
|
||||
|
@ -297,4 +289,40 @@ public class CapturingTransport implements Transport {
|
|||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
|
||||
synchronized (requestHandlerMutex) {
|
||||
if (requestHandlers.containsKey(reg.getAction())) {
|
||||
throw new IllegalArgumentException("transport handlers for action " + reg.getAction() + " is already registered");
|
||||
}
|
||||
requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public ResponseHandlers getResponseHandlers() {
|
||||
return responseHandlers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RequestHandlerRegistry getRequestHandler(String action) {
|
||||
return requestHandlers.get(action);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addConnectionListener(TransportConnectionListener listener) {
|
||||
if (this.listener != null) {
|
||||
throw new IllegalStateException("listener already set");
|
||||
}
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeConnectionListener(TransportConnectionListener listener) {
|
||||
if (listener == this.listener) {
|
||||
this.listener = null;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,6 +53,7 @@ import org.elasticsearch.transport.MockTcpTransport;
|
|||
import org.elasticsearch.transport.RequestHandlerRegistry;
|
||||
import org.elasticsearch.transport.TcpTransport;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportConnectionListener;
|
||||
import org.elasticsearch.transport.TransportException;
|
||||
import org.elasticsearch.transport.TransportInterceptor;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
|
@ -72,7 +73,6 @@ import java.util.Queue;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.LinkedBlockingDeque;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.Function;
|
||||
|
@ -169,17 +169,6 @@ public final class MockTransportService extends TransportService {
|
|||
}
|
||||
}
|
||||
|
||||
private volatile String executorName;
|
||||
|
||||
public void setExecutorName(final String executorName) {
|
||||
this.executorName = executorName;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExecutorService getExecutorService() {
|
||||
return executorName == null ? super.getExecutorService() : getThreadPool().executor(executorName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears all the registered rules.
|
||||
*/
|
||||
|
@ -559,8 +548,23 @@ public final class MockTransportService extends TransportService {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setTransportService(TransportService service) {
|
||||
transport.setTransportService(service);
|
||||
public void addConnectionListener(TransportConnectionListener listener) {
|
||||
transport.addConnectionListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeConnectionListener(TransportConnectionListener listener) {
|
||||
return transport.removeConnectionListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
|
||||
transport.registerRequestHandler(reg);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RequestHandlerRegistry getRequestHandler(String action) {
|
||||
return transport.getRequestHandler(action);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -595,11 +599,6 @@ public final class MockTransportService extends TransportService {
|
|||
return transport.getLocalAddresses();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long newRequestId() {
|
||||
return transport.newRequestId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection(DiscoveryNode node) {
|
||||
return new FilteredConnection(transport.getConnection(node)) {
|
||||
|
@ -627,6 +626,11 @@ public final class MockTransportService extends TransportService {
|
|||
return transport.getStats();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResponseHandlers getResponseHandlers() {
|
||||
return transport.getResponseHandlers();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Lifecycle.State lifecycleState() {
|
||||
return transport.lifecycleState();
|
||||
|
|
|
@ -205,7 +205,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
|
||||
public void testHelloWorld() {
|
||||
serviceA.registerRequestHandler("sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
assertThat("moshe", equalTo(request.message));
|
||||
try {
|
||||
|
@ -216,7 +216,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
});
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHello",
|
||||
new StringMessageRequest("moshe"), new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
public StringMessageResponse newInstance() {
|
||||
|
@ -247,7 +247,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertThat(e.getMessage(), false, equalTo(true));
|
||||
}
|
||||
|
||||
res = serviceB.submitRequest(nodeA, "sayHello", new StringMessageRequest("moshe"),
|
||||
res = serviceB.submitRequest(nodeA, "internal:sayHello", new StringMessageRequest("moshe"),
|
||||
TransportRequestOptions.builder().withCompress(true).build(), new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
public StringMessageResponse newInstance() {
|
||||
|
@ -281,7 +281,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
public void testThreadContext() throws ExecutionException, InterruptedException {
|
||||
|
||||
serviceA.registerRequestHandler("ping_pong", StringMessageRequest::new, ThreadPool.Names.GENERIC, (request, channel, task) -> {
|
||||
serviceA.registerRequestHandler("internal:ping_pong", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
assertEquals("ping_user", threadPool.getThreadContext().getHeader("test.ping.user"));
|
||||
assertNull(threadPool.getThreadContext().getTransient("my_private_context"));
|
||||
try {
|
||||
|
@ -325,7 +326,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
threadPool.getThreadContext().putHeader("test.ping.user", "ping_user");
|
||||
threadPool.getThreadContext().putTransient("my_private_context", context);
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "ping_pong", ping, responseHandler);
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:ping_pong", ping, responseHandler);
|
||||
|
||||
StringMessageResponse message = res.get();
|
||||
assertThat("pong", equalTo(message.message));
|
||||
|
@ -339,7 +340,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
// this should be a noop
|
||||
serviceA.disconnectFromNode(nodeA);
|
||||
final AtomicReference<Exception> exception = new AtomicReference<>();
|
||||
serviceA.registerRequestHandler("localNode", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:localNode", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
try {
|
||||
channel.sendResponse(new StringMessageResponse(request.message));
|
||||
|
@ -349,7 +350,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
});
|
||||
final AtomicReference<String> responseString = new AtomicReference<>();
|
||||
final CountDownLatch responseLatch = new CountDownLatch(1);
|
||||
serviceA.sendRequest(nodeA, "localNode", new StringMessageRequest("test"), new TransportResponseHandler<StringMessageResponse>() {
|
||||
serviceA.sendRequest(nodeA, "internal:localNode", new StringMessageRequest("test"),
|
||||
new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
public StringMessageResponse newInstance() {
|
||||
return new StringMessageResponse();
|
||||
|
@ -390,7 +392,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
fail(e.getMessage());
|
||||
}
|
||||
};
|
||||
final String ACTION = "action";
|
||||
final String ACTION = "internal:action";
|
||||
serviceA.registerRequestHandler(ACTION, TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
|
||||
requestHandler);
|
||||
serviceB.registerRequestHandler(ACTION, TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
|
||||
|
@ -485,7 +487,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testVoidMessageCompressed() {
|
||||
serviceA.registerRequestHandler("sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHello", TransportRequest.Empty::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
try {
|
||||
TransportResponseOptions responseOptions = TransportResponseOptions.builder().withCompress(true).build();
|
||||
|
@ -496,7 +498,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
});
|
||||
|
||||
TransportFuture<TransportResponse.Empty> res = serviceB.submitRequest(nodeA, "sayHello",
|
||||
TransportFuture<TransportResponse.Empty> res = serviceB.submitRequest(nodeA, "internal:sayHello",
|
||||
TransportRequest.Empty.INSTANCE, TransportRequestOptions.builder().withCompress(true).build(),
|
||||
new TransportResponseHandler<TransportResponse.Empty>() {
|
||||
@Override
|
||||
|
@ -529,7 +531,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testHelloWorldCompressed() {
|
||||
serviceA.registerRequestHandler("sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
new TransportRequestHandler<StringMessageRequest>() {
|
||||
@Override
|
||||
public void messageReceived(StringMessageRequest request, TransportChannel channel, Task task) {
|
||||
|
@ -544,7 +546,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
});
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHello",
|
||||
new StringMessageRequest("moshe"), TransportRequestOptions.builder().withCompress(true).build(),
|
||||
new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
|
@ -578,7 +580,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testErrorMessage() {
|
||||
serviceA.registerRequestHandler("sayHelloException", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHelloException", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
new TransportRequestHandler<StringMessageRequest>() {
|
||||
@Override
|
||||
public void messageReceived(StringMessageRequest request, TransportChannel channel, Task task) throws Exception {
|
||||
|
@ -587,7 +589,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
});
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHelloException",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHelloException",
|
||||
new StringMessageRequest("moshe"), new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
public StringMessageResponse newInstance() {
|
||||
|
@ -639,7 +641,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
public void testConcurrentSendRespondAndDisconnect() throws BrokenBarrierException, InterruptedException {
|
||||
Set<Exception> sendingErrors = ConcurrentCollections.newConcurrentSet();
|
||||
Set<Exception> responseErrors = ConcurrentCollections.newConcurrentSet();
|
||||
serviceA.registerRequestHandler("test", TestRequest::new,
|
||||
serviceA.registerRequestHandler("internal:test", TestRequest::new,
|
||||
randomBoolean() ? ThreadPool.Names.SAME : ThreadPool.Names.GENERIC, (request, channel, task) -> {
|
||||
try {
|
||||
channel.sendResponse(new TestResponse());
|
||||
|
@ -656,7 +658,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
logger.trace("caught exception while responding from node B", e);
|
||||
}
|
||||
};
|
||||
serviceB.registerRequestHandler("test", TestRequest::new, ThreadPool.Names.SAME, ignoringRequestHandler);
|
||||
serviceB.registerRequestHandler("internal:test", TestRequest::new, ThreadPool.Names.SAME, ignoringRequestHandler);
|
||||
|
||||
int halfSenders = scaledRandomIntBetween(3, 10);
|
||||
final CyclicBarrier go = new CyclicBarrier(halfSenders * 2 + 1);
|
||||
|
@ -712,7 +714,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
final String info = sender + "_" + iter;
|
||||
final DiscoveryNode node = nodeB; // capture now
|
||||
try {
|
||||
serviceA.sendRequest(node, "test", new TestRequest(info),
|
||||
serviceA.sendRequest(node, "internal:test", new TestRequest(info),
|
||||
new ActionListenerResponseHandler<>(listener, TestResponse::new));
|
||||
try {
|
||||
listener.actionGet();
|
||||
|
@ -742,7 +744,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
// simulate restart of nodeB
|
||||
serviceB.close();
|
||||
MockTransportService newService = buildService("TS_B_" + i, version1, null);
|
||||
newService.registerRequestHandler("test", TestRequest::new, ThreadPool.Names.SAME, ignoringRequestHandler);
|
||||
newService.registerRequestHandler("internal:test", TestRequest::new, ThreadPool.Names.SAME, ignoringRequestHandler);
|
||||
serviceB = newService;
|
||||
nodeB = newService.getLocalDiscoNode();
|
||||
serviceB.connectToNode(nodeA);
|
||||
|
@ -763,7 +765,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
public void testNotifyOnShutdown() throws Exception {
|
||||
final CountDownLatch latch2 = new CountDownLatch(1);
|
||||
try {
|
||||
serviceA.registerRequestHandler("foobar", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:foobar", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
try {
|
||||
latch2.await();
|
||||
|
@ -773,7 +775,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
fail(e.getMessage());
|
||||
}
|
||||
});
|
||||
TransportFuture<TransportResponse.Empty> foobar = serviceB.submitRequest(nodeA, "foobar",
|
||||
TransportFuture<TransportResponse.Empty> foobar = serviceB.submitRequest(nodeA, "internal:foobar",
|
||||
new StringMessageRequest(""), TransportRequestOptions.EMPTY, EmptyTransportResponseHandler.INSTANCE_SAME);
|
||||
latch2.countDown();
|
||||
try {
|
||||
|
@ -789,7 +791,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testTimeoutSendExceptionWithNeverSendingBackResponse() throws Exception {
|
||||
serviceA.registerRequestHandler("sayHelloTimeoutNoResponse", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHelloTimeoutNoResponse", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
new TransportRequestHandler<StringMessageRequest>() {
|
||||
@Override
|
||||
public void messageReceived(StringMessageRequest request, TransportChannel channel, Task task) {
|
||||
|
@ -798,7 +800,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
});
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHelloTimeoutNoResponse",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHelloTimeoutNoResponse",
|
||||
new StringMessageRequest("moshe"), TransportRequestOptions.builder().withTimeout(100).build(),
|
||||
new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
|
@ -834,7 +836,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
CountDownLatch waitForever = new CountDownLatch(1);
|
||||
CountDownLatch doneWaitingForever = new CountDownLatch(1);
|
||||
Semaphore inFlight = new Semaphore(Integer.MAX_VALUE);
|
||||
serviceA.registerRequestHandler("sayHelloTimeoutDelayedResponse", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHelloTimeoutDelayedResponse", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
new TransportRequestHandler<StringMessageRequest>() {
|
||||
@Override
|
||||
public void messageReceived(StringMessageRequest request, TransportChannel channel, Task task) throws InterruptedException {
|
||||
|
@ -862,7 +864,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
});
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHelloTimeoutDelayedResponse",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHelloTimeoutDelayedResponse",
|
||||
new StringMessageRequest("forever"), TransportRequestOptions.builder().withTimeout(100).build(),
|
||||
new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
|
@ -900,7 +902,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
for (int i = 0; i < 10; i++) {
|
||||
final int counter = i;
|
||||
// now, try and send another request, this times, with a short timeout
|
||||
TransportFuture<StringMessageResponse> result = serviceB.submitRequest(nodeA, "sayHelloTimeoutDelayedResponse",
|
||||
TransportFuture<StringMessageResponse> result = serviceB.submitRequest(nodeA, "internal:sayHelloTimeoutDelayedResponse",
|
||||
new StringMessageRequest(counter + "ms"), TransportRequestOptions.builder().withTimeout(3000).build(),
|
||||
new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
|
@ -975,12 +977,12 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
};
|
||||
|
||||
serviceA.registerRequestHandler("test", StringMessageRequest::new, ThreadPool.Names.SAME, handler);
|
||||
serviceA.registerRequestHandler("testError", StringMessageRequest::new, ThreadPool.Names.SAME, handlerWithError);
|
||||
serviceB.registerRequestHandler("test", StringMessageRequest::new, ThreadPool.Names.SAME, handler);
|
||||
serviceB.registerRequestHandler("testError", StringMessageRequest::new, ThreadPool.Names.SAME, handlerWithError);
|
||||
serviceA.registerRequestHandler("internal:test", StringMessageRequest::new, ThreadPool.Names.SAME, handler);
|
||||
serviceA.registerRequestHandler("internal:testError", StringMessageRequest::new, ThreadPool.Names.SAME, handlerWithError);
|
||||
serviceB.registerRequestHandler("internal:test", StringMessageRequest::new, ThreadPool.Names.SAME, handler);
|
||||
serviceB.registerRequestHandler("internal:testError", StringMessageRequest::new, ThreadPool.Names.SAME, handlerWithError);
|
||||
|
||||
final Tracer tracer = new Tracer(new HashSet<>(Arrays.asList("test", "testError")));
|
||||
final Tracer tracer = new Tracer(new HashSet<>(Arrays.asList("internal:test", "internal:testError")));
|
||||
// the tracer is invoked concurrently after the actual action is executed. that means a Tracer#requestSent can still be in-flight
|
||||
// from a handshake executed on connect in the setup method. this might confuse this test since it expects exact number of
|
||||
// invocations. To prevent any unrelated events messing with this test we filter on the actions we execute in this test.
|
||||
|
@ -991,7 +993,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
boolean timeout = randomBoolean();
|
||||
TransportRequestOptions options = timeout ? TransportRequestOptions.builder().withTimeout(1).build() :
|
||||
TransportRequestOptions.EMPTY;
|
||||
serviceA.sendRequest(nodeB, "test", new StringMessageRequest("", 10), options, noopResponseHandler);
|
||||
serviceA.sendRequest(nodeB, "internal:test", new StringMessageRequest("", 10), options, noopResponseHandler);
|
||||
requestCompleted.acquire();
|
||||
tracer.expectedEvents.get().await();
|
||||
assertThat("didn't see request sent", tracer.sawRequestSent, equalTo(true));
|
||||
|
@ -1001,7 +1003,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertThat("saw error sent", tracer.sawErrorSent, equalTo(false));
|
||||
|
||||
tracer.reset(4);
|
||||
serviceA.sendRequest(nodeB, "testError", new StringMessageRequest(""), noopResponseHandler);
|
||||
serviceA.sendRequest(nodeB, "internal:testError", new StringMessageRequest(""), noopResponseHandler);
|
||||
requestCompleted.acquire();
|
||||
tracer.expectedEvents.get().await();
|
||||
assertThat("didn't see request sent", tracer.sawRequestSent, equalTo(true));
|
||||
|
@ -1017,7 +1019,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
includeSettings = randomBoolean() ? "*" : "";
|
||||
excludeSettings = "*Error";
|
||||
} else {
|
||||
includeSettings = "test";
|
||||
includeSettings = "internal:test";
|
||||
excludeSettings = "DOESN'T_MATCH";
|
||||
}
|
||||
clusterSettings.applySettings(Settings.builder()
|
||||
|
@ -1026,7 +1028,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
.build());
|
||||
|
||||
tracer.reset(4);
|
||||
serviceA.sendRequest(nodeB, "test", new StringMessageRequest(""), noopResponseHandler);
|
||||
serviceA.sendRequest(nodeB, "internal:test", new StringMessageRequest(""), noopResponseHandler);
|
||||
requestCompleted.acquire();
|
||||
tracer.expectedEvents.get().await();
|
||||
assertThat("didn't see request sent", tracer.sawRequestSent, equalTo(true));
|
||||
|
@ -1036,7 +1038,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertThat("saw error sent", tracer.sawErrorSent, equalTo(false));
|
||||
|
||||
tracer.reset(2);
|
||||
serviceA.sendRequest(nodeB, "testError", new StringMessageRequest(""), noopResponseHandler);
|
||||
serviceA.sendRequest(nodeB, "internal:testError", new StringMessageRequest(""), noopResponseHandler);
|
||||
requestCompleted.acquire();
|
||||
tracer.expectedEvents.get().await();
|
||||
assertThat("saw request sent", tracer.sawRequestSent, equalTo(false));
|
||||
|
@ -1255,7 +1257,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testVersionFrom0to1() throws Exception {
|
||||
serviceB.registerRequestHandler("/version", Version1Request::new, ThreadPool.Names.SAME,
|
||||
serviceB.registerRequestHandler("internal:version", Version1Request::new, ThreadPool.Names.SAME,
|
||||
new TransportRequestHandler<Version1Request>() {
|
||||
@Override
|
||||
public void messageReceived(Version1Request request, TransportChannel channel, Task task) throws Exception {
|
||||
|
@ -1271,7 +1273,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
Version0Request version0Request = new Version0Request();
|
||||
version0Request.value1 = 1;
|
||||
Version0Response version0Response = serviceA.submitRequest(nodeB, "/version", version0Request,
|
||||
Version0Response version0Response = serviceA.submitRequest(nodeB, "internal:version", version0Request,
|
||||
new TransportResponseHandler<Version0Response>() {
|
||||
@Override
|
||||
public Version0Response newInstance() {
|
||||
|
@ -1299,7 +1301,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testVersionFrom1to0() throws Exception {
|
||||
serviceA.registerRequestHandler("/version", Version0Request::new, ThreadPool.Names.SAME,
|
||||
serviceA.registerRequestHandler("internal:version", Version0Request::new, ThreadPool.Names.SAME,
|
||||
new TransportRequestHandler<Version0Request>() {
|
||||
@Override
|
||||
public void messageReceived(Version0Request request, TransportChannel channel, Task task) throws Exception {
|
||||
|
@ -1314,7 +1316,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
Version1Request version1Request = new Version1Request();
|
||||
version1Request.value1 = 1;
|
||||
version1Request.value2 = 2;
|
||||
Version1Response version1Response = serviceB.submitRequest(nodeA, "/version", version1Request,
|
||||
Version1Response version1Response = serviceB.submitRequest(nodeA, "internal:version", version1Request,
|
||||
new TransportResponseHandler<Version1Response>() {
|
||||
@Override
|
||||
public Version1Response newInstance() {
|
||||
|
@ -1344,7 +1346,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testVersionFrom1to1() throws Exception {
|
||||
serviceB.registerRequestHandler("/version", Version1Request::new, ThreadPool.Names.SAME,
|
||||
serviceB.registerRequestHandler("internal:version", Version1Request::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertThat(request.value1, equalTo(1));
|
||||
assertThat(request.value2, equalTo(2));
|
||||
|
@ -1358,7 +1360,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
Version1Request version1Request = new Version1Request();
|
||||
version1Request.value1 = 1;
|
||||
version1Request.value2 = 2;
|
||||
Version1Response version1Response = serviceB.submitRequest(nodeB, "/version", version1Request,
|
||||
Version1Response version1Response = serviceB.submitRequest(nodeB, "internal:version", version1Request,
|
||||
new TransportResponseHandler<Version1Response>() {
|
||||
@Override
|
||||
public Version1Response newInstance() {
|
||||
|
@ -1388,7 +1390,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testVersionFrom0to0() throws Exception {
|
||||
serviceA.registerRequestHandler("/version", Version0Request::new, ThreadPool.Names.SAME,
|
||||
serviceA.registerRequestHandler("internal:version", Version0Request::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
assertThat(request.value1, equalTo(1));
|
||||
Version0Response response = new Version0Response();
|
||||
|
@ -1399,7 +1401,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
Version0Request version0Request = new Version0Request();
|
||||
version0Request.value1 = 1;
|
||||
Version0Response version0Response = serviceA.submitRequest(nodeA, "/version", version0Request,
|
||||
Version0Response version0Response = serviceA.submitRequest(nodeA, "internal:version", version0Request,
|
||||
new TransportResponseHandler<Version0Response>() {
|
||||
@Override
|
||||
public Version0Response newInstance() {
|
||||
|
@ -1427,7 +1429,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testMockFailToSendNoConnectRule() throws Exception {
|
||||
serviceA.registerRequestHandler("sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
assertThat("moshe", equalTo(request.message));
|
||||
throw new RuntimeException("bad message !!!");
|
||||
|
@ -1435,7 +1437,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
serviceB.addFailToSendNoConnectRule(serviceA);
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHello",
|
||||
new StringMessageRequest("moshe"), new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
public StringMessageResponse newInstance() {
|
||||
|
@ -1484,7 +1486,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testMockUnresponsiveRule() throws IOException {
|
||||
serviceA.registerRequestHandler("sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
serviceA.registerRequestHandler("internal:sayHello", StringMessageRequest::new, ThreadPool.Names.GENERIC,
|
||||
(request, channel, task) -> {
|
||||
assertThat("moshe", equalTo(request.message));
|
||||
throw new RuntimeException("bad message !!!");
|
||||
|
@ -1492,7 +1494,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
serviceB.addUnresponsiveRule(serviceA);
|
||||
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "sayHello",
|
||||
TransportFuture<StringMessageResponse> res = serviceB.submitRequest(nodeA, "internal:sayHello",
|
||||
new StringMessageRequest("moshe"), TransportRequestOptions.builder().withTimeout(100).build(),
|
||||
new TransportResponseHandler<StringMessageResponse>() {
|
||||
@Override
|
||||
|
@ -1539,15 +1541,12 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
final CountDownLatch latch = new CountDownLatch(2);
|
||||
final AtomicReference<TransportAddress> addressA = new AtomicReference<>();
|
||||
final AtomicReference<TransportAddress> addressB = new AtomicReference<>();
|
||||
serviceB.registerRequestHandler("action1", TestRequest::new, ThreadPool.Names.SAME, new TransportRequestHandler<TestRequest>() {
|
||||
@Override
|
||||
public void messageReceived(TestRequest request, TransportChannel channel, Task task) throws Exception {
|
||||
addressA.set(request.remoteAddress());
|
||||
channel.sendResponse(new TestResponse());
|
||||
latch.countDown();
|
||||
}
|
||||
serviceB.registerRequestHandler("internal:action1", TestRequest::new, ThreadPool.Names.SAME, (request, channel, task) -> {
|
||||
addressA.set(request.remoteAddress());
|
||||
channel.sendResponse(new TestResponse());
|
||||
latch.countDown();
|
||||
});
|
||||
serviceA.sendRequest(nodeB, "action1", new TestRequest(), new TransportResponseHandler<TestResponse>() {
|
||||
serviceA.sendRequest(nodeB, "internal:action1", new TestRequest(), new TransportResponseHandler<TestResponse>() {
|
||||
@Override
|
||||
public TestResponse newInstance() {
|
||||
return new TestResponse();
|
||||
|
@ -1582,7 +1581,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
try (TransportService service = buildService("TS_TEST", version0, null,
|
||||
Settings.EMPTY, false, false)) {
|
||||
AtomicBoolean requestProcessed = new AtomicBoolean(false);
|
||||
service.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
service.registerRequestHandler("internal:action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
requestProcessed.set(true);
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
|
@ -1594,7 +1593,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
Settings.EMPTY, true, false);
|
||||
try (Transport.Connection connection = serviceA.openConnection(node, null)) {
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
serviceA.sendRequest(connection, "action", new TestRequest(), TransportRequestOptions.EMPTY,
|
||||
serviceA.sendRequest(connection, "internal:action", new TestRequest(), TransportRequestOptions.EMPTY,
|
||||
new TransportResponseHandler<TestResponse>() {
|
||||
@Override
|
||||
public TestResponse newInstance() {
|
||||
|
@ -1756,7 +1755,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
if (randomBoolean() && request.resendCount++ < 20) {
|
||||
DiscoveryNode node = randomFrom(nodeA, nodeB, nodeC);
|
||||
logger.debug("send secondary request from {} to {} - {}", toNodeMap.get(service), node, request.info);
|
||||
service.sendRequest(node, "action1", new TestRequest("secondary " + request.info),
|
||||
service.sendRequest(node, "internal:action1", new TestRequest("secondary " + request.info),
|
||||
TransportRequestOptions.builder().withCompress(randomBoolean()).build(),
|
||||
new TransportResponseHandler<TestResponse>() {
|
||||
@Override
|
||||
|
@ -1800,11 +1799,11 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
|
||||
}
|
||||
}
|
||||
serviceB.registerRequestHandler("action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
serviceB.registerRequestHandler("internal:action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
new TestRequestHandler(serviceB));
|
||||
serviceC.registerRequestHandler("action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
serviceC.registerRequestHandler("internal:action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
new TestRequestHandler(serviceC));
|
||||
serviceA.registerRequestHandler("action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
serviceA.registerRequestHandler("internal:action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
new TestRequestHandler(serviceA));
|
||||
int iters = randomIntBetween(30, 60);
|
||||
CountDownLatch allRequestsDone = new CountDownLatch(iters);
|
||||
|
@ -1847,7 +1846,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
TransportService service = randomFrom(serviceC, serviceB, serviceA);
|
||||
DiscoveryNode node = randomFrom(nodeC, nodeB, nodeA);
|
||||
logger.debug("send from {} to {}", toNodeMap.get(service), node);
|
||||
service.sendRequest(node, "action1", new TestRequest("REQ[" + i + "]"),
|
||||
service.sendRequest(node, "internal:action1", new TestRequest("REQ[" + i + "]"),
|
||||
TransportRequestOptions.builder().withCompress(randomBoolean()).build(), new TestResponseHandler(i));
|
||||
}
|
||||
logger.debug("waiting for response");
|
||||
|
@ -1868,18 +1867,19 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testRegisterHandlerTwice() {
|
||||
serviceB.registerRequestHandler("action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
serviceB.registerRequestHandler("internal:action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
(request, message, task) -> {
|
||||
throw new AssertionError("boom");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () ->
|
||||
serviceB.registerRequestHandler("action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
serviceB.registerRequestHandler("internal:action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME,
|
||||
ThreadPool.Names.GENERIC),
|
||||
(request, message, task) -> {
|
||||
throw new AssertionError("boom");
|
||||
})
|
||||
);
|
||||
|
||||
serviceA.registerRequestHandler("action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
serviceA.registerRequestHandler("internal:action1", TestRequest::new, randomFrom(ThreadPool.Names.SAME, ThreadPool.Names.GENERIC),
|
||||
(request, message, task) -> {
|
||||
throw new AssertionError("boom");
|
||||
});
|
||||
|
@ -2066,7 +2066,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
public void testResponseHeadersArePreserved() throws InterruptedException {
|
||||
List<String> executors = new ArrayList<>(ThreadPool.THREAD_POOL_TYPES.keySet());
|
||||
CollectionUtil.timSort(executors); // makes sure it's reproducible
|
||||
serviceA.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceA.registerRequestHandler("internal:action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
|
||||
threadPool.getThreadContext().putTransient("boom", new Object());
|
||||
|
@ -2118,8 +2118,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
}
|
||||
};
|
||||
|
||||
serviceB.sendRequest(nodeA, "action", new TestRequest(randomFrom("fail", "pass")), transportResponseHandler);
|
||||
serviceA.sendRequest(nodeA, "action", new TestRequest(randomFrom("fail", "pass")), transportResponseHandler);
|
||||
serviceB.sendRequest(nodeA, "internal:action", new TestRequest(randomFrom("fail", "pass")), transportResponseHandler);
|
||||
serviceA.sendRequest(nodeA, "internal:action", new TestRequest(randomFrom("fail", "pass")), transportResponseHandler);
|
||||
latch.await();
|
||||
}
|
||||
|
||||
|
@ -2127,7 +2127,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
List<String> executors = new ArrayList<>(ThreadPool.THREAD_POOL_TYPES.keySet());
|
||||
CollectionUtil.timSort(executors); // makes sure it's reproducible
|
||||
TransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
|
||||
serviceC.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceC.registerRequestHandler("internal:action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
// do nothing
|
||||
});
|
||||
|
@ -2177,7 +2177,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
TransportRequestOptions.Type.STATE);
|
||||
try (Transport.Connection connection = serviceB.openConnection(serviceC.getLocalNode(), builder.build())) {
|
||||
serviceC.close();
|
||||
serviceB.sendRequest(connection, "action", new TestRequest("boom"), TransportRequestOptions.EMPTY,
|
||||
serviceB.sendRequest(connection, "internal:action", new TestRequest("boom"), TransportRequestOptions.EMPTY,
|
||||
transportResponseHandler);
|
||||
}
|
||||
latch.await();
|
||||
|
@ -2187,7 +2187,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
|
||||
CountDownLatch receivedLatch = new CountDownLatch(1);
|
||||
CountDownLatch sendResponseLatch = new CountDownLatch(1);
|
||||
serviceC.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceC.registerRequestHandler("internal:action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
// don't block on a network thread here
|
||||
threadPool.generic().execute(new AbstractRunnable() {
|
||||
|
@ -2242,7 +2242,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
TransportRequestOptions.Type.STATE);
|
||||
|
||||
try (Transport.Connection connection = serviceB.openConnection(serviceC.getLocalNode(), builder.build())) {
|
||||
serviceB.sendRequest(connection, "action", new TestRequest("hello world"), TransportRequestOptions.EMPTY,
|
||||
serviceB.sendRequest(connection, "internal:action", new TestRequest("hello world"), TransportRequestOptions.EMPTY,
|
||||
transportResponseHandler);
|
||||
receivedLatch.await();
|
||||
serviceC.close();
|
||||
|
@ -2255,7 +2255,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
MockTransportService serviceC = build(Settings.builder().put("name", "TS_TEST").build(), version0, null, true);
|
||||
CountDownLatch receivedLatch = new CountDownLatch(1);
|
||||
CountDownLatch sendResponseLatch = new CountDownLatch(1);
|
||||
serviceB.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceB.registerRequestHandler("internal:action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
// don't block on a network thread here
|
||||
threadPool.generic().execute(new AbstractRunnable() {
|
||||
|
@ -2322,7 +2322,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertEquals(25, transportStats.getRxSize().getBytes());
|
||||
assertEquals(45, transportStats.getTxSize().getBytes());
|
||||
});
|
||||
serviceC.sendRequest(connection, "action", new TestRequest("hello world"), TransportRequestOptions.EMPTY,
|
||||
serviceC.sendRequest(connection, "internal:action", new TestRequest("hello world"), TransportRequestOptions.EMPTY,
|
||||
transportResponseHandler);
|
||||
receivedLatch.await();
|
||||
assertBusy(() -> { // netty for instance invokes this concurrently so we better use assert busy here
|
||||
|
@ -2330,7 +2330,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertEquals(1, transportStats.getRxCount());
|
||||
assertEquals(2, transportStats.getTxCount());
|
||||
assertEquals(25, transportStats.getRxSize().getBytes());
|
||||
assertEquals(92, transportStats.getTxSize().getBytes());
|
||||
assertEquals(101, transportStats.getTxSize().getBytes());
|
||||
});
|
||||
sendResponseLatch.countDown();
|
||||
responseLatch.await();
|
||||
|
@ -2338,7 +2338,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertEquals(2, stats.getRxCount());
|
||||
assertEquals(2, stats.getTxCount());
|
||||
assertEquals(46, stats.getRxSize().getBytes());
|
||||
assertEquals(92, stats.getTxSize().getBytes());
|
||||
assertEquals(101, stats.getTxSize().getBytes());
|
||||
} finally {
|
||||
serviceC.close();
|
||||
}
|
||||
|
@ -2368,7 +2368,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
CountDownLatch sendResponseLatch = new CountDownLatch(1);
|
||||
Exception ex = new RuntimeException("boom");
|
||||
ex.setStackTrace(new StackTraceElement[0]);
|
||||
serviceB.registerRequestHandler("action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
serviceB.registerRequestHandler("internal:action", TestRequest::new, ThreadPool.Names.SAME,
|
||||
(request, channel, task) -> {
|
||||
// don't block on a network thread here
|
||||
threadPool.generic().execute(new AbstractRunnable() {
|
||||
|
@ -2437,7 +2437,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertEquals(25, transportStats.getRxSize().getBytes());
|
||||
assertEquals(45, transportStats.getTxSize().getBytes());
|
||||
});
|
||||
serviceC.sendRequest(connection, "action", new TestRequest("hello world"), TransportRequestOptions.EMPTY,
|
||||
serviceC.sendRequest(connection, "internal:action", new TestRequest("hello world"), TransportRequestOptions.EMPTY,
|
||||
transportResponseHandler);
|
||||
receivedLatch.await();
|
||||
assertBusy(() -> { // netty for instance invokes this concurrently so we better use assert busy here
|
||||
|
@ -2445,7 +2445,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
assertEquals(1, transportStats.getRxCount());
|
||||
assertEquals(2, transportStats.getTxCount());
|
||||
assertEquals(25, transportStats.getRxSize().getBytes());
|
||||
assertEquals(92, transportStats.getTxSize().getBytes());
|
||||
assertEquals(101, transportStats.getTxSize().getBytes());
|
||||
});
|
||||
sendResponseLatch.countDown();
|
||||
responseLatch.await();
|
||||
|
@ -2460,7 +2460,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
// 49 bytes are the non-exception message bytes that have been received. It should include the initial
|
||||
// handshake message and the header, version, etc bytes in the exception message.
|
||||
assertEquals(failedMessage, 49 + streamOutput.bytes().length(), stats.getRxSize().getBytes());
|
||||
assertEquals(92, stats.getTxSize().getBytes());
|
||||
assertEquals(101, stats.getTxSize().getBytes());
|
||||
} finally {
|
||||
serviceC.close();
|
||||
}
|
||||
|
@ -2638,10 +2638,9 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
|
|||
.toSet()));
|
||||
}
|
||||
|
||||
public void testChannelCloseWhileConnecting() throws IOException {
|
||||
public void testChannelCloseWhileConnecting() {
|
||||
try (MockTransportService service = build(Settings.builder().put("name", "close").build(), version0, null, true)) {
|
||||
service.setExecutorName(ThreadPool.Names.SAME); // make sure stuff is executed in a blocking fashion
|
||||
service.addConnectionListener(new TransportConnectionListener() {
|
||||
service.transport.addConnectionListener(new TransportConnectionListener() {
|
||||
@Override
|
||||
public void onConnectionOpened(final Transport.Connection connection) {
|
||||
try {
|
||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.license.XPackInfoResponse;
|
|||
import org.elasticsearch.license.XPackInfoResponse.FeatureSetsInfo.FeatureSet;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.XPackFeatureSet;
|
||||
|
||||
|
@ -52,8 +53,8 @@ public class TransportXPackInfoActionTests extends ESTestCase {
|
|||
featureSets.add(fs);
|
||||
}
|
||||
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportXPackInfoAction action = new TransportXPackInfoAction(Settings.EMPTY, transportService,
|
||||
mock(ActionFilters.class), licenseService, featureSets);
|
||||
|
||||
|
|
|
@ -5,8 +5,6 @@
|
|||
*/
|
||||
package org.elasticsearch.transport;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.test.SecurityIntegTestCase;
|
||||
import org.elasticsearch.xpack.core.XPackSettings;
|
||||
|
@ -24,15 +22,13 @@ public class SecurityServerTransportServiceTests extends SecurityIntegTestCase {
|
|||
|
||||
public void testSecurityServerTransportServiceWrapsAllHandlers() {
|
||||
for (TransportService transportService : internalCluster().getInstances(TransportService.class)) {
|
||||
for (Map.Entry<String, RequestHandlerRegistry> entry : transportService.requestHandlers.entrySet()) {
|
||||
RequestHandlerRegistry handler = entry.getValue();
|
||||
assertEquals(
|
||||
"handler not wrapped by " + SecurityServerTransportInterceptor.ProfileSecuredRequestHandler.class +
|
||||
"; do all the handler registration methods have overrides?",
|
||||
handler.toString(),
|
||||
"ProfileSecuredRequestHandler{action='" + handler.getAction() + "', executorName='" + handler.getExecutor()
|
||||
+ "', forceExecution=" + handler.isForceExecution() + "}");
|
||||
}
|
||||
RequestHandlerRegistry handler = transportService.transport.getRequestHandler(TransportService.HANDSHAKE_ACTION_NAME);
|
||||
assertEquals(
|
||||
"handler not wrapped by " + SecurityServerTransportInterceptor.ProfileSecuredRequestHandler.class +
|
||||
"; do all the handler registration methods have overrides?",
|
||||
handler.toString(),
|
||||
"ProfileSecuredRequestHandler{action='" + handler.getAction() + "', executorName='" + handler.getExecutor()
|
||||
+ "', forceExecution=" + handler.isForceExecution() + "}");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.role.DeleteRoleRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.role.DeleteRoleResponse;
|
||||
|
@ -42,8 +43,8 @@ public class TransportDeleteRoleActionTests extends ESTestCase {
|
|||
public void testReservedRole() {
|
||||
final String roleName = randomFrom(new ArrayList<>(ReservedRolesStore.names()));
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
(x) -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, (x) -> null, null, Collections.emptySet());
|
||||
TransportDeleteRoleAction action = new TransportDeleteRoleAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService);
|
||||
|
||||
|
@ -73,8 +74,8 @@ public class TransportDeleteRoleActionTests extends ESTestCase {
|
|||
public void testValidRole() {
|
||||
final String roleName = randomFrom("admin", "dept_a", "restricted");
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
(x) -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, (x) -> null, null, Collections.emptySet());
|
||||
TransportDeleteRoleAction action = new TransportDeleteRoleAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService);
|
||||
|
||||
|
@ -117,8 +118,8 @@ public class TransportDeleteRoleActionTests extends ESTestCase {
|
|||
final Exception e = randomFrom(new ElasticsearchSecurityException(""), new IllegalStateException());
|
||||
final String roleName = randomFrom("admin", "dept_a", "restricted");
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
(x) -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, (x) -> null, null, Collections.emptySet());
|
||||
TransportDeleteRoleAction action = new TransportDeleteRoleAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService);
|
||||
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.role.GetRolesRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.role.GetRolesResponse;
|
||||
|
@ -42,8 +43,8 @@ public class TransportGetRolesActionTests extends ESTestCase {
|
|||
|
||||
public void testReservedRoles() {
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportGetRolesAction action = new TransportGetRolesAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService, new ReservedRolesStore());
|
||||
|
||||
|
@ -88,8 +89,8 @@ public class TransportGetRolesActionTests extends ESTestCase {
|
|||
public void testStoreRoles() {
|
||||
final List<RoleDescriptor> storeRoleDescriptors = randomRoleDescriptors();
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportGetRolesAction action = new TransportGetRolesAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService, new ReservedRolesStore());
|
||||
|
||||
|
@ -140,8 +141,8 @@ public class TransportGetRolesActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportGetRolesAction action = new TransportGetRolesAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService, new ReservedRolesStore());
|
||||
|
||||
|
@ -204,8 +205,8 @@ public class TransportGetRolesActionTests extends ESTestCase {
|
|||
final Exception e = randomFrom(new ElasticsearchSecurityException(""), new IllegalStateException());
|
||||
final List<RoleDescriptor> storeRoleDescriptors = randomRoleDescriptors();
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportGetRolesAction action = new TransportGetRolesAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
rolesStore, transportService, new ReservedRolesStore());
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.role.PutRoleRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.role.PutRoleResponse;
|
||||
|
@ -43,8 +44,8 @@ public class TransportPutRoleActionTests extends ESTestCase {
|
|||
public void testReservedRole() {
|
||||
final String roleName = randomFrom(new ArrayList<>(ReservedRolesStore.names()));
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutRoleAction action = new TransportPutRoleAction(Settings.EMPTY, mock(ActionFilters.class), rolesStore, transportService);
|
||||
|
||||
PutRoleRequest request = new PutRoleRequest();
|
||||
|
@ -73,8 +74,8 @@ public class TransportPutRoleActionTests extends ESTestCase {
|
|||
public void testValidRole() {
|
||||
final String roleName = randomFrom("admin", "dept_a", "restricted");
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutRoleAction action = new TransportPutRoleAction(Settings.EMPTY, mock(ActionFilters.class), rolesStore, transportService);
|
||||
|
||||
final boolean created = randomBoolean();
|
||||
|
@ -116,8 +117,8 @@ public class TransportPutRoleActionTests extends ESTestCase {
|
|||
final Exception e = randomFrom(new ElasticsearchSecurityException(""), new IllegalStateException());
|
||||
final String roleName = randomFrom("admin", "dept_a", "restricted");
|
||||
NativeRolesStore rolesStore = mock(NativeRolesStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutRoleAction action = new TransportPutRoleAction(Settings.EMPTY, mock(ActionFilters.class), rolesStore, transportService);
|
||||
|
||||
PutRoleRequest request = new PutRoleRequest();
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.common.Strings;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.rolemapping.GetRoleMappingsRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.rolemapping.GetRoleMappingsResponse;
|
||||
|
@ -44,7 +45,7 @@ public class TransportGetRoleMappingsActionTests extends ESTestCase {
|
|||
@Before
|
||||
public void setupMocks() {
|
||||
store = mock(NativeRoleMappingStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
action = new TransportGetRoleMappingsAction(Settings.EMPTY, mock(ActionFilters.class), transportService, store);
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.action.support.PlainActionFuture;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.rolemapping.PutRoleMappingRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.rolemapping.PutRoleMappingResponse;
|
||||
|
@ -40,7 +41,7 @@ public class TransportPutRoleMappingActionTests extends ESTestCase {
|
|||
@Before
|
||||
public void setupMocks() {
|
||||
store = mock(NativeRoleMappingStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
action = new TransportPutRoleMappingAction(Settings.EMPTY, mock(ActionFilters.class), transportService, store);
|
||||
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.elasticsearch.tasks.Task;
|
|||
import org.elasticsearch.test.ClusterServiceUtils;
|
||||
import org.elasticsearch.test.client.NoOpClient;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.XPackSettings;
|
||||
import org.elasticsearch.xpack.core.security.action.saml.SamlInvalidateSessionRequest;
|
||||
|
@ -166,7 +167,7 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase {
|
|||
final ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
|
||||
tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, clusterService);
|
||||
|
||||
final TransportService transportService = new TransportService(Settings.EMPTY, null, null,
|
||||
final TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
final Realms realms = mock(Realms.class);
|
||||
action = new TransportSamlInvalidateSessionAction(settings, transportService, mock(ActionFilters.class),tokenService, realms);
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.env.TestEnvironment;
|
|||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ClusterServiceUtils;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.watcher.ResourceWatcherService;
|
||||
import org.elasticsearch.xpack.core.XPackSettings;
|
||||
|
@ -181,7 +182,7 @@ public class TransportSamlLogoutActionTests extends SamlTestCase {
|
|||
final ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool);
|
||||
tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, clusterService);
|
||||
|
||||
final TransportService transportService = new TransportService(Settings.EMPTY, null, null,
|
||||
final TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
final Realms realms = mock(Realms.class);
|
||||
action = new TransportSamlLogoutAction(settings, transportService, mock(ActionFilters.class), realms, tokenService);
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.SecurityContext;
|
||||
import org.elasticsearch.xpack.core.security.action.user.AuthenticateRequest;
|
||||
|
@ -37,8 +38,8 @@ public class TransportAuthenticateActionTests extends ESTestCase {
|
|||
public void testInternalUser() {
|
||||
SecurityContext securityContext = mock(SecurityContext.class);
|
||||
when(securityContext.getUser()).thenReturn(randomFrom(SystemUser.INSTANCE, XPackUser.INSTANCE));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportAuthenticateAction action = new TransportAuthenticateAction(Settings.EMPTY, transportService,
|
||||
mock(ActionFilters.class), securityContext);
|
||||
|
||||
|
@ -63,8 +64,8 @@ public class TransportAuthenticateActionTests extends ESTestCase {
|
|||
|
||||
public void testNullUser() {
|
||||
SecurityContext securityContext = mock(SecurityContext.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportAuthenticateAction action = new TransportAuthenticateAction(Settings.EMPTY, transportService,
|
||||
mock(ActionFilters.class), securityContext);
|
||||
|
||||
|
@ -91,8 +92,8 @@ public class TransportAuthenticateActionTests extends ESTestCase {
|
|||
final User user = randomFrom(new ElasticUser(true), new KibanaUser(true), new User("joe"));
|
||||
SecurityContext securityContext = mock(SecurityContext.class);
|
||||
when(securityContext.getUser()).thenReturn(user);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportAuthenticateAction action = new TransportAuthenticateAction(Settings.EMPTY, transportService,
|
||||
mock(ActionFilters.class), securityContext);
|
||||
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.SecuritySettingsSourceField;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.XPackSettings;
|
||||
import org.elasticsearch.xpack.core.security.action.user.ChangePasswordRequest;
|
||||
|
@ -53,7 +54,7 @@ public class TransportChangePasswordActionTests extends ESTestCase {
|
|||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
Settings passwordHashingSettings = Settings.builder().
|
||||
put(XPackSettings.PASSWORD_HASHING_ALGORITHM.getKey(), hashingAlgorithm).build();
|
||||
TransportService transportService = new TransportService(passwordHashingSettings, null, null,
|
||||
TransportService transportService = new TransportService(passwordHashingSettings, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportChangePasswordAction action = new TransportChangePasswordAction(settings, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
@ -88,7 +89,7 @@ public class TransportChangePasswordActionTests extends ESTestCase {
|
|||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
Settings passwordHashingSettings = Settings.builder().
|
||||
put(XPackSettings.PASSWORD_HASHING_ALGORITHM.getKey(), hashingAlgorithm).build();
|
||||
TransportService transportService = new TransportService(passwordHashingSettings, null, null,
|
||||
TransportService transportService = new TransportService(passwordHashingSettings, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportChangePasswordAction action = new TransportChangePasswordAction(Settings.EMPTY, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
@ -133,8 +134,8 @@ public class TransportChangePasswordActionTests extends ESTestCase {
|
|||
listener.onResponse(null);
|
||||
return null;
|
||||
}).when(usersStore).changePassword(eq(request), any(ActionListener.class));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
Settings passwordHashingSettings = Settings.builder().
|
||||
put(XPackSettings.PASSWORD_HASHING_ALGORITHM.getKey(), hashingAlgorithm).build();
|
||||
TransportChangePasswordAction action = new TransportChangePasswordAction(passwordHashingSettings, transportService,
|
||||
|
@ -168,8 +169,8 @@ public class TransportChangePasswordActionTests extends ESTestCase {
|
|||
request.passwordHash(hasher.hash(SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING));
|
||||
final AtomicReference<Throwable> throwableRef = new AtomicReference<>();
|
||||
final AtomicReference<ChangePasswordResponse> responseRef = new AtomicReference<>();
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
Settings passwordHashingSettings = Settings.builder().put(XPackSettings.PASSWORD_HASHING_ALGORITHM.getKey(),
|
||||
randomFrom("pbkdf2_50000", "pbkdf2_100000", "bcrypt11", "bcrypt8", "bcrypt")).build();
|
||||
TransportChangePasswordAction action = new TransportChangePasswordAction(passwordHashingSettings, transportService,
|
||||
|
@ -209,13 +210,12 @@ public class TransportChangePasswordActionTests extends ESTestCase {
|
|||
return null;
|
||||
}
|
||||
}).when(usersStore).changePassword(eq(request), any(ActionListener.class));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
Settings passwordHashingSettings = Settings.builder().
|
||||
put(XPackSettings.PASSWORD_HASHING_ALGORITHM.getKey(), hashingAlgorithm).build();
|
||||
TransportChangePasswordAction action = new TransportChangePasswordAction(passwordHashingSettings, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
||||
final AtomicReference<Throwable> throwableRef = new AtomicReference<>();
|
||||
final AtomicReference<ChangePasswordResponse> responseRef = new AtomicReference<>();
|
||||
action.doExecute(mock(Task.class), request, new ActionListener<ChangePasswordResponse>() {
|
||||
|
|
|
@ -11,6 +11,7 @@ import org.elasticsearch.action.support.ActionFilters;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.user.DeleteUserRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.user.DeleteUserResponse;
|
||||
|
@ -46,8 +47,8 @@ public class TransportDeleteUserActionTests extends ESTestCase {
|
|||
public void testAnonymousUser() {
|
||||
Settings settings = Settings.builder().put(AnonymousUser.ROLES_SETTING.getKey(), "superuser").build();
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportDeleteUserAction action = new TransportDeleteUserAction(settings, mock(ActionFilters.class), usersStore, transportService);
|
||||
|
||||
DeleteUserRequest request = new DeleteUserRequest(new AnonymousUser(settings).principal());
|
||||
|
@ -74,8 +75,8 @@ public class TransportDeleteUserActionTests extends ESTestCase {
|
|||
|
||||
public void testInternalUser() {
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportDeleteUserAction action = new TransportDeleteUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
@ -104,8 +105,8 @@ public class TransportDeleteUserActionTests extends ESTestCase {
|
|||
public void testReservedUser() {
|
||||
final User reserved = randomFrom(new ElasticUser(true), new KibanaUser(true));
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportDeleteUserAction action = new TransportDeleteUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
@ -134,8 +135,8 @@ public class TransportDeleteUserActionTests extends ESTestCase {
|
|||
public void testValidUser() {
|
||||
final User user = new User("joe");
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportDeleteUserAction action = new TransportDeleteUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
@ -175,8 +176,8 @@ public class TransportDeleteUserActionTests extends ESTestCase {
|
|||
final Exception e = randomFrom(new ElasticsearchSecurityException(""), new IllegalStateException(), new RuntimeException());
|
||||
final User user = new User("joe");
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportDeleteUserAction action = new TransportDeleteUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@ import org.elasticsearch.tasks.Task;
|
|||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.user.GetUsersRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.user.GetUsersResponse;
|
||||
|
@ -90,8 +91,8 @@ public class TransportGetUsersActionTests extends ESTestCase {
|
|||
AnonymousUser anonymousUser = new AnonymousUser(settings);
|
||||
ReservedRealm reservedRealm =
|
||||
new ReservedRealm(mock(Environment.class), settings, usersStore, anonymousUser, securityIndex, threadPool);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService, reservedRealm);
|
||||
|
||||
|
@ -125,8 +126,8 @@ public class TransportGetUsersActionTests extends ESTestCase {
|
|||
|
||||
public void testInternalUser() {
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService, mock(ReservedRealm.class));
|
||||
|
||||
|
@ -168,7 +169,7 @@ public class TransportGetUsersActionTests extends ESTestCase {
|
|||
final int size = randomIntBetween(1, allReservedUsers.size());
|
||||
final List<User> reservedUsers = randomSubsetOf(size, allReservedUsers);
|
||||
final List<String> names = reservedUsers.stream().map(User::principal).collect(Collectors.toList());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService, reservedRealm);
|
||||
|
@ -208,7 +209,7 @@ public class TransportGetUsersActionTests extends ESTestCase {
|
|||
ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap());
|
||||
ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(settings),
|
||||
securityIndex, threadPool);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService, reservedRealm);
|
||||
|
@ -255,7 +256,7 @@ public class TransportGetUsersActionTests extends ESTestCase {
|
|||
randomFrom(Collections.singletonList(new User("joe")), Arrays.asList(new User("jane"), new User("fred")), randomUsers());
|
||||
final String[] storeUsernames = storeUsers.stream().map(User::principal).collect(Collectors.toList()).toArray(Strings.EMPTY_ARRAY);
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService, mock(ReservedRealm.class));
|
||||
|
@ -303,7 +304,7 @@ public class TransportGetUsersActionTests extends ESTestCase {
|
|||
randomFrom(Collections.singletonList(new User("joe")), Arrays.asList(new User("jane"), new User("fred")), randomUsers());
|
||||
final String[] storeUsernames = storeUsers.stream().map(User::principal).collect(Collectors.toList()).toArray(Strings.EMPTY_ARRAY);
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService, mock(ReservedRealm.class));
|
||||
|
|
|
@ -19,6 +19,7 @@ import org.elasticsearch.mock.orig.Mockito;
|
|||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse;
|
||||
|
@ -58,9 +59,8 @@ public class TransportHasPrivilegesActionTests extends ESTestCase {
|
|||
user = new User(randomAlphaOfLengthBetween(4, 12));
|
||||
final ThreadPool threadPool = mock(ThreadPool.class);
|
||||
final ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
|
||||
final TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService
|
||||
.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
final TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
|
||||
final Authentication authentication = mock(Authentication.class);
|
||||
threadContext.putTransient(AuthenticationField.AUTHENTICATION_KEY, authentication);
|
||||
|
|
|
@ -17,6 +17,7 @@ import org.elasticsearch.tasks.Task;
|
|||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.SecuritySettingsSourceField;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.user.PutUserRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.user.PutUserResponse;
|
||||
|
@ -57,8 +58,8 @@ public class TransportPutUserActionTests extends ESTestCase {
|
|||
Settings settings = Settings.builder().put(AnonymousUser.ROLES_SETTING.getKey(), "superuser").build();
|
||||
final AnonymousUser anonymousUser = new AnonymousUser(settings);
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutUserAction action = new TransportPutUserAction(settings, mock(ActionFilters.class), usersStore, transportService);
|
||||
|
||||
PutUserRequest request = new PutUserRequest();
|
||||
|
@ -86,8 +87,8 @@ public class TransportPutUserActionTests extends ESTestCase {
|
|||
|
||||
public void testSystemUser() {
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutUserAction action = new TransportPutUserAction(Settings.EMPTY, mock(ActionFilters.class), usersStore, transportService);
|
||||
|
||||
PutUserRequest request = new PutUserRequest();
|
||||
|
@ -126,8 +127,8 @@ public class TransportPutUserActionTests extends ESTestCase {
|
|||
PlainActionFuture<Collection<User>> userFuture = new PlainActionFuture<>();
|
||||
reservedRealm.users(userFuture);
|
||||
final User reserved = randomFrom(userFuture.actionGet().toArray(new User[0]));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutUserAction action = new TransportPutUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
@ -156,8 +157,8 @@ public class TransportPutUserActionTests extends ESTestCase {
|
|||
public void testValidUser() {
|
||||
final User user = new User("joe");
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutUserAction action = new TransportPutUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
@ -203,8 +204,8 @@ public class TransportPutUserActionTests extends ESTestCase {
|
|||
final Exception e = randomFrom(new ElasticsearchSecurityException(""), new IllegalStateException(), new ValidationException());
|
||||
final User user = new User("joe");
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportPutUserAction action = new TransportPutUserAction(Settings.EMPTY, mock(ActionFilters.class),
|
||||
usersStore, transportService);
|
||||
|
||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext;
|
|||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.security.action.user.SetEnabledRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.user.SetEnabledResponse;
|
||||
|
@ -62,8 +63,8 @@ public class TransportSetEnabledActionTests extends ESTestCase {
|
|||
threadContext.putTransient(AuthenticationField.AUTHENTICATION_KEY, authentication);
|
||||
when(authentication.getUser()).thenReturn(user);
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportSetEnabledAction action = new TransportSetEnabledAction(settings, threadPool, transportService, mock(ActionFilters.class),
|
||||
usersStore);
|
||||
|
||||
|
@ -100,8 +101,8 @@ public class TransportSetEnabledActionTests extends ESTestCase {
|
|||
threadContext.putTransient(AuthenticationField.AUTHENTICATION_KEY, authentication);
|
||||
when(authentication.getUser()).thenReturn(user);
|
||||
NativeUsersStore usersStore = mock(NativeUsersStore.class);
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportSetEnabledAction action = new TransportSetEnabledAction(Settings.EMPTY, threadPool, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
||||
|
@ -154,8 +155,8 @@ public class TransportSetEnabledActionTests extends ESTestCase {
|
|||
}
|
||||
}).when(usersStore)
|
||||
.setEnabled(eq(user.principal()), eq(request.enabled()), eq(request.getRefreshPolicy()), any(ActionListener.class));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportSetEnabledAction action = new TransportSetEnabledAction(Settings.EMPTY, threadPool, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
||||
|
@ -206,8 +207,8 @@ public class TransportSetEnabledActionTests extends ESTestCase {
|
|||
}
|
||||
}).when(usersStore)
|
||||
.setEnabled(eq(user.principal()), eq(request.enabled()), eq(request.getRefreshPolicy()), any(ActionListener.class));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportSetEnabledAction action = new TransportSetEnabledAction(Settings.EMPTY, threadPool, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
||||
|
@ -246,8 +247,8 @@ public class TransportSetEnabledActionTests extends ESTestCase {
|
|||
request.username(user.principal());
|
||||
request.enabled(randomBoolean());
|
||||
request.setRefreshPolicy(randomFrom(RefreshPolicy.values()));
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR,
|
||||
x -> null, null, Collections.emptySet());
|
||||
TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null,
|
||||
TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet());
|
||||
TransportSetEnabledAction action = new TransportSetEnabledAction(Settings.EMPTY, threadPool, transportService,
|
||||
mock(ActionFilters.class), usersStore);
|
||||
|
||||
|
|
Loading…
Reference in New Issue