> getAttributes() {
+ return attributes;
+ }
+
+ @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);
+ }
+ if (attributes != null) {
+ b.append(", attributes=").append(attributes);
+ }
+ 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(name, other.name)
+ && Objects.equals(version, other.version)
+ && Objects.equals(roles, other.roles)
+ && Objects.equals(attributes, other.attributes);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(host, boundHosts, name, version, roles, attributes);
+ }
+
+ /**
+ * 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;
+ }
+
+ /**
+ * Teturns whether or not the node could be elected master.
+ */
+ public boolean isMasterEligible() {
+ return masterEligible;
+ }
+ /**
+ * Teturns whether or not the node stores data.
+ */
+ public boolean isData() {
+ return data;
+ }
+ /**
+ * Teturns whether or not 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..5f5296fe16b13
--- /dev/null
+++ b/client/rest/src/main/java/org/elasticsearch/client/NodeSelector.java
@@ -0,0 +1,90 @@
+/*
+ * 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.Iterator;
+
+/**
+ * Selects nodes that can receive requests. Used to keep requests away
+ * from master nodes or to send them to nodes with a particular attribute.
+ * Use with {@link RequestOptions.Builder#setNodeSelector(NodeSelector)}.
+ */
+public interface NodeSelector {
+ /**
+ * Select the {@link Node}s to which to send requests. This is called with
+ * a mutable {@link Iterable} of {@linkplain Node}s in the order that the
+ * rest client would prefer to use them and implementers should remove
+ * nodes from the that should not receive the request. Implementers may
+ * iterate the nodes as many times as they need.
+ *
+ * This may be called twice per request: first for "living" nodes that
+ * have not been blacklisted by previous errors. If the selector removes
+ * all nodes from the list or if there aren't any living nodes then the
+ * {@link RestClient} will call this method with a list of "dead" nodes.
+ *
+ * Implementers should not rely on the ordering of the nodes.
+ */
+ void select(Iterable nodes);
+ /*
+ * We were fairly careful with our choice of Iterable here. The caller has
+ * a List but reordering the list is likely to break round robin. Luckily
+ * Iterable doesn't allow any reordering.
+ */
+
+ /**
+ * Selector that matches any node.
+ */
+ NodeSelector ANY = new NodeSelector() {
+ @Override
+ public void select(Iterable nodes) {
+ // Intentionally does nothing
+ }
+
+ @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.
+ */
+ NodeSelector NOT_MASTER_ONLY = new NodeSelector() {
+ @Override
+ public void select(Iterable nodes) {
+ for (Iterator itr = nodes.iterator(); itr.hasNext();) {
+ Node node = itr.next();
+ if (node.getRoles() == null) continue;
+ if (node.getRoles().isMasterEligible()
+ && false == node.getRoles().isData()
+ && false == node.getRoles().isIngest()) {
+ itr.remove();
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "NOT_MASTER_ONLY";
+ }
+ };
+}
diff --git a/client/rest/src/main/java/org/elasticsearch/client/RequestLogger.java b/client/rest/src/main/java/org/elasticsearch/client/RequestLogger.java
index 07ff89b7e3fb0..7c56a7edf97a9 100644
--- a/client/rest/src/main/java/org/elasticsearch/client/RequestLogger.java
+++ b/client/rest/src/main/java/org/elasticsearch/client/RequestLogger.java
@@ -87,14 +87,14 @@ static void logResponse(Log logger, HttpUriRequest request, HttpHost host, HttpR
/**
* Logs a request that failed
*/
- static void logFailedRequest(Log logger, HttpUriRequest request, HttpHost host, Exception e) {
+ static void logFailedRequest(Log logger, HttpUriRequest request, Node node, Exception e) {
if (logger.isDebugEnabled()) {
- logger.debug("request [" + request.getMethod() + " " + host + getUri(request.getRequestLine()) + "] failed", e);
+ logger.debug("request [" + request.getMethod() + " " + node.getHost() + getUri(request.getRequestLine()) + "] failed", e);
}
if (tracer.isTraceEnabled()) {
String traceRequest;
try {
- traceRequest = buildTraceRequest(request, host);
+ traceRequest = buildTraceRequest(request, node.getHost());
} catch (IOException e1) {
tracer.trace("error while reading request for trace purposes", e);
traceRequest = "";
diff --git a/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java b/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java
index e31db17a336b0..97d150da3d3ff 100644
--- a/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java
+++ b/client/rest/src/main/java/org/elasticsearch/client/RequestOptions.java
@@ -37,18 +37,21 @@
*/
public final class RequestOptions {
public static final RequestOptions DEFAULT = new Builder(
- Collections.emptyList(), HeapBufferedResponseConsumerFactory.DEFAULT).build();
+ Collections.emptyList(), NodeSelector.ANY,
+ HeapBufferedResponseConsumerFactory.DEFAULT).build();
private final List headers;
+ private final NodeSelector nodeSelector;
private final HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory;
private RequestOptions(Builder builder) {
this.headers = Collections.unmodifiableList(new ArrayList<>(builder.headers));
+ this.nodeSelector = builder.nodeSelector;
this.httpAsyncResponseConsumerFactory = builder.httpAsyncResponseConsumerFactory;
}
public Builder toBuilder() {
- Builder builder = new Builder(headers, httpAsyncResponseConsumerFactory);
+ Builder builder = new Builder(headers, nodeSelector, httpAsyncResponseConsumerFactory);
return builder;
}
@@ -59,6 +62,14 @@ public List getHeaders() {
return headers;
}
+ /**
+ * The selector that chooses which nodes are valid destinations for
+ * {@link Request}s with these options.
+ */
+ public NodeSelector getNodeSelector() {
+ return nodeSelector;
+ }
+
/**
* The {@link HttpAsyncResponseConsumerFactory} used to create one
* {@link HttpAsyncResponseConsumer} callback per retry. Controls how the
@@ -82,6 +93,9 @@ public String toString() {
b.append(headers.get(h).toString());
}
}
+ if (nodeSelector != NodeSelector.ANY) {
+ b.append(", nodeSelector=").append(nodeSelector);
+ }
if (httpAsyncResponseConsumerFactory != HttpAsyncResponseConsumerFactory.DEFAULT) {
b.append(", consumerFactory=").append(httpAsyncResponseConsumerFactory);
}
@@ -99,20 +113,24 @@ public boolean equals(Object obj) {
RequestOptions other = (RequestOptions) obj;
return headers.equals(other.headers)
+ && nodeSelector.equals(other.nodeSelector)
&& httpAsyncResponseConsumerFactory.equals(other.httpAsyncResponseConsumerFactory);
}
@Override
public int hashCode() {
- return Objects.hash(headers, httpAsyncResponseConsumerFactory);
+ return Objects.hash(headers, nodeSelector, httpAsyncResponseConsumerFactory);
}
public static class Builder {
private final List headers;
+ private NodeSelector nodeSelector;
private HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory;
- private Builder(List headers, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory) {
+ private Builder(List headers, NodeSelector nodeSelector,
+ HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory) {
this.headers = new ArrayList<>(headers);
+ this.nodeSelector = nodeSelector;
this.httpAsyncResponseConsumerFactory = httpAsyncResponseConsumerFactory;
}
@@ -133,7 +151,15 @@ public void addHeader(String name, String value) {
}
/**
- * set the {@link HttpAsyncResponseConsumerFactory} used to create one
+ * Configure the selector that chooses which nodes are valid
+ * destinations for {@link Request}s with these options
+ */
+ public void setNodeSelector(NodeSelector nodeSelector) {
+ this.nodeSelector = Objects.requireNonNull(nodeSelector, "nodeSelector cannot be null");
+ }
+
+ /**
+ * Set 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.
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 0e603c4069ae4..82039cab5d04c 100644
--- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java
+++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java
@@ -46,10 +46,11 @@
import org.apache.http.nio.client.methods.HttpAsyncMethods;
import org.apache.http.nio.protocol.HttpAsyncRequestProducer;
import org.apache.http.nio.protocol.HttpAsyncResponseConsumer;
+import org.elasticsearch.client.DeadHostState.TimeSupplier;
-import javax.net.ssl.SSLHandshakeException;
import java.io.Closeable;
import java.io.IOException;
+import java.net.ConnectException;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.net.URISyntaxException;
@@ -57,11 +58,10 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
-import java.util.LinkedHashSet;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@@ -74,13 +74,16 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLHandshakeException;
+
+import static java.util.Collections.singletonList;
/**
* Client that connects to an Elasticsearch cluster through HTTP.
*
* Must be created using {@link RestClientBuilder}, which allows to set all the different options or just rely on defaults.
* The hosts that are part of the cluster need to be provided at creation time, but can also be replaced later
- * by calling {@link #setHosts(HttpHost...)}.
+ * by calling {@link #setNodes(Collection)}.
*
* The method {@link #performRequest(String, String, Map, HttpEntity, Header...)} allows to send a request to the cluster. When
* sending a request, a host gets selected out of the provided ones in a round-robin fashion. Failing hosts are marked dead and
@@ -102,53 +105,93 @@ 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) {
+ List 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.
+ *
+ * Prefer this to {@link #builder(HttpHost...)} if you have metadata up front about the nodes.
+ * If you don't either one is fine.
+ */
+ public static RestClientBuilder builder(Node... nodes) {
+ return new RestClientBuilder(nodes == null ? null : Arrays.asList(nodes));
+ }
+
+ /**
+ * 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.
+ *
+ * You can use this if you do not have metadata up front about the nodes. If you do, prefer
+ * {@link #builder(Node...)}.
+ * @see Node#Node(HttpHost)
*/
public static RestClientBuilder builder(HttpHost... hosts) {
- return new RestClientBuilder(hosts);
+ return new RestClientBuilder(hostsToNodes(hosts));
}
/**
- * Replaces the hosts that the client communicates with.
- * @see HttpHost
+ * Replaces the hosts with which the client communicates.
+ *
+ * @deprecated prefer {@link setNodes} because it allows you
+ * to set metadata for use with {@link NodeSelector}s
*/
- public synchronized void setHosts(HttpHost... hosts) {
- if (hosts == null || hosts.length == 0) {
- throw new IllegalArgumentException("hosts must not be null nor empty");
+ @Deprecated
+ public void setHosts(HttpHost... hosts) {
+ setNodes(hostsToNodes(hosts));
+ }
+
+ /**
+ * Replaces the nodes with which the client communicates.
+ */
+ public synchronized void setNodes(Collection nodes) {
+ if (nodes == null || nodes.isEmpty()) {
+ throw new IllegalArgumentException("nodes must not be null or empty");
}
- Set httpHosts = new LinkedHashSet<>();
AuthCache authCache = new BasicAuthCache();
- for (HttpHost host : hosts) {
- Objects.requireNonNull(host, "host cannot be null");
- httpHosts.add(host);
- authCache.put(host, new BasicScheme());
+
+ Map nodesByHost = new LinkedHashMap<>();
+ for (Node node : nodes) {
+ Objects.requireNonNull(node, "node cannot be null");
+ // TODO should we throw an IAE if we have two nodes with the same host?
+ nodesByHost.put(node.getHost(), node);
+ authCache.put(node.getHost(), new BasicScheme());
}
- this.hostTuple = new HostTuple<>(Collections.unmodifiableSet(httpHosts), authCache);
+ this.nodeTuple = new NodeTuple<>(
+ Collections.unmodifiableList(new ArrayList<>(nodesByHost.values())), authCache);
this.blacklist.clear();
}
+ private static List hostsToNodes(HttpHost[] hosts) {
+ if (hosts == null || hosts.length == 0) {
+ throw new IllegalArgumentException("hosts must not be null nor empty");
+ }
+ List nodes = new ArrayList<>(hosts.length);
+ for (int i = 0; i < hosts.length; i++) {
+ nodes.add(new Node(hosts[i]));
+ }
+ return nodes;
+ }
+
/**
- * Returns the configured hosts
+ * Get the list of nodes that the client knows about. The list is
+ * unmodifiable.
*/
- public List getHosts() {
- return new ArrayList<>(hostTuple.hosts);
+ public List getNodes() {
+ return nodeTuple.nodes;
}
/**
@@ -434,7 +477,7 @@ public void performRequestAsync(String method, String endpoint, Map requestParams = new HashMap<>(request.getParameters());
//ignore is a special parameter supported by the clients, shouldn't be sent to es
String ignoreString = requestParams.remove("ignore");
@@ -466,40 +509,40 @@ void performRequestAsyncNoCatch(Request request, ResponseListener listener) {
setHeaders(httpRequest, request.getOptions().getHeaders());
FailureTrackingResponseListener failureTrackingResponseListener = new FailureTrackingResponseListener(listener);
long startTime = System.nanoTime();
- performRequestAsync(startTime, nextHost(), httpRequest, ignoreErrorCodes,
+ performRequestAsync(startTime, nextNode(request.getOptions().getNodeSelector()), httpRequest, ignoreErrorCodes,
request.getOptions().getHttpAsyncResponseConsumerFactory(), failureTrackingResponseListener);
}
- private void performRequestAsync(final long startTime, final HostTuple> hostTuple, final HttpRequestBase request,
+ private void performRequestAsync(final long startTime, final NodeTuple> nodeTuple, final HttpRequestBase request,
final Set ignoreErrorCodes,
final HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory,
final FailureTrackingResponseListener listener) {
- final HttpHost host = hostTuple.hosts.next();
+ final Node node = nodeTuple.nodes.next();
//we stream the request body if the entity allows for it
- final HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(host, request);
+ final HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(node.getHost(), request);
final HttpAsyncResponseConsumer asyncResponseConsumer =
httpAsyncResponseConsumerFactory.createHttpAsyncResponseConsumer();
final HttpClientContext context = HttpClientContext.create();
- context.setAuthCache(hostTuple.authCache);
+ context.setAuthCache(nodeTuple.authCache);
client.execute(requestProducer, asyncResponseConsumer, context, new FutureCallback() {
@Override
public void completed(HttpResponse httpResponse) {
try {
- RequestLogger.logResponse(logger, request, host, httpResponse);
+ RequestLogger.logResponse(logger, request, node.getHost(), httpResponse);
int statusCode = httpResponse.getStatusLine().getStatusCode();
- Response response = new Response(request.getRequestLine(), host, httpResponse);
+ Response response = new Response(request.getRequestLine(), node.getHost(), httpResponse);
if (isSuccessfulResponse(statusCode) || ignoreErrorCodes.contains(response.getStatusLine().getStatusCode())) {
- onResponse(host);
+ onResponse(node);
listener.onSuccess(response);
} else {
ResponseException responseException = new ResponseException(response);
if (isRetryStatus(statusCode)) {
//mark host dead and retry against next one
- onFailure(host);
+ onFailure(node);
retryIfPossible(responseException);
} else {
//mark host alive and don't retry, as the error should be a request problem
- onResponse(host);
+ onResponse(node);
listener.onDefinitiveFailure(responseException);
}
}
@@ -511,8 +554,8 @@ public void completed(HttpResponse httpResponse) {
@Override
public void failed(Exception failure) {
try {
- RequestLogger.logFailedRequest(logger, request, host, failure);
- onFailure(host);
+ RequestLogger.logFailedRequest(logger, request, node, failure);
+ onFailure(node);
retryIfPossible(failure);
} catch(Exception e) {
listener.onDefinitiveFailure(e);
@@ -520,7 +563,7 @@ public void failed(Exception failure) {
}
private void retryIfPossible(Exception exception) {
- if (hostTuple.hosts.hasNext()) {
+ if (nodeTuple.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;
@@ -531,7 +574,7 @@ private void retryIfPossible(Exception exception) {
} else {
listener.trackFailure(exception);
request.reset();
- performRequestAsync(startTime, hostTuple, request, ignoreErrorCodes, httpAsyncResponseConsumerFactory, listener);
+ performRequestAsync(startTime, nodeTuple, request, ignoreErrorCodes, httpAsyncResponseConsumerFactory, listener);
}
} else {
listener.onDefinitiveFailure(exception);
@@ -560,54 +603,103 @@ private void setHeaders(HttpRequest httpRequest, Collection requestHeade
}
/**
- * 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.
+ * @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 (entry.getValue().shallBeRetried() == false) {
- filteredHosts.remove(entry.getKey());
- }
+ private NodeTuple> nextNode(NodeSelector nodeSelector) throws IOException {
+ NodeTuple> nodeTuple = this.nodeTuple;
+ List hosts = selectHosts(nodeTuple, blacklist, lastNodeIndex, nodeSelector);
+ return new NodeTuple<>(hosts.iterator(), nodeTuple.authCache);
+ }
+
+ /**
+ * Select hosts to try. Package private for testing.
+ */
+ static List selectHosts(NodeTuple> nodeTuple,
+ Map blacklist, AtomicInteger lastNodeIndex,
+ 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 hosts 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 o1.getValue().compareTo(o2.getValue());
- }
- });
- HttpHost deadHost = sortedHosts.get(0).getKey();
- logger.trace("resurrecting host [" + deadHost + "]");
- nextHosts = Collections.singleton(deadHost);
+ if (deadness.shallBeRetried()) {
+ livingNodes.add(node);
+ continue;
+ }
+ deadNodes.add(new DeadNode(node, deadness));
+ }
+
+ if (false == livingNodes.isEmpty()) {
+ /*
+ * Normal state: there is at least one living node. If the
+ * selector is ok with any over the living nodes then use them
+ * for the request.
+ */
+ List selectedLivingNodes = new ArrayList<>(livingNodes);
+ nodeSelector.select(selectedLivingNodes);
+ if (false == selectedLivingNodes.isEmpty()) {
+ /*
+ * Rotate the list so subsequent requests will prefer the
+ * nodes in a different order.
+ */
+ Collections.rotate(selectedLivingNodes, lastNodeIndex.getAndIncrement());
+ 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()) {
+ final List selectedDeadNodes = new ArrayList<>(deadNodes);
+ /*
+ * We'd like NodeSelectors to remove items directly from deadNodes
+ * so we can find the minimum after it is filtered without having
+ * to compare many things. This saves us a sort on the unfiltered
+ * list.
+ */
+ nodeSelector.select(new Iterable() {
+ @Override
+ public Iterator iterator() {
+ return new DeadNodeIteratorAdapter(selectedDeadNodes.iterator());
}
- } else {
- List rotatedHosts = new ArrayList<>(filteredHosts);
- Collections.rotate(rotatedHosts, rotatedHosts.size() - lastHostIndex.getAndIncrement());
- nextHosts = rotatedHosts;
+ });
+ if (false == selectedDeadNodes.isEmpty()) {
+ return singletonList(Collections.min(selectedDeadNodes).node);
}
- } while(nextHosts.isEmpty());
- return new HostTuple<>(nextHosts.iterator(), hostTuple.authCache);
+ }
+ throw new IOException("NodeSelector [" + nodeSelector + "] rejected all nodes, "
+ + "living " + livingNodes + " and dead " + deadNodes);
}
/**
* Called after each successful request call.
* Receives as an argument the host that was used for the successful request.
*/
- private void onResponse(HttpHost host) {
- DeadHostState removedHost = this.blacklist.remove(host);
+ private void onResponse(Node node) {
+ DeadHostState removedHost = this.blacklist.remove(node.getHost());
if (logger.isDebugEnabled() && removedHost != null) {
- logger.debug("removed host [" + host + "] from blacklist");
+ logger.debug("removed [" + node + "] from blacklist");
}
}
@@ -615,20 +707,25 @@ private void onResponse(HttpHost host) {
* Called after each failed attempt.
* Receives as an argument the host that was used for the failed attempt.
*/
- private void onFailure(HttpHost host) {
+ private void onFailure(Node node) {
while(true) {
- DeadHostState previousDeadHostState = blacklist.putIfAbsent(host, new DeadHostState(DeadHostState.TimeSupplier.DEFAULT));
+ DeadHostState previousDeadHostState =
+ blacklist.putIfAbsent(node.getHost(), new DeadHostState(TimeSupplier.DEFAULT));
if (previousDeadHostState == null) {
- logger.debug("added host [" + host + "] to blacklist");
+ if (logger.isDebugEnabled()) {
+ logger.debug("added [" + node + "] to blacklist");
+ }
break;
}
- if (blacklist.replace(host, previousDeadHostState,
- new DeadHostState(previousDeadHostState, DeadHostState.TimeSupplier.DEFAULT))) {
- logger.debug("updated host [" + host + "] already in blacklist");
+ if (blacklist.replace(node.getHost(), previousDeadHostState,
+ new DeadHostState(previousDeadHostState))) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("updated [" + node + "] already in blacklist");
+ }
break;
}
}
- failureListener.onFailure(host);
+ failureListener.onFailure(node);
}
@Override
@@ -840,6 +937,11 @@ Response get() throws IOException {
e.initCause(exception);
throw e;
}
+ if (exception instanceof ConnectException) {
+ ConnectException e = new ConnectException(exception.getMessage());
+ e.initCause(exception);
+ throw e;
+ }
if (exception instanceof IOException) {
throw new IOException(exception.getMessage(), exception);
}
@@ -862,27 +964,76 @@ Response get() throws IOException {
*/
public static class FailureListener {
/**
- * Notifies that the host provided as argument has just failed
+ * Notifies that the node provided as argument has just failed
*/
- public void onFailure(HttpHost host) {
-
- }
+ public void onFailure(Node node) {}
}
/**
- * {@code HostTuple} enables the {@linkplain HttpHost}s and {@linkplain AuthCache} to be set together in a thread
- * safe, volatile way.
+ * {@link NodeTuple} 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 DeadNode implements Comparable {
+ final Node node;
+ final DeadHostState deadness;
+
+ DeadNode(Node node, DeadHostState deadness) {
+ this.node = node;
+ this.deadness = deadness;
+ }
+
+ @Override
+ public String toString() {
+ return node.toString();
+ }
+
+ @Override
+ public int compareTo(DeadNode rhs) {
+ return deadness.compareTo(rhs.deadness);
+ }
+ }
+
+ /**
+ * Adapts an Iterator
into an
+ * Iterator
.
+ */
+ private static class DeadNodeIteratorAdapter implements Iterator {
+ private final Iterator itr;
+
+ private DeadNodeIteratorAdapter(Iterator itr) {
+ this.itr = itr;
+ }
+
+ @Override
+ public boolean hasNext() {
+ return itr.hasNext();
+ }
+
+ @Override
+ public Node next() {
+ return itr.next().node;
+ }
+
+ @Override
+ public void remove() {
+ itr.remove();
+ }
+ }
+
/**
* Add all headers from the provided varargs argument to a {@link Request}. This only exists
* to support methods that exist for backwards compatibility.
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 5f7831c67fc28..17d27248dfea9 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;
@@ -32,6 +31,7 @@
import java.security.AccessController;
import java.security.NoSuchAlgorithmException;
import java.security.PrivilegedAction;
+import java.util.List;
import java.util.Objects;
/**
@@ -48,7 +48,7 @@ public final class RestClientBuilder {
private static final Header[] EMPTY_HEADERS = new Header[0];
- private final HttpHost[] hosts;
+ private final List nodes;
private int maxRetryTimeout = DEFAULT_MAX_RETRY_TIMEOUT_MILLIS;
private Header[] defaultHeaders = EMPTY_HEADERS;
private RestClient.FailureListener failureListener;
@@ -59,18 +59,18 @@ public final class RestClientBuilder {
/**
* Creates a new builder instance and sets the hosts that the client will send requests to.
*
- * @throws NullPointerException if {@code hosts} or any host is {@code null}.
- * @throws IllegalArgumentException if {@code hosts} is empty.
+ * @throws IllegalArgumentException if {@code nodes} is {@code null} or empty.
*/
- RestClientBuilder(HttpHost... hosts) {
- Objects.requireNonNull(hosts, "hosts must not be null");
- if (hosts.length == 0) {
- throw new IllegalArgumentException("no hosts provided");
+ RestClientBuilder(List nodes) {
+ if (nodes == null || nodes.isEmpty()) {
+ 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;
}
/**
@@ -186,7 +186,7 @@ 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/test/java/org/elasticsearch/client/DeadHostStateTests.java b/client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java
index 75fbafd88f83c..daea27f896328 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java
@@ -21,11 +21,15 @@
import java.util.concurrent.TimeUnit;
+import org.elasticsearch.client.DeadHostState.TimeSupplier;
+
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
public class DeadHostStateTests extends RestClientTestCase {
@@ -42,7 +46,7 @@ public void testDeadHostStateFromPreviousDefaultTimeSupplier() {
DeadHostState previous = new DeadHostState(DeadHostState.TimeSupplier.DEFAULT);
int iters = randomIntBetween(5, 30);
for (int i = 0; i < iters; i++) {
- DeadHostState deadHostState = new DeadHostState(previous, DeadHostState.TimeSupplier.DEFAULT);
+ DeadHostState deadHostState = new DeadHostState(previous);
assertThat(deadHostState.getDeadUntilNanos(), greaterThan(previous.getDeadUntilNanos()));
assertThat(deadHostState.getFailedAttempts(), equalTo(previous.getFailedAttempts() + 1));
previous = deadHostState;
@@ -56,7 +60,7 @@ public void testCompareToDefaultTimeSupplier() {
if (i == 0) {
deadHostStates[i] = new DeadHostState(DeadHostState.TimeSupplier.DEFAULT);
} else {
- deadHostStates[i] = new DeadHostState(deadHostStates[i - 1], DeadHostState.TimeSupplier.DEFAULT);
+ deadHostStates[i] = new DeadHostState(deadHostStates[i - 1]);
}
}
for (int k = 1; k < deadHostStates.length; k++) {
@@ -65,6 +69,17 @@ public void testCompareToDefaultTimeSupplier() {
}
}
+ public void testCompareToDifferingTimeSupplier() {
+ try {
+ new DeadHostState(TimeSupplier.DEFAULT).compareTo(
+ new DeadHostState(new ConfigurableTimeSupplier()));
+ fail("expected failure");
+ } catch (IllegalArgumentException e) {
+ assertEquals("can't compare DeadHostStates with different clocks [nanoTime != configured[0]]",
+ e.getMessage());
+ }
+ }
+
public void testShallBeRetried() {
ConfigurableTimeSupplier timeSupplier = new ConfigurableTimeSupplier();
DeadHostState deadHostState = null;
@@ -74,7 +89,7 @@ public void testShallBeRetried() {
if (i == 0) {
deadHostState = new DeadHostState(timeSupplier);
} else {
- deadHostState = new DeadHostState(deadHostState, timeSupplier);
+ deadHostState = new DeadHostState(deadHostState);
}
for (int j = 0; j < expectedTimeoutSecond; j++) {
timeSupplier.nanoTime += TimeUnit.SECONDS.toNanos(1);
@@ -94,25 +109,29 @@ public void testDeadHostStateTimeouts() {
DeadHostState previous = new DeadHostState(zeroTimeSupplier);
for (long expectedTimeoutsSecond : EXPECTED_TIMEOUTS_SECONDS) {
assertThat(TimeUnit.NANOSECONDS.toSeconds(previous.getDeadUntilNanos()), equalTo(expectedTimeoutsSecond));
- previous = new DeadHostState(previous, zeroTimeSupplier);
+ previous = new DeadHostState(previous);
}
//check that from here on the timeout does not increase
int iters = randomIntBetween(5, 30);
for (int i = 0; i < iters; i++) {
- DeadHostState deadHostState = new DeadHostState(previous, zeroTimeSupplier);
+ DeadHostState deadHostState = new DeadHostState(previous);
assertThat(TimeUnit.NANOSECONDS.toSeconds(deadHostState.getDeadUntilNanos()),
equalTo(EXPECTED_TIMEOUTS_SECONDS[EXPECTED_TIMEOUTS_SECONDS.length - 1]));
previous = deadHostState;
}
}
- private static class ConfigurableTimeSupplier implements DeadHostState.TimeSupplier {
-
+ static class ConfigurableTimeSupplier implements DeadHostState.TimeSupplier {
long nanoTime;
@Override
public long nanoTime() {
return nanoTime;
}
+
+ @Override
+ public String toString() {
+ return "configured[" + nanoTime + "]";
+ }
}
}
diff --git a/client/rest/src/test/java/org/elasticsearch/client/HasAttributeNodeSelectorTests.java b/client/rest/src/test/java/org/elasticsearch/client/HasAttributeNodeSelectorTests.java
new file mode 100644
index 0000000000000..8a7c12e8c62de
--- /dev/null
+++ b/client/rest/src/test/java/org/elasticsearch/client/HasAttributeNodeSelectorTests.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 java.util.Map;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+import static org.junit.Assert.assertEquals;
+
+public class HasAttributeNodeSelectorTests extends RestClientTestCase {
+ public void testHasAttribute() {
+ Node hasAttributeValue = dummyNode(singletonMap("attr", singletonList("val")));
+ Node hasAttributeButNotValue = dummyNode(singletonMap("attr", singletonList("notval")));
+ Node hasAttributeValueInList = dummyNode(singletonMap("attr", Arrays.asList("val", "notval")));
+ Node notHasAttribute = dummyNode(singletonMap("notattr", singletonList("val")));
+ List nodes = new ArrayList<>();
+ nodes.add(hasAttributeValue);
+ nodes.add(hasAttributeButNotValue);
+ nodes.add(hasAttributeValueInList);
+ nodes.add(notHasAttribute);
+ List expected = new ArrayList<>();
+ expected.add(hasAttributeValue);
+ expected.add(hasAttributeValueInList);
+ new HasAttributeNodeSelector("attr", "val").select(nodes);
+ assertEquals(expected, nodes);
+ }
+
+ private static Node dummyNode(Map> attributes) {
+ return new Node(new HttpHost("dummy"), Collections.emptySet(),
+ randomAsciiAlphanumOfLength(5), randomAsciiAlphanumOfLength(5),
+ new Roles(randomBoolean(), randomBoolean(), randomBoolean()),
+ attributes);
+ }
+}
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..6c952fcf94759 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/HostsTrackingFailureListener.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/HostsTrackingFailureListener.java
@@ -22,6 +22,7 @@
import org.apache.http.HttpHost;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -29,14 +30,22 @@
import static org.junit.Assert.assertThat;
/**
- * {@link org.elasticsearch.client.RestClient.FailureListener} impl that allows to track when it gets called for which host.
+ * {@link RestClient.FailureListener} impl that allows to track when it gets called for which host.
*/
class HostsTrackingFailureListener extends RestClient.FailureListener {
private volatile Set hosts = new HashSet<>();
@Override
- public void onFailure(HttpHost host) {
- hosts.add(host);
+ public void onFailure(Node node) {
+ hosts.add(node.getHost());
+ }
+
+ void assertCalled(List nodes) {
+ HttpHost[] hosts = new HttpHost[nodes.size()];
+ for (int i = 0 ; i < nodes.size(); i++) {
+ hosts[i] = nodes.get(i).getHost();
+ }
+ assertCalled(hosts);
}
void assertCalled(HttpHost... hosts) {
@@ -48,4 +57,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..868ccdcab757d
--- /dev/null
+++ b/client/rest/src/test/java/org/elasticsearch/client/NodeSelectorTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.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()));
+ }
+ List expected = new ArrayList<>(nodes);
+ NodeSelector.ANY.select(nodes);
+ assertEquals(expected, nodes);
+ }
+
+ public void testNotMasterOnly() {
+ Node masterOnly = dummyNode(true, false, false);
+ Node all = dummyNode(true, true, true);
+ Node masterAndData = dummyNode(true, true, false);
+ Node masterAndIngest = dummyNode(true, false, true);
+ Node coordinatingOnly = dummyNode(false, false, false);
+ Node ingestOnly = dummyNode(false, false, true);
+ Node data = dummyNode(false, true, randomBoolean());
+ List nodes = new ArrayList<>();
+ nodes.add(masterOnly);
+ nodes.add(all);
+ nodes.add(masterAndData);
+ nodes.add(masterAndIngest);
+ nodes.add(coordinatingOnly);
+ nodes.add(ingestOnly);
+ nodes.add(data);
+ Collections.shuffle(nodes, getRandom());
+ List expected = new ArrayList<>(nodes);
+ expected.remove(masterOnly);
+ NodeSelector.NOT_MASTER_ONLY.select(nodes);
+ assertEquals(expected, nodes);
+ }
+
+ private static 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),
+ Collections.>emptyMap());
+ }
+}
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..9eeeb1144f485
--- /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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
+import static java.util.Collections.singletonMap;
+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 testToString() {
+ Map> attributes = new HashMap<>();
+ attributes.put("foo", singletonList("bar"));
+ attributes.put("baz", Arrays.asList("bort", "zoom"));
+ assertEquals("[host=http://1]", new Node(new HttpHost("1")).toString());
+ assertEquals("[host=http://1, attributes={foo=[bar], baz=[bort, zoom]}]",
+ new Node(new HttpHost("1"), null, null, null, null, attributes).toString());
+ assertEquals("[host=http://1, roles=mdi]", new Node(new HttpHost("1"),
+ null, null, null, new Roles(true, true, true), null).toString());
+ assertEquals("[host=http://1, version=ver]", new Node(new HttpHost("1"),
+ null, null, "ver", null, null).toString());
+ assertEquals("[host=http://1, name=nam]", new Node(new HttpHost("1"),
+ null, "nam", null, 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, null).toString());
+ assertEquals(
+ "[host=http://1, bound=[http://1, http://2], name=nam, version=ver, roles=m, attributes={foo=[bar], baz=[bort, zoom]}]",
+ new Node(new HttpHost("1"), new HashSet<>(Arrays.asList(new HttpHost("1"), new HttpHost("2"))),
+ "nam", "ver", new Roles(true, false, false), attributes).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),
+ randomBoolean() ? null : singletonMap("foo", singletonList("bar")));
+ 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(), node.getAttributes());
+ 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(), node.getAttributes())));
+ assertFalse(node.equals(new Node(host, new HashSet<>(Arrays.asList(host, new HttpHost(host.toHostString() + "changed"))),
+ node.getName(), node.getVersion(), node.getRoles(), node.getAttributes())));
+ assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName() + "changed",
+ node.getVersion(), node.getRoles(), node.getAttributes())));
+ assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName(),
+ node.getVersion() + "changed", node.getRoles(), node.getAttributes())));
+ assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName(),
+ node.getVersion(), new Roles(false, false, false), node.getAttributes())));
+ assertFalse(node.equals(new Node(host, node.getBoundHosts(), node.getName(),
+ node.getVersion(), node.getRoles(), singletonMap("bort", singletonList("bing")))));
+ }
+}
diff --git a/client/rest/src/test/java/org/elasticsearch/client/RequestOptionsTests.java b/client/rest/src/test/java/org/elasticsearch/client/RequestOptionsTests.java
index 19106792228d9..a78be6c126bae 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/RequestOptionsTests.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/RequestOptionsTests.java
@@ -114,6 +114,10 @@ static RequestOptions.Builder randomBuilder() {
}
}
+ if (randomBoolean()) {
+ builder.setNodeSelector(mock(NodeSelector.class));
+ }
+
if (randomBoolean()) {
builder.setHttpAsyncResponseConsumerFactory(new HeapBufferedResponseConsumerFactory(1));
}
@@ -127,12 +131,15 @@ private static RequestOptions copy(RequestOptions options) {
private static RequestOptions mutate(RequestOptions options) {
RequestOptions.Builder mutant = options.toBuilder();
- int mutationType = between(0, 1);
+ int mutationType = between(0, 2);
switch (mutationType) {
case 0:
mutant.addHeader("extra", "m");
return mutant.build();
case 1:
+ mutant.setNodeSelector(mock(NodeSelector.class));
+ return mutant.build();
+ case 2:
mutant.setHttpAsyncResponseConsumerFactory(new HeapBufferedResponseConsumerFactory(5));
return mutant.build();
default:
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 9657e782bda04..9fcb4978e28a7 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java
@@ -39,21 +39,42 @@ 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 nor empty", e.getMessage());
+ }
+
+ try {
+ RestClient.builder(new HttpHost[] {});
+ fail("should have failed");
+ } catch(IllegalArgumentException e) {
+ assertEquals("hosts must not be null nor empty", e.getMessage());
}
try {
- RestClient.builder();
+ RestClient.builder((Node[])null);
fail("should have failed");
} catch(IllegalArgumentException e) {
- assertEquals("no hosts provided", e.getMessage());
+ 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.builder(new HttpHost("localhost", 9200), null);
fail("should have failed");
- } catch(NullPointerException e) {
+ } catch(IllegalArgumentException e) {
assertEquals("host cannot be null", e.getMessage());
}
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 aa323276404cf..d09741ea25b6c 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java
@@ -29,9 +29,11 @@
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.Iterator;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
@@ -40,8 +42,11 @@
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.startsWith;
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}.
@@ -50,31 +55,37 @@
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 {
@@ -118,6 +129,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) {
@@ -182,6 +196,36 @@ public void onFailure(Exception exception) {
}
}
+ /**
+ * Test host selector against a real server and
+ * test what happens after calling
+ */
+ public void testNodeSelector() throws IOException {
+ Request request = new Request("GET", "/200");
+ RequestOptions.Builder options = request.getOptions().toBuilder();
+ options.setNodeSelector(firstPositionNodeSelector());
+ request.setOptions(options);
+ 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.
+ */
+ if (stoppedFirstHost) {
+ try {
+ restClient.performRequest(request);
+ fail("expected to fail to connect");
+ } catch (ConnectException e) {
+ // This is different in windows and linux but this matches both.
+ assertThat(e.getMessage(), startsWith("Connection refused"));
+ }
+ } else {
+ Response response = restClient.performRequest(request);
+ assertEquals(httpHosts[0], response.getHost());
+ }
+ }
+ }
+
private static class TestResponse {
private final String method;
private final int statusCode;
@@ -203,4 +247,17 @@ Response getResponse() {
throw new AssertionError("unexpected response " + response.getClass());
}
}
+
+ private NodeSelector firstPositionNodeSelector() {
+ return new NodeSelector() {
+ @Override
+ public void select(Iterable nodes) {
+ for (Iterator itr = nodes.iterator(); itr.hasNext();) {
+ if (httpHosts[0] != itr.next().getHost()) {
+ itr.remove();
+ }
+ }
+ }
+ };
+ }
}
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..d04b3cbb7554e 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,8 +43,11 @@
import java.io.IOException;
import java.net.SocketTimeoutException;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -71,7 +75,7 @@ public class RestClientMultipleHostsTests extends RestClientTestCase {
private ExecutorService exec = Executors.newFixedThreadPool(1);
private RestClient restClient;
- private HttpHost[] httpHosts;
+ private List nodes;
private HostsTrackingFailureListener failureListener;
@Before
@@ -108,13 +112,14 @@ 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 ArrayList<>(numNodes);
+ for (int i = 0; i < numNodes; i++) {
+ nodes.add(new Node(new HttpHost("localhost", 9200 + i)));
}
+ nodes = Collections.unmodifiableList(nodes);
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 +133,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.size(); j++) {
int statusCode = randomOkStatusCode(getRandom());
Response response = restClient.performRequest(randomHttpMethod(getRandom()), "/" + statusCode);
assertEquals(statusCode, response.getStatusLine().getStatusCode());
@@ -144,9 +148,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.size(); j++) {
String method = randomHttpMethod(getRandom());
int statusCode = randomErrorNoRetryStatusCode(getRandom());
try {
@@ -185,10 +188,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 +212,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 +233,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.size(); j++) {
retryEndpoint = randomErrorRetryEndpoint();
try {
restClient.performRequest(randomHttpMethod(getRandom()), retryEndpoint);
@@ -308,6 +308,58 @@ public void testRoundRobinRetryErrors() throws IOException {
}
}
+ public void testNodeSelector() throws IOException {
+ NodeSelector firstPositionOnly = new NodeSelector() {
+ @Override
+ public void select(Iterable restClientNodes) {
+ boolean found = false;
+ for (Iterator itr = restClientNodes.iterator(); itr.hasNext();) {
+ if (nodes.get(0) == itr.next()) {
+ found = true;
+ } else {
+ itr.remove();
+ }
+ }
+ assertTrue(found);
+ }
+ };
+ 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.
+ */
+ Request request = new Request("GET", "/200");
+ RequestOptions.Builder options = request.getOptions().toBuilder();
+ options.setNodeSelector(firstPositionOnly);
+ request.setOptions(options);
+ Response response = restClient.performRequest(request);
+ assertEquals(nodes.get(0).getHost(), response.getHost());
+ }
+ }
+
+ public void testSetNodes() throws IOException {
+ List newNodes = new ArrayList<>(nodes.size());
+ for (int i = 0; i < nodes.size(); i++) {
+ Roles roles = i == 0 ? new Roles(false, true, true) : new Roles(true, false, false);
+ newNodes.add(new Node(nodes.get(i).getHost(), null, null, null, roles, null));
+ }
+ restClient.setNodes(newNodes);
+ 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.
+ */
+ Request request = new Request("GET", "/200");
+ RequestOptions.Builder options = request.getOptions().toBuilder();
+ options.setNodeSelector(NodeSelector.NOT_MASTER_ONLY);
+ request.setOptions(options);
+ Response response = restClient.performRequest(request);
+ assertEquals(newNodes.get(0).getHost(), response.getHost());
+ }
+ }
+
private static String randomErrorRetryEndpoint() {
switch(RandomNumbers.randomIntBetween(getRandom(), 0, 3)) {
case 0:
@@ -321,4 +373,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 634929c5de156..5987fe7dd9849 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java
@@ -65,6 +65,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
+import static java.util.Collections.singletonList;
import static org.elasticsearch.client.RestClientTestUtil.getAllErrorStatusCodes;
import static org.elasticsearch.client.RestClientTestUtil.getHttpMethods;
import static org.elasticsearch.client.RestClientTestUtil.getOkStatusCodes;
@@ -94,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;
@@ -108,7 +109,7 @@ public void createRestClient() {
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();
@@ -146,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,
+ singletonList(node), null, failureListener);
}
/**
@@ -244,7 +246,7 @@ public void testErrorStatusCodes() throws IOException {
if (errorStatusCode <= 500 || expectedIgnores.contains(errorStatusCode)) {
failureListener.assertNotCalled();
} else {
- failureListener.assertCalled(httpHost);
+ failureListener.assertCalled(singletonList(node));
}
}
}
@@ -259,14 +261,14 @@ public void testIOExceptions() {
} catch(IOException e) {
assertThat(e, instanceOf(ConnectTimeoutException.class));
}
- failureListener.assertCalled(httpHost);
+ failureListener.assertCalled(singletonList(node));
try {
performRequest(method, "/soe");
fail("request should have failed");
} catch(IOException e) {
assertThat(e, instanceOf(SocketTimeoutException.class));
}
- failureListener.assertCalled(httpHost);
+ failureListener.assertCalled(singletonList(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 5fe5fcae78fee..04742ccab4f32 100644
--- a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java
+++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java
@@ -22,14 +22,23 @@
import org.apache.http.Header;
import org.apache.http.HttpHost;
import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.elasticsearch.client.DeadHostStateTests.ConfigurableTimeSupplier;
+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.Iterator;
+import java.util.List;
+import java.util.Map;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.TimeUnit;
+import static java.util.Collections.singletonList;
import static org.elasticsearch.client.RestClientTestUtil.getHttpMethods;
import static org.hamcrest.Matchers.instanceOf;
import static org.junit.Assert.assertEquals;
@@ -43,9 +52,9 @@
public class RestClientTests extends RestClientTestCase {
public void testCloseIsIdempotent() throws IOException {
- HttpHost[] hosts = new HttpHost[]{new HttpHost("localhost", 9200)};
+ List nodes = singletonList(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();
@@ -225,6 +234,7 @@ public void testBuildUriLeavesPathUntouched() {
}
}
+ @Deprecated
public void testSetHostsWrongArguments() throws IOException {
try (RestClient restClient = createRestClient()) {
restClient.setHosts((HttpHost[]) null);
@@ -241,45 +251,75 @@ public void testSetHostsWrongArguments() throws IOException {
try (RestClient restClient = createRestClient()) {
restClient.setHosts((HttpHost) null);
fail("setHosts should have failed");
- } catch (NullPointerException e) {
+ } catch (IllegalArgumentException e) {
assertEquals("host cannot be null", e.getMessage());
}
try (RestClient restClient = createRestClient()) {
restClient.setHosts(new HttpHost("localhost", 9200), null, new HttpHost("localhost", 9201));
fail("setHosts should have failed");
- } catch (NullPointerException e) {
+ } catch (IllegalArgumentException e) {
assertEquals("host cannot be null", e.getMessage());
}
}
- public void testSetHostsPreservesOrdering() throws Exception {
+ public void testSetNodesWrongArguments() throws IOException {
+ try (RestClient restClient = createRestClient()) {
+ restClient.setNodes(null);
+ fail("setNodes should have failed");
+ } catch (IllegalArgumentException e) {
+ assertEquals("nodes must not be null or empty", e.getMessage());
+ }
+ try (RestClient restClient = createRestClient()) {
+ restClient.setNodes(Collections.emptyList());
+ fail("setNodes should have failed");
+ } catch (IllegalArgumentException e) {
+ assertEquals("nodes must not be null or empty", e.getMessage());
+ }
+ try (RestClient restClient = createRestClient()) {
+ restClient.setNodes(Collections.singletonList((Node) null));
+ fail("setNodes should have failed");
+ } catch (NullPointerException e) {
+ assertEquals("node cannot be null", e.getMessage());
+ }
+ try (RestClient restClient = createRestClient()) {
+ restClient.setNodes(Arrays.asList(
+ new Node(new HttpHost("localhost", 9200)),
+ null,
+ new Node(new HttpHost("localhost", 9201))));
+ fail("setNodes should have failed");
+ } catch (NullPointerException e) {
+ assertEquals("node cannot be null", e.getMessage());
+ }
+ }
+
+ public void testSetNodesPreservesOrdering() throws Exception {
try (RestClient restClient = createRestClient()) {
- HttpHost[] hosts = randomHosts();
- restClient.setHosts(hosts);
- assertEquals(Arrays.asList(hosts), restClient.getHosts());
+ List nodes = randomNodes();
+ restClient.setNodes(nodes);
+ assertEquals(nodes, restClient.getNodes());
}
}
- private static HttpHost[] randomHosts() {
- int numHosts = randomIntBetween(1, 10);
- HttpHost[] hosts = new HttpHost[numHosts];
- for (int i = 0; i < hosts.length; i++) {
- hosts[i] = new HttpHost("host-" + i, 9200);
+ private static List randomNodes() {
+ int numNodes = randomIntBetween(1, 10);
+ List nodes = new ArrayList<>(numNodes);
+ for (int i = 0; i < numNodes; i++) {
+ nodes.add(new Node(new HttpHost("host-" + i, 9200)));
}
- return hosts;
+ return nodes;
}
- public void testSetHostsDuplicatedHosts() throws Exception {
+ public void testSetNodesDuplicatedHosts() throws Exception {
try (RestClient restClient = createRestClient()) {
- int numHosts = randomIntBetween(1, 10);
- HttpHost[] hosts = new HttpHost[numHosts];
- HttpHost host = new HttpHost("host", 9200);
- for (int i = 0; i < hosts.length; i++) {
- hosts[i] = host;
+ int numNodes = randomIntBetween(1, 10);
+ List nodes = new ArrayList<>(numNodes);
+ Node node = new Node(new HttpHost("host", 9200));
+ for (int i = 0; i < numNodes; i++) {
+ nodes.add(node);
}
- restClient.setHosts(hosts);
- assertEquals(1, restClient.getHosts().size());
- assertEquals(host, restClient.getHosts().get(0));
+ restClient.setNodes(nodes);
+ assertEquals(1, restClient.getNodes().size());
+ assertEquals(node, restClient.getNodes().get(0));
}
}
@@ -300,8 +340,143 @@ public void testNullPath() throws IOException {
}
}
+ public void testSelectHosts() throws IOException {
+ Node n1 = new Node(new HttpHost("1"), null, null, "1", null, null);
+ Node n2 = new Node(new HttpHost("2"), null, null, "2", null, null);
+ Node n3 = new Node(new HttpHost("3"), null, null, "3", null, null);
+
+ NodeSelector not1 = new NodeSelector() {
+ @Override
+ public void select(Iterable nodes) {
+ for (Iterator itr = nodes.iterator(); itr.hasNext();) {
+ if ("1".equals(itr.next().getVersion())) {
+ itr.remove();
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "NOT 1";
+ }
+ };
+ NodeSelector noNodes = new NodeSelector() {
+ @Override
+ public void select(Iterable nodes) {
+ for (Iterator itr = nodes.iterator(); itr.hasNext();) {
+ itr.next();
+ itr.remove();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "NONE";
+ }
+ };
+
+ NodeTuple> nodeTuple = new NodeTuple<>(Arrays.asList(n1, n2, n3), null);
+
+ Map emptyBlacklist = Collections.emptyMap();
+
+ // Normal cases where the node selector doesn't reject all living nodes
+ assertSelectLivingHosts(Arrays.asList(n1, n2, n3), nodeTuple, emptyBlacklist, NodeSelector.ANY);
+ assertSelectLivingHosts(Arrays.asList(n2, n3), nodeTuple, emptyBlacklist, not1);
+
+ /*
+ * Try a NodeSelector that excludes all nodes. This should
+ * throw an exception
+ */
+ {
+ 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, assertSelectAllRejected(nodeTuple, emptyBlacklist, noNodes));
+ }
+
+ // Mark all the nodes dead for a few test cases
+ {
+ ConfigurableTimeSupplier timeSupplier = new ConfigurableTimeSupplier();
+ Map blacklist = new HashMap<>();
+ blacklist.put(n1.getHost(), new DeadHostState(timeSupplier));
+ blacklist.put(n2.getHost(), new DeadHostState(new DeadHostState(timeSupplier)));
+ blacklist.put(n3.getHost(), new DeadHostState(new DeadHostState(new DeadHostState(timeSupplier))));
+
+ /*
+ * selectHosts will revive a single host if regardless of
+ * blacklist time. It'll revive the node that is closest
+ * to being revived that the NodeSelector is ok with.
+ */
+ assertEquals(singletonList(n1), RestClient.selectHosts(nodeTuple, blacklist, new AtomicInteger(), NodeSelector.ANY));
+ assertEquals(singletonList(n2), RestClient.selectHosts(nodeTuple, blacklist, new AtomicInteger(), not1));
+
+ /*
+ * Try a NodeSelector that excludes all nodes. This should
+ * return a failure, but a different failure than when the
+ * blacklist is empty so that the caller knows that all of
+ * their nodes are blacklisted AND blocked.
+ */
+ 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, assertSelectAllRejected(nodeTuple, blacklist, noNodes));
+
+ /*
+ * Now lets wind the clock forward, past the timeout for one of
+ * the dead nodes. We should return it.
+ */
+ timeSupplier.nanoTime = new DeadHostState(timeSupplier).getDeadUntilNanos();
+ assertSelectLivingHosts(Arrays.asList(n1), nodeTuple, blacklist, NodeSelector.ANY);
+
+ /*
+ * But if the NodeSelector rejects that node then we'll pick the
+ * first on that the NodeSelector doesn't reject.
+ */
+ assertSelectLivingHosts(Arrays.asList(n2), nodeTuple, blacklist, not1);
+
+ /*
+ * If we wind the clock way into the future, past any of the
+ * blacklist timeouts then we function as though the nodes aren't
+ * in the blacklist at all.
+ */
+ timeSupplier.nanoTime += DeadHostState.MAX_CONNECTION_TIMEOUT_NANOS;
+ assertSelectLivingHosts(Arrays.asList(n1, n2, n3), nodeTuple, blacklist, NodeSelector.ANY);
+ assertSelectLivingHosts(Arrays.asList(n2, n3), nodeTuple, blacklist, not1);
+ }
+ }
+
+ private void assertSelectLivingHosts(List expectedNodes, NodeTuple> nodeTuple,
+ Map blacklist, NodeSelector nodeSelector) throws IOException {
+ int iterations = 1000;
+ AtomicInteger lastNodeIndex = new AtomicInteger(0);
+ assertEquals(expectedNodes, RestClient.selectHosts(nodeTuple, blacklist, lastNodeIndex, nodeSelector));
+ // Calling it again rotates the set of results
+ for (int i = 1; i < iterations; i++) {
+ Collections.rotate(expectedNodes, 1);
+ assertEquals("iteration " + i, expectedNodes,
+ RestClient.selectHosts(nodeTuple, blacklist, lastNodeIndex, nodeSelector));
+ }
+ }
+
+ /**
+ * Assert that {@link RestClient#selectHosts} fails on the provided arguments.
+ * @return the message in the exception thrown by the failure
+ */
+ private String assertSelectAllRejected( NodeTuple> nodeTuple,
+ Map blacklist, NodeSelector nodeSelector) {
+ try {
+ RestClient.selectHosts(nodeTuple, blacklist, new AtomicInteger(0), nodeSelector);
+ throw new AssertionError("expected selectHosts to fail");
+ } catch (IOException e) {
+ return e.getMessage();
+ }
+ }
+
private static RestClient createRestClient() {
- HttpHost[] hosts = new HttpHost[]{new HttpHost("localhost", 9200)};
- return new RestClient(mock(CloseableHttpAsyncClient.class), randomIntBetween(1_000, 30_000), new Header[]{}, hosts, null, null);
+ List nodes = Collections.singletonList(new Node(new HttpHost("localhost", 9200)));
+ return new RestClient(mock(CloseableHttpAsyncClient.class), randomLongBetween(1_000, 30_000),
+ new Header[] {}, nodes, 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 d73c29bd91bc4..d3a0202747d25 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
@@ -36,7 +36,10 @@
import org.apache.http.ssl.SSLContextBuilder;
import org.apache.http.ssl.SSLContexts;
import org.apache.http.util.EntityUtils;
-import org.elasticsearch.client.HttpAsyncResponseConsumerFactory;
+import org.elasticsearch.client.HasAttributeNodeSelector;
+import org.elasticsearch.client.HttpAsyncResponseConsumerFactory.HeapBufferedResponseConsumerFactory;
+import org.elasticsearch.client.Node;
+import org.elasticsearch.client.NodeSelector;
import org.elasticsearch.client.Request;
import org.elasticsearch.client.RequestOptions;
import org.elasticsearch.client.Response;
@@ -72,6 +75,19 @@
*/
@SuppressWarnings("unused")
public class RestClientDocumentation {
+ private static final String TOKEN = "DUMMY";
+
+ // tag::rest-client-options-singleton
+ private static final RequestOptions COMMON_OPTIONS;
+ static {
+ RequestOptions.Builder builder = RequestOptions.DEFAULT.toBuilder();
+ builder.addHeader("Authorization", "Bearer " + TOKEN); // <1>
+ builder.setNodeSelector(NodeSelector.NOT_MASTER_ONLY); // <2>
+ builder.setHttpAsyncResponseConsumerFactory( // <3>
+ new HeapBufferedResponseConsumerFactory(30 * 1024 * 1024 * 1024));
+ COMMON_OPTIONS = builder.build();
+ }
+ // end::rest-client-options-singleton
@SuppressWarnings("unused")
public void testUsage() throws IOException, InterruptedException {
@@ -104,7 +120,7 @@ public void testUsage() throws IOException, InterruptedException {
RestClientBuilder builder = RestClient.builder(new HttpHost("localhost", 9200, "http"));
builder.setFailureListener(new RestClient.FailureListener() {
@Override
- public void onFailure(HttpHost host) {
+ public void onFailure(Node node) {
// <1>
}
});
@@ -172,21 +188,22 @@ public void onFailure(Exception exception) {
//tag::rest-client-body-shorter
request.setJsonEntity("{\"json\":\"text\"}");
//end::rest-client-body-shorter
+ //tag::rest-client-options-set-singleton
+ request.setOptions(COMMON_OPTIONS);
+ //end::rest-client-options-set-singleton
{
- //tag::rest-client-headers
- RequestOptions.Builder options = request.getOptions().toBuilder();
- options.addHeader("Accept", "text/plain");
- options.addHeader("Cache-Control", "no-cache");
+ //tag::rest-client-options-customize-header
+ RequestOptions.Builder options = COMMON_OPTIONS.toBuilder();
+ options.addHeader("cats", "knock things off of other things");
request.setOptions(options);
- //end::rest-client-headers
+ //end::rest-client-options-customize-header
}
{
- //tag::rest-client-response-consumer
- RequestOptions.Builder options = request.getOptions().toBuilder();
- options.setHttpAsyncResponseConsumerFactory(
- new HttpAsyncResponseConsumerFactory.HeapBufferedResponseConsumerFactory(30 * 1024 * 1024));
+ //tag::rest-client-options-customize-attribute
+ RequestOptions.Builder options = COMMON_OPTIONS.toBuilder();
+ options.setNodeSelector(new HasAttributeNodeSelector("rack", "c12")); // <1>
request.setOptions(options);
- //end::rest-client-response-consumer
+ //end::rest-client-options-customize-attribute
}
}
{
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
deleted file mode 100644
index 34a4988358653..0000000000000
--- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchHostsSniffer.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.client.sniff;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpHost;
-import org.elasticsearch.client.Response;
-import org.elasticsearch.client.RestClient;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-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.
- */
-public final class ElasticsearchHostsSniffer implements HostsSniffer {
-
- private static final Log logger = LogFactory.getLog(ElasticsearchHostsSniffer.class);
-
- public static final long DEFAULT_SNIFF_REQUEST_TIMEOUT = TimeUnit.SECONDS.toMillis(1);
-
- private final RestClient restClient;
- private final Map sniffRequestParams;
- private final Scheme scheme;
- private final JsonFactory jsonFactory = new JsonFactory();
-
- /**
- * Creates a new instance of the Elasticsearch sniffer. It will use the provided {@link RestClient} to fetch the hosts,
- * through the nodes info api, the default sniff request timeout value {@link #DEFAULT_SNIFF_REQUEST_TIMEOUT} and http
- * as the scheme for all the hosts.
- * @param restClient client used to fetch the hosts from elasticsearch through nodes info api. Usually the same instance
- * that is also provided to {@link Sniffer#builder(RestClient)}, so that the hosts are set to the same
- * client that was used to fetch them.
- */
- public ElasticsearchHostsSniffer(RestClient restClient) {
- this(restClient, DEFAULT_SNIFF_REQUEST_TIMEOUT, ElasticsearchHostsSniffer.Scheme.HTTP);
- }
-
- /**
- * Creates a new instance of the Elasticsearch sniffer. It will use the provided {@link RestClient} to fetch the hosts
- * through the nodes info api, the provided sniff request timeout value and scheme.
- * @param restClient client used to fetch the hosts from elasticsearch through nodes info api. Usually the same instance
- * that is also provided to {@link Sniffer#builder(RestClient)}, so that the hosts are set to the same
- * client that was used to sniff them.
- * @param sniffRequestTimeoutMillis the sniff request timeout (in milliseconds) to be passed in as a query string parameter
- * to elasticsearch. Allows to halt the request without any failure, as only the nodes
- * that have responded within this timeout will be returned.
- * @param scheme the scheme to associate sniffed nodes with (as it is not returned by elasticsearch)
- */
- public ElasticsearchHostsSniffer(RestClient restClient, long sniffRequestTimeoutMillis, Scheme scheme) {
- this.restClient = Objects.requireNonNull(restClient, "restClient cannot be null");
- if (sniffRequestTimeoutMillis < 0) {
- throw new IllegalArgumentException("sniffRequestTimeoutMillis must be greater than 0");
- }
- this.sniffRequestParams = Collections.singletonMap("timeout", sniffRequestTimeoutMillis + "ms");
- this.scheme = Objects.requireNonNull(scheme, "scheme cannot be null");
- }
-
- /**
- * Calls the elasticsearch nodes info api, parses the response and returns all the found http hosts
- */
- public List sniffHosts() throws IOException {
- Response response = restClient.performRequest("get", "/_nodes/http", sniffRequestParams);
- return readHosts(response.getEntity());
- }
-
- private List readHosts(HttpEntity entity) 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<>();
- while (parser.nextToken() != JsonToken.END_OBJECT) {
- if (parser.getCurrentToken() == JsonToken.START_OBJECT) {
- if ("nodes".equals(parser.getCurrentName())) {
- while (parser.nextToken() != JsonToken.END_OBJECT) {
- 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);
- }
- }
- } else {
- parser.skipChildren();
- }
- }
- }
- return hosts;
- }
- }
-
- private static HttpHost readHost(String nodeId, JsonParser parser, Scheme scheme) throws IOException {
- HttpHost httpHost = null;
- String fieldName = null;
- while (parser.nextToken() != JsonToken.END_OBJECT) {
- if (parser.getCurrentToken() == JsonToken.FIELD_NAME) {
- fieldName = parser.getCurrentName();
- } else if (parser.getCurrentToken() == JsonToken.START_OBJECT) {
- 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());
- } else if (parser.getCurrentToken() == JsonToken.START_OBJECT) {
- parser.skipChildren();
- }
- }
- } else {
- parser.skipChildren();
- }
- }
- }
- //http section is not present if http is not enabled on the node, ignore such nodes
- if (httpHost == null) {
- logger.debug("skipping node [" + nodeId + "] with http disabled");
- return null;
- }
- return httpHost;
- }
-
- public enum Scheme {
- HTTP("http"), HTTPS("https");
-
- private final String name;
-
- Scheme(String name) {
- this.name = name;
- }
-
- @Override
- public String toString() {
- return name;
- }
- }
-}
diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchNodesSniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchNodesSniffer.java
new file mode 100644
index 0000000000000..5c947f5625ba0
--- /dev/null
+++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/ElasticsearchNodesSniffer.java
@@ -0,0 +1,312 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import org.apache.commons.logging.Log;
+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.Request;
+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.HashMap;
+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;
+
+import static java.util.Collections.singletonList;
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableMap;
+
+/**
+ * Class responsible for sniffing the http hosts from elasticsearch through the nodes info api and returning them back.
+ * Compatible with elasticsearch 2.x+.
+ */
+public final class ElasticsearchNodesSniffer implements NodesSniffer {
+
+ private static final Log logger = LogFactory.getLog(ElasticsearchNodesSniffer.class);
+
+ public static final long DEFAULT_SNIFF_REQUEST_TIMEOUT = TimeUnit.SECONDS.toMillis(1);
+
+ private final RestClient restClient;
+ private final Request request;
+ private final Scheme scheme;
+ private final JsonFactory jsonFactory = new JsonFactory();
+
+ /**
+ * Creates a new instance of the Elasticsearch sniffer. It will use the provided {@link RestClient} to fetch the hosts,
+ * through the nodes info api, the default sniff request timeout value {@link #DEFAULT_SNIFF_REQUEST_TIMEOUT} and http
+ * as the scheme for all the hosts.
+ * @param restClient client used to fetch the hosts from elasticsearch through nodes info api. Usually the same instance
+ * that is also provided to {@link Sniffer#builder(RestClient)}, so that the hosts are set to the same
+ * client that was used to fetch them.
+ */
+ public ElasticsearchNodesSniffer(RestClient restClient) {
+ this(restClient, DEFAULT_SNIFF_REQUEST_TIMEOUT, ElasticsearchNodesSniffer.Scheme.HTTP);
+ }
+
+ /**
+ * Creates a new instance of the Elasticsearch sniffer. It will use the provided {@link RestClient} to fetch the hosts
+ * through the nodes info api, the provided sniff request timeout value and scheme.
+ * @param restClient client used to fetch the hosts from elasticsearch through nodes info api. Usually the same instance
+ * that is also provided to {@link Sniffer#builder(RestClient)}, so that the hosts are set to the same
+ * client that was used to sniff them.
+ * @param sniffRequestTimeoutMillis the sniff request timeout (in milliseconds) to be passed in as a query string parameter
+ * to elasticsearch. Allows to halt the request without any failure, as only the nodes
+ * that have responded within this timeout will be returned.
+ * @param scheme the scheme to associate sniffed nodes with (as it is not returned by elasticsearch)
+ */
+ public ElasticsearchNodesSniffer(RestClient restClient, long sniffRequestTimeoutMillis, Scheme scheme) {
+ this.restClient = Objects.requireNonNull(restClient, "restClient cannot be null");
+ if (sniffRequestTimeoutMillis < 0) {
+ throw new IllegalArgumentException("sniffRequestTimeoutMillis must be greater than 0");
+ }
+ this.request = new Request("GET", "/_nodes/http");
+ request.addParameter("timeout", sniffRequestTimeoutMillis + "ms");
+ this.scheme = Objects.requireNonNull(scheme, "scheme cannot be null");
+ }
+
+ /**
+ * Calls the elasticsearch nodes info api, parses the response and returns all the found http hosts
+ */
+ @Override
+ public List sniff() throws IOException {
+ Response response = restClient.performRequest(request);
+ return readHosts(response.getEntity(), scheme, jsonFactory);
+ }
+
+ 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 nodes = new ArrayList<>();
+ while (parser.nextToken() != JsonToken.END_OBJECT) {
+ if (parser.getCurrentToken() == JsonToken.START_OBJECT) {
+ if ("nodes".equals(parser.getCurrentName())) {
+ while (parser.nextToken() != JsonToken.END_OBJECT) {
+ JsonToken token = parser.nextToken();
+ assert token == JsonToken.START_OBJECT;
+ String nodeId = parser.getCurrentName();
+ Node node = readNode(nodeId, parser, scheme);
+ if (node != null) {
+ nodes.add(node);
+ }
+ }
+ } else {
+ parser.skipChildren();
+ }
+ }
+ }
+ return nodes;
+ }
+ }
+
+ private static Node readNode(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;
+ /*
+ * Multi-valued attributes come with key = `real_key.index` and we
+ * unflip them after reading them because we can't rely on the order
+ * that they arive.
+ */
+ final Map protoAttributes = new HashMap();
+
+ boolean sawRoles = false;
+ boolean master = false;
+ boolean data = false;
+ boolean ingest = false;
+
+ String fieldName = null;
+ while (parser.nextToken() != JsonToken.END_OBJECT) {
+ if (parser.getCurrentToken() == JsonToken.FIELD_NAME) {
+ fieldName = parser.getCurrentName();
+ } else if (parser.getCurrentToken() == JsonToken.START_OBJECT) {
+ if ("http".equals(fieldName)) {
+ while (parser.nextToken() != JsonToken.END_OBJECT) {
+ if (parser.getCurrentToken() == JsonToken.VALUE_STRING && "publish_address".equals(parser.getCurrentName())) {
+ 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) {
+ String oldValue = protoAttributes.put(parser.getCurrentName(), parser.getValueAsString());
+ if (oldValue != null) {
+ throw new IOException("repeated attribute key [" + parser.getCurrentName() + "]");
+ }
+ } else {
+ parser.skipChildren();
+ }
+ }
+ } 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 (publishedHost == null) {
+ logger.debug("skipping node [" + nodeId + "] with http disabled");
+ return null;
+ }
+
+ Map> realAttributes = new HashMap<>(protoAttributes.size());
+ List keys = new ArrayList<>(protoAttributes.keySet());
+ for (String key : keys) {
+ if (key.endsWith(".0")) {
+ String realKey = key.substring(0, key.length() - 2);
+ List values = new ArrayList<>();
+ int i = 0;
+ while (true) {
+ String value = protoAttributes.remove(realKey + "." + i);
+ if (value == null) {
+ break;
+ }
+ values.add(value);
+ i++;
+ }
+ realAttributes.put(realKey, unmodifiableList(values));
+ }
+ }
+ for (Map.Entry entry : protoAttributes.entrySet()) {
+ realAttributes.put(entry.getKey(), singletonList(entry.getValue()));
+ }
+
+ if (version.startsWith("2.")) {
+ /*
+ * 2.x doesn't send roles, instead we try to read them from
+ * attributes.
+ */
+ boolean clientAttribute = v2RoleAttributeValue(realAttributes, "client", false);
+ Boolean masterAttribute = v2RoleAttributeValue(realAttributes, "master", null);
+ Boolean dataAttribute = v2RoleAttributeValue(realAttributes, "data", null);
+ 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";
+ logger.trace("adding node [" + nodeId + "]");
+ return new Node(publishedHost, boundHosts, name, version, new Roles(master, data, ingest),
+ unmodifiableMap(realAttributes));
+ }
+
+ /**
+ * Returns {@code defaultValue} if the attribute didn't come back,
+ * {@code true} or {@code false} if it did come back as
+ * either of those, or throws an IOException if the attribute
+ * came back in a strange way.
+ */
+ private static Boolean v2RoleAttributeValue(Map> attributes,
+ String name, Boolean defaultValue) throws IOException {
+ List valueList = attributes.remove(name);
+ if (valueList == null) {
+ return defaultValue;
+ }
+ if (valueList.size() != 1) {
+ throw new IOException("expected only a single attribute value for [" + name + "] but got "
+ + valueList);
+ }
+ switch (valueList.get(0)) {
+ case "true":
+ return true;
+ case "false":
+ return false;
+ default:
+ throw new IOException("expected [" + name + "] to be either [true] or [false] but was ["
+ + valueList.get(0) + "]");
+ }
+ }
+
+ public enum Scheme {
+ HTTP("http"), HTTPS("https");
+
+ private final String name;
+
+ Scheme(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+}
diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/NodesSniffer.java
similarity index 85%
rename from client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java
rename to client/sniffer/src/main/java/org/elasticsearch/client/sniff/NodesSniffer.java
index 9eb7b34425944..c22c18f6eae32 100644
--- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/HostsSniffer.java
+++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/NodesSniffer.java
@@ -19,7 +19,7 @@
package org.elasticsearch.client.sniff;
-import org.apache.http.HttpHost;
+import org.elasticsearch.client.Node;
import java.io.IOException;
import java.util.List;
@@ -27,9 +27,9 @@
/**
* Responsible for sniffing the http hosts
*/
-public interface HostsSniffer {
+public interface NodesSniffer {
/**
- * Returns the sniffed http hosts
+ * Returns the sniffed Elasticsearch nodes.
*/
- List sniffHosts() throws IOException;
+ List sniff() throws IOException;
}
diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java
index 41051555bae2c..9d5627922823d 100644
--- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java
+++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java
@@ -19,7 +19,7 @@
package org.elasticsearch.client.sniff;
-import org.apache.http.HttpHost;
+import org.elasticsearch.client.Node;
import org.elasticsearch.client.RestClient;
import java.util.Objects;
@@ -54,7 +54,7 @@ public void setSniffer(Sniffer sniffer) {
}
@Override
- public void onFailure(HttpHost host) {
+ public void onFailure(Node node) {
if (sniffer == null) {
throw new IllegalStateException("sniffer was not set, unable to sniff on failure");
}
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 dc873ccd44e10..73780586e7617 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
@@ -21,7 +21,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 +29,7 @@
import java.io.IOException;
import java.security.AccessController;
import java.security.PrivilegedAction;
+import java.util.Collection;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@@ -43,7 +44,7 @@
/**
* Class responsible for sniffing nodes from some source (default is elasticsearch itself) and setting them to a provided instance of
* {@link RestClient}. Must be created via {@link SnifferBuilder}, which allows to set all of the different options or rely on defaults.
- * A background task fetches the nodes through the {@link HostsSniffer} and sets them to the {@link RestClient} instance.
+ * A background task fetches the nodes through the {@link NodesSniffer} and sets them to the {@link RestClient} instance.
* It is possible to perform sniffing on failure by creating a {@link SniffOnFailureListener} and providing it as an argument to
* {@link RestClientBuilder#setFailureListener(RestClient.FailureListener)}. The Sniffer implementation needs to be lazily set to the
* previously created SniffOnFailureListener through {@link SniffOnFailureListener#setSniffer(Sniffer)}.
@@ -53,7 +54,7 @@ public class Sniffer implements Closeable {
private static final Log logger = LogFactory.getLog(Sniffer.class);
private static final String SNIFFER_THREAD_NAME = "es_rest_client_sniffer";
- private final HostsSniffer hostsSniffer;
+ private final NodesSniffer nodesSniffer;
private final RestClient restClient;
private final long sniffIntervalMillis;
private final long sniffAfterFailureDelayMillis;
@@ -61,12 +62,12 @@ public class Sniffer implements Closeable {
private final AtomicBoolean initialized = new AtomicBoolean(false);
private volatile ScheduledTask nextScheduledTask;
- Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) {
- this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay);
+ Sniffer(RestClient restClient, NodesSniffer nodesSniffer, long sniffInterval, long sniffAfterFailureDelay) {
+ this(restClient, nodesSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay);
}
- Sniffer(RestClient restClient, HostsSniffer hostsSniffer, Scheduler scheduler, long sniffInterval, long sniffAfterFailureDelay) {
- this.hostsSniffer = hostsSniffer;
+ Sniffer(RestClient restClient, NodesSniffer nodesSniffer, Scheduler scheduler, long sniffInterval, long sniffAfterFailureDelay) {
+ this.nodesSniffer = nodesSniffer;
this.restClient = restClient;
this.sniffIntervalMillis = sniffInterval;
this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay;
@@ -205,14 +206,14 @@ boolean skip() {
}
final void sniff() throws IOException {
- List sniffedHosts = hostsSniffer.sniffHosts();
+ List sniffedNodes = nodesSniffer.sniff();
if (logger.isDebugEnabled()) {
- logger.debug("sniffed hosts: " + sniffedHosts);
+ logger.debug("sniffed nodes: " + sniffedNodes);
}
- if (sniffedHosts.isEmpty()) {
- logger.warn("no hosts to set, hosts will be updated at the next sniffing round");
+ if (sniffedNodes.isEmpty()) {
+ logger.warn("no nodes to set, nodes will be updated at the next sniffing round");
} else {
- restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()]));
+ restClient.setNodes(sniffedNodes);
}
}
@@ -227,7 +228,8 @@ public void close() {
/**
* Returns a new {@link SnifferBuilder} to help with {@link Sniffer} creation.
*
- * @param restClient the client that gets its hosts set (via {@link RestClient#setHosts(HttpHost...)}) once they are fetched
+ * @param restClient the client that gets its hosts set (via
+ * {@link RestClient#setNodes(Collection)}) once they are fetched
* @return a new instance of {@link SnifferBuilder}
*/
public static SnifferBuilder builder(RestClient restClient) {
diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SnifferBuilder.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SnifferBuilder.java
index 010a8a4a78d20..48ca52d423012 100644
--- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SnifferBuilder.java
+++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SnifferBuilder.java
@@ -34,7 +34,7 @@ public final class SnifferBuilder {
private final RestClient restClient;
private long sniffIntervalMillis = DEFAULT_SNIFF_INTERVAL;
private long sniffAfterFailureDelayMillis = DEFAULT_SNIFF_AFTER_FAILURE_DELAY;
- private HostsSniffer hostsSniffer;
+ private NodesSniffer nodesSniffer;
/**
* Creates a new builder instance by providing the {@link RestClient} that will be used to communicate with elasticsearch
@@ -69,13 +69,13 @@ public SnifferBuilder setSniffAfterFailureDelayMillis(int sniffAfterFailureDelay
}
/**
- * Sets the {@link HostsSniffer} to be used to read hosts. A default instance of {@link ElasticsearchHostsSniffer}
- * is created when not provided. This method can be used to change the configuration of the {@link ElasticsearchHostsSniffer},
+ * Sets the {@link NodesSniffer} to be used to read hosts. A default instance of {@link ElasticsearchNodesSniffer}
+ * is created when not provided. This method can be used to change the configuration of the {@link ElasticsearchNodesSniffer},
* or to provide a different implementation (e.g. in case hosts need to taken from a different source).
*/
- public SnifferBuilder setHostsSniffer(HostsSniffer hostsSniffer) {
- Objects.requireNonNull(hostsSniffer, "hostsSniffer cannot be null");
- this.hostsSniffer = hostsSniffer;
+ public SnifferBuilder setNodesSniffer(NodesSniffer nodesSniffer) {
+ Objects.requireNonNull(nodesSniffer, "nodesSniffer cannot be null");
+ this.nodesSniffer = nodesSniffer;
return this;
}
@@ -83,9 +83,9 @@ public SnifferBuilder setHostsSniffer(HostsSniffer hostsSniffer) {
* Creates the {@link Sniffer} based on the provided configuration.
*/
public Sniffer build() {
- if (hostsSniffer == null) {
- this.hostsSniffer = new ElasticsearchHostsSniffer(restClient);
+ if (nodesSniffer == null) {
+ this.nodesSniffer = new ElasticsearchNodesSniffer(restClient);
}
- return new Sniffer(restClient, hostsSniffer, sniffIntervalMillis, sniffAfterFailureDelayMillis);
+ return new Sniffer(restClient, nodesSniffer, sniffIntervalMillis, sniffAfterFailureDelayMillis);
}
}
diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchNodesSnifferParseTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchNodesSnifferParseTests.java
new file mode 100644
index 0000000000000..edc7330c13074
--- /dev/null
+++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchNodesSnifferParseTests.java
@@ -0,0 +1,121 @@
+/*
+ * 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.ElasticsearchNodesSniffer.Scheme;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.fasterxml.jackson.core.JsonFactory;
+
+import static java.util.Collections.singletonList;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test parsing the response from the {@code /_nodes/http} API from fixed
+ * versions of Elasticsearch.
+ */
+public class ElasticsearchNodesSnifferParseTests 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 = ElasticsearchNodesSniffer.readHosts(entity, Scheme.HTTP, new JsonFactory());
+ /*
+ * Use these assertions because the error messages are nicer
+ * than hasItems and we know the results are in order because
+ * that is how we generated the file.
+ */
+ assertThat(nodes, hasSize(expected.length));
+ for (int i = 0; i < expected.length; i++) {
+ assertEquals(expected[i], nodes.get(i));
+ }
+ } 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(9201, "m2", "2.0.0", true, true, false),
+ node(9202, "m3", "2.0.0", true, false, false),
+ node(9203, "d1", "2.0.0", false, true, false),
+ node(9204, "d2", "2.0.0", false, true, false),
+ node(9205, "d3", "2.0.0", false, true, false),
+ node(9206, "c1", "2.0.0", false, false, false),
+ node(9207, "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));
+ Map> attributes = new HashMap<>();
+ attributes.put("dummy", singletonList("everyone_has_me"));
+ attributes.put("number", singletonList(name.substring(1)));
+ attributes.put("array", Arrays.asList(name.substring(0, 1), name.substring(1)));
+ return new Node(host, boundHosts, name, version, new Roles(master, data, ingest), attributes);
+ }
+}
diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchNodesSnifferTests.java
similarity index 70%
rename from client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java
rename to client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchNodesSnifferTests.java
index ed2744df31c61..3d2a74685afcd 100644
--- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchHostsSnifferTests.java
+++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/ElasticsearchNodesSnifferTests.java
@@ -30,6 +30,7 @@
import org.apache.http.Consts;
import org.apache.http.HttpHost;
import org.apache.http.client.methods.HttpGet;
+import org.elasticsearch.client.Node;
import org.elasticsearch.client.Response;
import org.elasticsearch.client.ResponseException;
import org.elasticsearch.client.RestClient;
@@ -44,10 +45,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;
@@ -59,17 +60,17 @@
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
-public class ElasticsearchHostsSnifferTests extends RestClientTestCase {
+public class ElasticsearchNodesSnifferTests extends RestClientTestCase {
private int sniffRequestTimeout;
- private ElasticsearchHostsSniffer.Scheme scheme;
+ private ElasticsearchNodesSniffer.Scheme scheme;
private SniffResponse sniffResponse;
private HttpServer httpServer;
@Before
public void startHttpServer() throws IOException {
this.sniffRequestTimeout = RandomNumbers.randomIntBetween(getRandom(), 1000, 10000);
- this.scheme = RandomPicks.randomFrom(getRandom(), ElasticsearchHostsSniffer.Scheme.values());
+ this.scheme = RandomPicks.randomFrom(getRandom(), ElasticsearchNodesSniffer.Scheme.values());
if (rarely()) {
this.sniffResponse = SniffResponse.buildFailure();
} else {
@@ -86,7 +87,7 @@ public void stopHttpServer() throws IOException {
public void testConstructorValidation() throws IOException {
try {
- new ElasticsearchHostsSniffer(null, 1, ElasticsearchHostsSniffer.Scheme.HTTP);
+ new ElasticsearchNodesSniffer(null, 1, ElasticsearchNodesSniffer.Scheme.HTTP);
fail("should have failed");
} catch(NullPointerException e) {
assertEquals("restClient cannot be null", e.getMessage());
@@ -94,14 +95,14 @@ public void testConstructorValidation() throws IOException {
HttpHost httpHost = new HttpHost(httpServer.getAddress().getHostString(), httpServer.getAddress().getPort());
try (RestClient restClient = RestClient.builder(httpHost).build()) {
try {
- new ElasticsearchHostsSniffer(restClient, 1, null);
+ new ElasticsearchNodesSniffer(restClient, 1, null);
fail("should have failed");
} catch (NullPointerException e) {
assertEquals(e.getMessage(), "scheme cannot be null");
}
try {
- new ElasticsearchHostsSniffer(restClient, RandomNumbers.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0),
- ElasticsearchHostsSniffer.Scheme.HTTP);
+ new ElasticsearchNodesSniffer(restClient, RandomNumbers.randomIntBetween(getRandom(), Integer.MIN_VALUE, 0),
+ ElasticsearchNodesSniffer.Scheme.HTTP);
fail("should have failed");
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "sniffRequestTimeoutMillis must be greater than 0");
@@ -112,17 +113,13 @@ public void testConstructorValidation() throws IOException {
public void testSniffNodes() throws IOException {
HttpHost httpHost = new HttpHost(httpServer.getAddress().getHostString(), httpServer.getAddress().getPort());
try (RestClient restClient = RestClient.builder(httpHost).build()) {
- ElasticsearchHostsSniffer sniffer = new ElasticsearchHostsSniffer(restClient, sniffRequestTimeout, scheme);
+ ElasticsearchNodesSniffer sniffer = new ElasticsearchNodesSniffer(restClient, sniffRequestTimeout, scheme);
try {
- List sniffedHosts = sniffer.sniffHosts();
+ List sniffedNodes = sniffer.sniff();
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, sniffedNodes);
} catch(ResponseException e) {
Response response = e.getResponse();
if (sniffResponse.isFailure) {
@@ -173,9 +170,9 @@ public void handle(HttpExchange httpExchange) throws IOException {
}
}
- private static SniffResponse buildSniffResponse(ElasticsearchHostsSniffer.Scheme scheme) throws IOException {
+ private static SniffResponse buildSniffResponse(ElasticsearchNodesSniffer.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);
@@ -190,6 +187,35 @@ 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()));
+ }
+ }
+
+ int numAttributes = between(0, 5);
+ Map> attributes = new HashMap<>(numAttributes);
+ for (int j = 0; j < numAttributes; j++) {
+ int numValues = frequently() ? 1 : between(2, 5);
+ List values = new ArrayList<>();
+ for (int v = 0; v < numValues; v++) {
+ values.add(j + "value" + v);
+ }
+ attributes.put("attr" + j, values);
+ }
+
+ Node node = new Node(publishHost, boundHosts, randomAsciiAlphanumOfLength(5),
+ randomAsciiAlphanumOfLength(5),
+ new Node.Roles(randomBoolean(), randomBoolean(), randomBoolean()),
+ attributes);
+
generator.writeObjectFieldStart(nodeId);
if (getRandom().nextBoolean()) {
generator.writeObjectFieldStart("bogus_object");
@@ -203,56 +229,56 @@ 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();
- }
- int numAttributes = RandomNumbers.randomIntBetween(getRandom(), 0, 3);
- Map attributes = new HashMap<>(numAttributes);
- for (int j = 0; j < numAttributes; j++) {
- attributes.put("attr" + j, "value" + j);
}
+ generator.writeEndArray();
+
+ generator.writeFieldName("version");
+ generator.writeString(node.getVersion());
+ generator.writeFieldName("name");
+ generator.writeString(node.getName());
+
if (numAttributes > 0) {
generator.writeObjectFieldStart("attributes");
- }
- for (Map.Entry entry : attributes.entrySet()) {
- generator.writeStringField(entry.getKey(), entry.getValue());
- }
- if (numAttributes > 0) {
+ for (Map.Entry> entry : attributes.entrySet()) {
+ if (entry.getValue().size() == 1) {
+ generator.writeStringField(entry.getKey(), entry.getValue().get(0));
+ } else {
+ for (int v = 0; v < entry.getValue().size(); v++) {
+ generator.writeStringField(entry.getKey() + "." + v, entry.getValue().get(v));
+ }
+ }
+ }
generator.writeEndObject();
}
generator.writeEndObject();
@@ -260,18 +286,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();
@@ -281,11 +307,11 @@ private static class SniffResponse {
}
static SniffResponse buildFailure() {
- return new SniffResponse("", Collections.emptyList(), true);
+ return new SniffResponse("", Collections.emptyList(), true);
}
- static SniffResponse buildResponse(String nodesInfoBody, List hosts) {
- return new SniffResponse(nodesInfoBody, hosts, false);
+ static SniffResponse buildResponse(String nodesInfoBody, List nodes) {
+ return new SniffResponse(nodesInfoBody, nodes, 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/MockNodesSniffer.java
similarity index 78%
rename from client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java
rename to client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockNodesSniffer.java
index 7550459e9ea50..8acd929498e1b 100644
--- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java
+++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockNodesSniffer.java
@@ -20,16 +20,17 @@
package org.elasticsearch.client.sniff;
import org.apache.http.HttpHost;
+import org.elasticsearch.client.Node;
import java.util.Collections;
import java.util.List;
/**
- * Mock implementation of {@link HostsSniffer}. Useful to prevent any connection attempt while testing builders etc.
+ * Mock implementation of {@link NodesSniffer}. Useful to prevent any connection attempt while testing builders etc.
*/
-class MockHostsSniffer implements HostsSniffer {
+class MockNodesSniffer implements NodesSniffer {
@Override
- public List sniffHosts() {
- return Collections.singletonList(new HttpHost("localhost", 9200));
+ public List sniff() {
+ return Collections.singletonList(new Node(new HttpHost("localhost", 9200)));
}
}
diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SniffOnFailureListenerTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SniffOnFailureListenerTests.java
index 1fece270ffe0d..225bdb9a0097e 100644
--- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SniffOnFailureListenerTests.java
+++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SniffOnFailureListenerTests.java
@@ -20,6 +20,7 @@
package org.elasticsearch.client.sniff;
import org.apache.http.HttpHost;
+import org.elasticsearch.client.Node;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.client.RestClientTestCase;
@@ -46,7 +47,7 @@ public void testSetSniffer() throws Exception {
}
try (RestClient restClient = RestClient.builder(new HttpHost("localhost", 9200)).build()) {
- try (Sniffer sniffer = Sniffer.builder(restClient).setHostsSniffer(new MockHostsSniffer()).build()) {
+ try (Sniffer sniffer = Sniffer.builder(restClient).setNodesSniffer(new MockNodesSniffer()).build()) {
listener.setSniffer(sniffer);
try {
listener.setSniffer(sniffer);
@@ -54,7 +55,7 @@ public void testSetSniffer() throws Exception {
} catch(IllegalStateException e) {
assertEquals("sniffer can only be set once", e.getMessage());
}
- listener.onFailure(new HttpHost("localhost", 9200));
+ listener.onFailure(new Node(new HttpHost("localhost", 9200)));
}
}
}
diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java
index 9a7359e9c7215..f924a9fbebc81 100644
--- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java
+++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferBuilderTests.java
@@ -61,10 +61,10 @@ public void testBuild() throws Exception {
try {
- Sniffer.builder(client).setHostsSniffer(null);
+ Sniffer.builder(client).setNodesSniffer(null);
fail("should have failed");
} catch(NullPointerException e) {
- assertEquals("hostsSniffer cannot be null", e.getMessage());
+ assertEquals("nodesSniffer cannot be null", e.getMessage());
}
@@ -80,7 +80,7 @@ public void testBuild() throws Exception {
builder.setSniffAfterFailureDelayMillis(RandomNumbers.randomIntBetween(getRandom(), 1, Integer.MAX_VALUE));
}
if (getRandom().nextBoolean()) {
- builder.setHostsSniffer(new MockHostsSniffer());
+ builder.setNodesSniffer(new MockNodesSniffer());
}
try (Sniffer sniffer = builder.build()) {
diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java
index 8172774a77d80..00c5eb31d17e8 100644
--- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java
+++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java
@@ -20,11 +20,11 @@
package org.elasticsearch.client.sniff;
import org.apache.http.HttpHost;
+import org.elasticsearch.client.Node;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.client.RestClientTestCase;
import org.elasticsearch.client.sniff.Sniffer.DefaultScheduler;
import org.elasticsearch.client.sniff.Sniffer.Scheduler;
-import org.mockito.Matchers;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@@ -62,6 +62,7 @@
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyCollectionOf;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@@ -71,12 +72,12 @@
public class SnifferTests extends RestClientTestCase {
/**
- * Tests the {@link Sniffer#sniff()} method in isolation. Verifies that it uses the {@link HostsSniffer} implementation
+ * Tests the {@link Sniffer#sniff()} method in isolation. Verifies that it uses the {@link NodesSniffer} implementation
* to retrieve nodes and set them (when not empty) to the provided {@link RestClient} instance.
*/
public void testSniff() throws IOException {
- HttpHost initialHost = new HttpHost("localhost", 9200);
- try (RestClient restClient = RestClient.builder(initialHost).build()) {
+ Node initialNode = new Node(new HttpHost("localhost", 9200));
+ try (RestClient restClient = RestClient.builder(initialNode).build()) {
Scheduler noOpScheduler = new Scheduler() {
@Override
public Future> schedule(Sniffer.Task task, long delayMillis) {
@@ -88,53 +89,53 @@ public void shutdown() {
}
};
- CountingHostsSniffer hostsSniffer = new CountingHostsSniffer();
+ CountingNodesSniffer nodesSniffer = new CountingNodesSniffer();
int iters = randomIntBetween(5, 30);
- try (Sniffer sniffer = new Sniffer(restClient, hostsSniffer, noOpScheduler, 1000L, -1)){
+ try (Sniffer sniffer = new Sniffer(restClient, nodesSniffer, noOpScheduler, 1000L, -1)){
{
- assertEquals(1, restClient.getHosts().size());
- HttpHost httpHost = restClient.getHosts().get(0);
- assertEquals("localhost", httpHost.getHostName());
- assertEquals(9200, httpHost.getPort());
+ assertEquals(1, restClient.getNodes().size());
+ Node node = restClient.getNodes().get(0);
+ assertEquals("localhost", node.getHost().getHostName());
+ assertEquals(9200, node.getHost().getPort());
}
int emptyList = 0;
int failures = 0;
int runs = 0;
- List lastHosts = Collections.singletonList(initialHost);
+ List lastNodes = Collections.singletonList(initialNode);
for (int i = 0; i < iters; i++) {
try {
runs++;
sniffer.sniff();
- if (hostsSniffer.failures.get() > failures) {
+ if (nodesSniffer.failures.get() > failures) {
failures++;
- fail("should have failed given that hostsSniffer says it threw an exception");
- } else if (hostsSniffer.emptyList.get() > emptyList) {
+ fail("should have failed given that nodesSniffer says it threw an exception");
+ } else if (nodesSniffer.emptyList.get() > emptyList) {
emptyList++;
- assertEquals(lastHosts, restClient.getHosts());
+ assertEquals(lastNodes, restClient.getNodes());
} else {
- assertNotEquals(lastHosts, restClient.getHosts());
- List expectedHosts = CountingHostsSniffer.buildHosts(runs);
- assertEquals(expectedHosts, restClient.getHosts());
- lastHosts = restClient.getHosts();
+ assertNotEquals(lastNodes, restClient.getNodes());
+ List expectedNodes = CountingNodesSniffer.buildNodes(runs);
+ assertEquals(expectedNodes, restClient.getNodes());
+ lastNodes = restClient.getNodes();
}
} catch(IOException e) {
- if (hostsSniffer.failures.get() > failures) {
+ if (nodesSniffer.failures.get() > failures) {
failures++;
assertEquals("communication breakdown", e.getMessage());
}
}
}
- assertEquals(hostsSniffer.emptyList.get(), emptyList);
- assertEquals(hostsSniffer.failures.get(), failures);
- assertEquals(hostsSniffer.runs.get(), runs);
+ assertEquals(nodesSniffer.emptyList.get(), emptyList);
+ assertEquals(nodesSniffer.failures.get(), failures);
+ assertEquals(nodesSniffer.runs.get(), runs);
}
}
}
/**
- * Test multiple sniffing rounds by mocking the {@link Scheduler} as well as the {@link HostsSniffer}.
+ * Test multiple sniffing rounds by mocking the {@link Scheduler} as well as the {@link NodesSniffer}.
* Simulates the ordinary behaviour of {@link Sniffer} when sniffing on failure is not enabled.
- * The {@link CountingHostsSniffer} doesn't make any network connection but may throw exception or return no hosts, which makes
+ * The {@link CountingNodesSniffer} doesn't make any network connection but may throw exception or return no nodes, which makes
* it possible to verify that errors are properly handled and don't affect subsequent runs and their scheduling.
* The {@link Scheduler} implementation submits rather than scheduling tasks, meaning that it doesn't respect the requested sniff
* delays while allowing to assert that the requested delays for each requested run and the following one are the expected values.
@@ -143,7 +144,7 @@ public void testOrdinarySniffRounds() throws Exception {
final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE);
long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE);
RestClient restClient = mock(RestClient.class);
- CountingHostsSniffer hostsSniffer = new CountingHostsSniffer();
+ CountingNodesSniffer nodesSniffer = new CountingNodesSniffer();
final int iters = randomIntBetween(30, 100);
final Set> futures = new CopyOnWriteArraySet<>();
final CountDownLatch completionLatch = new CountDownLatch(1);
@@ -185,7 +186,7 @@ public void shutdown() {
}
};
try {
- new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay);
+ new Sniffer(restClient, nodesSniffer, scheduler, sniffInterval, sniffAfterFailureDelay);
assertTrue("timeout waiting for sniffing rounds to be completed", completionLatch.await(1000, TimeUnit.MILLISECONDS));
assertEquals(iters, futures.size());
//the last future is the only one that may not be completed yet, as the count down happens
@@ -200,10 +201,10 @@ public void shutdown() {
executor.shutdown();
assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS));
}
- int totalRuns = hostsSniffer.runs.get();
+ int totalRuns = nodesSniffer.runs.get();
assertEquals(iters, totalRuns);
- int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get();
- verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg());
+ int setNodesRuns = totalRuns - nodesSniffer.failures.get() - nodesSniffer.emptyList.get();
+ verify(restClient, times(setNodesRuns)).setNodes(anyCollectionOf(Node.class));
verifyNoMoreInteractions(restClient);
}
@@ -234,7 +235,7 @@ public void shutdown() {
}
};
- Sniffer sniffer = new Sniffer(restClient, new MockHostsSniffer(), scheduler, sniffInterval, sniffAfterFailureDelay);
+ Sniffer sniffer = new Sniffer(restClient, new MockNodesSniffer(), scheduler, sniffInterval, sniffAfterFailureDelay);
assertEquals(0, shutdown.get());
int iters = randomIntBetween(3, 10);
for (int i = 1; i <= iters; i++) {
@@ -246,7 +247,7 @@ public void shutdown() {
public void testSniffOnFailureNotInitialized() {
RestClient restClient = mock(RestClient.class);
- CountingHostsSniffer hostsSniffer = new CountingHostsSniffer();
+ CountingNodesSniffer nodesSniffer = new CountingNodesSniffer();
long sniffInterval = randomLongBetween(1, Long.MAX_VALUE);
long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE);
final AtomicInteger scheduleCalls = new AtomicInteger(0);
@@ -262,15 +263,15 @@ public void shutdown() {
}
};
- Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay);
+ Sniffer sniffer = new Sniffer(restClient, nodesSniffer, scheduler, sniffInterval, sniffAfterFailureDelay);
for (int i = 0; i < 10; i++) {
sniffer.sniffOnFailure();
}
assertEquals(1, scheduleCalls.get());
- int totalRuns = hostsSniffer.runs.get();
+ int totalRuns = nodesSniffer.runs.get();
assertEquals(0, totalRuns);
- int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get();
- verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg());
+ int setNodesRuns = totalRuns - nodesSniffer.failures.get() - nodesSniffer.emptyList.get();
+ verify(restClient, times(setNodesRuns)).setNodes(anyCollectionOf(Node.class));
verifyNoMoreInteractions(restClient);
}
@@ -281,7 +282,7 @@ public void shutdown() {
*/
public void testSniffOnFailure() throws Exception {
RestClient restClient = mock(RestClient.class);
- CountingHostsSniffer hostsSniffer = new CountingHostsSniffer();
+ CountingNodesSniffer nodesSniffer = new CountingNodesSniffer();
final AtomicBoolean initializing = new AtomicBoolean(true);
final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE);
final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE);
@@ -351,7 +352,7 @@ private Future> scheduleOrSubmit(Sniffer.Task task) {
public void shutdown() {
}
};
- final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay);
+ final Sniffer sniffer = new Sniffer(restClient, nodesSniffer, scheduler, sniffInterval, sniffAfterFailureDelay);
assertTrue("timeout waiting for sniffer to get initialized", initializingLatch.await(1000, TimeUnit.MILLISECONDS));
ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(5, 20));
@@ -413,9 +414,9 @@ public void run() {
}
assertEquals(onFailureTasks.size(), cancelledTasks);
- assertEquals(completedTasks, hostsSniffer.runs.get());
- int setHostsRuns = hostsSniffer.runs.get() - hostsSniffer.failures.get() - hostsSniffer.emptyList.get();
- verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg());
+ assertEquals(completedTasks, nodesSniffer.runs.get());
+ int setNodesRuns = nodesSniffer.runs.get() - nodesSniffer.failures.get() - nodesSniffer.emptyList.get();
+ verify(restClient, times(setNodesRuns)).setNodes(anyCollectionOf(Node.class));
verifyNoMoreInteractions(restClient);
} finally {
executor.shutdown();
@@ -446,7 +447,7 @@ private static boolean assertTaskCancelledOrCompleted(Sniffer.ScheduledTask task
public void testTaskCancelling() throws Exception {
RestClient restClient = mock(RestClient.class);
- HostsSniffer hostsSniffer = mock(HostsSniffer.class);
+ NodesSniffer nodesSniffer = mock(NodesSniffer.class);
Scheduler noOpScheduler = new Scheduler() {
@Override
public Future> schedule(Sniffer.Task task, long delayMillis) {
@@ -457,7 +458,7 @@ public Future> schedule(Sniffer.Task task, long delayMillis) {
public void shutdown() {
}
};
- Sniffer sniffer = new Sniffer(restClient, hostsSniffer, noOpScheduler, 0L, 0L);
+ Sniffer sniffer = new Sniffer(restClient, nodesSniffer, noOpScheduler, 0L, 0L);
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
try {
int numIters = randomIntBetween(50, 100);
@@ -540,18 +541,18 @@ boolean await() throws InterruptedException {
}
/**
- * Mock {@link HostsSniffer} implementation used for testing, which most of the times return a fixed host.
- * It rarely throws exception or return an empty list of hosts, to make sure that such situations are properly handled.
+ * Mock {@link NodesSniffer} implementation used for testing, which most of the times return a fixed node.
+ * It rarely throws exception or return an empty list of nodes, to make sure that such situations are properly handled.
* It also asserts that it never gets called concurrently, based on the assumption that only one sniff run can be run
* at a given point in time.
*/
- private static class CountingHostsSniffer implements HostsSniffer {
+ private static class CountingNodesSniffer implements NodesSniffer {
private final AtomicInteger runs = new AtomicInteger(0);
private final AtomicInteger failures = new AtomicInteger(0);
private final AtomicInteger emptyList = new AtomicInteger(0);
@Override
- public List sniffHosts() throws IOException {
+ public List sniff() throws IOException {
int run = runs.incrementAndGet();
if (rarely()) {
failures.incrementAndGet();
@@ -562,24 +563,23 @@ public List sniffHosts() throws IOException {
emptyList.incrementAndGet();
return Collections.emptyList();
}
- return buildHosts(run);
+ return buildNodes(run);
}
- private static List buildHosts(int run) {
+ private static List buildNodes(int run) {
int size = run % 5 + 1;
assert size > 0;
- List hosts = new ArrayList<>(size);
+ List nodes = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
- hosts.add(new HttpHost("sniffed-" + run, 9200 + i));
+ nodes.add(new Node(new HttpHost("sniffed-" + run, 9200 + i)));
}
- return hosts;
+ return nodes;
}
}
- @SuppressWarnings("unchecked")
public void testDefaultSchedulerSchedule() {
RestClient restClient = mock(RestClient.class);
- HostsSniffer hostsSniffer = mock(HostsSniffer.class);
+ NodesSniffer nodesSniffer = mock(NodesSniffer.class);
Scheduler noOpScheduler = new Scheduler() {
@Override
public Future> schedule(Sniffer.Task task, long delayMillis) {
@@ -591,7 +591,7 @@ public void shutdown() {
}
};
- Sniffer sniffer = new Sniffer(restClient, hostsSniffer, noOpScheduler, 0L, 0L);
+ Sniffer sniffer = new Sniffer(restClient, nodesSniffer, noOpScheduler, 0L, 0L);
Sniffer.Task task = sniffer.new Task(randomLongBetween(1, Long.MAX_VALUE));
ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class);
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..5f305024dba20 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,9 +20,10 @@
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;
+import org.elasticsearch.client.sniff.ElasticsearchNodesSniffer;
+import org.elasticsearch.client.sniff.NodesSniffer;
import org.elasticsearch.client.sniff.SniffOnFailureListener;
import org.elasticsearch.client.sniff.Sniffer;
@@ -91,12 +92,12 @@ public void testUsage() throws IOException {
RestClient restClient = RestClient.builder(
new HttpHost("localhost", 9200, "http"))
.build();
- HostsSniffer hostsSniffer = new ElasticsearchHostsSniffer(
+ NodesSniffer nodesSniffer = new ElasticsearchNodesSniffer(
restClient,
- ElasticsearchHostsSniffer.DEFAULT_SNIFF_REQUEST_TIMEOUT,
- ElasticsearchHostsSniffer.Scheme.HTTPS);
+ ElasticsearchNodesSniffer.DEFAULT_SNIFF_REQUEST_TIMEOUT,
+ ElasticsearchNodesSniffer.Scheme.HTTPS);
Sniffer sniffer = Sniffer.builder(restClient)
- .setHostsSniffer(hostsSniffer).build();
+ .setNodesSniffer(nodesSniffer).build();
//end::sniffer-https
}
{
@@ -104,28 +105,28 @@ public void testUsage() throws IOException {
RestClient restClient = RestClient.builder(
new HttpHost("localhost", 9200, "http"))
.build();
- HostsSniffer hostsSniffer = new ElasticsearchHostsSniffer(
+ NodesSniffer nodesSniffer = new ElasticsearchNodesSniffer(
restClient,
TimeUnit.SECONDS.toMillis(5),
- ElasticsearchHostsSniffer.Scheme.HTTP);
+ ElasticsearchNodesSniffer.Scheme.HTTP);
Sniffer sniffer = Sniffer.builder(restClient)
- .setHostsSniffer(hostsSniffer).build();
+ .setNodesSniffer(nodesSniffer).build();
//end::sniff-request-timeout
}
{
- //tag::custom-hosts-sniffer
+ //tag::custom-nodes-sniffer
RestClient restClient = RestClient.builder(
new HttpHost("localhost", 9200, "http"))
.build();
- HostsSniffer hostsSniffer = new HostsSniffer() {
+ NodesSniffer nodesSniffer = new NodesSniffer() {
@Override
- public List sniffHosts() throws IOException {
+ public List sniff() throws IOException {
return null; // <1>
}
};
Sniffer sniffer = Sniffer.builder(restClient)
- .setHostsSniffer(hostsSniffer).build();
- //end::custom-hosts-sniffer
+ .setNodesSniffer(nodesSniffer).build();
+ //end::custom-nodes-sniffer
}
}
}
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..22dc4ec13ed51
--- /dev/null
+++ b/client/sniffer/src/test/resources/2.0.0_nodes_http.json
@@ -0,0 +1,201 @@
+{
+ "cluster_name": "elasticsearch",
+ "nodes": {
+ "qr-SOrELSaGW8SlU8nflBw": {
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "m",
+ "data": "false",
+ "array.1": "1",
+ "master": "true"
+ },
+ "http": {
+ "bound_address": [
+ "127.0.0.1:9200",
+ "[::1]:9200"
+ ],
+ "publish_address": "127.0.0.1:9200",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "osfiXxUOQzCVIs-eepgSCA": {
+ "name": "m2",
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "m",
+ "array.1": "2",
+ "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
+ }
+ },
+ "lazeJFiIQ8eHHV4GeIdMPg": {
+ "name": "m3",
+ "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",
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "3",
+ "array.0": "m",
+ "data": "false",
+ "array.1": "3",
+ "master": "true"
+ },
+ "http": {
+ "bound_address": [
+ "127.0.0.1:9202",
+ "[::1]:9202"
+ ],
+ "publish_address": "127.0.0.1:9202",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "t9WxK-fNRsqV5G0Mm09KpQ": {
+ "name": "d1",
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "d",
+ "array.1": "1",
+ "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
+ }
+ },
+ "wgoDzluvTViwUjEsmVesKw": {
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "d",
+ "array.1": "2",
+ "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
+ }
+ },
+ "6j_t3pPhSm-oRTyypTzu5g": {
+ "name": "d3",
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "3",
+ "array.0": "d",
+ "array.1": "3",
+ "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
+ }
+ },
+ "PaEkm0z7Ssiuyfkh3aASag": {
+ "name": "c1",
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "c",
+ "data": "false",
+ "array.1": "1",
+ "master": "false"
+ },
+ "http": {
+ "bound_address": [
+ "127.0.0.1:9206",
+ "[::1]:9206"
+ ],
+ "publish_address": "127.0.0.1:9206",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "LAFKr2K_QmupqnM_atJqkQ": {
+ "name": "c2",
+ "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": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "c",
+ "data": "false",
+ "array.1": "2",
+ "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
+ }
+ }
+ }
+}
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..1358438237fc8
--- /dev/null
+++ b/client/sniffer/src/test/resources/5.0.0_nodes_http.json
@@ -0,0 +1,217 @@
+{
+ "_nodes": {
+ "total": 8,
+ "successful": 8,
+ "failed": 0
+ },
+ "cluster_name": "elasticsearch",
+ "nodes": {
+ "0S4r3NurTYSFSb8R9SxwWA": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "m",
+ "array.1": "1"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9200",
+ "127.0.0.1:9200"
+ ],
+ "publish_address": "127.0.0.1:9200",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "k_CBrMXARkS57Qb5-3Mw5g": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "m",
+ "array.1": "2"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9201",
+ "127.0.0.1:9201"
+ ],
+ "publish_address": "127.0.0.1:9201",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "6eynRPQ1RleJTeGDuTR9mw": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "3",
+ "array.0": "m",
+ "array.1": "3"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9202",
+ "127.0.0.1:9202"
+ ],
+ "publish_address": "127.0.0.1:9202",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "cbGC-ay1QNWaESvEh5513w": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "d",
+ "array.1": "1"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9203",
+ "127.0.0.1:9203"
+ ],
+ "publish_address": "127.0.0.1:9203",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "LexndPpXR2ytYsU5fTElnQ": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "d",
+ "array.1": "2"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9204",
+ "127.0.0.1:9204"
+ ],
+ "publish_address": "127.0.0.1:9204",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "SbNG1DKYSBu20zfOz2gDZQ": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "3",
+ "array.0": "d",
+ "array.1": "3"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9205",
+ "127.0.0.1:9205"
+ ],
+ "publish_address": "127.0.0.1:9205",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "fM4H-m2WTDWmsGsL7jIJew": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "c",
+ "array.1": "1"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9206",
+ "127.0.0.1:9206"
+ ],
+ "publish_address": "127.0.0.1:9206",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "pFoh7d0BTbqqI3HKd9na5A": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "c",
+ "array.1": "2"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9207",
+ "127.0.0.1:9207"
+ ],
+ "publish_address": "127.0.0.1:9207",
+ "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..f0535dfdfb00f
--- /dev/null
+++ b/client/sniffer/src/test/resources/6.0.0_nodes_http.json
@@ -0,0 +1,217 @@
+{
+ "_nodes": {
+ "total": 8,
+ "successful": 8,
+ "failed": 0
+ },
+ "cluster_name": "elasticsearch",
+ "nodes": {
+ "ikXK_skVTfWkhONhldnbkw": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "m",
+ "array.1": "1"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9200",
+ "127.0.0.1:9200"
+ ],
+ "publish_address": "127.0.0.1:9200",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "TMHa34w4RqeuYoHCfJGXZg": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "m",
+ "array.1": "2"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9201",
+ "127.0.0.1:9201"
+ ],
+ "publish_address": "127.0.0.1:9201",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "lzaMRJTVT166sgVZdQ5thA": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "3",
+ "array.0": "m",
+ "array.1": "3"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9202",
+ "127.0.0.1:9202"
+ ],
+ "publish_address": "127.0.0.1:9202",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "tGP5sUecSd6BLTWk1NWF8Q": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "d",
+ "array.1": "1"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9203",
+ "127.0.0.1:9203"
+ ],
+ "publish_address": "127.0.0.1:9203",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "c1UgW5ROTkSa2YnM_T56tw": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "d",
+ "array.1": "2"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9204",
+ "127.0.0.1:9204"
+ ],
+ "publish_address": "127.0.0.1:9204",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "QM9yjqjmS72MstpNYV_trg": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "3",
+ "array.0": "d",
+ "array.1": "3"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9205",
+ "127.0.0.1:9205"
+ ],
+ "publish_address": "127.0.0.1:9205",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "wLtzAssoQYeX_4TstgCj0Q": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "1",
+ "array.0": "c",
+ "array.1": "1"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9206",
+ "127.0.0.1:9206"
+ ],
+ "publish_address": "127.0.0.1:9206",
+ "max_content_length_in_bytes": 104857600
+ }
+ },
+ "ONOzpst8TH-ZebG7fxGwaA": {
+ "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"
+ ],
+ "attributes": {
+ "dummy": "everyone_has_me",
+ "number": "2",
+ "array.0": "c",
+ "array.1": "2"
+ },
+ "http": {
+ "bound_address": [
+ "[::1]:9207",
+ "127.0.0.1:9207"
+ ],
+ "publish_address": "127.0.0.1:9207",
+ "max_content_length_in_bytes": 104857600
+ }
+ }
+ }
+}
diff --git a/client/sniffer/src/test/resources/create_test_nodes_info.bash b/client/sniffer/src/test/resources/create_test_nodes_info.bash
new file mode 100644
index 0000000000000..f4f1c09882ea8
--- /dev/null
+++ b/client/sniffer/src/test/resources/create_test_nodes_info.bash
@@ -0,0 +1,107 @@
+#!/bin/bash
+
+# Recreates the v_nodes_http.json files in this directory. This is
+# meant to be an "every once in a while" thing that we do only when
+# we want to add a new version of Elasticsearch or configure the
+# nodes differently. That is why we don't do this in gradle. It also
+# allows us to play fast and loose with error handling. If something
+# goes wrong you have to manually clean up which is good because it
+# leaves around the kinds of things that we need to debug the failure.
+
+# I built this file so the next time I have to regenerate these
+# v_nodes_http.json files I won't have to reconfigure Elasticsearch
+# from scratch. While I was at it I took the time to make sure that
+# when we do rebuild the files they don't jump around too much. That
+# way the diffs are smaller.
+
+set -e
+
+script_path="$( cd "$(dirname "$0")" ; pwd -P )"
+work=$(mktemp -d)
+pushd ${work} >> /dev/null
+echo Working in ${work}
+
+wget https://download.elasticsearch.org/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.0.0/elasticsearch-2.0.0.tar.gz
+wget https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.0.0.tar.gz
+wget https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.0.0.tar.gz
+sha1sum -c - << __SHAs
+e369d8579bd3a2e8b5344278d5043f19f14cac88 elasticsearch-2.0.0.tar.gz
+d25f6547bccec9f0b5ea7583815f96a6f50849e0 elasticsearch-5.0.0.tar.gz
+__SHAs
+sha512sum -c - << __SHAs
+25bb622d2fc557d8b8eded634a9b333766f7b58e701359e1bcfafee390776eb323cb7ea7a5e02e8803e25d8b1d3aabec0ec1b0cf492d0bab5689686fe440181c elasticsearch-6.0.0.tar.gz
+__SHAs
+
+
+function do_version() {
+ local version=$1
+ local nodes='m1 m2 m3 d1 d2 d3 c1 c2'
+ rm -rf ${version}
+ mkdir -p ${version}
+ pushd ${version} >> /dev/null
+
+ tar xf ../elasticsearch-${version}.tar.gz
+ local http_port=9200
+ for node in ${nodes}; do
+ mkdir ${node}
+ cp -r elasticsearch-${version}/* ${node}
+ local master=$([[ "$node" =~ ^m.* ]] && echo true || echo false)
+ local data=$([[ "$node" =~ ^d.* ]] && echo true || echo false)
+ # m2 is always master and data for these test just so we have a node like that
+ data=$([[ "$node" == 'm2' ]] && echo true || echo ${data})
+ local attr=$([ ${version} == '2.0.0' ] && echo '' || echo '.attr')
+ local transport_port=$((http_port+100))
+
+ cat >> ${node}/config/elasticsearch.yml << __ES_YML
+node.name: ${node}
+node.master: ${master}
+node.data: ${data}
+node${attr}.dummy: everyone_has_me
+node${attr}.number: ${node:1}
+node${attr}.array: [${node:0:1}, ${node:1}]
+http.port: ${http_port}
+transport.tcp.port: ${transport_port}
+discovery.zen.minimum_master_nodes: 3
+discovery.zen.ping.unicast.hosts: ['localhost:9300','localhost:9301','localhost:9302']
+__ES_YML
+
+ if [ ${version} != '2.0.0' ]; then
+ perl -pi -e 's/-Xm([sx]).+/-Xm${1}512m/g' ${node}/config/jvm.options
+ fi
+
+ echo "starting ${version}/${node}..."
+ ${node}/bin/elasticsearch -d -p ${node}/pidfile
+
+ ((http_port++))
+ done
+
+ echo "waiting for cluster to form"
+ # got to wait for all the nodes
+ until curl -s localhost:9200; do
+ sleep .25
+ done
+
+ echo "waiting for all nodes to join"
+ until [ $(echo ${nodes} | wc -w) -eq $(curl -s localhost:9200/_cat/nodes | wc -l) ]; do
+ sleep .25
+ done
+
+ # jq sorts the nodes by their http host so the file doesn't jump around when we regenerate it
+ curl -s localhost:9200/_nodes/http?pretty \
+ | jq '[to_entries[] | ( select(.key == "nodes").value|to_entries|sort_by(.value.http.publish_address)|from_entries|{"key": "nodes", "value": .} ) // .] | from_entries' \
+ > ${script_path}/${version}_nodes_http.json
+
+ for node in ${nodes}; do
+ echo "stopping ${version}/${node}..."
+ kill $(cat ${node}/pidfile)
+ done
+
+ popd >> /dev/null
+}
+
+JAVA_HOME=$JAVA8_HOME do_version 2.0.0
+JAVA_HOME=$JAVA8_HOME do_version 5.0.0
+JAVA_HOME=$JAVA8_HOME do_version 6.0.0
+
+popd >> /dev/null
+rm -rf ${work}
diff --git a/client/sniffer/src/test/resources/readme.txt b/client/sniffer/src/test/resources/readme.txt
new file mode 100644
index 0000000000000..c6dd32a0410a5
--- /dev/null
+++ b/client/sniffer/src/test/resources/readme.txt
@@ -0,0 +1,6 @@
+`*_node_http.json` contains files created by spinning up toy clusters with a
+few nodes in different configurations locally at various versions. They are
+for testing `ElasticsearchNodesSniffer` against different versions of
+Elasticsearch.
+
+See create_test_nodes_info.bash for how to create these.
diff --git a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/DefaultShardsIT.java b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/DefaultShardsIT.java
index de736c84e45b6..74edfbd189a4c 100644
--- a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/DefaultShardsIT.java
+++ b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/DefaultShardsIT.java
@@ -19,6 +19,7 @@
package org.elasticsearch.test.rest;
+import org.apache.http.util.EntityUtils;
import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response;
@@ -30,11 +31,20 @@
public class DefaultShardsIT extends ESRestTestCase {
+ @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/31408")
public void testDefaultShards() throws IOException {
final Response response = client().performRequest(new Request("PUT", "/index"));
final String warning = response.getHeader("Warning");
+ if (warning == null) {
+ StringBuilder explanation = new StringBuilder("expected response to contain a warning but did not ");
+ explanation.append(response);
+ if (response.getEntity() != null) {
+ explanation.append(" entity:\n").append(EntityUtils.toString(response.getEntity()));
+ }
+ fail(explanation.toString());
+ }
final Matcher matcher = WARNING_HEADER_PATTERN.matcher(warning);
- assertTrue(matcher.matches());
+ assertTrue("warning didn't match warning header pattern but was [" + warning + "]", matcher.matches());
final String message = matcher.group(1);
assertThat(message, equalTo("the default number of shards will change from [5] to [1] in 7.0.0; "
+ "if you wish to continue using the default of [5] shards, "
diff --git a/docs/Versions.asciidoc b/docs/Versions.asciidoc
index e62bd0df9ff63..427825a77ddd7 100644
--- a/docs/Versions.asciidoc
+++ b/docs/Versions.asciidoc
@@ -5,6 +5,7 @@
:branch: 6.x
:jdk: 1.8.0_131
:jdk_major: 8
+:build_flavor: default
//////////
release-state can be: released | prerelease | unreleased
diff --git a/docs/build.gradle b/docs/build.gradle
index c6ded0292bc92..f28de4fa6fde9 100644
--- a/docs/build.gradle
+++ b/docs/build.gradle
@@ -20,7 +20,6 @@
apply plugin: 'elasticsearch.docs-test'
integTestCluster {
- distribution = 'oss-zip'
/* Enable regexes in painless so our tests don't complain about example
* snippets that use them. */
setting 'script.painless.regex.enabled', 'true'
diff --git a/docs/java-api/docs/index_.asciidoc b/docs/java-api/docs/index_.asciidoc
index b455a7ab01ff3..2ce19cfffa098 100644
--- a/docs/java-api/docs/index_.asciidoc
+++ b/docs/java-api/docs/index_.asciidoc
@@ -99,11 +99,13 @@ Note that you can also add arrays with `startArray(String)` and
other XContentBuilder objects.
If you need to see the generated JSON content, you can use the
-`string()` method.
+`Strings.toString()` method.
[source,java]
--------------------------------------------------
-String json = builder.string();
+import org.elasticsearch.common.Strings;
+
+String json = Strings.toString(builder);
--------------------------------------------------
diff --git a/docs/java-rest/high-level/getting-started.asciidoc b/docs/java-rest/high-level/getting-started.asciidoc
index 14a5058eb7272..3e9b9fa7ea08f 100644
--- a/docs/java-rest/high-level/getting-started.asciidoc
+++ b/docs/java-rest/high-level/getting-started.asciidoc
@@ -144,3 +144,13 @@ include-tagged::{doc-tests}/MiscellaneousDocumentationIT.java[rest-high-level-cl
In the rest of this documentation about the Java High Level Client, the `RestHighLevelClient` instance
will be referenced as `client`.
+
+[[java-rest-hight-getting-started-request-options]]
+=== RequestOptions
+
+All APIs in the `RestHighLevelClient` accept a `RequestOptions` which you can
+use to customize the request in ways that won't change how Elasticsearch
+executes the request. For example, this is the place where you'd specify a
+`NodeSelector` to control which node receives the request. See the
+<> for
+more examples of customizing the options.
diff --git a/docs/java-rest/high-level/indices/validate_query.asciidoc b/docs/java-rest/high-level/indices/validate_query.asciidoc
new file mode 100644
index 0000000000000..3b3b184b02875
--- /dev/null
+++ b/docs/java-rest/high-level/indices/validate_query.asciidoc
@@ -0,0 +1,113 @@
+[[java-rest-high-indices-validate-query]]
+=== Validate Query API
+
+[[java-rest-high-indices-validate-query-request]]
+==== Validate Query Request
+
+A `ValidateQueryRequest` requires one or more `indices` on which the query is validated. If no index
+is provided the request is executed on all indices.
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-request]
+--------------------------------------------------
+<1> The index on which to run the request.
+
+In addition it also needs the query that needs to be validated. The query can be built using the `QueryBuilders` utility class.
+The following code snippet builds a sample boolean query.
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-request-query]
+--------------------------------------------------
+<1> Build the desired query.
+<2> Set it to the request.
+
+==== Optional arguments
+The following arguments can optionally be provided:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-request-explain]
+--------------------------------------------------
+<1> The explain parameter can be set to true to get more detailed information about why a query failed
+
+By default, the request is executed on a single shard only, which is randomly selected. The detailed explanation of
+the query may depend on which shard is being hit, and therefore may vary from one request to another. So, in case of
+query rewrite the `allShards` parameter should be used to get response from all available shards.
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-request-allShards]
+--------------------------------------------------
+<1> Set the allShards parameter.
+
+When the query is valid, the explanation defaults to the string representation of that query. With rewrite set to true,
+the explanation is more detailed showing the actual Lucene query that will be executed
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-request-rewrite]
+--------------------------------------------------
+<1> Set the rewrite parameter.
+
+[[java-rest-high-indices-validate-query-sync]]
+==== Synchronous Execution
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-execute]
+--------------------------------------------------
+<1> Execute the request and get back the response in a ValidateQueryResponse object.
+
+[[java-rest-high-indices-validate-query-async]]
+==== Asynchronous Execution
+
+The asynchronous execution of a validate query request requires both the `ValidateQueryRequest`
+instance and an `ActionListener` instance to be passed to the asynchronous
+method:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-execute-async]
+--------------------------------------------------
+<1> The `ValidateQueryRequest` to execute and the `ActionListener` to use when
+the execution completes
+
+The asynchronous method does not block and returns immediately. Once it is
+completed the `ActionListener` is called back using the `onResponse` method
+if the execution successfully completed or using the `onFailure` method if
+it failed.
+
+A typical listener for `ValidateQueryResponse` looks like:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-execute-listener]
+--------------------------------------------------
+<1> Called when the execution is successfully completed. The response is
+provided as an argument
+<2> Called in case of failure. The raised exception is provided as an argument
+
+[[java-rest-high-indices-validate-query-response]]
+==== Validate Query Response
+
+The returned `ValidateQueryResponse` allows to retrieve information about the executed
+ operation as follows:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[validate-query-response]
+--------------------------------------------------
+<1> Check if the query is valid or not.
+<2> Get total number of shards.
+<3> Get number of shards that were successful.
+<4> Get number of shards that failed.
+<5> Get the shard failures as `DefaultShardOperationFailedException`.
+<6> Get the index of a failed shard.
+<7> Get the shard id of a failed shard.
+<8> Get the reason for shard failure.
+<9> Get the detailed explanation for the shards (if explain was set to `true`).
+<10> Get the index to which a particular explanation belongs.
+<11> Get the shard id to which a particular explanation belongs.
+<12> Get the actual explanation string.
\ No newline at end of file
diff --git a/docs/java-rest/high-level/script/delete_script.asciidoc b/docs/java-rest/high-level/script/delete_script.asciidoc
new file mode 100644
index 0000000000000..79b3b0b324715
--- /dev/null
+++ b/docs/java-rest/high-level/script/delete_script.asciidoc
@@ -0,0 +1,81 @@
+[[java-rest-high-delete-stored-script]]
+
+=== Delete Stored Script API
+
+[[java-rest-high-delete-stored-script-request]]
+==== Delete Stored Script Request
+
+A `DeleteStoredScriptRequest` requires an `id`:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-request]
+--------------------------------------------------
+<1> The id of the script
+
+==== Optional arguments
+The following arguments can optionally be provided:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-request-timeout]
+--------------------------------------------------
+<1> Timeout to wait for the all the nodes to acknowledge the stored script is deleted as a `TimeValue`
+<2> Timeout to wait for the all the nodes to acknowledge the stored script is deleted as a `String`
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-request-masterTimeout]
+--------------------------------------------------
+<1> Timeout to connect to the master node as a `TimeValue`
+<2> Timeout to connect to the master node as a `String`
+
+[[java-rest-high-delete-stored-script-sync]]
+==== Synchronous Execution
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-execute]
+--------------------------------------------------
+
+[[java-rest-high-delete-stored-script-async]]
+==== Asynchronous Execution
+
+The asynchronous execution of a delete stored script request requires both the `DeleteStoredScriptRequest`
+instance and an `ActionListener` instance to be passed to the asynchronous method:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-execute-async]
+--------------------------------------------------
+<1> The `DeleteStoredScriptRequest` to execute and the `ActionListener` to use when
+the execution completes
+
+[[java-rest-high-delete-stored-script-listener]]
+===== Action Listener
+
+The asynchronous method does not block and returns immediately. Once it is
+completed the `ActionListener` is called back using the `onResponse` method
+if the execution successfully completed or using the `onFailure` method if
+it failed.
+
+A typical listener for `DeleteStoredScriptResponse` looks like:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-execute-listener]
+--------------------------------------------------
+<1> Called when the execution is successfully completed. The response is
+provided as an argument
+<2> Called in case of failure. The raised exception is provided as an argument
+
+[[java-rest-high-delete-stored-script-response]]
+==== Delete Stored Script Response
+
+The returned `DeleteStoredScriptResponse` allows to retrieve information about the
+executed operation as follows:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[delete-stored-script-response]
+--------------------------------------------------
+<1> Indicates whether all of the nodes have acknowledged the request
\ No newline at end of file
diff --git a/docs/java-rest/high-level/script/get_script.asciidoc b/docs/java-rest/high-level/script/get_script.asciidoc
new file mode 100644
index 0000000000000..a38bdad2bd6af
--- /dev/null
+++ b/docs/java-rest/high-level/script/get_script.asciidoc
@@ -0,0 +1,77 @@
+[[java-rest-high-get-stored-script]]
+
+=== Get Stored Script API
+
+[[java-rest-high-get-stored-script-request]]
+==== Get Stored Script Request
+
+A `GetStoredScriptRequest` requires an `id`:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[get-stored-script-request]
+--------------------------------------------------
+<1> The id of the script
+
+==== Optional arguments
+The following arguments can optionally be provided:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[get-stored-script-request-masterTimeout]
+--------------------------------------------------
+<1> Timeout to connect to the master node as a `TimeValue`
+<2> Timeout to connect to the master node as a `String`
+
+[[java-rest-high-get-stored-script-sync]]
+==== Synchronous Execution
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[get-stored-script-execute]
+--------------------------------------------------
+
+[[java-rest-high-get-stored-script-async]]
+==== Asynchronous Execution
+
+The asynchronous execution of a get stored script request requires both the `GetStoredScriptRequest`
+instance and an `ActionListener` instance to be passed to the asynchronous method:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[get-stored-script-execute-async]
+--------------------------------------------------
+<1> The `GetStoredScriptRequest` to execute and the `ActionListener` to use when
+the execution completes
+
+[[java-rest-high-get-stored-script-listener]]
+===== Action Listener
+
+The asynchronous method does not block and returns immediately. Once it is
+completed the `ActionListener` is called back using the `onResponse` method
+if the execution successfully completed or using the `onFailure` method if
+it failed.
+
+A typical listener for `GetStoredScriptResponse` looks like:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[get-stored-script-execute-listener]
+--------------------------------------------------
+<1> Called when the execution is successfully completed. The response is
+provided as an argument
+<2> Called in case of failure. The raised exception is provided as an argument
+
+[[java-rest-high-get-stored-script-response]]
+==== Get Stored Script Response
+
+The returned `GetStoredScriptResponse` allows to retrieve information about the
+executed operation as follows:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/StoredScriptsDocumentationIT.java[get-stored-script-response]
+--------------------------------------------------
+<1> The script object consists of a content and a metadata
+<2> The language the script is written in, which defaults to `painless`.
+<3> The content of the script
+<4> Any named options that should be passed into the script.
\ No newline at end of file
diff --git a/docs/java-rest/high-level/supported-apis.asciidoc b/docs/java-rest/high-level/supported-apis.asciidoc
index b33c2421b06d3..17acc8f13c04d 100644
--- a/docs/java-rest/high-level/supported-apis.asciidoc
+++ b/docs/java-rest/high-level/supported-apis.asciidoc
@@ -73,6 +73,7 @@ Index Management::
* <>
* <>
* <>
+* <>
Mapping Management::
* <>
@@ -103,6 +104,7 @@ include::indices/get_alias.asciidoc[]
include::indices/put_settings.asciidoc[]
include::indices/get_settings.asciidoc[]
include::indices/put_template.asciidoc[]
+include::indices/validate_query.asciidoc[]
include::indices/get_templates.asciidoc[]
== Cluster APIs
@@ -149,3 +151,14 @@ The Java High Level REST Client supports the following Tasks APIs:
include::tasks/list_tasks.asciidoc[]
include::tasks/cancel_tasks.asciidoc[]
+
+== Script APIs
+
+The Java High Level REST Client supports the following Scripts APIs:
+
+* <>
+* <>
+
+include::script/get_script.asciidoc[]
+include::script/delete_script.asciidoc[]
+
diff --git a/docs/java-rest/low-level/sniffer.asciidoc b/docs/java-rest/low-level/sniffer.asciidoc
index 4f846847615ea..1ffaa519cfb50 100644
--- a/docs/java-rest/low-level/sniffer.asciidoc
+++ b/docs/java-rest/low-level/sniffer.asciidoc
@@ -55,7 +55,7 @@ dependencies {
Once a `RestClient` instance has been created as shown in <>,
a `Sniffer` can be associated to it. The `Sniffer` will make use of the provided `RestClient`
to periodically (every 5 minutes by default) fetch the list of current nodes from the cluster
-and update them by calling `RestClient#setHosts`.
+and update them by calling `RestClient#setNodes`.
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
@@ -105,7 +105,7 @@ on failure is not enabled like explained above.
The Elasticsearch Nodes Info api doesn't return the protocol to use when
connecting to the nodes but only their `host:port` key-pair, hence `http`
is used by default. In case `https` should be used instead, the
-`ElasticsearchHostsSniffer` instance has to be manually created and provided
+`ElasticsearchNodesSniffer` instance has to be manually created and provided
as follows:
["source","java",subs="attributes,callouts,macros"]
@@ -125,12 +125,12 @@ cluster, the ones that have responded until then.
include-tagged::{doc-tests}/SnifferDocumentation.java[sniff-request-timeout]
--------------------------------------------------
-Also, a custom `HostsSniffer` implementation can be provided for advanced
-use-cases that may require fetching the hosts from external sources rather
+Also, a custom `NodesSniffer` implementation can be provided for advanced
+use-cases that may require fetching the `Node`s from external sources rather
than from Elasticsearch:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
-include-tagged::{doc-tests}/SnifferDocumentation.java[custom-hosts-sniffer]
+include-tagged::{doc-tests}/SnifferDocumentation.java[custom-nodes-sniffer]
--------------------------------------------------
<1> Fetch the hosts from the external source
diff --git a/docs/java-rest/low-level/usage.asciidoc b/docs/java-rest/low-level/usage.asciidoc
index 012ce418226cd..1f8b302715f42 100644
--- a/docs/java-rest/low-level/usage.asciidoc
+++ b/docs/java-rest/low-level/usage.asciidoc
@@ -271,23 +271,58 @@ a `ContentType` of `application/json`.
include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-body-shorter]
--------------------------------------------------
-And you can add one or more headers to send with the request:
+[[java-rest-low-usage-request-options]]
+==== RequestOptions
+
+The `RequestOptions` class holds parts of the request that should be shared
+between many requests in the same application. You can make a singleton
+instance and share it between all requests:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
-include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-headers]
+include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-options-singleton]
--------------------------------------------------
+<1> Add any headers needed by all requests.
+<2> Set a `NodeSelector`.
+<3> Customize the response consumer.
+
+`addHeader` is for headers that are required for authorization or to work with
+a proxy in front of Elasticsearch. There is no need to set the `Content-Type`
+header because the client will automatically set that from the `HttpEntity`
+attached to the request.
+
+You can set the `NodeSelector` which controls which nodes will receive
+requests. `NodeSelector.NOT_MASTER_ONLY` is a good choice.
You can also customize the response consumer used to buffer the asynchronous
responses. The default consumer will buffer up to 100MB of response on the
JVM heap. If the response is larger then the request will fail. You could,
for example, lower the maximum size which might be useful if you are running
-in a heap constrained environment:
+in a heap constrained environment like the exmaple above.
+
+Once you've created the singleton you can use it when making requests:
["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
-include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-response-consumer]
+include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-options-set-singleton]
+--------------------------------------------------
+
+You can also customize these options on a per request basis. For example, this
+adds an extra header:
+
+["source","java",subs="attributes,callouts,macros"]
--------------------------------------------------
+include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-options-customize-header]
+--------------------------------------------------
+
+Or you can send requests to nodes with a particular attribute:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-options-customize-attribute]
+--------------------------------------------------
+<1> Replace the node selector with one that selects nodes on a particular rack.
+
==== Multiple parallel asynchronous actions
diff --git a/docs/plugins/discovery-azure-classic.asciidoc b/docs/plugins/discovery-azure-classic.asciidoc
index c56991b8f507f..1c1925de878aa 100644
--- a/docs/plugins/discovery-azure-classic.asciidoc
+++ b/docs/plugins/discovery-azure-classic.asciidoc
@@ -372,7 +372,7 @@ This command should give you a JSON result:
"cluster_uuid" : "AT69_T_DTp-1qgIJlatQqA",
"version" : {
"number" : "{version}",
- "build_flavor" : "oss",
+ "build_flavor" : "{build_flavor}",
"build_type" : "zip",
"build_hash" : "f27399d",
"build_date" : "2016-03-30T09:51:41.449Z",
diff --git a/docs/plugins/ingest-attachment.asciidoc b/docs/plugins/ingest-attachment.asciidoc
index 443d1fb578a6d..2f9564294d0b8 100644
--- a/docs/plugins/ingest-attachment.asciidoc
+++ b/docs/plugins/ingest-attachment.asciidoc
@@ -25,6 +25,7 @@ include::install_remove.asciidoc[]
| `field` | yes | - | The field to get the base64 encoded field from
| `target_field` | no | attachment | The field that will hold the attachment information
| `indexed_chars` | no | 100000 | The number of chars being used for extraction to prevent huge fields. Use `-1` for no limit.
+| `indexed_chars_field` | no | `null` | Field name from which you can overwrite the number of chars being used for extraction. See `indexed_chars`.
| `properties` | no | all properties | Array of properties to select to be stored. Can be `content`, `title`, `name`, `author`, `keywords`, `date`, `content_type`, `content_length`, `language`
| `ignore_missing` | no | `false` | If `true` and `field` does not exist, the processor quietly exits without modifying the document
|======
@@ -44,11 +45,11 @@ PUT _ingest/pipeline/attachment
}
]
}
-PUT my_index/my_type/my_id?pipeline=attachment
+PUT my_index/_doc/my_id?pipeline=attachment
{
"data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0="
}
-GET my_index/my_type/my_id
+GET my_index/_doc/my_id
--------------------------------------------------
// CONSOLE
@@ -59,7 +60,7 @@ Returns this:
{
"found": true,
"_index": "my_index",
- "_type": "my_type",
+ "_type": "_doc",
"_id": "my_id",
"_version": 1,
"_source": {
@@ -99,6 +100,115 @@ NOTE: Extracting contents from binary data is a resource intensive operation and
consumes a lot of resources. It is highly recommended to run pipelines
using this processor in a dedicated ingest node.
+[[ingest-attachment-extracted-chars]]
+==== Limit the number of extracted chars
+
+To prevent extracting too many chars and overload the node memory, the number of chars being used for extraction
+is limited by default to `100000`. You can change this value by setting `indexed_chars`. Use `-1` for no limit but
+ensure when setting this that your node will have enough HEAP to extract the content of very big documents.
+
+You can also define this limit per document by extracting from a given field the limit to set. If the document
+has that field, it will overwrite the `indexed_chars` setting. To set this field, define the `indexed_chars_field`
+setting.
+
+For example:
+
+[source,js]
+--------------------------------------------------
+PUT _ingest/pipeline/attachment
+{
+ "description" : "Extract attachment information",
+ "processors" : [
+ {
+ "attachment" : {
+ "field" : "data",
+ "indexed_chars" : 11,
+ "indexed_chars_field" : "max_size"
+ }
+ }
+ ]
+}
+PUT my_index/_doc/my_id?pipeline=attachment
+{
+ "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0="
+}
+GET my_index/_doc/my_id
+--------------------------------------------------
+// CONSOLE
+
+Returns this:
+
+[source,js]
+--------------------------------------------------
+{
+ "found": true,
+ "_index": "my_index",
+ "_type": "_doc",
+ "_id": "my_id",
+ "_version": 1,
+ "_source": {
+ "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0=",
+ "attachment": {
+ "content_type": "application/rtf",
+ "language": "sl",
+ "content": "Lorem ipsum",
+ "content_length": 11
+ }
+ }
+}
+--------------------------------------------------
+// TESTRESPONSE
+
+
+[source,js]
+--------------------------------------------------
+PUT _ingest/pipeline/attachment
+{
+ "description" : "Extract attachment information",
+ "processors" : [
+ {
+ "attachment" : {
+ "field" : "data",
+ "indexed_chars" : 11,
+ "indexed_chars_field" : "max_size"
+ }
+ }
+ ]
+}
+PUT my_index/_doc/my_id_2?pipeline=attachment
+{
+ "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0=",
+ "max_size": 5
+}
+GET my_index/_doc/my_id_2
+--------------------------------------------------
+// CONSOLE
+
+Returns this:
+
+[source,js]
+--------------------------------------------------
+{
+ "found": true,
+ "_index": "my_index",
+ "_type": "_doc",
+ "_id": "my_id_2",
+ "_version": 1,
+ "_source": {
+ "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0=",
+ "max_size": 5,
+ "attachment": {
+ "content_type": "application/rtf",
+ "language": "ro",
+ "content": "Lorem",
+ "content_length": 5
+ }
+ }
+}
+--------------------------------------------------
+// TESTRESPONSE
+
+
[[ingest-attachment-with-arrays]]
==== Using the Attachment Processor with arrays
@@ -150,7 +260,7 @@ PUT _ingest/pipeline/attachment
}
]
}
-PUT my_index/my_type/my_id?pipeline=attachment
+PUT my_index/_doc/my_id?pipeline=attachment
{
"attachments" : [
{
@@ -163,7 +273,7 @@ PUT my_index/my_type/my_id?pipeline=attachment
}
]
}
-GET my_index/my_type/my_id
+GET my_index/_doc/my_id
--------------------------------------------------
// CONSOLE
@@ -172,7 +282,7 @@ Returns this:
--------------------------------------------------
{
"_index" : "my_index",
- "_type" : "my_type",
+ "_type" : "_doc",
"_id" : "my_id",
"_version" : 1,
"found" : true,
diff --git a/docs/reference/analysis/tokenfilters.asciidoc b/docs/reference/analysis/tokenfilters.asciidoc
index 6e77b4498650d..dd5cb2e702cff 100644
--- a/docs/reference/analysis/tokenfilters.asciidoc
+++ b/docs/reference/analysis/tokenfilters.asciidoc
@@ -95,4 +95,6 @@ include::tokenfilters/decimal-digit-tokenfilter.asciidoc[]
include::tokenfilters/fingerprint-tokenfilter.asciidoc[]
-include::tokenfilters/minhash-tokenfilter.asciidoc[]
\ No newline at end of file
+include::tokenfilters/minhash-tokenfilter.asciidoc[]
+
+include::tokenfilters/remove-duplicates-tokenfilter.asciidoc[]
\ No newline at end of file
diff --git a/docs/reference/analysis/tokenfilters/remove-duplicates-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/remove-duplicates-tokenfilter.asciidoc
new file mode 100644
index 0000000000000..594e18eaf7f7e
--- /dev/null
+++ b/docs/reference/analysis/tokenfilters/remove-duplicates-tokenfilter.asciidoc
@@ -0,0 +1,5 @@
+[[analysis-remove-duplicates-tokenfilter]]
+=== Remove Duplicates Token Filter
+
+A token filter of type `remove_duplicates` that drops identical tokens at the
+same position.
diff --git a/docs/reference/cat/nodeattrs.asciidoc b/docs/reference/cat/nodeattrs.asciidoc
index 196f142cc35e1..6c474c2117943 100644
--- a/docs/reference/cat/nodeattrs.asciidoc
+++ b/docs/reference/cat/nodeattrs.asciidoc
@@ -9,15 +9,23 @@ For example:
GET /_cat/nodeattrs?v
--------------------------------------------------
// CONSOLE
+// TEST[s/\?v/\?v&s=node,attr/]
+// Sort the resulting attributes so we can assert on them more easilly
Could look like:
[source,txt]
--------------------------------------------------
node host ip attr value
-EK_AsJb 127.0.0.1 127.0.0.1 testattr test
+...
+node-0 127.0.0.1 127.0.0.1 testattr test
+...
--------------------------------------------------
-// TESTRESPONSE[s/EK_AsJb/.+/ _cat]
+// TESTRESPONSE[s/\.\.\.\n$/\n(.+ xpack\\.installed true\n)?\n/]
+// TESTRESPONSE[s/\.\.\.\n/(.+ ml\\..+\n)*/ _cat]
+// If xpack is not installed then neither ... with match anything
+// If xpack is installed then the first ... contains ml attributes
+// and the second contains xpack.installed=true
The first few columns (`node`, `host`, `ip`) give you basic info per node
and the `attr` and `value` columns give you the custom node attributes,
@@ -46,15 +54,24 @@ mode (`v`). The header name will match the supplied value (e.g.,
GET /_cat/nodeattrs?v&h=name,pid,attr,value
--------------------------------------------------
// CONSOLE
+// TEST[s/,value/,value&s=node,attr/]
+// Sort the resulting attributes so we can assert on them more easilly
Might look like:
[source,txt]
--------------------------------------------------
name pid attr value
-EK_AsJb 19566 testattr test
+...
+node-0 19566 testattr test
+...
--------------------------------------------------
-// TESTRESPONSE[s/EK_AsJb/.+/ s/19566/\\d*/ _cat]
+// TESTRESPONSE[s/19566/\\d*/]
+// TESTRESPONSE[s/\.\.\.\n$/\n(.+ xpack\\.installed true\n)?\n/]
+// TESTRESPONSE[s/\.\.\.\n/(.+ ml\\..+\n)*/ _cat]
+// If xpack is not installed then neither ... with match anything
+// If xpack is installed then the first ... contains ml attributes
+// and the second contains xpack.installed=true
[cols="<,<,<,<,<",options="header",subs="normal"]
|=======================================================================
diff --git a/docs/reference/cat/templates.asciidoc b/docs/reference/cat/templates.asciidoc
index bc221d13552c0..076e84b72b5d3 100644
--- a/docs/reference/cat/templates.asciidoc
+++ b/docs/reference/cat/templates.asciidoc
@@ -8,9 +8,16 @@ The `templates` command provides information about existing templates.
GET /_cat/templates?v&s=name
--------------------------------------------------
// CONSOLE
+// TEST[s/templates/templates\/template*/]
// TEST[s/^/PUT _template\/template0\n{"index_patterns": "te*", "order": 0}\n/]
// TEST[s/^/PUT _template\/template1\n{"index_patterns": "tea*", "order": 1}\n/]
// TEST[s/^/PUT _template\/template2\n{"index_patterns": "teak*", "order": 2, "version": 7}\n/]
+// The substitions do two things:
+// 1. Filter the response to just templates matching the te* pattern
+// so that we only get the templates we expect regardless of which
+// templates exist. If xpack is installed there will be unexpected
+// templates.
+// 2. Create some templates to expect in the response.
which looks like
diff --git a/docs/reference/cat/thread_pool.asciidoc b/docs/reference/cat/thread_pool.asciidoc
index f1c7664ae33f6..0f8e74c670335 100644
--- a/docs/reference/cat/thread_pool.asciidoc
+++ b/docs/reference/cat/thread_pool.asciidoc
@@ -18,19 +18,19 @@ node-0 analyze 0 0 0
node-0 fetch_shard_started 0 0 0
node-0 fetch_shard_store 0 0 0
node-0 flush 0 0 0
-node-0 force_merge 0 0 0
-node-0 generic 0 0 0
-node-0 get 0 0 0
-node-0 index 0 0 0
-node-0 listener 0 0 0
-node-0 management 1 0 0
-node-0 refresh 0 0 0
-node-0 search 0 0 0
-node-0 snapshot 0 0 0
-node-0 warmer 0 0 0
+...
node-0 write 0 0 0
--------------------------------------------------
+// TESTRESPONSE[s/\.\.\./(node-0 .+ 0 0 0\n)+/]
// TESTRESPONSE[s/\d+/\\d+/ _cat]
+// The substitutions do two things:
+// 1. Expect any number of extra thread pools. This allows us to only list a
+// few thread pools. The list would be super long otherwise. In addition,
+// if xpack is installed then the list will contain more thread pools and
+// this way we don't have to assert about them.
+// 2. Expect any number of active, queued, or rejected items. We really don't
+// know how many there will be and we just want to assert that there are
+// numbers in the response, not *which* numbers are there.
The first column is the node name
@@ -54,10 +54,16 @@ get
index
listener
management
+ml_autodetect (default distro only)
+ml_datafeed (default distro only)
+ml_utility (default distro only)
refresh
+rollup_indexing (default distro only)`
search
+security-token-key (default distro only)
snapshot
warmer
+watcher (default distro only)
write
--------------------------------------------------
diff --git a/docs/reference/cluster/nodes-info.asciidoc b/docs/reference/cluster/nodes-info.asciidoc
index 6522d0f5ad68a..2cd61dd905ff6 100644
--- a/docs/reference/cluster/nodes-info.asciidoc
+++ b/docs/reference/cluster/nodes-info.asciidoc
@@ -142,7 +142,7 @@ The result will look similar to:
"host": "node-0.elastic.co",
"ip": "192.168.17",
"version": "{version}",
- "build_flavor": "oss",
+ "build_flavor": "{build_flavor}",
"build_type": "zip",
"build_hash": "587409e",
"roles": [
@@ -237,7 +237,7 @@ The result will look similar to:
"host": "node-0.elastic.co",
"ip": "192.168.17",
"version": "{version}",
- "build_flavor": "oss",
+ "build_flavor": "{build_flavor}",
"build_type": "zip",
"build_hash": "587409e",
"roles": [],
diff --git a/docs/reference/cluster/stats.asciidoc b/docs/reference/cluster/stats.asciidoc
index 6efb4dced8bb8..191da2660d668 100644
--- a/docs/reference/cluster/stats.asciidoc
+++ b/docs/reference/cluster/stats.asciidoc
@@ -192,23 +192,23 @@ Will return, for example:
"description": "Ingest processor that extracts information from a user agent",
"classname": "org.elasticsearch.ingest.useragent.IngestUserAgentPlugin",
"has_native_controller": false
- }
+ },
+ ...
],
- "network_types" : {
- "transport_types" : {
- "netty4" : 1
- },
- "http_types" : {
- "netty4" : 1
- }
- }
+ ...
}
}
--------------------------------------------------
// TESTRESPONSE[s/"plugins": \[[^\]]*\]/"plugins": $body.$_path/]
+// TESTRESPONSE[s/\.\.\./"network_types": "replace_me"/]
// TESTRESPONSE[s/: (\-)?[0-9]+/: $body.$_path/]
// TESTRESPONSE[s/: "[^"]*"/: $body.$_path/]
-////
-The TESTRESPONSE above replace all the fields values by the expected ones in the test,
-because we don't really care about the field values but we want to check the fields names.
-////
\ No newline at end of file
+// These replacements do a few things:
+// 1. Ignore the contents of the `plugins` object because we don't know all of
+// the plugins that will be in it. And because we figure folks don't need to
+// see an exhaustive list anyway.
+// 2. The last ... contains more things that we don't think are important to
+// include in the output.
+// 3. All of the numbers and strings on the right hand side of *every* field in
+// the response are ignored. So we're really only asserting things about the
+// the shape of this response, not the values in it.
diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc
index f05acab559ce1..bdbffb0a08d5d 100644
--- a/docs/reference/docs/reindex.asciidoc
+++ b/docs/reference/docs/reindex.asciidoc
@@ -422,11 +422,11 @@ POST _reindex
// TEST[s/"username": "user",//]
// TEST[s/"password": "pass"//]
-The `host` parameter must contain a scheme, host, and port (e.g.
-`https://otherhost:9200`). The `username` and `password` parameters are
-optional, and when they are present `_reindex` will connect to the remote
-Elasticsearch node using basic auth. Be sure to use `https` when using
-basic auth or the password will be sent in plain text.
+The `host` parameter must contain a scheme, host, port (e.g.
+`https://otherhost:9200`) and optional path (e.g. `https://otherhost:9200/proxy`).
+The `username` and `password` parameters are optional, and when they are present `_reindex`
+will connect to the remote Elasticsearch node using basic auth. Be sure to use `https` when
+using basic auth or the password will be sent in plain text.
Remote hosts have to be explicitly whitelisted in elasticsearch.yaml using the
`reindex.remote.whitelist` property. It can be set to a comma delimited list
diff --git a/docs/reference/migration/index.asciidoc b/docs/reference/migration/index.asciidoc
index 6c654a4564531..9a034b30f2681 100644
--- a/docs/reference/migration/index.asciidoc
+++ b/docs/reference/migration/index.asciidoc
@@ -17,7 +17,10 @@ As a general rule:
* Migration between non-consecutive major versions -- e.g. `2.x` to `6.x` --
is not supported.
-See <> for more info.
+For more information, see <>.
+
+See also <> and <>.
+
--
include::migrate_6_0.asciidoc[]
diff --git a/docs/reference/migration/migrate_6_1.asciidoc b/docs/reference/migration/migrate_6_1.asciidoc
index 0dfb945826987..579d00fa778fc 100644
--- a/docs/reference/migration/migrate_6_1.asciidoc
+++ b/docs/reference/migration/migrate_6_1.asciidoc
@@ -1,6 +1,9 @@
[[breaking-changes-6.1]]
== Breaking changes in 6.1
+This section discusses the changes that you need to be aware of when migrating
+your application to Elasticsearch 6.1.
+
[[breaking_61_packaging]]
[float]
=== Bootstrap checks now apply to link-local addresses
diff --git a/docs/reference/migration/migrate_6_2.asciidoc b/docs/reference/migration/migrate_6_2.asciidoc
index 81dac028ce300..6a78363721f80 100644
--- a/docs/reference/migration/migrate_6_2.asciidoc
+++ b/docs/reference/migration/migrate_6_2.asciidoc
@@ -1,6 +1,9 @@
[[breaking-changes-6.2]]
== Breaking changes in 6.2
+This section discusses the changes that you need to be aware of when migrating
+your application to Elasticsearch 6.2
+
[[breaking_62_packaging]]
[float]
=== All permission bootstrap check
diff --git a/docs/reference/migration/migrate_6_3.asciidoc b/docs/reference/migration/migrate_6_3.asciidoc
index 07523da849589..e18021d46222d 100644
--- a/docs/reference/migration/migrate_6_3.asciidoc
+++ b/docs/reference/migration/migrate_6_3.asciidoc
@@ -1,6 +1,15 @@
[[breaking-changes-6.3]]
== Breaking changes in 6.3
+This section discusses the changes that you need to be aware of when migrating
+your application to Elasticsearch 6.3.
+
+* <>
+* <>
+* <>
+
+See also <> and <>.
+
[[breaking_63_api_changes]]
=== API changes
@@ -46,6 +55,12 @@ accept these permissions either by keeping standard input open and attaching a
TTY (i.e., using interactive mode to accept the permissions), or by passing the
`--batch` flag.
+==== Implementing custom realms with SPI instead of XPackExtension
+
+The legacy `XPackExtension` extension mechanism has been removed and replaced
+with an SPI based extension mechanism that is installed and built as an
+elasticsearch plugin. For more information about using SPI loaded security extensions in custom realms, see {stack-ov}/custom-realms.html[Integrating with other authentication systems].
+
[[breaking_63_settings_changes]]
=== Settings changes
@@ -79,3 +94,11 @@ place) you can start Elasticsearch with the JVM option
`-Des.thread_pool.write.use_bulk_as_display_name=true` to have Elasticsearch
continue to display the name of this thread pool as `bulk`. Elasticsearch will
stop observing this system property in 7.0.0.
+
+==== Enabling monitoring
+
+By default when you install {xpack}, monitoring is enabled but data collection
+is disabled. To enable data collection, use the new
+`xpack.monitoring.collection.enabled` setting. You can update this setting by
+using the <>. For more
+information, see <>.
diff --git a/docs/reference/migration/migrate_6_4.asciidoc b/docs/reference/migration/migrate_6_4.asciidoc
index d7a77102c3b75..16a78fae80149 100644
--- a/docs/reference/migration/migrate_6_4.asciidoc
+++ b/docs/reference/migration/migrate_6_4.asciidoc
@@ -1,6 +1,14 @@
[[breaking-changes-6.4]]
== Breaking changes in 6.4
+This section discusses the changes that you need to be aware of when migrating
+your application to Elasticsearch 6.4.
+
+* <>
+* <>
+
+See also <> and <>.
+
[[breaking_64_api_changes]]
=== API changes
@@ -37,4 +45,27 @@ Stored script formats that don't use top level `script` object have been depreca
support for these formats will be removed in the next major release.
This includes `template` stored scripts format and
-formats that do not encapsulate the script inside a `script` json object.
\ No newline at end of file
+formats that do not encapsulate the script inside a `script` json object.
+
+[[breaking_64_rest_client_changes]]
+=== REST Client
+
+==== Old low level ++performRequest++s deprecated
+The versions of `performRequest` and `performRequestAsync` that were in the
+low level client in 6.3 have been deprecated in favor of
+`performRequest(Request)` and `performRequestAsync(Request, ActionListener)`.
+These will allow us to add more features to the client without adding more
+variants of `performRequest`.
+
+==== Old high level request methods deprecated
+All request methods on the high level client have been deprecated in favor
+of a new variant that takes `RequestOptions`. This allows users of the high
+level client to customize many options on individual requests that were not
+available otherwise.
+
+==== HostSniffer renamed to NodeSniffer and signature changed
+To provide allow the `Sniffer` to pick up more metadata we changed it from
+sniffing ++HttpHost++s to sniffing ++Node++s, a new class introduced to contain
+both connection information and metadata like the node's role or any
+attributes defined in elasticsearch.yml. These can be used by the new
+`RequestOptions#setNodeSelector`.
diff --git a/docs/reference/release-notes/6.3.asciidoc b/docs/reference/release-notes/6.3.asciidoc
index 1c14668e099df..e47f59118a2bc 100644
--- a/docs/reference/release-notes/6.3.asciidoc
+++ b/docs/reference/release-notes/6.3.asciidoc
@@ -1,5 +1,5 @@
[[release-notes-6.3.0]]
-== 6.3.0 Release Notes
+== {es} version 6.3.0
Also see <>.
@@ -23,16 +23,11 @@ Packaging::
Plugins::
* Remove silent batch mode from install plugin {pull}29359[#29359]
-Search::
-* Fail _search request with trailing tokens {pull}29428[#29428] (issue: {issue}28995[#28995])
-
Security::
* The legacy `XPackExtension` extension mechanism has been removed and replaced
with an SPI based extension mechanism that is installed and built as an
elasticsearch plugin.
-Task Management::
-* Remove metadata customs that can break serialization {pull}30945[#30945] (issues: {issue}30731[#30731], {issue}30857[#30857])
[[breaking-java-6.3.0]]
@@ -67,6 +62,7 @@ REST API::
Search::
* Deprecate slicing on `_uid`. {pull}29353[#29353]
+* Generate deprecation warning for _search request with trailing tokens {pull}29428[#29428] (issue: {issue}28995[#28995])
Stats::
* Deprecate the suggest metrics {pull}29627[#29627] (issue: {issue}29589[#29589])
@@ -227,8 +223,8 @@ Mapping::
Machine Learning::
* Synchronize long and short tests for periodicity {ml-pull}62[#62]
* Improvements to trend modelling and periodicity testing for forecasting {ml-pull}7[#7] (issue: {ml-issue}5[#5])
-* [ML] Clean left behind model state docs {pull}30659[#30659] (issue: {issue}30551[#30551])
-* [ML] Hide internal Job update options from the REST API {pull}30537[#30537] (issue: {issue}30512[#30512])
+* Clean left behind model state docs {pull}30659[#30659] (issue: {issue}30551[#30551])
+* Hide internal Job update options from the REST API {pull}30537[#30537] (issue: {issue}30512[#30512])
Packaging::
* Configure heap dump path for archive packages {pull}29130[#29130] (issue: {issue}26755[#26755])
diff --git a/docs/reference/release-notes/highlights-6.3.0.asciidoc b/docs/reference/release-notes/highlights-6.3.0.asciidoc
index 94c89b5755713..fb396d2832bcc 100644
--- a/docs/reference/release-notes/highlights-6.3.0.asciidoc
+++ b/docs/reference/release-notes/highlights-6.3.0.asciidoc
@@ -50,10 +50,3 @@ versions (and EOL dates), you can continue to use Java 8. See the
https://www.elastic.co/support/matrix#matrix_jvm[support matrix]
for all of the JVM options for {es}.
-[float]
-=== Improved trend modeling and periodicity testing for forecasting
-
-{stack-ov}/ml-overview.html#ml-forecasting[Forecasting] is now more reliable and
-has greatly improved confidence intervals--particularly for longer time ranges.
-These improvements also affect trend and seasonality modeling during anomaly
-detection.
diff --git a/docs/reference/setup/install/check-running.asciidoc b/docs/reference/setup/install/check-running.asciidoc
index 0cfc4b329ecfa..7b95a10158d2f 100644
--- a/docs/reference/setup/install/check-running.asciidoc
+++ b/docs/reference/setup/install/check-running.asciidoc
@@ -19,7 +19,7 @@ which should give you a response something like this:
"cluster_uuid" : "AT69_T_DTp-1qgIJlatQqA",
"version" : {
"number" : "{version}",
- "build_flavor" : "oss",
+ "build_flavor" : "{build_flavor}",
"build_type" : "zip",
"build_hash" : "f27399d",
"build_date" : "2016-03-30T09:51:41.449Z",
diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java
index 722d75a9293f7..cdd8101a73c70 100644
--- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java
@@ -37,6 +37,7 @@
import org.apache.lucene.analysis.cjk.CJKAnalyzer;
import org.apache.lucene.analysis.cjk.CJKBigramFilter;
import org.apache.lucene.analysis.cjk.CJKWidthFilter;
+import org.apache.lucene.analysis.ckb.SoraniAnalyzer;
import org.apache.lucene.analysis.ckb.SoraniNormalizationFilter;
import org.apache.lucene.analysis.commongrams.CommonGramsFilter;
import org.apache.lucene.analysis.core.DecimalDigitFilter;
@@ -52,17 +53,27 @@
import org.apache.lucene.analysis.de.GermanAnalyzer;
import org.apache.lucene.analysis.de.GermanNormalizationFilter;
import org.apache.lucene.analysis.de.GermanStemFilter;
+import org.apache.lucene.analysis.el.GreekAnalyzer;
import org.apache.lucene.analysis.en.EnglishAnalyzer;
import org.apache.lucene.analysis.en.KStemFilter;
import org.apache.lucene.analysis.en.PorterStemFilter;
+import org.apache.lucene.analysis.es.SpanishAnalyzer;
import org.apache.lucene.analysis.eu.BasqueAnalyzer;
+import org.apache.lucene.analysis.fa.PersianAnalyzer;
import org.apache.lucene.analysis.fa.PersianNormalizationFilter;
import org.apache.lucene.analysis.fi.FinnishAnalyzer;
import org.apache.lucene.analysis.fr.FrenchAnalyzer;
+import org.apache.lucene.analysis.ga.IrishAnalyzer;
import org.apache.lucene.analysis.gl.GalicianAnalyzer;
+import org.apache.lucene.analysis.hi.HindiAnalyzer;
import org.apache.lucene.analysis.hi.HindiNormalizationFilter;
+import org.apache.lucene.analysis.hu.HungarianAnalyzer;
import org.apache.lucene.analysis.hy.ArmenianAnalyzer;
+import org.apache.lucene.analysis.id.IndonesianAnalyzer;
import org.apache.lucene.analysis.in.IndicNormalizationFilter;
+import org.apache.lucene.analysis.it.ItalianAnalyzer;
+import org.apache.lucene.analysis.lt.LithuanianAnalyzer;
+import org.apache.lucene.analysis.lv.LatvianAnalyzer;
import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter;
import org.apache.lucene.analysis.miscellaneous.DisableGraphAttribute;
import org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilter;
@@ -79,19 +90,26 @@
import org.apache.lucene.analysis.ngram.NGramTokenFilter;
import org.apache.lucene.analysis.ngram.NGramTokenizer;
import org.apache.lucene.analysis.nl.DutchAnalyzer;
+import org.apache.lucene.analysis.no.NorwegianAnalyzer;
import org.apache.lucene.analysis.path.PathHierarchyTokenizer;
import org.apache.lucene.analysis.pattern.PatternTokenizer;
import org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilter;
import org.apache.lucene.analysis.payloads.TypeAsPayloadTokenFilter;
+import org.apache.lucene.analysis.pt.PortugueseAnalyzer;
import org.apache.lucene.analysis.reverse.ReverseStringFilter;
+import org.apache.lucene.analysis.ro.RomanianAnalyzer;
+import org.apache.lucene.analysis.ru.RussianAnalyzer;
import org.apache.lucene.analysis.shingle.ShingleFilter;
import org.apache.lucene.analysis.snowball.SnowballFilter;
import org.apache.lucene.analysis.standard.ClassicFilter;
import org.apache.lucene.analysis.standard.ClassicTokenizer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.analysis.standard.UAX29URLEmailTokenizer;
+import org.apache.lucene.analysis.sv.SwedishAnalyzer;
+import org.apache.lucene.analysis.th.ThaiAnalyzer;
import org.apache.lucene.analysis.th.ThaiTokenizer;
import org.apache.lucene.analysis.tr.ApostropheFilter;
+import org.apache.lucene.analysis.tr.TurkishAnalyzer;
import org.apache.lucene.analysis.util.ElisionFilter;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
@@ -130,6 +148,8 @@ public Map>> getAn
analyzers.put("standard_html_strip", StandardHtmlStripAnalyzerProvider::new);
analyzers.put("pattern", PatternAnalyzerProvider::new);
analyzers.put("snowball", SnowballAnalyzerProvider::new);
+
+ // Language analyzers:
analyzers.put("arabic", ArabicAnalyzerProvider::new);
analyzers.put("armenian", ArmenianAnalyzerProvider::new);
analyzers.put("basque", BasqueAnalyzerProvider::new);
@@ -147,6 +167,24 @@ public Map>> getAn
analyzers.put("french", FrenchAnalyzerProvider::new);
analyzers.put("galician", GalicianAnalyzerProvider::new);
analyzers.put("german", GermanAnalyzerProvider::new);
+ analyzers.put("greek", GreekAnalyzerProvider::new);
+ analyzers.put("hindi", HindiAnalyzerProvider::new);
+ analyzers.put("hungarian", HungarianAnalyzerProvider::new);
+ analyzers.put("indonesian", IndonesianAnalyzerProvider::new);
+ analyzers.put("irish", IrishAnalyzerProvider::new);
+ analyzers.put("italian", ItalianAnalyzerProvider::new);
+ analyzers.put("latvian", LatvianAnalyzerProvider::new);
+ analyzers.put("lithuanian", LithuanianAnalyzerProvider::new);
+ analyzers.put("norwegian", NorwegianAnalyzerProvider::new);
+ analyzers.put("persian", PersianAnalyzerProvider::new);
+ analyzers.put("portuguese", PortugueseAnalyzerProvider::new);
+ analyzers.put("romanian", RomanianAnalyzerProvider::new);
+ analyzers.put("russian", RussianAnalyzerProvider::new);
+ analyzers.put("sorani", SoraniAnalyzerProvider::new);
+ analyzers.put("spanish", SpanishAnalyzerProvider::new);
+ analyzers.put("swedish", SwedishAnalyzerProvider::new);
+ analyzers.put("turkish", TurkishAnalyzerProvider::new);
+ analyzers.put("thai", ThaiAnalyzerProvider::new);
return analyzers;
}
@@ -194,6 +232,7 @@ public Map> getTokenFilters() {
filters.put("pattern_replace", requriesAnalysisSettings(PatternReplaceTokenFilterFactory::new));
filters.put("persian_normalization", PersianNormalizationFilterFactory::new);
filters.put("porter_stem", PorterStemTokenFilterFactory::new);
+ filters.put("remove_duplicates", RemoveDuplicatesTokenFilterFactory::new);
filters.put("reverse", ReverseTokenFilterFactory::new);
filters.put("russian_stem", RussianStemTokenFilterFactory::new);
filters.put("scandinavian_folding", ScandinavianFoldingFilterFactory::new);
@@ -247,13 +286,15 @@ public Map> getTokenizers() {
@Override
public List getPreBuiltAnalyzerProviderFactories() {
List analyzers = new ArrayList<>();
- analyzers.add(new PreBuiltAnalyzerProviderFactory("standard_html_strip", CachingStrategy.LUCENE,
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("standard_html_strip", CachingStrategy.ELASTICSEARCH,
() -> new StandardHtmlStripAnalyzer(CharArraySet.EMPTY_SET)));
analyzers.add(new PreBuiltAnalyzerProviderFactory("pattern", CachingStrategy.ELASTICSEARCH,
() -> new PatternAnalyzer(Regex.compile("\\W+" /*PatternAnalyzer.NON_WORD_PATTERN*/, null), true,
CharArraySet.EMPTY_SET)));
analyzers.add(new PreBuiltAnalyzerProviderFactory("snowball", CachingStrategy.LUCENE,
() -> new SnowballAnalyzer("English", StopAnalyzer.ENGLISH_STOP_WORDS_SET)));
+
+ // Language analyzers:
analyzers.add(new PreBuiltAnalyzerProviderFactory("arabic", CachingStrategy.LUCENE, ArabicAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("armenian", CachingStrategy.LUCENE, ArmenianAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("basque", CachingStrategy.LUCENE, BasqueAnalyzer::new));
@@ -262,7 +303,7 @@ public List getPreBuiltAnalyzerProviderFactorie
analyzers.add(new PreBuiltAnalyzerProviderFactory("bulgarian", CachingStrategy.LUCENE, BulgarianAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("catalan", CachingStrategy.LUCENE, CatalanAnalyzer::new));
// chinese analyzer: only for old indices, best effort
- analyzers.add(new PreBuiltAnalyzerProviderFactory("chinese", CachingStrategy.LUCENE, StandardAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("chinese", CachingStrategy.ONE, StandardAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("cjk", CachingStrategy.LUCENE, CJKAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("czech", CachingStrategy.LUCENE, CzechAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("danish", CachingStrategy.LUCENE, DanishAnalyzer::new));
@@ -272,6 +313,24 @@ public List getPreBuiltAnalyzerProviderFactorie
analyzers.add(new PreBuiltAnalyzerProviderFactory("french", CachingStrategy.LUCENE, FrenchAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("galician", CachingStrategy.LUCENE, GalicianAnalyzer::new));
analyzers.add(new PreBuiltAnalyzerProviderFactory("german", CachingStrategy.LUCENE, GermanAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("greek", CachingStrategy.LUCENE, GreekAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("hindi", CachingStrategy.LUCENE, HindiAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("hungarian", CachingStrategy.LUCENE, HungarianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("indonesian", CachingStrategy.LUCENE, IndonesianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("irish", CachingStrategy.LUCENE, IrishAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("italian", CachingStrategy.LUCENE, ItalianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("latvian", CachingStrategy.LUCENE, LatvianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("lithuanian", CachingStrategy.LUCENE, LithuanianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("norwegian", CachingStrategy.LUCENE, NorwegianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("persian", CachingStrategy.LUCENE, PersianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("portuguese", CachingStrategy.LUCENE, PortugueseAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("romanian", CachingStrategy.LUCENE, RomanianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("russian", CachingStrategy.LUCENE, RussianAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("sorani", CachingStrategy.LUCENE, SoraniAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("spanish", CachingStrategy.LUCENE, SpanishAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("swedish", CachingStrategy.LUCENE, SwedishAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("turkish", CachingStrategy.LUCENE, TurkishAnalyzer::new));
+ analyzers.add(new PreBuiltAnalyzerProviderFactory("thai", CachingStrategy.LUCENE, ThaiAnalyzer::new));
return analyzers;
}
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/GreekAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/GreekAnalyzerProvider.java
similarity index 84%
rename from server/src/main/java/org/elasticsearch/index/analysis/GreekAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/GreekAnalyzerProvider.java
index 012fc64c97166..3cf1f911e48cc 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/GreekAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/GreekAnalyzerProvider.java
@@ -17,18 +17,20 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.el.GreekAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class GreekAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final GreekAnalyzer analyzer;
- public GreekAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ GreekAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new GreekAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, GreekAnalyzer.getDefaultStopSet()));
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/HindiAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/HindiAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/HindiAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/HindiAnalyzerProvider.java
index 43ebe2677aec7..7b73c119a86f6 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/HindiAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/HindiAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.hi.HindiAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class HindiAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final HindiAnalyzer analyzer;
- public HindiAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ HindiAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new HindiAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, HindiAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/HungarianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/HungarianAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/HungarianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/HungarianAnalyzerProvider.java
index 81745ba0f2716..5ce948819c6ae 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/HungarianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/HungarianAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.hu.HungarianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class HungarianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final HungarianAnalyzer analyzer;
- public HungarianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ HungarianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new HungarianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, HungarianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/IndonesianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/IndonesianAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/IndonesianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/IndonesianAnalyzerProvider.java
index d5633025804e9..d20185deb4509 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/IndonesianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/IndonesianAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.id.IndonesianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class IndonesianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final IndonesianAnalyzer analyzer;
- public IndonesianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ IndonesianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new IndonesianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, IndonesianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/IrishAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/IrishAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/IrishAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/IrishAnalyzerProvider.java
index 69250df736052..dae7862b76bb6 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/IrishAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/IrishAnalyzerProvider.java
@@ -17,13 +17,15 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.ga.IrishAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
/**
* Provider for {@link IrishAnalyzer}
@@ -32,7 +34,7 @@ public class IrishAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final ItalianAnalyzer analyzer;
- public ItalianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ ItalianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new ItalianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, ItalianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/LatvianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/LatvianAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/LatvianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/LatvianAnalyzerProvider.java
index a7731f352b997..668f91f6ea478 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/LatvianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/LatvianAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.lv.LatvianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class LatvianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final LatvianAnalyzer analyzer;
- public LatvianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ LatvianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new LatvianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, LatvianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/LithuanianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/LithuanianAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/LithuanianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/LithuanianAnalyzerProvider.java
index 307904b89427f..8a355c9c3f8c8 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/LithuanianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/LithuanianAnalyzerProvider.java
@@ -17,13 +17,15 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.lt.LithuanianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
/**
* Provider for {@link LithuanianAnalyzer}
@@ -32,7 +34,7 @@ public class LithuanianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final NorwegianAnalyzer analyzer;
- public NorwegianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ NorwegianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new NorwegianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, NorwegianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/PersianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/PersianAnalyzerProvider.java
similarity index 84%
rename from server/src/main/java/org/elasticsearch/index/analysis/PersianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/PersianAnalyzerProvider.java
index ed92e3e0c02d6..74c3a95a57766 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/PersianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/PersianAnalyzerProvider.java
@@ -17,18 +17,20 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.fa.PersianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class PersianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final PersianAnalyzer analyzer;
- public PersianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ PersianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new PersianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, PersianAnalyzer.getDefaultStopSet()));
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/PortugueseAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/PortugueseAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/PortugueseAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/PortugueseAnalyzerProvider.java
index 73dc016fe6965..aba9c9980c9e1 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/PortugueseAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/PortugueseAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.pt.PortugueseAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class PortugueseAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final PortugueseAnalyzer analyzer;
- public PortugueseAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ PortugueseAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new PortugueseAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, PortugueseAnalyzer.getDefaultStopSet()),
diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RemoveDuplicatesTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RemoveDuplicatesTokenFilterFactory.java
new file mode 100644
index 0000000000000..a136c5573121e
--- /dev/null
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RemoveDuplicatesTokenFilterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.analysis.common;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.miscellaneous.RemoveDuplicatesTokenFilter;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.env.Environment;
+import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractTokenFilterFactory;
+
+/**
+ * Filter factory for the lucene RemoveDuplicatesTokenFilter
+ */
+class RemoveDuplicatesTokenFilterFactory extends AbstractTokenFilterFactory {
+
+ RemoveDuplicatesTokenFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ super(indexSettings, name, settings);
+ }
+
+ @Override
+ public TokenStream create(TokenStream tokenStream) {
+ return new RemoveDuplicatesTokenFilter(tokenStream);
+ }
+}
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/RomanianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RomanianAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/RomanianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RomanianAnalyzerProvider.java
index 5eeb22f5118d2..f1ff399272c38 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/RomanianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RomanianAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.ro.RomanianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class RomanianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final RomanianAnalyzer analyzer;
- public RomanianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ RomanianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new RomanianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, RomanianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/RussianAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RussianAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/RussianAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RussianAnalyzerProvider.java
index 6e57603794cd2..2d20398a7fff8 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/RussianAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/RussianAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.ru.RussianAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class RussianAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final RussianAnalyzer analyzer;
- public RussianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ RussianAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new RussianAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, RussianAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/SoraniAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SoraniAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/SoraniAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SoraniAnalyzerProvider.java
index d3b9fcd3f5c47..e2f565a7ec560 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/SoraniAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SoraniAnalyzerProvider.java
@@ -17,13 +17,15 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.ckb.SoraniAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
/**
* Provider for {@link SoraniAnalyzer}
@@ -32,7 +34,7 @@ public class SoraniAnalyzerProvider extends AbstractIndexAnalyzerProvider {
private final SpanishAnalyzer analyzer;
- public SpanishAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
+ SpanishAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) {
super(indexSettings, name, settings);
analyzer = new SpanishAnalyzer(
Analysis.parseStopWords(env, indexSettings.getIndexVersionCreated(), settings, SpanishAnalyzer.getDefaultStopSet()),
diff --git a/server/src/main/java/org/elasticsearch/index/analysis/SwedishAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SwedishAnalyzerProvider.java
similarity index 85%
rename from server/src/main/java/org/elasticsearch/index/analysis/SwedishAnalyzerProvider.java
rename to modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SwedishAnalyzerProvider.java
index 066d2eef78bd1..3fb3279e79a3e 100644
--- a/server/src/main/java/org/elasticsearch/index/analysis/SwedishAnalyzerProvider.java
+++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SwedishAnalyzerProvider.java
@@ -17,19 +17,21 @@
* under the License.
*/
-package org.elasticsearch.index.analysis;
+package org.elasticsearch.analysis.common;
import org.apache.lucene.analysis.CharArraySet;
import org.apache.lucene.analysis.sv.SwedishAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
+import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
+import org.elasticsearch.index.analysis.Analysis;
public class SwedishAnalyzerProvider extends AbstractIndexAnalyzerProvider