diff --git a/client/rest/src/main/java/org/elasticsearch/client/AbstractRestClientActions.java b/client/rest/src/main/java/org/elasticsearch/client/AbstractRestClientActions.java new file mode 100644 index 0000000000000..4039194c68694 --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/AbstractRestClientActions.java @@ -0,0 +1,219 @@ +/* + * 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.client; + +import java.io.IOException; +import java.net.ConnectException; +import java.net.SocketTimeoutException; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import javax.net.ssl.SSLHandshakeException; + +import org.apache.http.ConnectionClosedException; +import org.apache.http.Header; +import org.apache.http.HttpEntity; +import org.apache.http.conn.ConnectTimeoutException; + +/** + * Abstract implementation of {@link RestClientActions} shared by + * {@link RestClient} and {@link RestClientView}. + */ +abstract class AbstractRestClientActions implements RestClientActions { + /** + * Build a {@link SyncResponseListener} to convert requests from + * asynchronous to synchronous. + */ + abstract SyncResponseListener syncResponseListener(); + + /** + * Perform the actual request asynchronously, letting any the caller + * handle all exceptions. + */ + abstract void performRequestAsyncNoCatch(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + ResponseListener responseListener, Header[] headers) throws IOException; + + @Override + public final Response performRequest(String method, String endpoint, Header... headers) throws IOException { + return performRequest(method, endpoint, Collections.emptyMap(), null, headers); + } + + @Override + public final Response performRequest(String method, String endpoint, Map params, Header... headers) throws IOException { + return performRequest(method, endpoint, params, (HttpEntity)null, headers); + } + + @Override // TODO this method is not final so the tests for the High Level REST Client don't have to change. We'll fix this soon. + public Response performRequest(String method, String endpoint, Map params, + HttpEntity entity, Header... headers) throws IOException { + return performRequest(method, endpoint, params, entity, HttpAsyncResponseConsumerFactory.DEFAULT, headers); + } + + @Override + public final Response performRequest(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + Header... headers) throws IOException { + SyncResponseListener listener = syncResponseListener(); + performRequestAsyncNoCatch(method, endpoint, params, entity, httpAsyncResponseConsumerFactory, + listener, headers); + return listener.get(); + } + + public final void performRequestAsync(String method, String endpoint, ResponseListener responseListener, Header... headers) { + performRequestAsync(method, endpoint, Collections.emptyMap(), null, responseListener, headers); + } + + @Override + public final void performRequestAsync(String method, String endpoint, Map params, + ResponseListener responseListener, Header... headers) { + performRequestAsync(method, endpoint, params, null, responseListener, headers); + } + + @Override // TODO this method is not final so the tests for the High Level REST Client don't have to change. We'll fix this soon. + public void performRequestAsync(String method, String endpoint, Map params, + HttpEntity entity, ResponseListener responseListener, Header... headers) { + performRequestAsync(method, endpoint, params, entity, HttpAsyncResponseConsumerFactory.DEFAULT, responseListener, headers); + } + + @Override + public final void performRequestAsync(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + ResponseListener responseListener, Header... headers) { + try { + performRequestAsyncNoCatch(method, endpoint, params, entity, httpAsyncResponseConsumerFactory, + responseListener, headers); + } catch (Exception e) { + responseListener.onFailure(e); + } + } + + /** + * Listener used in any sync performRequest calls, it waits for a response or an exception back up to a timeout + */ + static class SyncResponseListener implements ResponseListener { + private final CountDownLatch latch = new CountDownLatch(1); + private final AtomicReference response = new AtomicReference<>(); + private final AtomicReference exception = new AtomicReference<>(); + + private final long timeout; + + SyncResponseListener(long timeout) { + assert timeout > 0; + this.timeout = timeout; + } + + @Override + public void onSuccess(Response response) { + Objects.requireNonNull(response, "response must not be null"); + boolean wasResponseNull = this.response.compareAndSet(null, response); + if (wasResponseNull == false) { + throw new IllegalStateException("response is already set"); + } + + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + Objects.requireNonNull(exception, "exception must not be null"); + boolean wasExceptionNull = this.exception.compareAndSet(null, exception); + if (wasExceptionNull == false) { + throw new IllegalStateException("exception is already set"); + } + latch.countDown(); + } + + /** + * Waits (up to a timeout) for some result of the request: either a response, or an exception. + */ + Response get() throws IOException { + try { + //providing timeout is just a safety measure to prevent everlasting waits + //the different client timeouts should already do their jobs + if (latch.await(timeout, TimeUnit.MILLISECONDS) == false) { + throw new IOException("listener timeout after waiting for [" + timeout + "] ms"); + } + } catch (InterruptedException e) { + throw new RuntimeException("thread waiting for the response was interrupted", e); + } + + Exception exception = this.exception.get(); + Response response = this.response.get(); + if (exception != null) { + if (response != null) { + IllegalStateException e = new IllegalStateException("response and exception are unexpectedly set at the same time"); + e.addSuppressed(exception); + throw e; + } + /* + * Wrap and rethrow whatever exception we received, copying the type + * where possible so the synchronous API looks as much as possible + * like the asynchronous API. We wrap the exception so that the caller's + * signature shows up in any exception we throw. + */ + if (exception instanceof ResponseException) { + throw new ResponseException((ResponseException) exception); + } + if (exception instanceof ConnectException) { + ConnectException e = new ConnectException(exception.getMessage()); + e.initCause(exception); + throw e; + } + if (exception instanceof ConnectTimeoutException) { + ConnectTimeoutException e = new ConnectTimeoutException(exception.getMessage()); + e.initCause(exception); + throw e; + } + if (exception instanceof SocketTimeoutException) { + SocketTimeoutException e = new SocketTimeoutException(exception.getMessage()); + e.initCause(exception); + throw e; + } + if (exception instanceof ConnectionClosedException) { + ConnectionClosedException e = new ConnectionClosedException(exception.getMessage()); + e.initCause(exception); + throw e; + } + if (exception instanceof SSLHandshakeException) { + SSLHandshakeException e = new SSLHandshakeException(exception.getMessage()); + e.initCause(exception); + throw e; + } + if (exception instanceof IOException) { + throw new IOException(exception.getMessage(), exception); + } + if (exception instanceof RuntimeException){ + throw new RuntimeException(exception.getMessage(), exception); + } + throw new RuntimeException("error while performing request", exception); + } + + if (response == null) { + throw new IllegalStateException("response not set and no exception caught either"); + } + return response; + } + } +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java b/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java index a7b222da70e1d..1412fb589e620 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java +++ b/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java @@ -53,6 +53,11 @@ private DeadHostState() { this.failedAttempts = previousDeadHostState.failedAttempts + 1; } + DeadHostState(int failedAttempts, long deadUntilNanos) { + this.failedAttempts = failedAttempts; + this.deadUntilNanos = deadUntilNanos; + } + /** * Returns the timestamp (nanos) till the host is supposed to stay dead without being retried. * After that the host should be retried. diff --git a/client/rest/src/main/java/org/elasticsearch/client/Node.java b/client/rest/src/main/java/org/elasticsearch/client/Node.java new file mode 100644 index 0000000000000..b26a0fa603c99 --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/Node.java @@ -0,0 +1,235 @@ +/* + * 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.client; + +import static java.util.Collections.unmodifiableSet; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; + +import org.apache.http.HttpHost; + +/** + * Metadata about an {@link HttpHost} running Elasticsearch. + */ +public class Node { + /** + * Address that this host claims is its primary contact point. + */ + private final HttpHost host; + /** + * Addresses on which the host is listening. These are useful to have + * around because they allow you to find a host based on any address it + * is listening on. + */ + private final Set boundHosts; + /** + * Name of the node as configured by the {@code node.name} attribute. + */ + private final String name; + /** + * Version of Elasticsearch that the node is running or {@code null} + * if we don't know the version. + */ + private final String version; + /** + * Roles that the Elasticsearch process on the host has or {@code null} + * if we don't know what roles the node has. + */ + private final Roles roles; + + /** + * Create a {@linkplain Node} with metadata. All parameters except + * {@code host} are nullable and implementations of {@link NodeSelector} + * need to decide what to do in their absence. + */ + public Node(HttpHost host, Set boundHosts, String name, String version, Roles roles) { + if (host == null) { + throw new IllegalArgumentException("host cannot be null"); + } + this.host = host; + this.boundHosts = boundHosts; + this.name = name; + this.version = version; + this.roles = roles; + } + + /** + * Create a {@linkplain Node} without any metadata. + */ + public Node(HttpHost host) { + this(host, null, null, null, null); + } + + /** + * Make a copy of this {@link Node} but replacing its + * {@link #getHost() host}. Use this when the sniffing implementation + * returns a {@link #getHost() host} that is not useful to the client. + */ + public Node withHost(HttpHost host) { + /* + * If the new host isn't in the bound hosts list we add it so the + * result looks sane. + */ + Set boundHosts = this.boundHosts; + if (false == boundHosts.contains(host)) { + boundHosts = new HashSet<>(boundHosts); + boundHosts.add(host); + boundHosts = unmodifiableSet(boundHosts); + } + return new Node(host, boundHosts, name, version, roles); + } + + /** + * Contact information for the host. + */ + public HttpHost getHost() { + return host; + } + + /** + * Addresses on which the host is listening. These are useful to have + * around because they allow you to find a host based on any address it + * is listening on. + */ + public Set getBoundHosts() { + return boundHosts; + } + + /** + * @return the name + */ + public String getName() { + return name; + } + + /** + * Version of Elasticsearch that the node is running or {@code null} + * if we don't know the version. + */ + public String getVersion() { + return version; + } + + /** + * Roles that the Elasticsearch process on the host has or {@code null} + * if we don't know what roles the node has. + */ + public Roles getRoles() { + return roles; + } + + @Override + public String toString() { + StringBuilder b = new StringBuilder(); + b.append("[host=").append(host); + if (boundHosts != null) { + b.append(", bound=").append(boundHosts); + } + if (name != null) { + b.append(", name=").append(name); + } + if (version != null) { + b.append(", version=").append(version); + } + if (roles != null) { + b.append(", roles=").append(roles); + } + return b.append(']').toString(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || obj.getClass() != getClass()) { + return false; + } + Node other = (Node) obj; + return host.equals(other.host) + && Objects.equals(boundHosts, other.boundHosts) + && Objects.equals(version, other.version) + && Objects.equals(name, other.name) + && Objects.equals(roles, other.roles); + } + + @Override + public int hashCode() { + return Objects.hash(host, boundHosts, name, version, roles); + } + + /** + * Role information about an Elasticsearch process. + */ + public static final class Roles { + private final boolean masterEligible; + private final boolean data; + private final boolean ingest; + + public Roles(boolean masterEligible, boolean data, boolean ingest) { + this.masterEligible = masterEligible; + this.data = data; + this.ingest = ingest; + } + + /** + * The node could be elected master. + */ + public boolean isMasterEligible() { + return masterEligible; + } + /** + * The node stores data. + */ + public boolean isData() { + return data; + } + /** + * The node runs ingest pipelines. + */ + public boolean isIngest() { + return ingest; + } + + @Override + public String toString() { + StringBuilder result = new StringBuilder(3); + if (masterEligible) result.append('m'); + if (data) result.append('d'); + if (ingest) result.append('i'); + return result.toString(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || obj.getClass() != getClass()) { + return false; + } + Roles other = (Roles) obj; + return masterEligible == other.masterEligible + && data == other.data + && ingest == other.ingest; + } + + @Override + public int hashCode() { + return Objects.hash(masterEligible, data, ingest); + } + } +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/NodeSelector.java b/client/rest/src/main/java/org/elasticsearch/client/NodeSelector.java new file mode 100644 index 0000000000000..c95fd43395f86 --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/NodeSelector.java @@ -0,0 +1,115 @@ +/* + * 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.client; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * Selects nodes that can receive requests. Use with + * {@link RestClientActions#withNodeSelector withNodeSelector}. + */ +public interface NodeSelector { + /** + * Select the {@link Node}s to which to send requests. This may be called + * twice per request, once for "living" nodes that have not had been + * blacklisted by previous errors if there are any. If it returns an + * empty list when sent the living nodes or if there aren't any living + * nodes left then this will be called with a list of "dead" nodes that + * have been blacklisted by previous failures. In both cases it should + * return a list of nodes sorted by its preference for which node is used. + * If it is operating on "living" nodes that it returns function as + * fallbacks in case of request failures. If it is operating on dead nodes + * then the dead node that it returns is attempted but no others. + * + * @param nodes an unmodifiable list of {@linkplain Node}s in the order + * that the {@link RestClient} would prefer to use them + * @return a subset of the provided list of {@linkplain Node}s that the + * selector approves of, in the order that the selector would prefer + * to use them. + */ + List select(List nodes); + + /** + * Selector that matches any node. + */ + NodeSelector ANY = new NodeSelector() { + @Override + public List select(List nodes) { + return nodes; + } + + @Override + public String toString() { + return "ANY"; + } + }; + + /** + * Selector that matches any node that has metadata and doesn't + * have the {@code master} role OR it has the data {@code data} + * role. It does not reorder the nodes sent to it. + */ + NodeSelector NOT_MASTER_ONLY = new NodeSelector() { + @Override + public List select(List nodes) { + List subset = new ArrayList<>(nodes.size()); + for (Node node : nodes) { + if (node.getRoles() == null) continue; + if (false == node.getRoles().isMasterEligible() || node.getRoles().isData()) { + subset.add(node); + } + } + return subset; + } + + @Override + public String toString() { + return "NOT_MASTER_ONLY"; + } + }; + + /** + * Selector that composes two selectors, running the "right" most selector + * first and then running the "left" selector on the results of the "right" + * selector. + */ + class Compose implements NodeSelector { + private final NodeSelector lhs; + private final NodeSelector rhs; + + public Compose(NodeSelector lhs, NodeSelector rhs) { + this.lhs = Objects.requireNonNull(lhs, "lhs is required"); + this.rhs = Objects.requireNonNull(rhs, "rhs is required"); + } + + @Override + public List select(List nodes) { + return lhs.select(rhs.select(nodes)); + } + + @Override + public String toString() { + // . as in haskell's "compose" operator + return lhs + "." + rhs; + } + } +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/Response.java b/client/rest/src/main/java/org/elasticsearch/client/Response.java index 02aedb4765abe..39bbf769713b2 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/Response.java +++ b/client/rest/src/main/java/org/elasticsearch/client/Response.java @@ -40,7 +40,7 @@ public class Response { Response(RequestLine requestLine, HttpHost host, HttpResponse response) { Objects.requireNonNull(requestLine, "requestLine cannot be null"); - Objects.requireNonNull(host, "node cannot be null"); + Objects.requireNonNull(host, "host cannot be null"); Objects.requireNonNull(response, "response cannot be null"); this.requestLine = requestLine; this.host = host; diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index 4aa1a9d815cf4..0316fef30bf59 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -16,18 +16,17 @@ * specific language governing permissions and limitations * under the License. */ + package org.elasticsearch.client; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.http.ConnectionClosedException; import org.apache.http.Header; import org.apache.http.HttpEntity; import org.apache.http.HttpHost; import org.apache.http.HttpRequest; import org.apache.http.HttpResponse; import org.apache.http.client.AuthCache; -import org.apache.http.client.ClientProtocolException; import org.apache.http.client.methods.HttpEntityEnclosingRequestBase; import org.apache.http.client.methods.HttpHead; import org.apache.http.client.methods.HttpOptions; @@ -39,7 +38,6 @@ import org.apache.http.client.protocol.HttpClientContext; import org.apache.http.client.utils.URIBuilder; import org.apache.http.concurrent.FutureCallback; -import org.apache.http.conn.ConnectTimeoutException; import org.apache.http.impl.auth.BasicScheme; import org.apache.http.impl.client.BasicAuthCache; import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; @@ -47,9 +45,10 @@ import org.apache.http.nio.protocol.HttpAsyncRequestProducer; import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; +import static java.util.Collections.singletonList; + import java.io.Closeable; import java.io.IOException; -import java.net.SocketTimeoutException; import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; @@ -67,12 +66,9 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import javax.net.ssl.SSLHandshakeException; /** * Client that connects to an Elasticsearch cluster through HTTP. @@ -91,8 +87,7 @@ *

* Requests can be traced by enabling trace logging for "tracer". The trace logger outputs requests and responses in curl format. */ -public class RestClient implements Closeable { - +public class RestClient extends AbstractRestClientActions implements Closeable { private static final Log logger = LogFactory.getLog(RestClient.class); private final CloseableHttpAsyncClient client; @@ -101,221 +96,125 @@ public class RestClient implements Closeable { final List

defaultHeaders; private final long maxRetryTimeoutMillis; private final String pathPrefix; - private final AtomicInteger lastHostIndex = new AtomicInteger(0); - private volatile HostTuple> hostTuple; + private final AtomicInteger lastNodeIndex = new AtomicInteger(0); private final ConcurrentMap blacklist = new ConcurrentHashMap<>(); private final FailureListener failureListener; + private volatile NodeTuple> nodeTuple; RestClient(CloseableHttpAsyncClient client, long maxRetryTimeoutMillis, Header[] defaultHeaders, - HttpHost[] hosts, String pathPrefix, FailureListener failureListener) { + Node[] nodes, String pathPrefix, FailureListener failureListener) { this.client = client; this.maxRetryTimeoutMillis = maxRetryTimeoutMillis; this.defaultHeaders = Collections.unmodifiableList(Arrays.asList(defaultHeaders)); this.failureListener = failureListener; this.pathPrefix = pathPrefix; - setHosts(hosts); + setNodes(nodes); } /** * Returns a new {@link RestClientBuilder} to help with {@link RestClient} creation. - * Creates a new builder instance and sets the hosts that the client will send requests to. + * Creates a new builder instance and sets the nodes that the client will send requests to. + * @see Node#Node(HttpHost) */ public static RestClientBuilder builder(HttpHost... hosts) { - return new RestClientBuilder(hosts); + return builder(hostsToNodes(hosts)); } /** - * Replaces the hosts that the client communicates with. - * @see HttpHost + * Returns a new {@link RestClientBuilder} to help with {@link RestClient} creation. + * Creates a new builder instance and sets the nodes that the client will send requests to. + *

+ * Prefer this to {@link #builder(Node...)} if you have metadata up front about the nodes. + * If y ou don't either one is fine. */ - public synchronized void setHosts(HttpHost... hosts) { - if (hosts == null || hosts.length == 0) { - throw new IllegalArgumentException("hosts must not be null nor empty"); - } - Set httpHosts = new HashSet<>(); - AuthCache authCache = new BasicAuthCache(); - for (HttpHost host : hosts) { - Objects.requireNonNull(host, "host cannot be null"); - httpHosts.add(host); - authCache.put(host, new BasicScheme()); - } - this.hostTuple = new HostTuple<>(Collections.unmodifiableSet(httpHosts), authCache); - this.blacklist.clear(); + public static RestClientBuilder builder(Node... nodes) { + return new RestClientBuilder(nodes); } /** - * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response - * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, Header...)} but without parameters - * and request body. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param headers the optional request headers - * @return the response returned by Elasticsearch - * @throws IOException in case of a problem or the connection was aborted - * @throws ClientProtocolException in case of an http protocol error - * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + * Replaces the nodes that the client communicates without providing any + * metadata about any of the nodes. */ - public Response performRequest(String method, String endpoint, Header... headers) throws IOException { - return performRequest(method, endpoint, Collections.emptyMap(), null, headers); + public void setHosts(HttpHost... hosts) { + setNodes(hostsToNodes(hosts)); } - /** - * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response - * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, Header...)} but without request body. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param params the query_string parameters - * @param headers the optional request headers - * @return the response returned by Elasticsearch - * @throws IOException in case of a problem or the connection was aborted - * @throws ClientProtocolException in case of an http protocol error - * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error - */ - public Response performRequest(String method, String endpoint, Map params, Header... headers) throws IOException { - return performRequest(method, endpoint, params, (HttpEntity)null, headers); + private static Node[] hostsToNodes(HttpHost[] hosts) { + if (hosts == null || hosts.length == 0) { + throw new IllegalArgumentException("hosts must not be null or empty"); + } + Node[] nodes = new Node[hosts.length]; + for (int i = 0; i < hosts.length; i++) { + nodes[i] = new Node(hosts[i]); + } + return nodes; } /** - * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response - * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, HttpAsyncResponseConsumerFactory, Header...)} - * which doesn't require specifying an {@link HttpAsyncResponseConsumerFactory} instance, - * {@link HttpAsyncResponseConsumerFactory} will be used to create the needed instances of {@link HttpAsyncResponseConsumer}. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param params the query_string parameters - * @param entity the body of the request, null if not applicable - * @param headers the optional request headers - * @return the response returned by Elasticsearch - * @throws IOException in case of a problem or the connection was aborted - * @throws ClientProtocolException in case of an http protocol error - * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + * Replaces the nodes that the client communicates with. Prefer this to + * {@link #setHosts(HttpHost...)} if you have metadata about the hosts + * like their Elasticsearch version of which roles they implement. */ - public Response performRequest(String method, String endpoint, Map params, - HttpEntity entity, Header... headers) throws IOException { - return performRequest(method, endpoint, params, entity, HttpAsyncResponseConsumerFactory.DEFAULT, headers); + public synchronized void setNodes(Node... nodes) { + if (nodes == null || nodes.length == 0) { + throw new IllegalArgumentException("nodes must not be null or empty"); + } + AuthCache authCache = new BasicAuthCache(); + + for (Node node : nodes) { + if (node == null) { + throw new IllegalArgumentException("node cannot be null"); + } + authCache.put(node.getHost(), new BasicScheme()); + } + this.nodeTuple = new NodeTuple<>(Collections.unmodifiableList( + Arrays.asList(nodes)), authCache); + this.blacklist.clear(); } /** - * Sends a request to the Elasticsearch cluster that the client points to. Blocks until the request is completed and returns - * its response or fails by throwing an exception. Selects a host out of the provided ones in a round-robin fashion. Failing hosts - * are marked dead and retried after a certain amount of time (minimum 1 minute, maximum 30 minutes), depending on how many times - * they previously failed (the more failures, the later they will be retried). In case of failures all of the alive nodes (or dead - * nodes that deserve a retry) are retried until one responds or none of them does, in which case an {@link IOException} will be thrown. - * - * This method works by performing an asynchronous call and waiting - * for the result. If the asynchronous call throws an exception we wrap - * it and rethrow it so that the stack trace attached to the exception - * contains the call site. While we attempt to preserve the original - * exception this isn't always possible and likely haven't covered all of - * the cases. You can get the original exception from - * {@link Exception#getCause()}. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param params the query_string parameters - * @param entity the body of the request, null if not applicable - * @param httpAsyncResponseConsumerFactory the {@link HttpAsyncResponseConsumerFactory} used to create one - * {@link HttpAsyncResponseConsumer} callback per retry. Controls how the response body gets streamed from a non-blocking HTTP - * connection on the client side. - * @param headers the optional request headers - * @return the response returned by Elasticsearch - * @throws IOException in case of a problem or the connection was aborted - * @throws ClientProtocolException in case of an http protocol error - * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + * Copy of the list of nodes that the client knows about. */ - public Response performRequest(String method, String endpoint, Map params, - HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, - Header... headers) throws IOException { - SyncResponseListener listener = new SyncResponseListener(maxRetryTimeoutMillis); - performRequestAsyncNoCatch(method, endpoint, params, entity, httpAsyncResponseConsumerFactory, - listener, headers); - return listener.get(); + public List getNodes() { + return nodeTuple.nodes; } - /** - * Sends a request to the Elasticsearch cluster that the client points to. Doesn't wait for the response, instead - * the provided {@link ResponseListener} will be notified upon completion or failure. Shortcut to - * {@link #performRequestAsync(String, String, Map, HttpEntity, ResponseListener, Header...)} but without parameters and request body. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails - * @param headers the optional request headers - */ - public void performRequestAsync(String method, String endpoint, ResponseListener responseListener, Header... headers) { - performRequestAsync(method, endpoint, Collections.emptyMap(), null, responseListener, headers); + @Override + final SyncResponseListener syncResponseListener() { + return new SyncResponseListener(maxRetryTimeoutMillis); } - /** - * Sends a request to the Elasticsearch cluster that the client points to. Doesn't wait for the response, instead - * the provided {@link ResponseListener} will be notified upon completion or failure. Shortcut to - * {@link #performRequestAsync(String, String, Map, HttpEntity, ResponseListener, Header...)} but without request body. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param params the query_string parameters - * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails - * @param headers the optional request headers - */ - public void performRequestAsync(String method, String endpoint, Map params, - ResponseListener responseListener, Header... headers) { - performRequestAsync(method, endpoint, params, null, responseListener, headers); + @Override + public RestClientActions withNodeSelector(NodeSelector nodeSelector) { + return new RestClientView(this, nodeSelector); } - /** - * Sends a request to the Elasticsearch cluster that the client points to. Doesn't wait for the response, instead - * the provided {@link ResponseListener} will be notified upon completion or failure. - * Shortcut to {@link #performRequestAsync(String, String, Map, HttpEntity, HttpAsyncResponseConsumerFactory, ResponseListener, - * Header...)} which doesn't require specifying an {@link HttpAsyncResponseConsumerFactory} instance, - * {@link HttpAsyncResponseConsumerFactory} will be used to create the needed instances of {@link HttpAsyncResponseConsumer}. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param params the query_string parameters - * @param entity the body of the request, null if not applicable - * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails - * @param headers the optional request headers - */ - public void performRequestAsync(String method, String endpoint, Map params, - HttpEntity entity, ResponseListener responseListener, Header... headers) { - performRequestAsync(method, endpoint, params, entity, HttpAsyncResponseConsumerFactory.DEFAULT, responseListener, headers); + // TODO this exists entirely to so we don't have to change much in the high level rest client tests. We'll remove in a followup. + @Override + public Response performRequest(String method, String endpoint, Map params, + HttpEntity entity, Header... headers) throws IOException { + return super.performRequest(method, endpoint, params, entity, headers); } - /** - * Sends a request to the Elasticsearch cluster that the client points to. The request is executed asynchronously - * and the provided {@link ResponseListener} gets notified upon request completion or failure. - * Selects a host out of the provided ones in a round-robin fashion. Failing hosts are marked dead and retried after a certain - * amount of time (minimum 1 minute, maximum 30 minutes), depending on how many times they previously failed (the more failures, - * the later they will be retried). In case of failures all of the alive nodes (or dead nodes that deserve a retry) are retried - * until one responds or none of them does, in which case an {@link IOException} will be thrown. - * - * @param method the http method - * @param endpoint the path of the request (without host and port) - * @param params the query_string parameters - * @param entity the body of the request, null if not applicable - * @param httpAsyncResponseConsumerFactory the {@link HttpAsyncResponseConsumerFactory} used to create one - * {@link HttpAsyncResponseConsumer} callback per retry. Controls how the response body gets streamed from a non-blocking HTTP - * connection on the client side. - * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails - * @param headers the optional request headers - */ + // TODO this exists entirely to so we don't have to change much in the high level rest client tests. We'll remove in a followup. + @Override public void performRequestAsync(String method, String endpoint, Map params, - HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, - ResponseListener responseListener, Header... headers) { - try { - performRequestAsyncNoCatch(method, endpoint, params, entity, httpAsyncResponseConsumerFactory, - responseListener, headers); - } catch (Exception e) { - responseListener.onFailure(e); - } + HttpEntity entity, ResponseListener responseListener, Header... headers) { + super.performRequestAsync(method, endpoint, params, entity, responseListener, headers); + } + + @Override + final void performRequestAsyncNoCatch(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + ResponseListener responseListener, Header[] headers) throws IOException { + // Requests made directly to the client use the noop NodeSelector. + performRequestAsyncNoCatch(method, endpoint, params, entity, httpAsyncResponseConsumerFactory, + responseListener, NodeSelector.ANY, headers); } void performRequestAsyncNoCatch(String method, String endpoint, Map params, - HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, - ResponseListener responseListener, Header... headers) { + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + ResponseListener responseListener, NodeSelector nodeSelector, Header[] headers) throws IOException { Objects.requireNonNull(params, "params must not be null"); Map requestParams = new HashMap<>(params); //ignore is a special parameter supported by the clients, shouldn't be sent to es @@ -348,15 +247,15 @@ void performRequestAsyncNoCatch(String method, String endpoint, Map> hostTuple, final HttpRequestBase request, + private void performRequestAsync(final long startTime, final NodeTuple> hostTuple, final HttpRequestBase request, final Set ignoreErrorCodes, final HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, final FailureTrackingResponseListener listener) { - final HttpHost host = hostTuple.hosts.next(); + final HttpHost host = hostTuple.nodes.next().getHost(); //we stream the request body if the entity allows for it final HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(host, request); final HttpAsyncResponseConsumer asyncResponseConsumer = @@ -402,7 +301,7 @@ public void failed(Exception failure) { } private void retryIfPossible(Exception exception) { - if (hostTuple.hosts.hasNext()) { + if (hostTuple.nodes.hasNext()) { //in case we are retrying, check whether maxRetryTimeout has been reached long timeElapsedMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime); long timeout = maxRetryTimeoutMillis - timeElapsedMillis; @@ -444,44 +343,93 @@ private void setHeaders(HttpRequest httpRequest, Header[] requestHeaders) { } /** - * Returns an {@link Iterable} of hosts to be used for a request call. - * Ideally, the first host is retrieved from the iterable and used successfully for the request. - * Otherwise, after each failure the next host has to be retrieved from the iterator so that the request can be retried until - * there are no more hosts available to retry against. The maximum total of attempts is equal to the number of hosts in the iterable. - * The iterator returned will never be empty. In case there are no healthy hosts available, or dead ones to be be retried, - * one dead host gets returned so that it can be retried. + * Returns a non-empty {@link Iterator} of nodes to be used for a request + * that match the {@link NodeSelector}. + *

+ * If there are no living nodes that match the {@link NodeSelector} + * this will return the dead node that matches the {@link NodeSelector} + * that is closest to being revived. + *

+ * If no living and no dead nodes match the selector we retry a few + * times to handle concurrent modifications of the list of dead nodes. + * We never block the thread or {@link Thread#sleep} or anything like + * that. If the retries fail this throws a {@link IOException}. + * @throws IOException if no nodes are available */ - private HostTuple> nextHost() { - final HostTuple> hostTuple = this.hostTuple; - Collection nextHosts = Collections.emptySet(); - do { - Set filteredHosts = new HashSet<>(hostTuple.hosts); - for (Map.Entry entry : blacklist.entrySet()) { - if (System.nanoTime() - entry.getValue().getDeadUntilNanos() < 0) { - filteredHosts.remove(entry.getKey()); - } + private NodeTuple> nextNode(NodeSelector nodeSelector) throws IOException { + NodeTuple> nodeTuple = this.nodeTuple; + List hosts = selectHosts(nodeTuple, blacklist, lastNodeIndex, System.nanoTime(), nodeSelector); + return new NodeTuple<>(hosts.iterator(), nodeTuple.authCache); + } + + static List selectHosts(NodeTuple> nodeTuple, + Map blacklist, AtomicInteger lastNodeIndex, + long now, NodeSelector nodeSelector) throws IOException { + /* + * Sort the nodes into living and dead lists. + */ + List livingNodes = new ArrayList<>(nodeTuple.nodes.size() - blacklist.size()); + List deadNodes = new ArrayList<>(blacklist.size()); + for (Node node : nodeTuple.nodes) { + DeadHostState deadness = blacklist.get(node.getHost()); + if (deadness == null) { + livingNodes.add(node); + continue; } - if (filteredHosts.isEmpty()) { - //last resort: if there are no good host to use, return a single dead one, the one that's closest to being retried - List> sortedHosts = new ArrayList<>(blacklist.entrySet()); - if (sortedHosts.size() > 0) { - Collections.sort(sortedHosts, new Comparator>() { - @Override - public int compare(Map.Entry o1, Map.Entry o2) { - return Long.compare(o1.getValue().getDeadUntilNanos(), o2.getValue().getDeadUntilNanos()); - } - }); - HttpHost deadHost = sortedHosts.get(0).getKey(); - logger.trace("resurrecting host [" + deadHost + "]"); - nextHosts = Collections.singleton(deadHost); + long nanosUntilRevival = now - deadness.getDeadUntilNanos(); + if (nanosUntilRevival > 0) { + livingNodes.add(node); + continue; + } + deadNodes.add(new DeadNodeAndRevival(node, nanosUntilRevival)); + } + + if (false == livingNodes.isEmpty()) { + /* + * Normal state: there is at least one living node. Rotate the + * list so subsequent requests to will prefer the nodes in a + * different order then run them through the NodeSelector so it + * can have its say in which nodes are ok and their ordering. If + * the selector is ok with any over the living nodes then use + * them for the request. + */ + Collections.rotate(livingNodes, lastNodeIndex.getAndIncrement()); + List selectedLivingNodes = nodeSelector.select(livingNodes); + if (false == selectedLivingNodes.isEmpty()) { + return selectedLivingNodes; + } + } + + /* + * Last resort: If there are no good nodes to use, either because + * the selector rejected all the living nodes or because there aren't + * any living ones. Either way, we want to revive a single dead node + * that the NodeSelectors are OK with. We do this by sorting the dead + * nodes by their revival time and passing them through the + * NodeSelector so it can have its say in which nodes are ok and their + * ordering. If the selector is ok with any of the nodes then use just + * the first one in the list because we only want to revive a single + * node. + */ + if (false == deadNodes.isEmpty()) { + Collections.sort(deadNodes, new Comparator() { + @Override + public int compare(DeadNodeAndRevival lhs, DeadNodeAndRevival rhs) { + return Long.compare(rhs.nanosUntilRevival, lhs.nanosUntilRevival); } - } else { - List rotatedHosts = new ArrayList<>(filteredHosts); - Collections.rotate(rotatedHosts, rotatedHosts.size() - lastHostIndex.getAndIncrement()); - nextHosts = rotatedHosts; + }); + + List selectedDeadNodes = new ArrayList<>(deadNodes.size()); + for (DeadNodeAndRevival n : deadNodes) { + selectedDeadNodes.add(n.node); + } + selectedDeadNodes = nodeSelector.select(selectedDeadNodes); + if (false == selectedDeadNodes.isEmpty()) { + return singletonList(selectedDeadNodes.get(0)); } - } while(nextHosts.isEmpty()); - return new HostTuple<>(nextHosts.iterator(), hostTuple.authCache); + } + throw new IOException("NodeSelector [" + nodeSelector + "] rejected all nodes, " + + "living " + livingNodes + " and dead " + deadNodes); } /** @@ -636,109 +584,6 @@ void trackFailure(Exception exception) { } } - /** - * Listener used in any sync performRequest calls, it waits for a response or an exception back up to a timeout - */ - static class SyncResponseListener implements ResponseListener { - private final CountDownLatch latch = new CountDownLatch(1); - private final AtomicReference response = new AtomicReference<>(); - private final AtomicReference exception = new AtomicReference<>(); - - private final long timeout; - - SyncResponseListener(long timeout) { - assert timeout > 0; - this.timeout = timeout; - } - - @Override - public void onSuccess(Response response) { - Objects.requireNonNull(response, "response must not be null"); - boolean wasResponseNull = this.response.compareAndSet(null, response); - if (wasResponseNull == false) { - throw new IllegalStateException("response is already set"); - } - - latch.countDown(); - } - - @Override - public void onFailure(Exception exception) { - Objects.requireNonNull(exception, "exception must not be null"); - boolean wasExceptionNull = this.exception.compareAndSet(null, exception); - if (wasExceptionNull == false) { - throw new IllegalStateException("exception is already set"); - } - latch.countDown(); - } - - /** - * Waits (up to a timeout) for some result of the request: either a response, or an exception. - */ - Response get() throws IOException { - try { - //providing timeout is just a safety measure to prevent everlasting waits - //the different client timeouts should already do their jobs - if (latch.await(timeout, TimeUnit.MILLISECONDS) == false) { - throw new IOException("listener timeout after waiting for [" + timeout + "] ms"); - } - } catch (InterruptedException e) { - throw new RuntimeException("thread waiting for the response was interrupted", e); - } - - Exception exception = this.exception.get(); - Response response = this.response.get(); - if (exception != null) { - if (response != null) { - IllegalStateException e = new IllegalStateException("response and exception are unexpectedly set at the same time"); - e.addSuppressed(exception); - throw e; - } - /* - * Wrap and rethrow whatever exception we received, copying the type - * where possible so the synchronous API looks as much as possible - * like the asynchronous API. We wrap the exception so that the caller's - * signature shows up in any exception we throw. - */ - if (exception instanceof ResponseException) { - throw new ResponseException((ResponseException) exception); - } - if (exception instanceof ConnectTimeoutException) { - ConnectTimeoutException e = new ConnectTimeoutException(exception.getMessage()); - e.initCause(exception); - throw e; - } - if (exception instanceof SocketTimeoutException) { - SocketTimeoutException e = new SocketTimeoutException(exception.getMessage()); - e.initCause(exception); - throw e; - } - if (exception instanceof ConnectionClosedException) { - ConnectionClosedException e = new ConnectionClosedException(exception.getMessage()); - e.initCause(exception); - throw e; - } - if (exception instanceof SSLHandshakeException) { - SSLHandshakeException e = new SSLHandshakeException(exception.getMessage()); - e.initCause(exception); - throw e; - } - if (exception instanceof IOException) { - throw new IOException(exception.getMessage(), exception); - } - if (exception instanceof RuntimeException){ - throw new RuntimeException(exception.getMessage(), exception); - } - throw new RuntimeException("error while performing request", exception); - } - - if (response == null) { - throw new IllegalStateException("response not set and no exception caught either"); - } - return response; - } - } - /** * Listener that allows to be notified whenever a failure happens. Useful when sniffing is enabled, so that we can sniff on failure. * The default implementation is a no-op. @@ -753,16 +598,35 @@ public void onFailure(HttpHost host) { } /** - * {@code HostTuple} enables the {@linkplain HttpHost}s and {@linkplain AuthCache} to be set together in a thread + * {@code HostTuple} enables the {@linkplain Node}s and {@linkplain AuthCache} to be set together in a thread * safe, volatile way. */ - private static class HostTuple { - final T hosts; + static class NodeTuple { + final T nodes; final AuthCache authCache; - HostTuple(final T hosts, final AuthCache authCache) { - this.hosts = hosts; + NodeTuple(final T nodes, final AuthCache authCache) { + this.nodes = nodes; this.authCache = authCache; } } + + /** + * Contains a reference to a blacklisted node and the time until it is + * revived. We use this so we can do a single pass over the blacklist. + */ + private static class DeadNodeAndRevival { + final Node node; + final long nanosUntilRevival; + + DeadNodeAndRevival(Node node, long nanosUntilRevival) { + this.node = node; + this.nanosUntilRevival = nanosUntilRevival; + } + + @Override + public String toString() { + return node.toString(); + } + } } diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClientActions.java b/client/rest/src/main/java/org/elasticsearch/client/RestClientActions.java new file mode 100644 index 0000000000000..c9ad4ac483cc0 --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClientActions.java @@ -0,0 +1,192 @@ +/* + * 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.client; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; + +import org.apache.http.Header; +import org.apache.http.HttpEntity; +import org.apache.http.client.ClientProtocolException; +import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; + +/** + * Actions that can be taken on a {@link RestClient} or the stateless views + * returned by {@link RestClientActions#withNodeSelector withNodeSelector}. + */ +public interface RestClientActions { + /** + * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response + * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, Header...)} but without parameters + * and request body. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param headers the optional request headers + * @return the response returned by Elasticsearch + * @throws IOException in case of a problem or the connection was aborted + * @throws ClientProtocolException in case of an http protocol error + * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + */ + Response performRequest(String method, String endpoint, Header... headers) throws IOException; + + /** + * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response + * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, Header...)} but without request body. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param params the query_string parameters + * @param headers the optional request headers + * @return the response returned by Elasticsearch + * @throws IOException in case of a problem or the connection was aborted + * @throws ClientProtocolException in case of an http protocol error + * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + */ + Response performRequest(String method, String endpoint, Map params, Header... headers) throws IOException; + + /** + * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response + * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, HttpAsyncResponseConsumerFactory, Header...)} + * which doesn't require specifying an {@link HttpAsyncResponseConsumerFactory} instance, + * {@link HttpAsyncResponseConsumerFactory} will be used to create the needed instances of {@link HttpAsyncResponseConsumer}. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param params the query_string parameters + * @param entity the body of the request, null if not applicable + * @param headers the optional request headers + * @return the response returned by Elasticsearch + * @throws IOException in case of a problem or the connection was aborted + * @throws ClientProtocolException in case of an http protocol error + * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + */ + Response performRequest(String method, String endpoint, Map params, + HttpEntity entity, Header... headers) throws IOException; + + /** + * Sends a request to the Elasticsearch cluster that the client points to. Blocks until the request is completed and returns + * its response or fails by throwing an exception. Selects a host out of the provided ones in a round-robin fashion. Failing hosts + * are marked dead and retried after a certain amount of time (minimum 1 minute, maximum 30 minutes), depending on how many times + * they previously failed (the more failures, the later they will be retried). In case of failures all of the alive nodes (or dead + * nodes that deserve a retry) are retried until one responds or none of them does, in which case an {@link IOException} will be thrown. + * + * This method works by performing an asynchronous call and waiting + * for the result. If the asynchronous call throws an exception we wrap + * it and rethrow it so that the stack trace attached to the exception + * contains the call site. While we attempt to preserve the original + * exception this isn't always possible and likely haven't covered all of + * the cases. You can get the original exception from + * {@link Exception#getCause()}. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param params the query_string parameters + * @param entity the body of the request, null if not applicable + * @param httpAsyncResponseConsumerFactory the {@link HttpAsyncResponseConsumerFactory} used to create one + * {@link HttpAsyncResponseConsumer} callback per retry. Controls how the response body gets streamed from a non-blocking HTTP + * connection on the client side. + * @param headers the optional request headers + * @return the response returned by Elasticsearch + * @throws IOException in case of a problem or the connection was aborted + * @throws ClientProtocolException in case of an http protocol error + * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + */ + Response performRequest(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + Header... headers) throws IOException; + + /** + * Sends a request to the Elasticsearch cluster that the client points to. Doesn't wait for the response, instead + * the provided {@link ResponseListener} will be notified upon completion or failure. Shortcut to + * {@link #performRequestAsync(String, String, Map, HttpEntity, ResponseListener, Header...)} but without parameters and request body. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails + * @param headers the optional request headers + */ + void performRequestAsync(String method, String endpoint, ResponseListener responseListener, Header... headers); + + /** + * Sends a request to the Elasticsearch cluster that the client points to. Doesn't wait for the response, instead + * the provided {@link ResponseListener} will be notified upon completion or failure. Shortcut to + * {@link #performRequestAsync(String, String, Map, HttpEntity, ResponseListener, Header...)} but without request body. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param params the query_string parameters + * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails + * @param headers the optional request headers + */ + void performRequestAsync(String method, String endpoint, Map params, + ResponseListener responseListener, Header... headers); + + /** + * Sends a request to the Elasticsearch cluster that the client points to. Doesn't wait for the response, instead + * the provided {@link ResponseListener} will be notified upon completion or failure. + * Shortcut to {@link #performRequestAsync(String, String, Map, HttpEntity, HttpAsyncResponseConsumerFactory, ResponseListener, + * Header...)} which doesn't require specifying an {@link HttpAsyncResponseConsumerFactory} instance, + * {@link HttpAsyncResponseConsumerFactory} will be used to create the needed instances of {@link HttpAsyncResponseConsumer}. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param params the query_string parameters + * @param entity the body of the request, null if not applicable + * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails + * @param headers the optional request headers + */ + void performRequestAsync(String method, String endpoint, Map params, + HttpEntity entity, ResponseListener responseListener, Header... headers); + + /** + * Sends a request to the Elasticsearch cluster that the client points to. The request is executed asynchronously + * and the provided {@link ResponseListener} gets notified upon request completion or failure. + * Selects a host out of the provided ones in a round-robin fashion. Failing hosts are marked dead and retried after a certain + * amount of time (minimum 1 minute, maximum 30 minutes), depending on how many times they previously failed (the more failures, + * the later they will be retried). In case of failures all of the alive nodes (or dead nodes that deserve a retry) are retried + * until one responds or none of them does, in which case an {@link IOException} will be thrown. + * + * @param method the http method + * @param endpoint the path of the request (without host and port) + * @param params the query_string parameters + * @param entity the body of the request, null if not applicable + * @param httpAsyncResponseConsumerFactory the {@link HttpAsyncResponseConsumerFactory} used to create one + * {@link HttpAsyncResponseConsumer} callback per retry. Controls how the response body gets streamed from a non-blocking HTTP + * connection on the client side. + * @param responseListener the {@link ResponseListener} to notify when the request is completed or fails + * @param headers the optional request headers + */ + void performRequestAsync(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + ResponseListener responseListener, Header... headers); + + /** + * Create a "stateless view" of this client that only runs requests on + * selected hosts. The result of this method is "stateless" because it + * backs all of its requests to the {@link RestClient} that created it + * so there is no need to manage this view with try-with-resources and + * it does not extend {@link Closeable}. Closing the + * {@linkplain RestClient} that created the view disposes of the + * underlying connection, making the view useless. + */ + RestClientActions withNodeSelector(NodeSelector nodeSelector); +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java b/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java index 286ed7dd53910..0f0acc4e420be 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java @@ -20,7 +20,6 @@ package org.elasticsearch.client; import org.apache.http.Header; -import org.apache.http.HttpHost; import org.apache.http.client.config.RequestConfig; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; @@ -49,7 +48,7 @@ public final class RestClientBuilder { private static final Header[] EMPTY_HEADERS = new Header[0]; - private final HttpHost[] hosts; + private final Node[] nodes; private int maxRetryTimeout = DEFAULT_MAX_RETRY_TIMEOUT_MILLIS; private Header[] defaultHeaders = EMPTY_HEADERS; private RestClient.FailureListener failureListener; @@ -63,15 +62,16 @@ public final class RestClientBuilder { * @throws NullPointerException if {@code hosts} or any host is {@code null}. * @throws IllegalArgumentException if {@code hosts} is empty. */ - RestClientBuilder(HttpHost... hosts) { - Objects.requireNonNull(hosts, "hosts must not be null"); - if (hosts.length == 0) { - throw new IllegalArgumentException("no hosts provided"); + RestClientBuilder(Node[] nodes) { + if (nodes == null || nodes.length == 0) { + throw new IllegalArgumentException("nodes must not be null or empty"); } - for (HttpHost host : hosts) { - Objects.requireNonNull(host, "host cannot be null"); + for (Node node : nodes) { + if (node == null) { + throw new IllegalArgumentException("node cannot be null"); + } } - this.hosts = hosts; + this.nodes = nodes; } /** @@ -187,7 +187,8 @@ public CloseableHttpAsyncClient run() { return createHttpClient(); } }); - RestClient restClient = new RestClient(httpClient, maxRetryTimeout, defaultHeaders, hosts, pathPrefix, failureListener); + RestClient restClient = new RestClient(httpClient, maxRetryTimeout, defaultHeaders, nodes, + pathPrefix, failureListener); httpClient.start(); return restClient; } diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClientView.java b/client/rest/src/main/java/org/elasticsearch/client/RestClientView.java new file mode 100644 index 0000000000000..2639443819be6 --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClientView.java @@ -0,0 +1,63 @@ +/* + * 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.client; + +import java.io.IOException; +import java.util.Map; + +import org.apache.http.Header; +import org.apache.http.HttpEntity; + +/** + * Stateless view into a {@link RestClient} with customized parameters. + */ +class RestClientView extends AbstractRestClientActions { + /** + * {@linkplain RestClient} to which to delegate all requests. + */ + private final RestClient delegate; + /** + * Selects which hosts are valid destinations requests. + */ + private final NodeSelector nodeSelector; + + RestClientView(RestClient delegate, NodeSelector nodeSelector) { + this.delegate = delegate; + this.nodeSelector = nodeSelector; + } + + @Override + final SyncResponseListener syncResponseListener() { + return delegate.syncResponseListener(); + } + + @Override + public final RestClientView withNodeSelector(final NodeSelector newNodeSelector) { + return new RestClientView(delegate, new NodeSelector.Compose(nodeSelector, newNodeSelector)); + } + + @Override + final void performRequestAsyncNoCatch(String method, String endpoint, Map params, + HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, + ResponseListener responseListener, Header[] headers) throws IOException { + delegate.performRequestAsyncNoCatch(method, endpoint, params, entity, httpAsyncResponseConsumerFactory, + responseListener, nodeSelector, headers); + } +} diff --git a/client/rest/src/test/java/org/elasticsearch/client/HostsTrackingFailureListener.java b/client/rest/src/test/java/org/elasticsearch/client/HostsTrackingFailureListener.java index e2f0ba81f6ed7..fb5deada773de 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/HostsTrackingFailureListener.java +++ b/client/rest/src/test/java/org/elasticsearch/client/HostsTrackingFailureListener.java @@ -39,6 +39,14 @@ public void onFailure(HttpHost host) { hosts.add(host); } + void assertCalled(Node... nodes) { + HttpHost[] hosts = new HttpHost[nodes.length]; + for (int i = 0; i < nodes.length; i++) { + hosts[i] = nodes[i].getHost(); + } + assertCalled(hosts); + } + void assertCalled(HttpHost... hosts) { assertEquals(hosts.length, this.hosts.size()); assertThat(this.hosts, containsInAnyOrder(hosts)); @@ -48,4 +56,4 @@ void assertCalled(HttpHost... hosts) { void assertNotCalled() { assertEquals(0, hosts.size()); } -} \ No newline at end of file +} diff --git a/client/rest/src/test/java/org/elasticsearch/client/NodeSelectorTests.java b/client/rest/src/test/java/org/elasticsearch/client/NodeSelectorTests.java new file mode 100644 index 0000000000000..e8aa7a175be8b --- /dev/null +++ b/client/rest/src/test/java/org/elasticsearch/client/NodeSelectorTests.java @@ -0,0 +1,59 @@ +/* + * 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.client; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.Node.Roles; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class NodeSelectorTests extends RestClientTestCase { + public void testAny() { + List nodes = new ArrayList<>(); + int size = between(2, 5); + for (int i = 0; i < size; i++) { + nodes.add(dummyNode(randomBoolean(), randomBoolean(), randomBoolean())); + } + assertEquals(nodes, NodeSelector.ANY.select(nodes)); + } + + public void testNotMasterOnly() { + Node masterOnly = dummyNode(true, false, randomBoolean()); + Node masterAndData = dummyNode(true, true, randomBoolean()); + Node coordinatingOnly = dummyNode(false, false, randomBoolean()); + Node data = dummyNode(false, true, randomBoolean()); + List nodes = Arrays.asList(masterOnly, masterAndData, coordinatingOnly, data); + Collections.shuffle(nodes, getRandom()); + List expected = new ArrayList<>(nodes); + expected.remove(masterOnly); + assertEquals(expected, NodeSelector.NOT_MASTER_ONLY.select(nodes)); + } + + private Node dummyNode(boolean master, boolean data, boolean ingest) { + return new Node(new HttpHost("dummy"), Collections.emptySet(), + randomAsciiAlphanumOfLength(5), randomAsciiAlphanumOfLength(5), + new Roles(master, data, ingest)); + } +} diff --git a/client/rest/src/test/java/org/elasticsearch/client/NodeTests.java b/client/rest/src/test/java/org/elasticsearch/client/NodeTests.java new file mode 100644 index 0000000000000..989861df50293 --- /dev/null +++ b/client/rest/src/test/java/org/elasticsearch/client/NodeTests.java @@ -0,0 +1,89 @@ +/* + * 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.client; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.Node.Roles; + +import java.util.Arrays; +import java.util.HashSet; + +import static java.util.Collections.singleton; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class NodeTests extends RestClientTestCase { + public void testWithHost() { + HttpHost h1 = new HttpHost("1"); + HttpHost h2 = new HttpHost("2"); + HttpHost h3 = new HttpHost("3"); + + Node n = new Node(h1, new HashSet<>(Arrays.asList(h1, h2)), + randomAsciiAlphanumOfLength(5), randomAsciiAlphanumOfLength(5), + new Roles(randomBoolean(), randomBoolean(), randomBoolean())); + + // Host is in the bound hosts list + assertEquals(h2, n.withHost(h2).getHost()); + assertEquals(n.getBoundHosts(), n.withHost(h2).getBoundHosts()); + + // Host not in the bound hosts list + assertEquals(h3, n.withHost(h3).getHost()); + assertEquals(new HashSet<>(Arrays.asList(h1, h2, h3)), n.withHost(h3).getBoundHosts()); + } + + public void testToString() { + assertEquals("[host=http://1]", new Node(new HttpHost("1")).toString()); + assertEquals("[host=http://1, roles=mdi]", new Node(new HttpHost("1"), + null, null, null, new Roles(true, true, true)).toString()); + assertEquals("[host=http://1, version=ver]", new Node(new HttpHost("1"), + null, null, "ver", null).toString()); + assertEquals("[host=http://1, name=nam]", new Node(new HttpHost("1"), + null, "nam", null, null).toString()); + assertEquals("[host=http://1, bound=[http://1, http://2]]", new Node(new HttpHost("1"), + new HashSet<>(Arrays.asList(new HttpHost("1"), new HttpHost("2"))), null, null, null).toString()); + assertEquals("[host=http://1, bound=[http://1, http://2], name=nam, version=ver, roles=m]", + new Node(new HttpHost("1"), new HashSet<>(Arrays.asList(new HttpHost("1"), new HttpHost("2"))), + "nam", "ver", new Roles(true, false, false)).toString()); + + } + + public void testEqualsAndHashCode() { + HttpHost host = new HttpHost(randomAsciiAlphanumOfLength(5)); + Node node = new Node(host, + randomBoolean() ? null : singleton(host), + randomBoolean() ? null : randomAsciiAlphanumOfLength(5), + randomBoolean() ? null : randomAsciiAlphanumOfLength(5), + randomBoolean() ? null : new Roles(true, true, true)); + assertFalse(node.equals(null)); + assertTrue(node.equals(node)); + assertEquals(node.hashCode(), node.hashCode()); + Node copy = new Node(host, node.getBoundHosts(), node.getName(), node.getVersion(), node.getRoles()); + assertTrue(node.equals(copy)); + assertEquals(node.hashCode(), copy.hashCode()); + assertFalse(node.equals(new Node(new HttpHost(host.toHostString() + "changed"), node.getBoundHosts(), + node.getName(), node.getVersion(), node.getRoles()))); + assertFalse(node.equals(new Node(host, new HashSet<>(Arrays.asList(host, new HttpHost(host.toHostString() + "changed"))), + node.getName(), node.getVersion(), node.getRoles()))); + assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName() + "changed", node.getVersion(), node.getRoles()))); + assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName(), node.getVersion() + "changed", node.getRoles()))); + assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName(), node.getVersion(), new Roles(false, false, false)))); + } +} diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java index c9243d3aaf6ce..b11d240f27bd4 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java @@ -39,24 +39,45 @@ public void testBuild() throws IOException { try { RestClient.builder((HttpHost[])null); fail("should have failed"); - } catch(NullPointerException e) { - assertEquals("hosts must not be null", e.getMessage()); + } catch(IllegalArgumentException e) { + assertEquals("hosts must not be null or empty", e.getMessage()); } try { - RestClient.builder(); + RestClient.builder(new HttpHost[] {}); fail("should have failed"); } catch(IllegalArgumentException e) { - assertEquals("no hosts provided", e.getMessage()); + assertEquals("hosts must not be null or empty", e.getMessage()); } try { RestClient.builder(new HttpHost("localhost", 9200), null); fail("should have failed"); - } catch(NullPointerException e) { + } catch(IllegalArgumentException e) { assertEquals("host cannot be null", e.getMessage()); } + try { + RestClient.builder((Node[])null); + fail("should have failed"); + } catch(IllegalArgumentException e) { + assertEquals("nodes must not be null or empty", e.getMessage()); + } + + try { + RestClient.builder(new Node[] {}); + fail("should have failed"); + } catch(IllegalArgumentException e) { + assertEquals("nodes must not be null or empty", e.getMessage()); + } + + try { + RestClient.builder(new Node(new HttpHost("localhost", 9200)), null); + fail("should have failed"); + } catch(IllegalArgumentException e) { + assertEquals("node cannot be null", e.getMessage()); + } + try (RestClient restClient = RestClient.builder(new HttpHost("localhost", 9200)).build()) { assertNotNull(restClient); } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java index da5a960c2e84c..4ad9484365b12 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java @@ -30,19 +30,25 @@ import org.junit.BeforeClass; import java.io.IOException; +import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import static java.util.Collections.singletonList; import static org.elasticsearch.client.RestClientTestUtil.getAllStatusCodes; import static org.elasticsearch.client.RestClientTestUtil.randomErrorNoRetryStatusCode; import static org.elasticsearch.client.RestClientTestUtil.randomOkStatusCode; +import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Integration test to check interaction between {@link RestClient} and {@link org.apache.http.client.HttpClient}. @@ -53,31 +59,38 @@ public class RestClientMultipleHostsIntegTests extends RestClientTestCase { private static HttpServer[] httpServers; - private static RestClient restClient; + private static HttpHost[] httpHosts; + private static boolean stoppedFirstHost = false; + private static String pathPrefixWithoutLeadingSlash; private static String pathPrefix; + private static RestClient restClient; @BeforeClass public static void startHttpServer() throws Exception { - String pathPrefixWithoutLeadingSlash; if (randomBoolean()) { - pathPrefixWithoutLeadingSlash = "testPathPrefix/" + randomAsciiOfLengthBetween(1, 5); + pathPrefixWithoutLeadingSlash = "testPathPrefix/" + + randomAsciiLettersOfLengthBetween(1, 5); pathPrefix = "/" + pathPrefixWithoutLeadingSlash; } else { pathPrefix = pathPrefixWithoutLeadingSlash = ""; } int numHttpServers = randomIntBetween(2, 4); httpServers = new HttpServer[numHttpServers]; - HttpHost[] httpHosts = new HttpHost[numHttpServers]; + httpHosts = new HttpHost[numHttpServers]; for (int i = 0; i < numHttpServers; i++) { HttpServer httpServer = createHttpServer(); httpServers[i] = httpServer; httpHosts[i] = new HttpHost(httpServer.getAddress().getHostString(), httpServer.getAddress().getPort()); } + restClient = buildRestClient(); + } + + private static RestClient buildRestClient() { RestClientBuilder restClientBuilder = RestClient.builder(httpHosts); if (pathPrefix.length() > 0) { restClientBuilder.setPathPrefix((randomBoolean() ? "/" : "") + pathPrefixWithoutLeadingSlash); } - restClient = restClientBuilder.build(); + return restClientBuilder.build(); } private static HttpServer createHttpServer() throws Exception { @@ -123,6 +136,9 @@ public void stopRandomHost() { if (httpServers.length > 1 && randomBoolean()) { List updatedHttpServers = new ArrayList<>(httpServers.length - 1); int nodeIndex = randomInt(httpServers.length - 1); + if (0 == nodeIndex) { + stoppedFirstHost = true; + } for (int i = 0; i < httpServers.length; i++) { HttpServer httpServer = httpServers[i]; if (i == nodeIndex) { @@ -187,6 +203,57 @@ public void onFailure(Exception exception) { } } + /** + * Test host selector against a real server and + * test what happens after calling + */ + public void testWithNodeSelector() throws IOException { + RestClientActions withNodeSelector = restClient.withNodeSelector(firstPositionNodeSelector()); + int rounds = between(1, 10); + for (int i = 0; i < rounds; i++) { + /* + * Run the request more than once to verify that the + * NodeSelector overrides the round robin behavior. + */ + performRequestAndAssertOnFirstHost(withNodeSelector); + } + } + + /** + * Tests that stopping the {@link RestClient} backing the result of + * {@link RestClientActions#withNodeSelector(NodeSelector)} causes + * subsequent uses of the view to throw sensible exceptions. + */ + public void testStoppedView() throws IOException { + RestClientActions withNodeSelector; + // Build our own RestClient for this test because we're going to close it. + try (RestClient toStop = buildRestClient()) { + withNodeSelector = toStop.withNodeSelector(firstPositionNodeSelector()); + performRequestAndAssertOnFirstHost(withNodeSelector); + } + try { + withNodeSelector.performRequest("GET", "/200"); + fail("expected a failure"); + } catch (IllegalStateException e) { + assertThat(e.getMessage(), containsString("status: STOPPED")); + } + } + + private void performRequestAndAssertOnFirstHost(RestClientActions withNodeSelector) throws IOException { + if (stoppedFirstHost) { + try { + withNodeSelector.performRequest("GET", "/200"); + fail("expected to fail to connect"); + } catch (ConnectException e) { + assertEquals("Connection refused", e.getMessage()); + } + } else { + Response response = withNodeSelector.performRequest("GET", "/200"); + assertEquals(httpHosts[0], response.getHost()); + } + + } + private static class TestResponse { private final String method; private final int statusCode; @@ -205,7 +272,24 @@ Response getResponse() { if (response instanceof ResponseException) { return ((ResponseException) response).getResponse(); } + if (response instanceof Exception) { + throw new AssertionError("unexpected response " + response.getClass(), (Exception) response); + } throw new AssertionError("unexpected response " + response.getClass()); } } + + private NodeSelector firstPositionNodeSelector() { + return new NodeSelector() { + @Override + public List select(List nodes) { + for (Node node : nodes) { + if (httpHosts[0] == node.getHost()) { + return singletonList(node); + } + } + return Collections.emptyList(); + } + }; + } } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java index a3a834ff3204b..03212a700a186 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java @@ -35,6 +35,7 @@ import org.apache.http.message.BasicStatusLine; import org.apache.http.nio.protocol.HttpAsyncRequestProducer; import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; +import org.elasticsearch.client.Node.Roles; import org.junit.After; import org.junit.Before; import org.mockito.invocation.InvocationOnMock; @@ -42,19 +43,22 @@ import java.io.IOException; import java.net.SocketTimeoutException; -import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static java.util.Collections.singletonList; import static org.elasticsearch.client.RestClientTestUtil.randomErrorNoRetryStatusCode; import static org.elasticsearch.client.RestClientTestUtil.randomErrorRetryStatusCode; import static org.elasticsearch.client.RestClientTestUtil.randomHttpMethod; import static org.elasticsearch.client.RestClientTestUtil.randomOkStatusCode; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -71,7 +75,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase { private ExecutorService exec = Executors.newFixedThreadPool(1); private RestClient restClient; - private HttpHost[] httpHosts; + private Node[] nodes; private HostsTrackingFailureListener failureListener; @Before @@ -108,13 +112,13 @@ public void run() { return null; } }); - int numHosts = RandomNumbers.randomIntBetween(getRandom(), 2, 5); - httpHosts = new HttpHost[numHosts]; - for (int i = 0; i < numHosts; i++) { - httpHosts[i] = new HttpHost("localhost", 9200 + i); + int numNodes = RandomNumbers.randomIntBetween(getRandom(), 2, 5); + nodes = new Node[numNodes]; + for (int i = 0; i < numNodes; i++) { + nodes[i] = new Node(new HttpHost("localhost", 9200 + i)); } failureListener = new HostsTrackingFailureListener(); - restClient = new RestClient(httpClient, 10000, new Header[0], httpHosts, null, failureListener); + restClient = new RestClient(httpClient, 10000, new Header[0], nodes, null, failureListener); } /** @@ -128,9 +132,8 @@ public void shutdownExec() { public void testRoundRobinOkStatusCodes() throws IOException { int numIters = RandomNumbers.randomIntBetween(getRandom(), 1, 5); for (int i = 0; i < numIters; i++) { - Set hostsSet = new HashSet<>(); - Collections.addAll(hostsSet, httpHosts); - for (int j = 0; j < httpHosts.length; j++) { + Set hostsSet = hostsSet(); + for (int j = 0; j < nodes.length; j++) { int statusCode = randomOkStatusCode(getRandom()); Response response = restClient.performRequest(randomHttpMethod(getRandom()), "/" + statusCode); assertEquals(statusCode, response.getStatusLine().getStatusCode()); @@ -144,9 +147,8 @@ public void testRoundRobinOkStatusCodes() throws IOException { public void testRoundRobinNoRetryErrors() throws IOException { int numIters = RandomNumbers.randomIntBetween(getRandom(), 1, 5); for (int i = 0; i < numIters; i++) { - Set hostsSet = new HashSet<>(); - Collections.addAll(hostsSet, httpHosts); - for (int j = 0; j < httpHosts.length; j++) { + Set hostsSet = hostsSet(); + for (int j = 0; j < nodes.length; j++) { String method = randomHttpMethod(getRandom()); int statusCode = randomErrorNoRetryStatusCode(getRandom()); try { @@ -185,10 +187,9 @@ public void testRoundRobinRetryErrors() throws IOException { * the caller. It wraps the exception that contains the failed hosts. */ e = (ResponseException) e.getCause(); - Set hostsSet = new HashSet<>(); - Collections.addAll(hostsSet, httpHosts); + Set hostsSet = hostsSet(); //first request causes all the hosts to be blacklisted, the returned exception holds one suppressed exception each - failureListener.assertCalled(httpHosts); + failureListener.assertCalled(nodes); do { Response response = e.getResponse(); assertEquals(Integer.parseInt(retryEndpoint.substring(1)), response.getStatusLine().getStatusCode()); @@ -210,10 +211,9 @@ public void testRoundRobinRetryErrors() throws IOException { * the caller. It wraps the exception that contains the failed hosts. */ e = (IOException) e.getCause(); - Set hostsSet = new HashSet<>(); - Collections.addAll(hostsSet, httpHosts); + Set hostsSet = hostsSet(); //first request causes all the hosts to be blacklisted, the returned exception holds one suppressed exception each - failureListener.assertCalled(httpHosts); + failureListener.assertCalled(nodes); do { HttpHost httpHost = HttpHost.create(e.getMessage()); assertTrue("host [" + httpHost + "] not found, most likely used multiple times", hostsSet.remove(httpHost)); @@ -232,9 +232,8 @@ public void testRoundRobinRetryErrors() throws IOException { int numIters = RandomNumbers.randomIntBetween(getRandom(), 2, 5); for (int i = 1; i <= numIters; i++) { //check that one different host is resurrected at each new attempt - Set hostsSet = new HashSet<>(); - Collections.addAll(hostsSet, httpHosts); - for (int j = 0; j < httpHosts.length; j++) { + Set hostsSet = hostsSet(); + for (int j = 0; j < nodes.length; j++) { retryEndpoint = randomErrorRetryEndpoint(); try { restClient.performRequest(randomHttpMethod(getRandom()), retryEndpoint); @@ -308,6 +307,45 @@ public void testRoundRobinRetryErrors() throws IOException { } } + public void testWithNodeSelector() throws IOException { + NodeSelector firstPositionOnly = new NodeSelector() { + @Override + public List select(List restClientNodes) { + assertThat(restClientNodes, hasItem(nodes[0])); + return singletonList(nodes[0]); + } + }; + RestClientActions withNodeSelector = restClient.withNodeSelector(firstPositionOnly); + int rounds = between(1, 10); + for (int i = 0; i < rounds; i++) { + /* + * Run the request more than once to verify that the + * NodeSelector overrides the round robin behavior. + */ + Response response = withNodeSelector.performRequest("GET", "/200"); + assertEquals(nodes[0].getHost(), response.getHost()); + } + } + + public void testSetNodes() throws IOException { + Node[] newNodes = new Node[nodes.length]; + for (int i = 0; i < nodes.length; i++) { + Roles roles = i == 0 ? new Roles(false, true, true) : new Roles(true, false, false); + newNodes[i] = new Node(nodes[i].getHost(), null, null, null, roles); + } + restClient.setNodes(newNodes); + RestClientActions withNodeSelector = restClient.withNodeSelector(NodeSelector.NOT_MASTER_ONLY); + int rounds = between(1, 10); + for (int i = 0; i < rounds; i++) { + /* + * Run the request more than once to verify that the + * NodeSelector overrides the round robin behavior. + */ + Response response = withNodeSelector.performRequest("GET", "/200"); + assertEquals(newNodes[0].getHost(), response.getHost()); + } + } + private static String randomErrorRetryEndpoint() { switch(RandomNumbers.randomIntBetween(getRandom(), 0, 3)) { case 0: @@ -321,4 +359,16 @@ private static String randomErrorRetryEndpoint() { } throw new UnsupportedOperationException(); } + + /** + * Build a mutable {@link Set} containing all the {@link Node#getHost() hosts} + * in use by the test. + */ + private Set hostsSet() { + Set hosts = new HashSet<>(); + for (Node node : nodes) { + hosts.add(node.getHost()); + } + return hosts; + } } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java index caf9ce6be2e07..f33fe26f34e06 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java @@ -95,7 +95,7 @@ public class RestClientSingleHostTests extends RestClientTestCase { private ExecutorService exec = Executors.newFixedThreadPool(1); private RestClient restClient; private Header[] defaultHeaders; - private HttpHost httpHost; + private Node node; private CloseableHttpAsyncClient httpClient; private HostsTrackingFailureListener failureListener; @@ -109,7 +109,7 @@ public void createRestClient() throws IOException { public Future answer(InvocationOnMock invocationOnMock) throws Throwable { HttpAsyncRequestProducer requestProducer = (HttpAsyncRequestProducer) invocationOnMock.getArguments()[0]; HttpClientContext context = (HttpClientContext) invocationOnMock.getArguments()[2]; - assertThat(context.getAuthCache().get(httpHost), instanceOf(BasicScheme.class)); + assertThat(context.getAuthCache().get(node.getHost()), instanceOf(BasicScheme.class)); final FutureCallback futureCallback = (FutureCallback) invocationOnMock.getArguments()[3]; HttpUriRequest request = (HttpUriRequest)requestProducer.generateRequest(); @@ -147,9 +147,10 @@ public void run() { }); defaultHeaders = RestClientTestUtil.randomHeaders(getRandom(), "Header-default"); - httpHost = new HttpHost("localhost", 9200); + node = new Node(new HttpHost("localhost", 9200)); failureListener = new HostsTrackingFailureListener(); - restClient = new RestClient(httpClient, 10000, defaultHeaders, new HttpHost[]{httpHost}, null, failureListener); + restClient = new RestClient(httpClient, 10000, defaultHeaders, new Node[] {node}, + null, failureListener); } /** @@ -196,33 +197,6 @@ public void testInternalHttpRequest() throws Exception { } } - public void testSetHosts() throws IOException { - try { - restClient.setHosts((HttpHost[]) null); - fail("setHosts should have failed"); - } catch (IllegalArgumentException e) { - assertEquals("hosts must not be null nor empty", e.getMessage()); - } - try { - restClient.setHosts(); - fail("setHosts should have failed"); - } catch (IllegalArgumentException e) { - assertEquals("hosts must not be null nor empty", e.getMessage()); - } - try { - restClient.setHosts((HttpHost) null); - fail("setHosts should have failed"); - } catch (NullPointerException e) { - assertEquals("host cannot be null", e.getMessage()); - } - try { - restClient.setHosts(new HttpHost("localhost", 9200), null, new HttpHost("localhost", 9201)); - fail("setHosts should have failed"); - } catch (NullPointerException e) { - assertEquals("host cannot be null", e.getMessage()); - } - } - /** * End to end test for ok status codes */ @@ -283,7 +257,7 @@ public void testErrorStatusCodes() throws IOException { if (errorStatusCode <= 500 || expectedIgnores.contains(errorStatusCode)) { failureListener.assertNotCalled(); } else { - failureListener.assertCalled(httpHost); + failureListener.assertCalled(node); } } } @@ -298,14 +272,14 @@ public void testIOExceptions() throws IOException { } catch(IOException e) { assertThat(e, instanceOf(ConnectTimeoutException.class)); } - failureListener.assertCalled(httpHost); + failureListener.assertCalled(node); try { performRequest(method, "/soe"); fail("request should have failed"); } catch(IOException e) { assertThat(e, instanceOf(SocketTimeoutException.class)); } - failureListener.assertCalled(httpHost); + failureListener.assertCalled(node); } } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java index 33323d39663e2..cca16162a7de6 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java @@ -22,12 +22,20 @@ import org.apache.http.Header; import org.apache.http.HttpHost; import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; +import org.elasticsearch.client.RestClient.NodeTuple; import java.io.IOException; import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import static java.util.Collections.singletonList; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -39,9 +47,10 @@ public class RestClientTests extends RestClientTestCase { public void testCloseIsIdempotent() throws IOException { - HttpHost[] hosts = new HttpHost[]{new HttpHost("localhost", 9200)}; + Node[] nodes = new Node[] {new Node(new HttpHost("localhost", 9200))}; CloseableHttpAsyncClient closeableHttpAsyncClient = mock(CloseableHttpAsyncClient.class); - RestClient restClient = new RestClient(closeableHttpAsyncClient, 1_000, new Header[0], hosts, null, null); + RestClient restClient = new RestClient(closeableHttpAsyncClient, 1_000, new Header[0], + nodes, null, null); restClient.close(); verify(closeableHttpAsyncClient, times(1)).close(); restClient.close(); @@ -147,8 +156,209 @@ public void testBuildUriLeavesPathUntouched() { } } + public void testSelectHosts() throws IOException { + int iterations = 1000; + Node n1 = new Node(new HttpHost("1"), null, null, "1", null); + Node n2 = new Node(new HttpHost("2"), null, null, "2", null); + Node n3 = new Node(new HttpHost("3"), null, null, "3", null); + List nodes = Arrays.asList(n1, n2, n3); + + NodeSelector not1 = new NodeSelector() { + @Override + public List select(List nodes) { + List result = new ArrayList<>(); + for (Node node : nodes) { + if (false == "1".equals(node.getVersion())) { + result.add(node); + } + } + return result; + } + + @Override + public String toString() { + return "NOT 1"; + } + }; + NodeSelector noNodes = new NodeSelector() { + @Override + public List select(List nodes) { + return Collections.emptyList(); + } + + @Override + public String toString() { + return "NONE"; + } + }; + + NodeTuple> nodeTuple = new NodeTuple<>(nodes, null); + Map blacklist = new HashMap<>(); + AtomicInteger lastNodeIndex = new AtomicInteger(0); + long now = 0; + + // Normal case + List expectedNodes = Arrays.asList(n1, n2, n3); + assertEquals(expectedNodes, RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, NodeSelector.ANY)); + // Calling it again rotates the set of results + for (int i = 0; i < iterations; i++) { + Collections.rotate(expectedNodes, 1); + assertEquals(expectedNodes, RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, NodeSelector.ANY)); + } + + // Exclude some node + lastNodeIndex.set(0); + // h1 excluded + assertEquals(Arrays.asList(n2, n3), RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, not1)); + // Calling it again rotates the set of results + assertEquals(Arrays.asList(n3, n2), RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, not1)); + // And again, same + assertEquals(Arrays.asList(n2, n3), RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, not1)); + /* + * But this time it doesn't because the list being filtered changes + * from (h1, h2, h3) to (h2, h3, h1) which both look the same when + * you filter out h1. + */ + assertEquals(Arrays.asList(n2, n3), RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, not1)); + + /* + * Try a NodeSelector that excludes all nodes. This should + * throw an exception + */ + lastNodeIndex.set(0); + try { + RestClient.selectHosts(nodeTuple, blacklist, lastNodeIndex, now, noNodes); + fail("expected selectHosts to fail"); + } catch (IOException e) { + String message = "NodeSelector [NONE] rejected all nodes, living [" + + "[host=http://1, version=1], [host=http://2, version=2], " + + "[host=http://3, version=3]] and dead []"; + assertEquals(message, e.getMessage()); + } + + /* + * Mark all nodes as dead and look up at a time *after* the + * revival time. This should return all nodes. + */ + blacklist.put(n1.getHost(), new DeadHostState(1, 1)); + blacklist.put(n2.getHost(), new DeadHostState(1, 2)); + blacklist.put(n3.getHost(), new DeadHostState(1, 3)); + lastNodeIndex.set(0); + now = 1000; + expectedNodes = Arrays.asList(n1, n2, n3); + assertEquals(expectedNodes, RestClient.selectHosts(nodeTuple, blacklist, lastNodeIndex, + now, NodeSelector.ANY)); + // Calling it again rotates the set of results + for (int i = 0; i < iterations; i++) { + Collections.rotate(expectedNodes, 1); + assertEquals(expectedNodes, RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, NodeSelector.ANY)); + } + + /* + * Now try with the nodes dead and *not* past their dead time. + * Only the node closest to revival should come back. + */ + now = 0; + assertEquals(singletonList(n1), RestClient.selectHosts(nodeTuple, blacklist, lastNodeIndex, + now, NodeSelector.ANY)); + + /* + * Now try with the nodes dead and *not* past their dead time + * *and* a node selector that removes the node that is closest + * to being revived. The second closest node should come back. + */ + assertEquals(singletonList(n2), RestClient.selectHosts(nodeTuple, blacklist, + lastNodeIndex, now, not1)); + + /* + * Try a NodeSelector that excludes all nodes. This should + * return a failure, but a different failure than normal + * because it'll block revival rather than outright reject + * healthy nodes. + */ + lastNodeIndex.set(0); + try { + RestClient.selectHosts(nodeTuple, blacklist, lastNodeIndex, now, noNodes); + fail("expected selectHosts to fail"); + } catch (IOException e) { + String message = "NodeSelector [NONE] rejected all nodes, living [] and dead [" + + "[host=http://1, version=1], [host=http://2, version=2], " + + "[host=http://3, version=3]]"; + assertEquals(message, e.getMessage()); + } + } + + public void testSetHostsFailures() throws IOException { + RestClient restClient = createRestClient(); + try { + restClient.setHosts((HttpHost[]) null); + fail("setHosts should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("hosts must not be null or empty", e.getMessage()); + } + try { + restClient.setHosts(); + fail("setHosts should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("hosts must not be null or empty", e.getMessage()); + } + try { + restClient.setHosts((HttpHost) null); + fail("setHosts should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("host cannot be null", e.getMessage()); + } + try { + restClient.setHosts(new HttpHost("localhost", 9200), null, new HttpHost("localhost", 9201)); + fail("setHosts should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("host cannot be null", e.getMessage()); + } + } + + public void testSetNodesFailures() throws IOException { + RestClient restClient = createRestClient(); + try { + restClient.setNodes((Node[]) null); + fail("setNodes should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("nodes must not be null or empty", e.getMessage()); + } + try { + restClient.setNodes(); + fail("setNodes should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("nodes must not be null or empty", e.getMessage()); + } + try { + restClient.setNodes((Node) null); + fail("setNodes should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("node cannot be null", e.getMessage()); + } + try { + restClient.setNodes( + new Node(new HttpHost("localhost", 9200)), + null, + new Node(new HttpHost("localhost", 9201))); + fail("setNodes should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("node cannot be null", e.getMessage()); + } + } + private static RestClient createRestClient() { - HttpHost[] hosts = new HttpHost[]{new HttpHost("localhost", 9200)}; - return new RestClient(mock(CloseableHttpAsyncClient.class), randomLongBetween(1_000, 30_000), new Header[]{}, hosts, null, null); + Node[] hosts = new Node[] { + new Node(new HttpHost("localhost", 9200)) + }; + return new RestClient(mock(CloseableHttpAsyncClient.class), randomLongBetween(1_000, 30_000), + new Header[] {}, hosts, null, null); } } diff --git a/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java b/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java index 1bad6b5f6d6fd..bdf977c943588 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java +++ b/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java @@ -28,6 +28,7 @@ import org.apache.http.client.CredentialsProvider; import org.apache.http.client.config.RequestConfig; import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.http.impl.nio.reactor.IOReactorConfig; @@ -36,10 +37,12 @@ import org.apache.http.ssl.SSLContextBuilder; import org.apache.http.ssl.SSLContexts; import org.apache.http.util.EntityUtils; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.client.HttpAsyncResponseConsumerFactory; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseListener; import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientActions; import org.elasticsearch.client.RestClientBuilder; import javax.net.ssl.SSLContext; @@ -253,7 +256,18 @@ public void onFailure(Exception exception) { latch.await(); //end::rest-client-async-example } + } + @SuppressWarnings("unused") + public void testNodeSelector() throws IOException { + // TODO link me to docs + try (RestClient restClient = RestClient.builder( + new HttpHost("localhost", 9200, "http"), + new HttpHost("localhost", 9201, "http")).build()) { + RestClientActions client = restClient.withNodeSelector(NodeSelector.NOT_MASTER_ONLY); + client.performRequest("POST", "/test_index/test_type", Collections.emptyMap(), + new StringEntity("{\"test\":\"test\"}", ContentType.APPLICATION_JSON)); + } } @SuppressWarnings("unused") diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchHostsSniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchHostsSniffer.java index 34a4988358653..5f91191f731bd 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchHostsSniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchHostsSniffer.java @@ -26,22 +26,26 @@ import org.apache.commons.logging.LogFactory; import org.apache.http.HttpEntity; import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.Node.Roles; import java.io.IOException; import java.io.InputStream; import java.net.URI; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.TimeUnit; /** * Class responsible for sniffing the http hosts from elasticsearch through the nodes info api and returning them back. - * Compatible with elasticsearch 5.x and 2.x. + * Compatible with elasticsearch 2.x+. */ public final class ElasticsearchHostsSniffer implements HostsSniffer { @@ -89,18 +93,19 @@ public ElasticsearchHostsSniffer(RestClient restClient, long sniffRequestTimeout /** * Calls the elasticsearch nodes info api, parses the response and returns all the found http hosts */ - public List sniffHosts() throws IOException { + @Override + public List sniffHosts() throws IOException { Response response = restClient.performRequest("get", "/_nodes/http", sniffRequestParams); - return readHosts(response.getEntity()); + return readHosts(response.getEntity(), scheme, jsonFactory); } - private List readHosts(HttpEntity entity) throws IOException { + static List readHosts(HttpEntity entity, Scheme scheme, JsonFactory jsonFactory) throws IOException { try (InputStream inputStream = entity.getContent()) { JsonParser parser = jsonFactory.createParser(inputStream); if (parser.nextToken() != JsonToken.START_OBJECT) { throw new IOException("expected data to start with an object"); } - List hosts = new ArrayList<>(); + List nodes = new ArrayList<>(); while (parser.nextToken() != JsonToken.END_OBJECT) { if (parser.getCurrentToken() == JsonToken.START_OBJECT) { if ("nodes".equals(parser.getCurrentName())) { @@ -108,10 +113,9 @@ private List readHosts(HttpEntity entity) throws IOException { JsonToken token = parser.nextToken(); assert token == JsonToken.START_OBJECT; String nodeId = parser.getCurrentName(); - HttpHost sniffedHost = readHost(nodeId, parser, this.scheme); - if (sniffedHost != null) { - logger.trace("adding node [" + nodeId + "]"); - hosts.add(sniffedHost); + Node node = readHost(nodeId, parser, scheme); + if (node != null) { + nodes.add(node); } } } else { @@ -119,13 +123,31 @@ private List readHosts(HttpEntity entity) throws IOException { } } } - return hosts; + return nodes; } } - private static HttpHost readHost(String nodeId, JsonParser parser, Scheme scheme) throws IOException { - HttpHost httpHost = null; + private static Node readHost(String nodeId, JsonParser parser, Scheme scheme) throws IOException { + HttpHost publishedHost = null; + /* + * We sniff the bound hosts so we can look up the node based on any + * address on which it is listening. This is useful in Elasticsearch's + * test framework where we sometimes publish ipv6 addresses but the + * tests contact the node on ipv4. + */ + Set boundHosts = new HashSet<>(); + String name = null; + String version = null; String fieldName = null; + // Used to read roles from 5.0+ + boolean sawRoles = false; + boolean master = false; + boolean data = false; + boolean ingest = false; + // Used to read roles from 2.x + Boolean masterAttribute = null; + Boolean dataAttribute = null; + boolean clientAttribute = false; while (parser.nextToken() != JsonToken.END_OBJECT) { if (parser.getCurrentToken() == JsonToken.FIELD_NAME) { fieldName = parser.getCurrentName(); @@ -133,9 +155,27 @@ private static HttpHost readHost(String nodeId, JsonParser parser, Scheme scheme if ("http".equals(fieldName)) { while (parser.nextToken() != JsonToken.END_OBJECT) { if (parser.getCurrentToken() == JsonToken.VALUE_STRING && "publish_address".equals(parser.getCurrentName())) { - URI boundAddressAsURI = URI.create(scheme + "://" + parser.getValueAsString()); - httpHost = new HttpHost(boundAddressAsURI.getHost(), boundAddressAsURI.getPort(), - boundAddressAsURI.getScheme()); + URI publishAddressAsURI = URI.create(scheme + "://" + parser.getValueAsString()); + publishedHost = new HttpHost(publishAddressAsURI.getHost(), publishAddressAsURI.getPort(), + publishAddressAsURI.getScheme()); + } else if (parser.currentToken() == JsonToken.START_ARRAY && "bound_address".equals(parser.getCurrentName())) { + while (parser.nextToken() != JsonToken.END_ARRAY) { + URI boundAddressAsURI = URI.create(scheme + "://" + parser.getValueAsString()); + boundHosts.add(new HttpHost(boundAddressAsURI.getHost(), boundAddressAsURI.getPort(), + boundAddressAsURI.getScheme())); + } + } else if (parser.getCurrentToken() == JsonToken.START_OBJECT) { + parser.skipChildren(); + } + } + } else if ("attributes".equals(fieldName)) { + while (parser.nextToken() != JsonToken.END_OBJECT) { + if (parser.getCurrentToken() == JsonToken.VALUE_STRING && "master".equals(parser.getCurrentName())) { + masterAttribute = toBoolean(parser.getValueAsString()); + } else if (parser.getCurrentToken() == JsonToken.VALUE_STRING && "data".equals(parser.getCurrentName())) { + dataAttribute = toBoolean(parser.getValueAsString()); + } else if (parser.getCurrentToken() == JsonToken.VALUE_STRING && "client".equals(parser.getCurrentName())) { + clientAttribute = toBoolean(parser.getValueAsString()); } else if (parser.getCurrentToken() == JsonToken.START_OBJECT) { parser.skipChildren(); } @@ -143,14 +183,55 @@ private static HttpHost readHost(String nodeId, JsonParser parser, Scheme scheme } else { parser.skipChildren(); } + } else if (parser.currentToken() == JsonToken.START_ARRAY) { + if ("roles".equals(fieldName)) { + sawRoles = true; + while (parser.nextToken() != JsonToken.END_ARRAY) { + switch (parser.getText()) { + case "master": + master = true; + break; + case "data": + data = true; + break; + case "ingest": + ingest = true; + break; + default: + logger.warn("unknown role [" + parser.getText() + "] on node [" + nodeId + "]"); + } + } + } else { + parser.skipChildren(); + } + } else if (parser.currentToken().isScalarValue()) { + if ("version".equals(fieldName)) { + version = parser.getText(); + } else if ("name".equals(fieldName)) { + name = parser.getText(); + } } } //http section is not present if http is not enabled on the node, ignore such nodes - if (httpHost == null) { + if (publishedHost == null) { logger.debug("skipping node [" + nodeId + "] with http disabled"); return null; + } else { + logger.trace("adding node [" + nodeId + "]"); + if (version.startsWith("2.")) { + /* + * 2.x doesn't send roles, instead we try to read them from + * attributes. + */ + master = masterAttribute == null ? false == clientAttribute : masterAttribute; + data = dataAttribute == null ? false == clientAttribute : dataAttribute; + } else { + assert sawRoles : "didn't see roles for [" + nodeId + "]"; + } + assert boundHosts.contains(publishedHost) : + "[" + nodeId + "] doesn't make sense! publishedHost should be in boundHosts"; + return new Node(publishedHost, boundHosts, name, version, new Roles(master, data, ingest)); } - return httpHost; } public enum Scheme { @@ -167,4 +248,15 @@ public String toString() { return name; } } + + private static boolean toBoolean(String string) { + switch (string) { + case "true": + return true; + case "false": + return false; + default: + throw new IllegalArgumentException("[" + string + "] is not a valid boolean"); + } + } } diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java index 9eb7b34425944..9de8134208ec1 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java @@ -20,16 +20,19 @@ package org.elasticsearch.client.sniff; import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; import java.io.IOException; import java.util.List; +import java.util.Map; /** - * Responsible for sniffing the http hosts + * Responsible for sniffing the http hosts. */ public interface HostsSniffer { /** - * Returns the sniffed http hosts + * Returns a {@link Map} from sniffed {@link HttpHost} to metadata + * sniffed about the host. */ - List sniffHosts() throws IOException; + List sniffHosts() throws IOException; } diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index c655babd9ed3d..95332a90957ad 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -22,6 +22,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; @@ -29,6 +30,7 @@ import java.io.IOException; import java.security.AccessController; import java.security.PrivilegedAction; +import java.util.Iterator; import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -116,15 +118,19 @@ void sniffOnFailure(HttpHost failedHost) { void sniff(HttpHost excludeHost, long nextSniffDelayMillis) { if (running.compareAndSet(false, true)) { try { - List sniffedHosts = hostsSniffer.sniffHosts(); + final List sniffedHosts = hostsSniffer.sniffHosts(); logger.debug("sniffed hosts: " + sniffedHosts); if (excludeHost != null) { - sniffedHosts.remove(excludeHost); + for (Iterator itr = sniffedHosts.iterator(); itr.hasNext();) { + if (itr.next().getHost().equals(excludeHost)) { + itr.remove(); + } + } } if (sniffedHosts.isEmpty()) { logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); } else { - this.restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); + this.restClient.setNodes(sniffedHosts.toArray(new Node[0])); } } catch (Exception e) { logger.error("error while sniffing nodes", e); diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferParseExampleTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferParseExampleTests.java new file mode 100644 index 0000000000000..41156d9b58863 --- /dev/null +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferParseExampleTests.java @@ -0,0 +1,112 @@ +/* + * 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.client.sniff; + +import org.apache.http.HttpEntity; +import org.apache.http.HttpHost; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.InputStreamEntity; +import org.elasticsearch.client.Node; +import org.elasticsearch.client.RestClientTestCase; +import org.elasticsearch.client.Node.Roles; +import org.elasticsearch.client.sniff.ElasticsearchHostsSniffer.Scheme; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import com.fasterxml.jackson.core.JsonFactory; + +import static org.hamcrest.Matchers.arrayContainingInAnyOrder; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertThat; + +/** + * Test parsing the response from the {@code /_nodes/http} API from fixed + * versions of Elasticsearch. + */ +public class ElasticsearchHostsSnifferParseExampleTests extends RestClientTestCase { + private void checkFile(String file, Node... expected) throws IOException { + InputStream in = Thread.currentThread().getContextClassLoader().getResourceAsStream(file); + if (in == null) { + throw new IllegalArgumentException("Couldn't find [" + file + "]"); + } + try { + HttpEntity entity = new InputStreamEntity(in, ContentType.APPLICATION_JSON); + List nodes = ElasticsearchHostsSniffer.readHosts(entity, Scheme.HTTP, new JsonFactory()); + // Use these assertions because the error messages are nicer than hasItems. + assertThat(nodes, hasSize(expected.length)); + for (Node expectedNode : expected) { + assertThat(nodes, hasItem(expectedNode)); + } + } finally { + in.close(); + } + } + + public void test2x() throws IOException { + checkFile("2.0.0_nodes_http.json", + node(9200, "m1", "2.0.0", true, false, false), + node(9202, "m2", "2.0.0", true, true, false), + node(9201, "m3", "2.0.0", true, false, false), + node(9205, "d1", "2.0.0", false, true, false), + node(9204, "d2", "2.0.0", false, true, false), + node(9203, "d3", "2.0.0", false, true, false), + node(9207, "c1", "2.0.0", false, false, false), + node(9206, "c2", "2.0.0", false, false, false)); + } + + public void test5x() throws IOException { + checkFile("5.0.0_nodes_http.json", + node(9200, "m1", "5.0.0", true, false, true), + node(9201, "m2", "5.0.0", true, true, true), + node(9202, "m3", "5.0.0", true, false, true), + node(9203, "d1", "5.0.0", false, true, true), + node(9204, "d2", "5.0.0", false, true, true), + node(9205, "d3", "5.0.0", false, true, true), + node(9206, "c1", "5.0.0", false, false, true), + node(9207, "c2", "5.0.0", false, false, true)); + } + + public void test6x() throws IOException { + checkFile("6.0.0_nodes_http.json", + node(9200, "m1", "6.0.0", true, false, true), + node(9201, "m2", "6.0.0", true, true, true), + node(9202, "m3", "6.0.0", true, false, true), + node(9203, "d1", "6.0.0", false, true, true), + node(9204, "d2", "6.0.0", false, true, true), + node(9205, "d3", "6.0.0", false, true, true), + node(9206, "c1", "6.0.0", false, false, true), + node(9207, "c2", "6.0.0", false, false, true)); + } + + private Node node(int port, String name, String version, boolean master, boolean data, boolean ingest) { + HttpHost host = new HttpHost("127.0.0.1", port); + Set boundHosts = new HashSet<>(2); + boundHosts.add(host); + boundHosts.add(new HttpHost("[::1]", port)); + return new Node(host, boundHosts, name, version, new Roles(master, data, ingest)); + } +} diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java index 483b7df62f95a..8ce55f4256530 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java @@ -31,6 +31,7 @@ import org.apache.http.HttpHost; import org.apache.http.client.methods.HttpGet; import org.codehaus.mojo.animal_sniffer.IgnoreJRERequirement; +import org.elasticsearch.client.Node; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; @@ -45,10 +46,10 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -117,15 +118,11 @@ public void testSniffNodes() throws IOException { try (RestClient restClient = RestClient.builder(httpHost).build()) { ElasticsearchHostsSniffer sniffer = new ElasticsearchHostsSniffer(restClient, sniffRequestTimeout, scheme); try { - List sniffedHosts = sniffer.sniffHosts(); + List result = sniffer.sniffHosts(); if (sniffResponse.isFailure) { fail("sniffNodes should have failed"); } - assertThat(sniffedHosts.size(), equalTo(sniffResponse.hosts.size())); - Iterator responseHostsIterator = sniffResponse.hosts.iterator(); - for (HttpHost sniffedHost : sniffedHosts) { - assertEquals(sniffedHost, responseHostsIterator.next()); - } + assertEquals(sniffResponse.result, result); } catch(ResponseException e) { Response response = e.getResponse(); if (sniffResponse.isFailure) { @@ -180,7 +177,7 @@ public void handle(HttpExchange httpExchange) throws IOException { private static SniffResponse buildSniffResponse(ElasticsearchHostsSniffer.Scheme scheme) throws IOException { int numNodes = RandomNumbers.randomIntBetween(getRandom(), 1, 5); - List hosts = new ArrayList<>(numNodes); + List nodes = new ArrayList<>(numNodes); JsonFactory jsonFactory = new JsonFactory(); StringWriter writer = new StringWriter(); JsonGenerator generator = jsonFactory.createGenerator(writer); @@ -195,6 +192,23 @@ private static SniffResponse buildSniffResponse(ElasticsearchHostsSniffer.Scheme generator.writeObjectFieldStart("nodes"); for (int i = 0; i < numNodes; i++) { String nodeId = RandomStrings.randomAsciiOfLengthBetween(getRandom(), 5, 10); + String host = "host" + i; + int port = RandomNumbers.randomIntBetween(getRandom(), 9200, 9299); + HttpHost publishHost = new HttpHost(host, port, scheme.toString()); + Set boundHosts = new HashSet<>(); + boundHosts.add(publishHost); + + if (randomBoolean()) { + int bound = between(1, 5); + for (int b = 0; b < bound; b++) { + boundHosts.add(new HttpHost(host + b, port, scheme.toString())); + } + } + + Node node = new Node(publishHost, boundHosts, randomAsciiAlphanumOfLength(5), + randomAsciiAlphanumOfLength(5), + new Node.Roles(randomBoolean(), randomBoolean(), randomBoolean())); + generator.writeObjectFieldStart(nodeId); if (getRandom().nextBoolean()) { generator.writeObjectFieldStart("bogus_object"); @@ -208,44 +222,45 @@ private static SniffResponse buildSniffResponse(ElasticsearchHostsSniffer.Scheme } boolean isHttpEnabled = rarely() == false; if (isHttpEnabled) { - String host = "host" + i; - int port = RandomNumbers.randomIntBetween(getRandom(), 9200, 9299); - HttpHost httpHost = new HttpHost(host, port, scheme.toString()); - hosts.add(httpHost); + nodes.add(node); generator.writeObjectFieldStart("http"); - if (getRandom().nextBoolean()) { - generator.writeArrayFieldStart("bound_address"); - generator.writeString("[fe80::1]:" + port); - generator.writeString("[::1]:" + port); - generator.writeString("127.0.0.1:" + port); - generator.writeEndArray(); + generator.writeArrayFieldStart("bound_address"); + for (HttpHost bound : boundHosts) { + generator.writeString(bound.toHostString()); } + generator.writeEndArray(); if (getRandom().nextBoolean()) { generator.writeObjectFieldStart("bogus_object"); generator.writeEndObject(); } - generator.writeStringField("publish_address", httpHost.toHostString()); + generator.writeStringField("publish_address", publishHost.toHostString()); if (getRandom().nextBoolean()) { generator.writeNumberField("max_content_length_in_bytes", 104857600); } generator.writeEndObject(); } - if (getRandom().nextBoolean()) { - String[] roles = {"master", "data", "ingest"}; - int numRoles = RandomNumbers.randomIntBetween(getRandom(), 0, 3); - Set nodeRoles = new HashSet<>(numRoles); - for (int j = 0; j < numRoles; j++) { - String role; - do { - role = RandomPicks.randomFrom(getRandom(), roles); - } while(nodeRoles.add(role) == false); + + List roles = Arrays.asList(new String[] {"master", "data", "ingest"}); + Collections.shuffle(roles, getRandom()); + generator.writeArrayFieldStart("roles"); + for (String role : roles) { + if ("master".equals(role) && node.getRoles().isMasterEligible()) { + generator.writeString("master"); } - generator.writeArrayFieldStart("roles"); - for (String nodeRole : nodeRoles) { - generator.writeString(nodeRole); + if ("data".equals(role) && node.getRoles().isData()) { + generator.writeString("data"); + } + if ("ingest".equals(role) && node.getRoles().isIngest()) { + generator.writeString("ingest"); } - generator.writeEndArray(); } + generator.writeEndArray(); + + generator.writeFieldName("version"); + generator.writeString(node.getVersion()); + generator.writeFieldName("name"); + generator.writeString(node.getName()); + int numAttributes = RandomNumbers.randomIntBetween(getRandom(), 0, 3); Map attributes = new HashMap<>(numAttributes); for (int j = 0; j < numAttributes; j++) { @@ -265,18 +280,18 @@ private static SniffResponse buildSniffResponse(ElasticsearchHostsSniffer.Scheme generator.writeEndObject(); generator.writeEndObject(); generator.close(); - return SniffResponse.buildResponse(writer.toString(), hosts); + return SniffResponse.buildResponse(writer.toString(), nodes); } private static class SniffResponse { private final String nodesInfoBody; private final int nodesInfoResponseCode; - private final List hosts; + private final List result; private final boolean isFailure; - SniffResponse(String nodesInfoBody, List hosts, boolean isFailure) { + SniffResponse(String nodesInfoBody, List result, boolean isFailure) { this.nodesInfoBody = nodesInfoBody; - this.hosts = hosts; + this.result = result; this.isFailure = isFailure; if (isFailure) { this.nodesInfoResponseCode = randomErrorResponseCode(); @@ -286,11 +301,11 @@ private static class SniffResponse { } static SniffResponse buildFailure() { - return new SniffResponse("", Collections.emptyList(), true); + return new SniffResponse("", null, true); } - static SniffResponse buildResponse(String nodesInfoBody, List hosts) { - return new SniffResponse(nodesInfoBody, hosts, false); + static SniffResponse buildResponse(String nodesInfoBody, List result) { + return new SniffResponse(nodesInfoBody, result, false); } } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java index 5a52151d76e01..bb5b443ff0f9f 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java @@ -20,6 +20,7 @@ package org.elasticsearch.client.sniff; import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; import java.io.IOException; import java.util.Collections; @@ -30,7 +31,9 @@ */ class MockHostsSniffer implements HostsSniffer { @Override - public List sniffHosts() throws IOException { - return Collections.singletonList(new HttpHost("localhost", 9200)); + public List sniffHosts() throws IOException { + return Collections.singletonList(new Node( + new HttpHost("localhost", 9200), Collections.emptySet(), "mock node name", + "mock version", new Node.Roles(false, false, false))); } } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java index 199632d478f81..541f3822f7505 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java @@ -20,6 +20,7 @@ package org.elasticsearch.client.sniff.documentation; import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.sniff.ElasticsearchHostsSniffer; import org.elasticsearch.client.sniff.HostsSniffer; @@ -28,6 +29,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; /** @@ -119,7 +121,7 @@ public void testUsage() throws IOException { .build(); HostsSniffer hostsSniffer = new HostsSniffer() { @Override - public List sniffHosts() throws IOException { + public List sniffHosts() throws IOException { return null; // <1> } }; diff --git a/client/sniffer/src/test/resources/2.0.0_nodes_http.json b/client/sniffer/src/test/resources/2.0.0_nodes_http.json new file mode 100644 index 0000000000000..b370e78e16011 --- /dev/null +++ b/client/sniffer/src/test/resources/2.0.0_nodes_http.json @@ -0,0 +1,141 @@ +{ + "cluster_name" : "elasticsearch", + "nodes" : { + "qYUZ_8bTRwODPxukDlFw6Q" : { + "name" : "d2", + "transport_address" : "127.0.0.1:9304", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9204", + "attributes" : { + "master" : "false" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9204", "[::1]:9204" ], + "publish_address" : "127.0.0.1:9204", + "max_content_length_in_bytes" : 104857600 + } + }, + "Yej5UVNgR2KgBjUFHOQpCw" : { + "name" : "c1", + "transport_address" : "127.0.0.1:9307", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9207", + "attributes" : { + "data" : "false", + "master" : "false" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9207", "[::1]:9207" ], + "publish_address" : "127.0.0.1:9207", + "max_content_length_in_bytes" : 104857600 + } + }, + "mHttJwhwReangKEx9EGuAg" : { + "name" : "m3", + "transport_address" : "127.0.0.1:9301", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9201", + "attributes" : { + "data" : "false", + "master" : "true" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9201", "[::1]:9201" ], + "publish_address" : "127.0.0.1:9201", + "max_content_length_in_bytes" : 104857600 + } + }, + "6Erdptt_QRGLxMiLi9mTkg" : { + "name" : "c2", + "transport_address" : "127.0.0.1:9306", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9206", + "attributes" : { + "data" : "false", + "client" : "true" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9206", "[::1]:9206" ], + "publish_address" : "127.0.0.1:9206", + "max_content_length_in_bytes" : 104857600 + } + }, + "mLRCZBypTiys6e8KY5DMnA" : { + "name" : "m1", + "transport_address" : "127.0.0.1:9300", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9200", + "attributes" : { + "data" : "false" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9200", "[::1]:9200" ], + "publish_address" : "127.0.0.1:9200", + "max_content_length_in_bytes" : 104857600 + } + }, + "pVqOhytXQwetsZVzCBppYw" : { + "name" : "m2", + "transport_address" : "127.0.0.1:9302", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9202", + "http" : { + "bound_address" : [ "127.0.0.1:9202", "[::1]:9202" ], + "publish_address" : "127.0.0.1:9202", + "max_content_length_in_bytes" : 104857600 + } + }, + "ARyzVfpJSw2a9TOIUpbsBA" : { + "name" : "d1", + "transport_address" : "127.0.0.1:9305", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9205", + "attributes" : { + "master" : "false" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9205", "[::1]:9205" ], + "publish_address" : "127.0.0.1:9205", + "max_content_length_in_bytes" : 104857600 + } + }, + "2Hpid-g5Sc2BKCevhN6VQw" : { + "name" : "d3", + "transport_address" : "127.0.0.1:9303", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "2.0.0", + "build" : "de54438", + "http_address" : "127.0.0.1:9203", + "attributes" : { + "master" : "false" + }, + "http" : { + "bound_address" : [ "127.0.0.1:9203", "[::1]:9203" ], + "publish_address" : "127.0.0.1:9203", + "max_content_length_in_bytes" : 104857600 + } + } + } +} diff --git a/client/sniffer/src/test/resources/5.0.0_nodes_http.json b/client/sniffer/src/test/resources/5.0.0_nodes_http.json new file mode 100644 index 0000000000000..7a7d143ecaf43 --- /dev/null +++ b/client/sniffer/src/test/resources/5.0.0_nodes_http.json @@ -0,0 +1,169 @@ +{ + "_nodes" : { + "total" : 8, + "successful" : 8, + "failed" : 0 + }, + "cluster_name" : "test", + "nodes" : { + "DXz_rhcdSF2xJ96qyjaLVw" : { + "name" : "m1", + "transport_address" : "127.0.0.1:9300", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "master", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9200", + "127.0.0.1:9200" + ], + "publish_address" : "127.0.0.1:9200", + "max_content_length_in_bytes" : 104857600 + } + }, + "53Mi6jYdRgeR1cdyuoNfQQ" : { + "name" : "m2", + "transport_address" : "127.0.0.1:9301", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "master", + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9201", + "127.0.0.1:9201" + ], + "publish_address" : "127.0.0.1:9201", + "max_content_length_in_bytes" : 104857600 + } + }, + "XBIghcHiRlWP9c4vY6rETw" : { + "name" : "c2", + "transport_address" : "127.0.0.1:9307", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9207", + "127.0.0.1:9207" + ], + "publish_address" : "127.0.0.1:9207", + "max_content_length_in_bytes" : 104857600 + } + }, + "cFM30FlyS8K1njH_bovwwQ" : { + "name" : "d1", + "transport_address" : "127.0.0.1:9303", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9203", + "127.0.0.1:9203" + ], + "publish_address" : "127.0.0.1:9203", + "max_content_length_in_bytes" : 104857600 + } + }, + "eoVUVRGNRDyyOapqIcrsIA" : { + "name" : "d2", + "transport_address" : "127.0.0.1:9304", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9204", + "127.0.0.1:9204" + ], + "publish_address" : "127.0.0.1:9204", + "max_content_length_in_bytes" : 104857600 + } + }, + "xPN76uDcTP-DyXaRzPg2NQ" : { + "name" : "c1", + "transport_address" : "127.0.0.1:9306", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9206", + "127.0.0.1:9206" + ], + "publish_address" : "127.0.0.1:9206", + "max_content_length_in_bytes" : 104857600 + } + }, + "RY0oW2d7TISEqazk-U4Kcw" : { + "name" : "d3", + "transport_address" : "127.0.0.1:9305", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9205", + "127.0.0.1:9205" + ], + "publish_address" : "127.0.0.1:9205", + "max_content_length_in_bytes" : 104857600 + } + }, + "tU0rXEZmQ9GsWfn2TQ4kow" : { + "name" : "m3", + "transport_address" : "127.0.0.1:9302", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "5.0.0", + "build_hash" : "253032b", + "roles" : [ + "master", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9202", + "127.0.0.1:9202" + ], + "publish_address" : "127.0.0.1:9202", + "max_content_length_in_bytes" : 104857600 + } + } + } +} diff --git a/client/sniffer/src/test/resources/6.0.0_nodes_http.json b/client/sniffer/src/test/resources/6.0.0_nodes_http.json new file mode 100644 index 0000000000000..5a8905da64c89 --- /dev/null +++ b/client/sniffer/src/test/resources/6.0.0_nodes_http.json @@ -0,0 +1,169 @@ +{ + "_nodes" : { + "total" : 8, + "successful" : 8, + "failed" : 0 + }, + "cluster_name" : "test", + "nodes" : { + "FX9npqGQSL2mOGF8Zkf3hw" : { + "name" : "m2", + "transport_address" : "127.0.0.1:9301", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "master", + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9201", + "127.0.0.1:9201" + ], + "publish_address" : "127.0.0.1:9201", + "max_content_length_in_bytes" : 104857600 + } + }, + "jmUqzYLGTbWCg127kve3Tg" : { + "name" : "d1", + "transport_address" : "127.0.0.1:9303", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9203", + "127.0.0.1:9203" + ], + "publish_address" : "127.0.0.1:9203", + "max_content_length_in_bytes" : 104857600 + } + }, + "soBU6bzvTOqdLxPstSbJ2g" : { + "name" : "d3", + "transport_address" : "127.0.0.1:9305", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9205", + "127.0.0.1:9205" + ], + "publish_address" : "127.0.0.1:9205", + "max_content_length_in_bytes" : 104857600 + } + }, + "mtYDAhURTP6twdmNAkMnOg" : { + "name" : "m3", + "transport_address" : "127.0.0.1:9302", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "master", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9202", + "127.0.0.1:9202" + ], + "publish_address" : "127.0.0.1:9202", + "max_content_length_in_bytes" : 104857600 + } + }, + "URxHiUQPROOt1G22Ev6lXw" : { + "name" : "c2", + "transport_address" : "127.0.0.1:9307", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9207", + "127.0.0.1:9207" + ], + "publish_address" : "127.0.0.1:9207", + "max_content_length_in_bytes" : 104857600 + } + }, + "_06S_kWoRqqFR8Z8CS3JRw" : { + "name" : "c1", + "transport_address" : "127.0.0.1:9306", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9206", + "127.0.0.1:9206" + ], + "publish_address" : "127.0.0.1:9206", + "max_content_length_in_bytes" : 104857600 + } + }, + "QZE5Bd6DQJmnfVs2dglOvA" : { + "name" : "d2", + "transport_address" : "127.0.0.1:9304", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "data", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9204", + "127.0.0.1:9204" + ], + "publish_address" : "127.0.0.1:9204", + "max_content_length_in_bytes" : 104857600 + } + }, + "_3mTXg6dSweZn5ReB2fQqw" : { + "name" : "m1", + "transport_address" : "127.0.0.1:9300", + "host" : "127.0.0.1", + "ip" : "127.0.0.1", + "version" : "6.0.0", + "build_hash" : "8f0685b", + "roles" : [ + "master", + "ingest" + ], + "http" : { + "bound_address" : [ + "[::1]:9200", + "127.0.0.1:9200" + ], + "publish_address" : "127.0.0.1:9200", + "max_content_length_in_bytes" : 104857600 + } + } + } +} diff --git a/client/sniffer/src/test/resources/readme.txt b/client/sniffer/src/test/resources/readme.txt new file mode 100644 index 0000000000000..ad13fb3170046 --- /dev/null +++ b/client/sniffer/src/test/resources/readme.txt @@ -0,0 +1,4 @@ +`*_node_http.json` contains files created by spining up toy clusters with a +few nodes in different configurations locally at various versions. They are +for testing `ElasticsearchHostsSniffer` against different versions of +Elasticsearch. diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/README.asciidoc b/rest-api-spec/src/main/resources/rest-api-spec/test/README.asciidoc index c93873a5be429..75135881b8353 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/README.asciidoc +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/README.asciidoc @@ -198,6 +198,25 @@ header. The warnings must match exactly. Using it looks like this: .... +If the arguments to `do` include `node_selector` then the request is only +sent to nodes that match the `node_selector`. Currently only the `version` +selector is supported and it has the same logic as the `version` field in +`skip`. It looks like this: + +.... +"test id": + - skip: + features: node_selector + - do: + node_selector: + version: " - 6.9.99" + index: + index: test-weird-index-中文 + type: weird.type + id: 1 + body: { foo: bar } +.... + === `set` For some tests, it is necessary to extract a value from the previous `response`, in diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/10_with_id.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/10_with_id.yml index daac81849fb5e..056a4aec1dfd4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/10_with_id.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/10_with_id.yml @@ -1,7 +1,11 @@ --- "Index with ID": - + # NOCOMMIT check that everything is wired up. This should fail if run. + - skip: + features: node_selector - do: + node_selector: + version: " - 6.9.99" index: index: test-weird-index-中文 type: weird.type diff --git a/test/framework/build.gradle b/test/framework/build.gradle index 193fcb30988c6..5f1bc524da599 100644 --- a/test/framework/build.gradle +++ b/test/framework/build.gradle @@ -21,6 +21,7 @@ import org.elasticsearch.gradle.precommit.PrecommitTasks; dependencies { compile "org.elasticsearch.client:elasticsearch-rest-client:${version}" + compile "org.elasticsearch.client:elasticsearch-rest-client-sniffer:${version}" compile "org.elasticsearch:elasticsearch-nio:${version}" compile "org.elasticsearch:elasticsearch:${version}" compile "org.elasticsearch:elasticsearch-cli:${version}" diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlDocsTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlDocsTestClient.java index dacd67ccadc32..4ef45ed5db1da 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlDocsTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlDocsTestClient.java @@ -22,9 +22,11 @@ import org.apache.http.HttpEntity; import org.apache.http.HttpHost; import org.elasticsearch.Version; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestSpec; import java.io.IOException; @@ -45,9 +47,9 @@ public ClientYamlDocsTestClient(ClientYamlSuiteRestSpec restSpec, RestClient res super(restSpec, restClient, hosts, esVersion); } - public ClientYamlTestResponse callApi(String apiName, Map params, HttpEntity entity, Map headers) - throws IOException { - + @Override + public ClientYamlTestResponse callApi(String apiName, Map params, HttpEntity entity, + Map headers, NodeSelector nodeSelector) throws IOException { if ("raw".equals(apiName)) { // Raw requests are bit simpler.... Map queryStringParams = new HashMap<>(params); @@ -61,6 +63,6 @@ public ClientYamlTestResponse callApi(String apiName, Map params throw new ClientYamlTestResponseException(e); } } - return super.callApi(apiName, params, entity, headers); + return super.callApi(apiName, params, entity, headers, nodeSelector); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java index f5e834aa90c69..89fc3843a72ec 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java @@ -28,9 +28,11 @@ import org.apache.http.util.EntityUtils; import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestApi; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestPath; @@ -75,8 +77,8 @@ public Version getEsVersion() { /** * Calls an api with the provided parameters and body */ - public ClientYamlTestResponse callApi(String apiName, Map params, HttpEntity entity, Map headers) - throws IOException { + public ClientYamlTestResponse callApi(String apiName, Map params, HttpEntity entity, + Map headers, NodeSelector nodeSelector) throws IOException { ClientYamlSuiteRestApi restApi = restApi(apiName); @@ -170,7 +172,9 @@ public ClientYamlTestResponse callApi(String apiName, Map params logger.debug("calling api [{}]", apiName); try { - Response response = restClient.performRequest(requestMethod, requestPath, queryStringParams, entity, requestHeaders); + Response response = restClient + .withNodeSelector(nodeSelector) + .performRequest(requestMethod, requestPath, queryStringParams, entity, requestHeaders); return new ClientYamlTestResponse(response); } catch(ResponseException e) { throw new ClientYamlTestResponseException(e); diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java index ca04c0c53d12a..1556fa0b88c49 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java @@ -25,6 +25,8 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; +import org.elasticsearch.client.NodeSelector; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -57,9 +59,13 @@ public class ClientYamlTestExecutionContext { private final boolean randomizeContentType; - ClientYamlTestExecutionContext(ClientYamlTestClient clientYamlTestClient, boolean randomizeContentType) { + private final CheckedRunnable setNodeMetadata; + + ClientYamlTestExecutionContext(ClientYamlTestClient clientYamlTestClient, CheckedRunnable setNodeMetadata, + boolean randomizeContentType) { this.clientYamlTestClient = clientYamlTestClient; this.randomizeContentType = randomizeContentType; + this.setNodeMetadata = setNodeMetadata; } /** @@ -68,6 +74,15 @@ public class ClientYamlTestExecutionContext { */ public ClientYamlTestResponse callApi(String apiName, Map params, List> bodies, Map headers) throws IOException { + return callApi(apiName, params, bodies, headers, NodeSelector.ANY); + } + + /** + * Calls an elasticsearch api with the parameters and request body provided as arguments. + * Saves the obtained response in the execution context. + */ + public ClientYamlTestResponse callApi(String apiName, Map params, List> bodies, + Map headers, NodeSelector nodeSelector) throws IOException { //makes a copy of the parameters before modifying them for this specific request Map requestParams = new HashMap<>(params); requestParams.putIfAbsent("error_trace", "true"); // By default ask for error traces, this my be overridden by params @@ -85,9 +100,13 @@ public ClientYamlTestResponse callApi(String apiName, Map params } } + if (nodeSelector != NodeSelector.ANY) { + setNodeMetadata.run(); + } + HttpEntity entity = createEntity(bodies, requestHeaders); try { - response = callApiInternal(apiName, requestParams, entity, requestHeaders); + response = callApiInternal(apiName, requestParams, entity, requestHeaders, nodeSelector); return response; } catch(ClientYamlTestResponseException e) { response = e.getRestTestResponse(); @@ -154,8 +173,8 @@ private BytesRef bodyAsBytesRef(Map bodyAsMap, XContentType xCon // pkg-private for testing ClientYamlTestResponse callApiInternal(String apiName, Map params, - HttpEntity entity, Map headers) throws IOException { - return clientYamlTestClient.callApi(apiName, params, entity, headers); + HttpEntity entity, Map headers, NodeSelector nodeSelector) throws IOException { + return clientYamlTestClient.callApi(apiName, params, entity, headers, nodeSelector); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java index 927f9b46c966a..bc785fbc9748f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java @@ -22,9 +22,11 @@ import com.carrotsearch.randomizedtesting.RandomizedTest; import org.apache.http.HttpHost; import org.elasticsearch.Version; +import org.elasticsearch.client.Node; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.sniff.ElasticsearchHostsSniffer; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.PathUtils; @@ -42,12 +44,15 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; /** * Runs a suite of yaml tests shared with all the official Elasticsearch clients against against an elasticsearch cluster. @@ -120,8 +125,10 @@ public void initAndResetContext() throws Exception { } } ClientYamlTestClient clientYamlTestClient = initClientYamlTestClient(restSpec, restClient, hosts, esVersion); - restTestExecutionContext = new ClientYamlTestExecutionContext(clientYamlTestClient, randomizeContentType()); - adminExecutionContext = new ClientYamlTestExecutionContext(clientYamlTestClient, false); + restTestExecutionContext = new ClientYamlTestExecutionContext(clientYamlTestClient, + () -> sniffHostMetadata(client()), randomizeContentType()); + adminExecutionContext = new ClientYamlTestExecutionContext(clientYamlTestClient, + () -> sniffHostMetadata(adminClient()), false); String[] blacklist = resolvePathsProperty(REST_TESTS_BLACKLIST, null); blacklistPathMatchers = new ArrayList<>(); for (String entry : blacklist) { @@ -354,4 +361,77 @@ private String errorMessage(ExecutableSection executableSection, Throwable t) { protected boolean randomizeContentType() { return true; } + + + /** + * Sniff the cluster for host metadata if it hasn't already been sniffed. This isn't the + * same thing as using the {@link Sniffer} because: + *

    + *
  • It doesn't replace the hosts that that {@link #client} communicates with + *
  • If there is already host metadata it skips running. This behavior isn't + * thread safe but it doesn't have to be for our tests. + *
+ */ + private void sniffHostMetadata(RestClient client) throws IOException { + List nodes = client.getNodes(); + boolean allHaveRoles = true; + for (Node node : nodes) { + if (node.getRoles() == null) { + allHaveRoles = false; + break; + } + } + if (allHaveRoles) { + // We already have resolved metadata. + return; + } + // No resolver, sniff one time and resolve metadata against the results + ElasticsearchHostsSniffer.Scheme scheme = + ElasticsearchHostsSniffer.Scheme.valueOf(getProtocol().toUpperCase(Locale.ROOT)); + /* + * We don't want to change the list of nodes that the client communicates with + * because that'd just be rude. So instead we replace the nodes find the nodes + * returned by the sniffer that correspond with the nodes already the client + * and set the nodes to them. That *shouldn't* change the nodes that the client + * communicates with. + */ + ElasticsearchHostsSniffer sniffer = new ElasticsearchHostsSniffer( + adminClient(), ElasticsearchHostsSniffer.DEFAULT_SNIFF_REQUEST_TIMEOUT, scheme); + attachSniffedMetadataOnClient(client, nodes, sniffer.sniffHosts()); + } + + static void attachSniffedMetadataOnClient(RestClient client, List originalNodes, List nodesWithMetadata) { + Set originalHosts = originalNodes.stream() + .map(Node::getHost) + .collect(Collectors.toSet()); + List sniffed = new ArrayList<>(); + for (Node node : nodesWithMetadata) { + /* + * getHost is the publish_address of the node which, sometimes, is + * ipv6 and, sometimes, our original address for the node is ipv4. + * In that case the ipv4 address should be in getBoundHosts. If it + * isn't then we'll end up without the right number of hosts which + * will fail down below with a pretty error message. + */ + if (originalHosts.contains(node.getHost())) { + sniffed.add(node); + } else { + for (HttpHost bound : node.getBoundHosts()) { + if (originalHosts.contains(bound)) { + sniffed.add(node.withHost(bound)); + break; + } + } + } + } + int missing = originalNodes.size() - sniffed.size(); + if (missing > 0) { + List hosts = originalNodes.stream() + .map(Node::getHost) + .collect(Collectors.toList()); + throw new IllegalStateException("Didn't sniff metadata for all nodes. Wanted metadata for " + + hosts + " but got " + sniffed); + } + client.setNodes(sniffed.toArray(new Node[0])); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java index ab9be65514a96..31fa59857cfe2 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java @@ -39,6 +39,7 @@ public final class Features { "catch_unauthorized", "embedded_stash_key", "headers", + "node_selector", "stash_in_key", "stash_in_path", "stash_path_replace", diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/package-info.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/package-info.java deleted file mode 100644 index de63b46eff313..0000000000000 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/parser/package-info.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/** - * Parses YAML test {@link org.elasticsearch.test.rest.yaml.section.ClientYamlTestSuite}s containing - * {@link org.elasticsearch.test.rest.yaml.section.ClientYamlTestSection}s. - */ -package org.elasticsearch.test.rest.yaml.parser; diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ApiCallSection.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ApiCallSection.java index 4553845458541..de73fefaea776 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ApiCallSection.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ApiCallSection.java @@ -24,6 +24,8 @@ import java.util.List; import java.util.Map; +import org.elasticsearch.client.NodeSelector; + import static java.util.Collections.unmodifiableMap; /** @@ -35,6 +37,7 @@ public class ApiCallSection { private final Map params = new HashMap<>(); private final Map headers = new HashMap<>(); private final List> bodies = new ArrayList<>(); + private NodeSelector nodeSelector = NodeSelector.ANY; public ApiCallSection(String api) { this.api = api; @@ -76,4 +79,18 @@ public void addBody(Map body) { public boolean hasBody() { return bodies.size() > 0; } + + /** + * Selects the node on which to run this request. + */ + public NodeSelector getNodeSelector() { + return nodeSelector; + } + + /** + * Set the selector that decides which node can run this request. + */ + public void setNodeSelector(NodeSelector nodeSelector) { + this.nodeSelector = nodeSelector; + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSection.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSection.java index 321d22ed70aa7..1ec2382fac596 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSection.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSection.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.test.rest.yaml.section; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.common.xcontent.XContentParser; @@ -91,6 +92,12 @@ public void addExecutableSection(ExecutableSection executableSection) { + "runners that do not support the [warnings] section can skip the test at line [" + doSection.getLocation().lineNumber + "]"); } + if (NodeSelector.ANY != doSection.getApiCallSection().getNodeSelector() + && false == skipSection.getFeatures().contains("node_selector")) { + throw new IllegalArgumentException("Attempted to add a [do] with a [node_selector] section without a corresponding " + + "[skip] so runners that do not support the [node_selector] section can skip the test at line [" + + doSection.getLocation().lineNumber + "]"); + } } this.executableSections.add(executableSection); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/DoSection.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/DoSection.java index 7c6647d65f044..7c7147434e3f4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/DoSection.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/DoSection.java @@ -19,7 +19,11 @@ package org.elasticsearch.test.rest.yaml.section; +import org.apache.http.HttpHost; import org.apache.logging.log4j.Logger; +import org.elasticsearch.Version; +import org.elasticsearch.client.Node; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; @@ -84,6 +88,7 @@ public static DoSection parse(XContentParser parser) throws IOException { DoSection doSection = new DoSection(parser.getTokenLocation()); ApiCallSection apiCallSection = null; + NodeSelector nodeSelector = NodeSelector.ANY; Map headers = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); List expectedWarnings = new ArrayList<>(); @@ -120,6 +125,17 @@ public static DoSection parse(XContentParser parser) throws IOException { headers.put(headerName, parser.text()); } } + } else if ("node_selector".equals(currentFieldName)) { + String selectorName = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + selectorName = parser.currentName(); + } else if (token.isValue()) { + NodeSelector newSelector = buildNodeSelector( + parser.getTokenLocation(), selectorName, parser.text()); + nodeSelector = new NodeSelector.Compose(nodeSelector, newSelector); + } + } } else if (currentFieldName != null) { // must be part of API call then apiCallSection = new ApiCallSection(currentFieldName); String paramName = null; @@ -152,6 +168,7 @@ public static DoSection parse(XContentParser parser) throws IOException { throw new IllegalArgumentException("client call section is mandatory within a do section"); } apiCallSection.addHeaders(headers); + apiCallSection.setNodeSelector(nodeSelector); doSection.setApiCallSection(apiCallSection); doSection.setExpectedWarningHeaders(unmodifiableList(expectedWarnings)); } finally { @@ -160,7 +177,6 @@ public static DoSection parse(XContentParser parser) throws IOException { return doSection; } - private static final Logger logger = Loggers.getLogger(DoSection.class); private final XContentLocation location; @@ -221,7 +237,7 @@ public void execute(ClientYamlTestExecutionContext executionContext) throws IOEx try { ClientYamlTestResponse response = executionContext.callApi(apiCallSection.getApi(), apiCallSection.getParams(), - apiCallSection.getBodies(), apiCallSection.getHeaders()); + apiCallSection.getBodies(), apiCallSection.getHeaders(), apiCallSection.getNodeSelector()); if (Strings.hasLength(catchParam)) { String catchStatusCode; if (catches.containsKey(catchParam)) { @@ -337,4 +353,35 @@ private String formatStatusCodeMessage(ClientYamlTestResponse restTestResponse, not(equalTo(408)), not(equalTo(409))))); } + + private static NodeSelector buildNodeSelector(XContentLocation location, String name, String value) { + switch (name) { + case "version": + Version[] range = SkipSection.parseVersionRange(value); + return new NodeSelector() { + @Override + public List select(List nodes) { + List result = new ArrayList<>(nodes.size()); + for (Node node : nodes) { + if (node.getVersion() == null) { + throw new IllegalStateException("expected [version] metadata to be set but got " + + node); + } + Version version = Version.fromString(node.getVersion()); + if (version.onOrAfter(range[0]) && version.onOrBefore(range[1])) { + result.add(node); + } + } + return result; + } + + @Override + public String toString() { + return "version between [" + range[0] + "] and [" + range[1] + "]"; + } + }; + default: + throw new IllegalArgumentException("unknown node_selector [" + name + "]"); + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/SkipSection.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/SkipSection.java index eb1fea4b79aed..e487f8e74da3b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/SkipSection.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/SkipSection.java @@ -153,7 +153,7 @@ public boolean isEmpty() { return EMPTY.equals(this); } - private Version[] parseVersionRange(String versionRange) { + static Version[] parseVersionRange(String versionRange) { if (versionRange == null) { return new Version[] { null, null }; } diff --git a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContextTests.java b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContextTests.java index 2150baf59eab0..c54847088eb7a 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContextTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContextTests.java @@ -20,24 +20,28 @@ package org.elasticsearch.test.rest.yaml; import org.apache.http.HttpEntity; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.test.ESTestCase; import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; + public class ClientYamlTestExecutionContextTests extends ESTestCase { public void testHeadersSupportStashedValueReplacement() throws IOException { final AtomicReference> headersRef = new AtomicReference<>(); final ClientYamlTestExecutionContext context = - new ClientYamlTestExecutionContext(null, randomBoolean()) { + new ClientYamlTestExecutionContext(null, () -> {}, randomBoolean()) { @Override ClientYamlTestResponse callApiInternal(String apiName, Map params, - HttpEntity entity, - Map headers) { + HttpEntity entity, Map headers, NodeSelector nodeSelector) { headersRef.set(headers); return null; } @@ -57,4 +61,20 @@ ClientYamlTestResponse callApiInternal(String apiName, Map param assertEquals("foo2", headersRef.get().get("foo")); assertEquals("baz bar1", headersRef.get().get("foo1")); } + + public void testNonDefaultNodeSelectorSetsNodeMetadata() throws IOException { + AtomicBoolean setHostMetadata = new AtomicBoolean(false); + final ClientYamlTestExecutionContext context = + new ClientYamlTestExecutionContext(null, () -> setHostMetadata.set(true), randomBoolean()) { + @Override + ClientYamlTestResponse callApiInternal(String apiName, Map params, + HttpEntity entity, Map headers, NodeSelector nodeSelector) { + return null; + } + }; + context.callApi(randomAlphaOfLength(2), emptyMap(), emptyList(), emptyMap(), NodeSelector.ANY); + assertFalse(setHostMetadata.get()); + context.callApi(randomAlphaOfLength(2), emptyMap(), emptyList(), emptyMap(), NodeSelector.NOT_MASTER_ONLY); + assertTrue(setHostMetadata.get()); + } } diff --git a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCaseTests.java b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCaseTests.java index ae64dbc893d81..b40b0048a538a 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCaseTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCaseTests.java @@ -20,14 +20,26 @@ import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; +import org.elasticsearch.client.RestClient; import org.elasticsearch.test.ESTestCase; +import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.Matchers.greaterThan; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; public class ESClientYamlSuiteTestCaseTests extends ESTestCase { @@ -91,4 +103,56 @@ private static void assertSingleFile(Set files, String dirName, String fil assertThat(file.getFileName().toString(), equalTo(fileName)); assertThat(file.toAbsolutePath().getParent().getFileName().toString(), equalTo(dirName)); } + + public void testAttachSniffedMetadataOnClientOk() { + RestClient client = mock(RestClient.class); + List originalNodes = Arrays.asList( + new Node(new HttpHost("1")), + new Node(new HttpHost("2")), + new Node(new HttpHost("3"))); + List nodesWithMetadata = Arrays.asList(new Node[] { + // This node matches exactly: + new Node(new HttpHost("1"), emptySet(), randomAlphaOfLength(5), + randomAlphaOfLength(5), randomRoles()), + // This node also matches exactly but has bound hosts which don't matter: + new Node(new HttpHost("2"), + new HashSet<>(Arrays.asList(new HttpHost("2"), new HttpHost("not2"))), + randomAlphaOfLength(5), randomAlphaOfLength(5), randomRoles()), + // This node's host doesn't match but one of its published hosts does so + // we return a modified version of it: + new Node(new HttpHost("not3"), + new HashSet<>(Arrays.asList(new HttpHost("not3"), new HttpHost("3"))), + randomAlphaOfLength(5), randomAlphaOfLength(5), randomRoles()), + // This node isn't in the original list so it isn't added: + new Node(new HttpHost("4"), emptySet(), randomAlphaOfLength(5), + randomAlphaOfLength(5), randomRoles()), + }); + ESClientYamlSuiteTestCase.attachSniffedMetadataOnClient(client, originalNodes, nodesWithMetadata); + verify(client).setNodes(new Node[] { + nodesWithMetadata.get(0), + nodesWithMetadata.get(1), + nodesWithMetadata.get(2).withHost(new HttpHost("3")), + }); + } + + public void testAttachSniffedMetadataOnClientNotEnoughNodes() { + // Try a version of the call that should fail because it doesn't have all the results + RestClient client = mock(RestClient.class); + List originalNodes = Arrays.asList( + new Node(new HttpHost("1")), + new Node(new HttpHost("2"))); + List nodesWithMetadata = Arrays.asList(new Node[] { + // This node matches exactly: + new Node(new HttpHost("1"), emptySet(), "n", "v", new Node.Roles(true, true, true)), + }); + IllegalStateException e = expectThrows(IllegalStateException.class, () -> + ESClientYamlSuiteTestCase.attachSniffedMetadataOnClient(client, originalNodes, nodesWithMetadata)); + assertEquals(e.getMessage(), "Didn't sniff metadata for all nodes. Wanted metadata for " + + "[http://1, http://2] but got [[host=http://1, bound=[], name=n, version=v, roles=mdi]]"); + verify(client, never()).setNodes(any(Node[].class)); + } + + private Node.Roles randomRoles() { + return new Node.Roles(randomBoolean(), randomBoolean(), randomBoolean()); + } } diff --git a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSectionTests.java b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSectionTests.java index ecee131c7a28e..87f2d7f9a53f8 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSectionTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSectionTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.test.rest.yaml.section; import org.elasticsearch.Version; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.common.xcontent.XContentParser; @@ -35,11 +36,12 @@ import static org.hamcrest.Matchers.nullValue; public class ClientYamlTestSectionTests extends AbstractClientYamlTestFragmentParserTestCase { - public void testAddingDoWithoutWarningWithoutSkip() { + public void testAddingDoWithoutSkips() { int lineNumber = between(1, 10000); ClientYamlTestSection section = new ClientYamlTestSection(new XContentLocation(0, 0), "test"); section.setSkipSection(SkipSection.EMPTY); DoSection doSection = new DoSection(new XContentLocation(lineNumber, 0)); + doSection.setApiCallSection(new ApiCallSection("test")); section.addExecutableSection(doSection); } @@ -49,6 +51,7 @@ public void testAddingDoWithWarningWithSkip() { section.setSkipSection(new SkipSection(null, singletonList("warnings"), null)); DoSection doSection = new DoSection(new XContentLocation(lineNumber, 0)); doSection.setExpectedWarningHeaders(singletonList("foo")); + doSection.setApiCallSection(new ApiCallSection("test")); section.addExecutableSection(doSection); } @@ -58,11 +61,37 @@ public void testAddingDoWithWarningWithSkipButNotWarnings() { section.setSkipSection(new SkipSection(null, singletonList("yaml"), null)); DoSection doSection = new DoSection(new XContentLocation(lineNumber, 0)); doSection.setExpectedWarningHeaders(singletonList("foo")); + doSection.setApiCallSection(new ApiCallSection("test")); Exception e = expectThrows(IllegalArgumentException.class, () -> section.addExecutableSection(doSection)); assertEquals("Attempted to add a [do] with a [warnings] section without a corresponding [skip] so runners that do not support the" + " [warnings] section can skip the test at line [" + lineNumber + "]", e.getMessage()); } + public void testAddingDoWithNodeSelectorWithSkip() { + int lineNumber = between(1, 10000); + ClientYamlTestSection section = new ClientYamlTestSection(new XContentLocation(0, 0), "test"); + section.setSkipSection(new SkipSection(null, singletonList("node_selector"), null)); + DoSection doSection = new DoSection(new XContentLocation(lineNumber, 0)); + ApiCallSection apiCall = new ApiCallSection("test"); + apiCall.setNodeSelector(NodeSelector.NOT_MASTER_ONLY); + doSection.setApiCallSection(apiCall); + section.addExecutableSection(doSection); + } + + public void testAddingDoWithNodeSelectorWithSkipButNotWarnings() { + int lineNumber = between(1, 10000); + ClientYamlTestSection section = new ClientYamlTestSection(new XContentLocation(0, 0), "test"); + section.setSkipSection(new SkipSection(null, singletonList("yaml"), null)); + DoSection doSection = new DoSection(new XContentLocation(lineNumber, 0)); + ApiCallSection apiCall = new ApiCallSection("test"); + apiCall.setNodeSelector(NodeSelector.NOT_MASTER_ONLY); + doSection.setApiCallSection(apiCall); + Exception e = expectThrows(IllegalArgumentException.class, () -> section.addExecutableSection(doSection)); + assertEquals("Attempted to add a [do] with a [node_selector] section without a corresponding" + + " [skip] so runners that do not support the [node_selector] section can skip the test at" + + " line [" + lineNumber + "]", e.getMessage()); + } + public void testWrongIndentation() throws Exception { { XContentParser parser = createParser(YamlXContent.yamlXContent, diff --git a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/DoSectionTests.java b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/DoSectionTests.java index 982eac4b80274..24731d2d52d5a 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/DoSectionTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/rest/yaml/section/DoSectionTests.java @@ -19,23 +19,34 @@ package org.elasticsearch.test.rest.yaml.section; +import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; +import org.elasticsearch.client.NodeSelector; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.yaml.YamlXContent; +import org.elasticsearch.test.rest.yaml.ClientYamlTestExecutionContext; +import org.elasticsearch.test.rest.yaml.ClientYamlTestResponse; import org.hamcrest.MatcherAssert; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class DoSectionTests extends AbstractClientYamlTestFragmentParserTestCase { @@ -496,7 +507,35 @@ public void testParseDoSectionExpectedWarnings() throws Exception { assertThat(doSection.getApiCallSection(), notNullValue()); assertThat(doSection.getExpectedWarningHeaders(), equalTo(singletonList( "just one entry this time"))); + } + + public void testNodeSelector() throws IOException { + parser = createParser(YamlXContent.yamlXContent, + "node_selector:\n" + + " version: 5.2.0-6.0.0\n" + + "indices.get_field_mapping:\n" + + " index: test_index" + ); + + DoSection doSection = DoSection.parse(parser); + assertNotSame(NodeSelector.ANY, doSection.getApiCallSection().getNodeSelector()); + Node v170 = nodeWithVersion("1.7.0"); + Node v521 = nodeWithVersion("5.2.1"); + Node v550 = nodeWithVersion("5.5.0"); + Node v612 = nodeWithVersion("6.1.2"); + assertEquals(Arrays.asList(v521, v550), doSection.getApiCallSection().getNodeSelector() + .select(Arrays.asList(v170, v521, v550, v612))); + ClientYamlTestExecutionContext context = mock(ClientYamlTestExecutionContext.class); + ClientYamlTestResponse mockResponse = mock(ClientYamlTestResponse.class); + when(context.callApi("indices.get_field_mapping", singletonMap("index", "test_index"), + emptyList(), emptyMap(), doSection.getApiCallSection().getNodeSelector())).thenReturn(mockResponse); + doSection.execute(context); + verify(context).callApi("indices.get_field_mapping", singletonMap("index", "test_index"), + emptyList(), emptyMap(), doSection.getApiCallSection().getNodeSelector()); + } + private Node nodeWithVersion(String version) { + return new Node(new HttpHost("dummy"), null, null, version, null); } private void assertJsonEquals(Map actual, String expected) throws IOException {