diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 03b2674a4cc8c..f1f78910e9a4d 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -41,7 +41,7 @@ We enjoy working with contributors to get their code accepted. There are many ap Note that it is unlikely the project will merge refactors for the sake of refactoring. These types of pull requests have a high cost to maintainers in reviewing and testing with little to no tangible benefit. This especially includes changes generated by tools. For example, -converting all generic interface instances to use the diamond operator. +converting all generic interface instances to use the diamond operator. The process for contributing to any of the [Elastic repositories](https://github.com/elastic/) is similar. Details for individual projects can be found below. @@ -209,6 +209,95 @@ Before submitting your changes, run the test suite to make sure that nothing is ./gradlew check ``` +### Project layout + +This repository is split into many top level directories. The most important +ones are: + +#### `docs` +Documentation for the project. + +#### `distribution` +Builds our tar and zip archives and our rpm and deb packages. + +#### `libs` +Libraries used to build other parts of the project. These are meant to be +internal rather than general purpose. We have no plans to +[semver](https://semver.org/) their APIs or accept feature requests for them. +We publish them to maven central because they are dependencies of our plugin +test framework, high level rest client, and jdbc driver but they really aren't +general purpose enough to *belong* in maven central. We're still working out +what to do here. + +#### `modules` +Features that are shipped with Elasticsearch by default but are not built in to +the server. We typically separate features from the server because they require +permissions that we don't believe *all* of Elasticsearch should have or because +they depend on libraries that we don't believe *all* of Elasticsearch should +depend on. + +For example, reindex requires the `connect` permission so it can perform +reindex-from-remote but we don't believe that the *all* of Elasticsearch should +have the "connect". For another example, Painless is implemented using antlr4 +and asm and we don't believe that *all* of Elasticsearch should have access to +them. + +#### `plugins` +Officially supported plugins to Elasticsearch. We decide that a feature should +be a plugin rather than shipped as a module because we feel that it is only +important to a subset of users, especially if it requires extra dependencies. + +The canonical example of this is the ICU analysis plugin. It is important for +folks who want the fairly language neutral ICU analyzer but the library to +implement the analyzer is 11MB so we don't ship it with Elasticsearch by +default. + +Another example is the `discovery-gce` plugin. It is *vital* to folks running +in [GCP](https://cloud.google.com/) but useless otherwise and it depends on a +dozen extra jars. + +#### `qa` +Honestly this is kind of in flux and we're not 100% sure where we'll end up. +Right now the directory contains +* Tests that require multiple modules or plugins to work +* Tests that form a cluster made up of multiple versions of Elasticsearch like +full cluster restart, rolling restarts, and mixed version tests +* Tests that test the Elasticsearch clients in "interesting" places like the +`wildfly` project. +* Tests that test Elasticsearch in funny configurations like with ingest +disabled +* Tests that need to do strange things like install plugins that thrown +uncaught `Throwable`s or add a shutdown hook +But we're not convinced that all of these things *belong* in the qa directory. +We're fairly sure that tests that require multiple modules or plugins to work +should just pick a "home" plugin. We're fairly sure that the multi-version +tests *do* belong in qa. Beyond that, we're not sure. If you want to add a new +qa project, open a PR and be ready to discuss options. + +#### `server` +The server component of Elasticsearch that contains all of the modules and +plugins. Right now things like the high level rest client depend on the server +but we'd like to fix that in the future. + +#### `test` +Our test framework and test fixtures. We use the test framework for testing the +server, the plugins, and modules, and pretty much everything else. We publish +the test framework so folks who develop Elasticsearch plugins can use it to +test the plugins. The test fixtures are external processes that we start before +running specific tests that rely on them. + +For example, we have an hdfs test that uses mini-hdfs to test our +repository-hdfs plugin. + +#### `x-pack` +Commercially licensed code that integrates with the rest of Elasticsearch. The +`docs` subdirectory functions just like the top level `docs` subdirectory and +the `qa` subdirectory functions just like the top level `qa` subdirectory. The +`plugin` subdirectory contains the x-pack module which runs inside the +Elasticsearch process. The `transport-client` subdirectory contains extensions +to Elasticsearch's standard transport client to work properly with x-pack. + + Contributing as part of a class ------------------------------- In general Elasticsearch is happy to accept contributions that were created as diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy index a44b9c849d333..9cc5bb82552ab 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy @@ -497,10 +497,15 @@ class BuildPlugin implements Plugin { project.afterEvaluate { project.tasks.withType(JavaCompile) { final JavaVersion targetCompatibilityVersion = JavaVersion.toVersion(it.targetCompatibility) - // we fork because compiling lots of different classes in a shared jvm can eventually trigger GC overhead limitations - options.fork = true - options.forkOptions.javaHome = new File(project.compilerJavaHome) - options.forkOptions.memoryMaximumSize = "512m" + final compilerJavaHomeFile = new File(project.compilerJavaHome) + // we only fork if the Gradle JDK is not the same as the compiler JDK + if (compilerJavaHomeFile.canonicalPath == Jvm.current().javaHome.canonicalPath) { + options.fork = false + } else { + options.fork = true + options.forkOptions.javaHome = compilerJavaHomeFile + options.forkOptions.memoryMaximumSize = "512m" + } if (targetCompatibilityVersion == JavaVersion.VERSION_1_8) { // compile with compact 3 profile by default // NOTE: this is just a compile time check: does not replace testing with a compact3 JRE @@ -530,10 +535,15 @@ class BuildPlugin implements Plugin { } // also apply release flag to groovy, which is used in build-tools project.tasks.withType(GroovyCompile) { - final JavaVersion targetCompatibilityVersion = JavaVersion.toVersion(it.targetCompatibility) - options.fork = true - options.forkOptions.javaHome = new File(project.compilerJavaHome) - options.compilerArgs << '--release' << targetCompatibilityVersion.majorVersion + final compilerJavaHomeFile = new File(project.compilerJavaHome) + // we only fork if the Gradle JDK is not the same as the compiler JDK + if (compilerJavaHomeFile.canonicalPath == Jvm.current().javaHome.canonicalPath) { + options.fork = false + } else { + options.fork = true + options.forkOptions.javaHome = compilerJavaHomeFile + options.compilerArgs << '--release' << JavaVersion.toVersion(it.targetCompatibility).majorVersion + } } } } @@ -549,6 +559,11 @@ class BuildPlugin implements Plugin { javadoc.classpath = javadoc.getClasspath().filter { f -> return classes.contains(f) == false } + /* + * Generate docs using html5 to suppress a warning from `javadoc` + * that the default will change to html5 in the future. + */ + javadoc.options.addBooleanOption('html5', true) } configureJavadocJar(project) } diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy index 95ec00beca7e0..15a4f21b17543 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy @@ -141,9 +141,11 @@ public class RestTestsFromSnippetsTask extends SnippetsTask { private static final String SYNTAX = { String method = /(?GET|PUT|POST|HEAD|OPTIONS|DELETE)/ String pathAndQuery = /(?[^\n]+)/ - String badBody = /GET|PUT|POST|HEAD|OPTIONS|DELETE|#/ + String badBody = /GET|PUT|POST|HEAD|OPTIONS|DELETE|startyaml|#/ String body = /(?(?:\n(?!$badBody)[^\n]+)+)/ - String nonComment = /$method\s+$pathAndQuery$body?/ + String rawRequest = /(?:$method\s+$pathAndQuery$body?)/ + String yamlRequest = /(?:startyaml(?s)(?.+?)(?-s)endyaml)/ + String nonComment = /(?:$rawRequest|$yamlRequest)/ String comment = /(?#.+)/ /(?:$comment|$nonComment)\n+/ }() @@ -333,6 +335,11 @@ public class RestTestsFromSnippetsTask extends SnippetsTask { // Comment return } + String yamlRequest = matcher.group("yaml"); + if (yamlRequest != null) { + current.println(yamlRequest) + return + } String method = matcher.group("method") String pathAndQuery = matcher.group("pathAndQuery") String body = matcher.group("body") diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java index 248d86c7c4217..5aa64a5c1375e 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java @@ -51,6 +51,8 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsResponse; import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeResponse; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateResponse; import java.io.IOException; import java.util.Collections; @@ -456,4 +458,26 @@ public void putSettingsAsync(UpdateSettingsRequest updateSettingsRequest, Action UpdateSettingsResponse::fromXContent, listener, emptySet(), headers); } + /** + * Puts an index template using the Index Templates API + *

+ * See Index Templates API + * on elastic.co + */ + public PutIndexTemplateResponse putTemplate(PutIndexTemplateRequest putIndexTemplateRequest, Header... headers) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(putIndexTemplateRequest, RequestConverters::putTemplate, + PutIndexTemplateResponse::fromXContent, emptySet(), headers); + } + + /** + * Asynchronously puts an index template using the Index Templates API + *

+ * See Index Templates API + * on elastic.co + */ + public void putTemplateAsync(PutIndexTemplateRequest putIndexTemplateRequest, + ActionListener listener, Header... headers) { + restHighLevelClient.performRequestAsyncAndParseEntity(putIndexTemplateRequest, RequestConverters::putTemplate, + PutIndexTemplateResponse::fromXContent, listener, emptySet(), headers); + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java index 705d8dfc9d252..2e7b4ba74cc39 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java @@ -29,6 +29,7 @@ import org.apache.http.entity.ContentType; import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; @@ -47,6 +48,7 @@ import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeType; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; @@ -77,7 +79,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.rankeval.RankEvalRequest; -import org.elasticsearch.rest.action.RestFieldCapabilitiesAction; import org.elasticsearch.rest.action.search.RestSearchAction; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; @@ -86,10 +87,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.nio.charset.Charset; -import java.util.Collections; -import java.util.HashMap; import java.util.Locale; -import java.util.Map; import java.util.StringJoiner; final class RequestConverters { @@ -647,6 +645,32 @@ static Request indexPutSettings(UpdateSettingsRequest updateSettingsRequest) thr return request; } + static Request getRepositories(GetRepositoriesRequest getRepositoriesRequest) { + String[] repositories = getRepositoriesRequest.repositories() == null ? Strings.EMPTY_ARRAY : getRepositoriesRequest.repositories(); + String endpoint = new EndpointBuilder().addPathPartAsIs("_snapshot").addCommaSeparatedPathParts(repositories).build(); + Request request = new Request(HttpGet.METHOD_NAME, endpoint); + + Params parameters = new Params(request); + parameters.withMasterTimeout(getRepositoriesRequest.masterNodeTimeout()); + parameters.withLocal(getRepositoriesRequest.local()); + return request; + } + + static Request putTemplate(PutIndexTemplateRequest putIndexTemplateRequest) throws IOException { + String endpoint = new EndpointBuilder().addPathPartAsIs("_template").addPathPart(putIndexTemplateRequest.name()).build(); + Request request = new Request(HttpPut.METHOD_NAME, endpoint); + Params params = new Params(request); + params.withMasterTimeout(putIndexTemplateRequest.masterNodeTimeout()); + if (putIndexTemplateRequest.create()) { + params.putParam("create", Boolean.TRUE.toString()); + } + if (Strings.hasText(putIndexTemplateRequest.cause())) { + params.putParam("cause", putIndexTemplateRequest.cause()); + } + request.setEntity(createEntity(putIndexTemplateRequest, REQUEST_BODY_CONTENT_TYPE)); + return request; + } + private static HttpEntity createEntity(ToXContent toXContent, XContentType xContentType) throws IOException { BytesRef source = XContentHelper.toXContent(toXContent, xContentType, false).toBytesRef(); return new ByteArrayEntity(source.bytes, source.offset, source.length, createContentType(xContentType)); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java index 9de8a748dadca..1985d6bd06dd4 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java @@ -26,6 +26,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteRequest; @@ -189,6 +191,7 @@ public class RestHighLevelClient implements Closeable { private final IndicesClient indicesClient = new IndicesClient(this); private final ClusterClient clusterClient = new ClusterClient(this); + private final SnapshotClient snapshotClient = new SnapshotClient(this); /** * Creates a {@link RestHighLevelClient} given the low level {@link RestClientBuilder} that allows to build the @@ -252,6 +255,15 @@ public final ClusterClient cluster() { return clusterClient; } + /** + * Provides a {@link SnapshotClient} which can be used to access the Snapshot API. + * + * See Snapshot API on elastic.co + */ + public final SnapshotClient snapshot() { + return snapshotClient; + } + /** * Executes a bulk request using the Bulk API * diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java new file mode 100644 index 0000000000000..e526fbe7164f9 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java @@ -0,0 +1,70 @@ +/* + * 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.Header; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; + +import java.io.IOException; + +import static java.util.Collections.emptySet; + +/** + * A wrapper for the {@link RestHighLevelClient} that provides methods for accessing the Snapshot API. + *

+ * See Snapshot API on elastic.co + */ +public final class SnapshotClient { + private final RestHighLevelClient restHighLevelClient; + + SnapshotClient(RestHighLevelClient restHighLevelClient) { + this.restHighLevelClient = restHighLevelClient; + } + + /** + * Gets a list of snapshot repositories. If the list of repositories is empty or it contains a single element "_all", all + * registered repositories are returned. + *

+ * See Snapshot and Restore + * API on elastic.co + */ + public GetRepositoriesResponse getRepositories(GetRepositoriesRequest getRepositoriesRequest, Header... headers) + throws IOException { + return restHighLevelClient.performRequestAndParseEntity(getRepositoriesRequest, RequestConverters::getRepositories, + GetRepositoriesResponse::fromXContent, emptySet(), headers); + } + + /** + * Asynchronously gets a list of snapshot repositories. If the list of repositories is empty or it contains a single element "_all", all + * registered repositories are returned. + *

+ * See Snapshot and Restore + * API on elastic.co + */ + public void getRepositoriesAsync(GetRepositoriesRequest getRepositoriesRequest, + ActionListener listener, Header... headers) { + restHighLevelClient.performRequestAsyncAndParseEntity(getRepositoriesRequest, RequestConverters::getRepositories, + GetRepositoriesResponse::fromXContent, listener, emptySet(), headers); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java new file mode 100644 index 0000000000000..597d35a99967b --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java @@ -0,0 +1,219 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.client; + +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.get.MultiGetRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.RestStatus; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +public class BulkProcessorRetryIT extends ESRestHighLevelClientTestCase { + + private static final String INDEX_NAME = "index"; + private static final String TYPE_NAME = "type"; + + private static BulkProcessor.Builder initBulkProcessorBuilder(BulkProcessor.Listener listener) { + return BulkProcessor.builder(highLevelClient()::bulkAsync, listener); + } + + public void testBulkRejectionLoadWithoutBackoff() throws Exception { + boolean rejectedExecutionExpected = true; + executeBulkRejectionLoad(BackoffPolicy.noBackoff(), rejectedExecutionExpected); + } + + public void testBulkRejectionLoadWithBackoff() throws Throwable { + boolean rejectedExecutionExpected = false; + executeBulkRejectionLoad(BackoffPolicy.exponentialBackoff(), rejectedExecutionExpected); + } + + private void executeBulkRejectionLoad(BackoffPolicy backoffPolicy, boolean rejectedExecutionExpected) throws Exception { + final CorrelatingBackoffPolicy internalPolicy = new CorrelatingBackoffPolicy(backoffPolicy); + final int numberOfAsyncOps = randomIntBetween(600, 700); + final CountDownLatch latch = new CountDownLatch(numberOfAsyncOps); + final Set responses = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + BulkProcessor bulkProcessor = initBulkProcessorBuilder(new BulkProcessor.Listener() { + @Override + public void beforeBulk(long executionId, BulkRequest request) { + } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + internalPolicy.logResponse(response); + responses.add(response); + latch.countDown(); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + responses.add(failure); + latch.countDown(); + } + }).setBulkActions(1) + .setConcurrentRequests(randomIntBetween(0, 100)) + .setBackoffPolicy(internalPolicy) + .build(); + + MultiGetRequest multiGetRequest = indexDocs(bulkProcessor, numberOfAsyncOps); + latch.await(10, TimeUnit.SECONDS); + bulkProcessor.close(); + + assertEquals(responses.size(), numberOfAsyncOps); + + boolean rejectedAfterAllRetries = false; + for (Object response : responses) { + if (response instanceof BulkResponse) { + BulkResponse bulkResponse = (BulkResponse) response; + for (BulkItemResponse bulkItemResponse : bulkResponse.getItems()) { + if (bulkItemResponse.isFailed()) { + BulkItemResponse.Failure failure = bulkItemResponse.getFailure(); + if (failure.getStatus() == RestStatus.TOO_MANY_REQUESTS) { + if (rejectedExecutionExpected == false) { + Iterator backoffState = internalPolicy.backoffStateFor(bulkResponse); + assertNotNull("backoffState is null (indicates a bulk request got rejected without retry)", backoffState); + if (backoffState.hasNext()) { + // we're not expecting that we overwhelmed it even once when we maxed out the number of retries + throw new AssertionError("Got rejected although backoff policy would allow more retries", + failure.getCause()); + } else { + rejectedAfterAllRetries = true; + logger.debug("We maxed out the number of bulk retries and got rejected (this is ok)."); + } + } + } else { + throw new AssertionError("Unexpected failure with status: " + failure.getStatus()); + } + } + } + } else { + Throwable t = (Throwable) response; + // we're not expecting any other errors + throw new AssertionError("Unexpected failure", t); + } + } + + highLevelClient().indices().refresh(new RefreshRequest()); + int multiGetResponsesCount = highLevelClient().multiGet(multiGetRequest).getResponses().length; + + if (rejectedExecutionExpected) { + assertThat(multiGetResponsesCount, lessThanOrEqualTo(numberOfAsyncOps)); + } else if (rejectedAfterAllRetries) { + assertThat(multiGetResponsesCount, lessThan(numberOfAsyncOps)); + } else { + assertThat(multiGetResponsesCount, equalTo(numberOfAsyncOps)); + } + + } + + private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) { + MultiGetRequest multiGetRequest = new MultiGetRequest(); + for (int i = 1; i <= numDocs; i++) { + processor.add(new IndexRequest(INDEX_NAME, TYPE_NAME, Integer.toString(i)) + .source(XContentType.JSON, "field", randomRealisticUnicodeOfCodepointLengthBetween(1, 30))); + multiGetRequest.add(INDEX_NAME, TYPE_NAME, Integer.toString(i)); + } + return multiGetRequest; + } + + /** + * Internal helper class to correlate backoff states with bulk responses. This is needed to check whether we maxed out the number + * of retries but still got rejected (which is perfectly fine and can also happen from time to time under heavy load). + * + * This implementation relies on an implementation detail in Retry, namely that the bulk listener is notified on the same thread + * as the last call to the backoff policy's iterator. The advantage is that this is non-invasive to the rest of the production code. + */ + private static class CorrelatingBackoffPolicy extends BackoffPolicy { + private final Map> correlations = new ConcurrentHashMap<>(); + // this is intentionally *not* static final. We will only ever have one instance of this class per test case and want the + // thread local to be eligible for garbage collection right after the test to avoid leaks. + private final ThreadLocal> iterators = new ThreadLocal<>(); + + private final BackoffPolicy delegate; + + private CorrelatingBackoffPolicy(BackoffPolicy delegate) { + this.delegate = delegate; + } + + public Iterator backoffStateFor(BulkResponse response) { + return correlations.get(response); + } + + // Assumption: This method is called from the same thread as the last call to the internal iterator's #hasNext() / #next() + // see also Retry.AbstractRetryHandler#onResponse(). + public void logResponse(BulkResponse response) { + Iterator iterator = iterators.get(); + // did we ever retry? + if (iterator != null) { + // we should correlate any iterator only once + iterators.remove(); + correlations.put(response, iterator); + } + } + + @Override + public Iterator iterator() { + return new CorrelatingIterator(iterators, delegate.iterator()); + } + + private static class CorrelatingIterator implements Iterator { + private final Iterator delegate; + private final ThreadLocal> iterators; + + private CorrelatingIterator(ThreadLocal> iterators, Iterator delegate) { + this.iterators = iterators; + this.delegate = delegate; + } + + @Override + public boolean hasNext() { + // update on every invocation as we might get rescheduled on a different thread. Unfortunately, there is a chance that + // we pollute the thread local map with stale values. Due to the implementation of Retry and the life cycle of the + // enclosing class CorrelatingBackoffPolicy this should not pose a major problem though. + iterators.set(this); + return delegate.hasNext(); + } + + @Override + public TimeValue next() { + // update on every invocation + iterators.set(this); + return delegate.next(); + } + } + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java index eb09084200bd2..931447d85d44a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java @@ -56,11 +56,14 @@ import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeResponse; import org.elasticsearch.action.admin.indices.shrink.ResizeType; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.broadcast.BroadcastResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -73,11 +76,19 @@ import org.elasticsearch.rest.RestStatus; import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; import java.util.Map; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; +import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractRawValues; +import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractValue; import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.startsWith; @@ -812,4 +823,59 @@ public void testIndexPutSettingNonExistent() throws IOException { + "or check the breaking changes documentation for removed settings]")); } + @SuppressWarnings("unchecked") + public void testPutTemplate() throws Exception { + PutIndexTemplateRequest putTemplateRequest = new PutIndexTemplateRequest() + .name("my-template") + .patterns(Arrays.asList("pattern-1", "name-*")) + .order(10) + .create(randomBoolean()) + .settings(Settings.builder().put("number_of_shards", "3").put("number_of_replicas", "0")) + .mapping("doc", "host_name", "type=keyword", "description", "type=text") + .alias(new Alias("alias-1").indexRouting("abc")).alias(new Alias("{index}-write").searchRouting("xyz")); + + PutIndexTemplateResponse putTemplateResponse = execute(putTemplateRequest, + highLevelClient().indices()::putTemplate, highLevelClient().indices()::putTemplateAsync); + assertThat(putTemplateResponse.isAcknowledged(), equalTo(true)); + + Map templates = getAsMap("/_template/my-template"); + assertThat(templates.keySet(), hasSize(1)); + assertThat(extractValue("my-template.order", templates), equalTo(10)); + assertThat(extractRawValues("my-template.index_patterns", templates), contains("pattern-1", "name-*")); + assertThat(extractValue("my-template.settings.index.number_of_shards", templates), equalTo("3")); + assertThat(extractValue("my-template.settings.index.number_of_replicas", templates), equalTo("0")); + assertThat(extractValue("my-template.mappings.doc.properties.host_name.type", templates), equalTo("keyword")); + assertThat(extractValue("my-template.mappings.doc.properties.description.type", templates), equalTo("text")); + assertThat((Map) extractValue("my-template.aliases.alias-1", templates), hasEntry("index_routing", "abc")); + assertThat((Map) extractValue("my-template.aliases.{index}-write", templates), hasEntry("search_routing", "xyz")); + } + + public void testPutTemplateBadRequests() throws Exception { + RestHighLevelClient client = highLevelClient(); + + // Failed to validate because index patterns are missing + PutIndexTemplateRequest withoutPattern = new PutIndexTemplateRequest("t1"); + ValidationException withoutPatternError = expectThrows(ValidationException.class, + () -> execute(withoutPattern, client.indices()::putTemplate, client.indices()::putTemplateAsync)); + assertThat(withoutPatternError.validationErrors(), contains("index patterns are missing")); + + // Create-only specified but an template exists already + PutIndexTemplateRequest goodTemplate = new PutIndexTemplateRequest("t2").patterns(Arrays.asList("qa-*", "prod-*")); + assertTrue(execute(goodTemplate, client.indices()::putTemplate, client.indices()::putTemplateAsync).isAcknowledged()); + goodTemplate.create(true); + ElasticsearchException alreadyExistsError = expectThrows(ElasticsearchException.class, + () -> execute(goodTemplate, client.indices()::putTemplate, client.indices()::putTemplateAsync)); + assertThat(alreadyExistsError.getDetailedMessage(), + containsString("[type=illegal_argument_exception, reason=index_template [t2] already exists]")); + goodTemplate.create(false); + assertTrue(execute(goodTemplate, client.indices()::putTemplate, client.indices()::putTemplateAsync).isAcknowledged()); + + // Rejected due to unknown settings + PutIndexTemplateRequest unknownSettingTemplate = new PutIndexTemplateRequest("t3") + .patterns(Collections.singletonList("any")) + .settings(Settings.builder().put("this-setting-does-not-exist", 100)); + ElasticsearchStatusException unknownSettingError = expectThrows(ElasticsearchStatusException.class, + () -> execute(unknownSettingTemplate, client.indices()::putTemplate, client.indices()::putTemplateAsync)); + assertThat(unknownSettingError.getDetailedMessage(), containsString("unknown setting [index.this-setting-does-not-exist]")); + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index 1953c820b8af8..2d4ef8b6413d9 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -26,12 +26,12 @@ import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; import org.apache.http.entity.ByteArrayEntity; -import org.apache.http.entity.ContentType; -import org.apache.http.entity.StringEntity; import org.apache.http.util.EntityUtils; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; @@ -46,10 +46,11 @@ import org.elasticsearch.action.admin.indices.open.OpenIndexRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.rollover.RolloverRequest; -import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest; +import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeType; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.delete.DeleteRequest; @@ -70,6 +71,7 @@ import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestConverters.EndpointBuilder; import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; @@ -77,15 +79,13 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.client.RequestConverters.EndpointBuilder; -import org.elasticsearch.client.RequestConverters.Params; import org.elasticsearch.index.RandomCreateIndexGenerator; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.query.TermQueryBuilder; @@ -94,7 +94,6 @@ import org.elasticsearch.index.rankeval.RankEvalSpec; import org.elasticsearch.index.rankeval.RatedRequest; import org.elasticsearch.index.rankeval.RestRankEvalAction; -import org.elasticsearch.rest.action.RestFieldCapabilitiesAction; import org.elasticsearch.rest.action.search.RestSearchAction; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; @@ -111,8 +110,6 @@ import java.io.IOException; import java.io.InputStream; -import java.lang.reflect.Constructor; -import java.lang.reflect.Modifier; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -121,7 +118,6 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Set; import java.util.StringJoiner; import java.util.function.Consumer; import java.util.function.Function; @@ -1432,6 +1428,68 @@ public void testIndexPutSettings() throws IOException { assertEquals(expectedParams, request.getParameters()); } + public void testGetRepositories() { + Map expectedParams = new HashMap<>(); + StringBuilder endpoint = new StringBuilder("/_snapshot"); + + GetRepositoriesRequest getRepositoriesRequest = new GetRepositoriesRequest(); + setRandomMasterTimeout(getRepositoriesRequest, expectedParams); + setRandomLocal(getRepositoriesRequest, expectedParams); + + if (randomBoolean()) { + String[] entries = new String[] {"a", "b", "c"}; + getRepositoriesRequest.repositories(entries); + endpoint.append("/" + String.join(",", entries)); + } + + Request request = RequestConverters.getRepositories(getRepositoriesRequest); + assertThat(endpoint.toString(), equalTo(request.getEndpoint())); + assertThat(HttpGet.METHOD_NAME, equalTo(request.getMethod())); + assertThat(expectedParams, equalTo(request.getParameters())); + } + + public void testPutTemplateRequest() throws Exception { + Map names = new HashMap<>(); + names.put("log", "log"); + names.put("template#1", "template%231"); + names.put("-#template", "-%23template"); + names.put("foo^bar", "foo%5Ebar"); + + PutIndexTemplateRequest putTemplateRequest = new PutIndexTemplateRequest() + .name(randomFrom(names.keySet())) + .patterns(Arrays.asList(generateRandomStringArray(20, 100, false, false))); + if (randomBoolean()) { + putTemplateRequest.order(randomInt()); + } + if (randomBoolean()) { + putTemplateRequest.version(randomInt()); + } + if (randomBoolean()) { + putTemplateRequest.settings(Settings.builder().put("setting-" + randomInt(), randomTimeValue())); + } + if (randomBoolean()) { + putTemplateRequest.mapping("doc-" + randomInt(), "field-" + randomInt(), "type=" + randomFrom("text", "keyword")); + } + if (randomBoolean()) { + putTemplateRequest.alias(new Alias("alias-" + randomInt())); + } + Map expectedParams = new HashMap<>(); + if (randomBoolean()) { + expectedParams.put("create", Boolean.TRUE.toString()); + putTemplateRequest.create(true); + } + if (randomBoolean()) { + String cause = randomUnicodeOfCodepointLengthBetween(1, 50); + putTemplateRequest.cause(cause); + expectedParams.put("cause", cause); + } + setRandomMasterTimeout(putTemplateRequest, expectedParams); + Request request = RequestConverters.putTemplate(putTemplateRequest); + assertThat(request.getEndpoint(), equalTo("/_template/" + names.get(putTemplateRequest.name()))); + assertThat(request.getParameters(), equalTo(expectedParams)); + assertToXContentBody(putTemplateRequest, request.getEntity()); + } + private static void assertToXContentBody(ToXContent expectedBody, HttpEntity actualEntity) throws IOException { BytesReference expectedBytes = XContentHelper.toXContent(expectedBody, REQUEST_BODY_CONTENT_TYPE, false); assertEquals(XContentType.JSON.mediaTypeWithoutParameters(), actualEntity.getContentType().getValue()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java new file mode 100644 index 0000000000000..ab2c632bfeb58 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SnapshotIT.java @@ -0,0 +1,82 @@ +/* + * 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.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; +import java.util.Collections; + +import static org.hamcrest.Matchers.equalTo; + +public class SnapshotIT extends ESRestHighLevelClientTestCase { + + public void testModulesGetRepositoriesUsingParams() throws IOException { + String repository = "test"; + String repositorySettings = "{\"type\":\"fs\", \"settings\":{\"location\": \".\"}}"; + highLevelClient().getLowLevelClient().performRequest("put", "_snapshot/" + repository, Collections.emptyMap(), + new StringEntity(repositorySettings, ContentType.APPLICATION_JSON)); + + highLevelClient().getLowLevelClient().performRequest("put", "_snapshot/" + repository + "_other", Collections.emptyMap(), + new StringEntity(repositorySettings, ContentType.APPLICATION_JSON)); + + { + GetRepositoriesRequest request = new GetRepositoriesRequest(); + request.repositories(new String[]{repository}); + GetRepositoriesResponse response = execute(request, highLevelClient().snapshot()::getRepositories, + highLevelClient().snapshot()::getRepositoriesAsync); + assertThat(1, equalTo(response.repositories().size())); + } + { + GetRepositoriesRequest request = new GetRepositoriesRequest(); + GetRepositoriesResponse response = execute(request, highLevelClient().snapshot()::getRepositories, + highLevelClient().snapshot()::getRepositoriesAsync); + assertThat(2, equalTo(response.repositories().size())); + } + } + + public void testModulesGetDefaultRepositories() throws IOException { + String repositorySettings = "{\"type\":\"fs\", \"settings\":{\"location\": \".\"}}"; + GetRepositoriesRequest request = new GetRepositoriesRequest(); + + highLevelClient().getLowLevelClient().performRequest("put", "_snapshot/test", Collections.emptyMap(), + new StringEntity(repositorySettings, ContentType.APPLICATION_JSON)); + + GetRepositoriesResponse response = execute(request, highLevelClient().snapshot()::getRepositories, + highLevelClient().snapshot()::getRepositoriesAsync); + assertThat(1, equalTo(response.repositories().size())); + } + + public void testModulesGetRepositoriesNonExistent() throws IOException { + String repository = "doesnotexist"; + GetRepositoriesRequest request = new GetRepositoriesRequest(new String[]{repository}); + ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> execute(request, + highLevelClient().snapshot()::getRepositories, highLevelClient().snapshot()::getRepositoriesAsync)); + + assertThat(exception.status(), equalTo(RestStatus.NOT_FOUND)); + assertThat(exception.getMessage(), equalTo( + "Elasticsearch exception [type=repository_missing_exception, reason=[" + repository + "] missing]")); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java index 33cd12152851b..1dd9834d8f53f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java @@ -55,6 +55,10 @@ import org.elasticsearch.action.admin.indices.shrink.ResizeRequest; import org.elasticsearch.action.admin.indices.shrink.ResizeResponse; import org.elasticsearch.action.admin.indices.shrink.ResizeType; +import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; +import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateResponse; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest; +import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.IndicesOptions; @@ -71,11 +75,14 @@ import org.elasticsearch.rest.RestStatus; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import static org.hamcrest.Matchers.equalTo; + /** * This class is used to generate the Java Indices API documentation. * You need to wrap your code between two tags like: @@ -1598,4 +1605,164 @@ public void onFailure(Exception e) { assertTrue(latch.await(30L, TimeUnit.SECONDS)); } + public void testPutTemplate() throws Exception { + RestHighLevelClient client = highLevelClient(); + + // tag::put-template-request + PutIndexTemplateRequest request = new PutIndexTemplateRequest("my-template"); // <1> + request.patterns(Arrays.asList("pattern-1", "log-*")); // <2> + // end::put-template-request + + // tag::put-template-request-settings + request.settings(Settings.builder() // <1> + .put("index.number_of_shards", 3) + .put("index.number_of_replicas", 1) + ); + // end::put-template-request-settings + + { + // tag::put-template-request-mappings-json + request.mapping("tweet", // <1> + "{\n" + + " \"tweet\": {\n" + + " \"properties\": {\n" + + " \"message\": {\n" + + " \"type\": \"text\"\n" + + " }\n" + + " }\n" + + " }\n" + + "}", // <2> + XContentType.JSON); + // end::put-template-request-mappings-json + assertTrue(client.indices().putTemplate(request).isAcknowledged()); + } + { + //tag::put-template-request-mappings-map + Map jsonMap = new HashMap<>(); + Map message = new HashMap<>(); + message.put("type", "text"); + Map properties = new HashMap<>(); + properties.put("message", message); + Map tweet = new HashMap<>(); + tweet.put("properties", properties); + jsonMap.put("tweet", tweet); + request.mapping("tweet", jsonMap); // <1> + //end::put-template-request-mappings-map + assertTrue(client.indices().putTemplate(request).isAcknowledged()); + } + { + //tag::put-template-request-mappings-xcontent + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + { + builder.startObject("tweet"); + { + builder.startObject("properties"); + { + builder.startObject("message"); + { + builder.field("type", "text"); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + request.mapping("tweet", builder); // <1> + //end::put-template-request-mappings-xcontent + assertTrue(client.indices().putTemplate(request).isAcknowledged()); + } + { + //tag::put-template-request-mappings-shortcut + request.mapping("tweet", "message", "type=text"); // <1> + //end::put-template-request-mappings-shortcut + assertTrue(client.indices().putTemplate(request).isAcknowledged()); + } + + // tag::put-template-request-aliases + request.alias(new Alias("twitter_alias").filter(QueryBuilders.termQuery("user", "kimchy"))); // <1> + request.alias(new Alias("{index}_alias").searchRouting("xyz")); // <2> + // end::put-template-request-aliases + + // tag::put-template-request-order + request.order(20); // <1> + // end::put-template-request-order + + // tag::put-template-request-version + request.version(4); // <1> + // end::put-template-request-version + + // tag::put-template-whole-source + request.source("{\n" + + " \"index_patterns\": [\n" + + " \"log-*\",\n" + + " \"pattern-1\"\n" + + " ],\n" + + " \"order\": 1,\n" + + " \"settings\": {\n" + + " \"number_of_shards\": 1\n" + + " },\n" + + " \"mappings\": {\n" + + " \"tweet\": {\n" + + " \"properties\": {\n" + + " \"message\": {\n" + + " \"type\": \"text\"\n" + + " }\n" + + " }\n" + + " }\n" + + " },\n" + + " \"aliases\": {\n" + + " \"alias-1\": {},\n" + + " \"{index}-alias\": {}\n" + + " }\n" + + "}", XContentType.JSON); // <1> + // end::put-template-whole-source + + // tag::put-template-request-create + request.create(true); // <1> + // end::put-template-request-create + + // tag::put-template-request-masterTimeout + request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1> + request.masterNodeTimeout("1m"); // <2> + // end::put-template-request-masterTimeout + + request.create(false); // make test happy + + // tag::put-template-execute + PutIndexTemplateResponse putTemplateResponse = client.indices().putTemplate(request); + // end::put-template-execute + + // tag::put-template-response + boolean acknowledged = putTemplateResponse.isAcknowledged(); // <1> + // end::put-template-response + assertTrue(acknowledged); + + // tag::put-template-execute-listener + ActionListener listener = + new ActionListener() { + @Override + public void onResponse(PutIndexTemplateResponse putTemplateResponse) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::put-template-execute-listener + + // Replace the empty listener by a blocking listener in test + final CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::put-template-execute-async + client.indices().putTemplateAsync(request, listener); // <1> + // end::put-template-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java new file mode 100644 index 0000000000000..1044cc9da3332 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SnapshotClientDocumentationIT.java @@ -0,0 +1,135 @@ +/* + * 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.documentation; + +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; +import org.elasticsearch.client.ESRestHighLevelClientTestCase; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.common.unit.TimeValue; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.equalTo; + +/** + * This class is used to generate the Java Cluster API documentation. + * You need to wrap your code between two tags like: + * // tag::example + * // end::example + * + * Where example is your tag name. + * + * Then in the documentation, you can extract what is between tag and end tags with + * ["source","java",subs="attributes,callouts,macros"] + * -------------------------------------------------- + * include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[example] + * -------------------------------------------------- + * + * The column width of the code block is 84. If the code contains a line longer + * than 84, the line will be cut and a horizontal scroll bar will be displayed. + * (the code indentation of the tag is not included in the width) + */ +public class SnapshotClientDocumentationIT extends ESRestHighLevelClientTestCase { + + private static final String testRepository = "test_repository"; + + public void testSnapshotGetRepository() throws IOException { + RestHighLevelClient client = highLevelClient(); + + createTestRepositories(); + + // tag::get-repository-request + GetRepositoriesRequest request = new GetRepositoriesRequest(); + // end::get-repository-request + + // tag::get-repository-request-repositories + String [] repositories = new String[] { testRepository }; + request.repositories(repositories); // <1> + // end::get-repository-request-repositories + // tag::get-repository-request-local + request.local(true); // <1> + // end::get-repository-request-local + // tag::get-repository-request-masterTimeout + request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1> + request.masterNodeTimeout("1m"); // <2> + // end::get-repository-request-masterTimeout + + // tag::get-repository-execute + GetRepositoriesResponse response = client.snapshot().getRepositories(request); + // end::get-repository-execute + + // tag::get-repository-response + List repositoryMetaDataResponse = response.repositories(); + // end::get-repository-response + assertThat(1, equalTo(repositoryMetaDataResponse.size())); + assertThat(testRepository, equalTo(repositoryMetaDataResponse.get(0).name())); + } + + public void testSnapshotGetRepositoryAsync() throws InterruptedException { + RestHighLevelClient client = highLevelClient(); + { + GetRepositoriesRequest request = new GetRepositoriesRequest(); + + // tag::get-repository-execute-listener + ActionListener listener = + new ActionListener() { + @Override + public void onResponse(GetRepositoriesResponse getRepositoriesResponse) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::get-repository-execute-listener + + // Replace the empty listener by a blocking listener in test + final CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::get-repository-execute-async + client.snapshot().getRepositoriesAsync(request, listener); // <1> + // end::get-repository-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + + } + + private void createTestRepositories() throws IOException { + RestHighLevelClient client = highLevelClient(); + String repositorySettings = "{\"type\":\"fs\", \"settings\":{\"location\": \".\"}}"; + highLevelClient().getLowLevelClient().performRequest("put", "_snapshot/" + testRepository, Collections.emptyMap(), + new StringEntity(repositorySettings, ContentType.APPLICATION_JSON)); + + } +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/Request.java b/client/rest/src/main/java/org/elasticsearch/client/Request.java index bf0012339fb41..92610239cae92 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/Request.java +++ b/client/rest/src/main/java/org/elasticsearch/client/Request.java @@ -19,8 +19,10 @@ package org.elasticsearch.client; +import org.apache.http.entity.ContentType; import org.apache.http.Header; import org.apache.http.HttpEntity; +import org.apache.http.nio.entity.NStringEntity; import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; import java.util.Arrays; @@ -103,6 +105,17 @@ public void setEntity(HttpEntity entity) { this.entity = entity; } + /** + * Set the body of the request to a string. If not set or set to + * {@code null} then no body is sent with the request. The + * {@code Content-Type} will be sent as {@code application/json}. + * If you need a different content type then use + * {@link #setEntity(HttpEntity)}. + */ + public void setJsonEntity(String entity) { + setEntity(entity == null ? null : new NStringEntity(entity, ContentType.APPLICATION_JSON)); + } + /** * The body of the request. If {@code null} then no body * is sent with the request. 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 1d4036c210308..05fa4d536b3b6 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -210,7 +210,9 @@ public void performRequestAsync(Request request, ResponseListener responseListen * @throws IOException in case of a problem or the connection was aborted * @throws ClientProtocolException in case of an http protocol error * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + * @deprecated prefer {@link #performRequest(Request)} */ + @Deprecated public Response performRequest(String method, String endpoint, Header... headers) throws IOException { Request request = new Request(method, endpoint); request.setHeaders(headers); @@ -229,7 +231,9 @@ public Response performRequest(String method, String endpoint, Header... headers * @throws IOException in case of a problem or the connection was aborted * @throws ClientProtocolException in case of an http protocol error * @throws ResponseException in case Elasticsearch responded with a status code that indicated an error + * @deprecated prefer {@link #performRequest(Request)} */ + @Deprecated public Response performRequest(String method, String endpoint, Map params, Header... headers) throws IOException { Request request = new Request(method, endpoint); addParameters(request, params); @@ -252,7 +256,9 @@ public Response performRequest(String method, String endpoint, Map params, HttpEntity entity, Header... headers) throws IOException { Request request = new Request(method, endpoint); @@ -289,7 +295,9 @@ public Response performRequest(String method, String endpoint, Map params, HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, Header... headers) throws IOException { @@ -310,7 +318,9 @@ public Response performRequest(String method, String endpoint, Map params, ResponseListener responseListener, Header... headers) { Request request; @@ -361,7 +373,9 @@ public void performRequestAsync(String method, String endpoint, Map params, HttpEntity entity, ResponseListener responseListener, Header... headers) { Request request; @@ -394,7 +408,9 @@ public void performRequestAsync(String method, String endpoint, Map params, HttpEntity entity, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, ResponseListener responseListener, Header... headers) { diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java b/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java index 286ed7dd53910..8768c07161989 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClientBuilder.java @@ -43,7 +43,6 @@ public final class RestClientBuilder { public static final int DEFAULT_CONNECT_TIMEOUT_MILLIS = 1000; public static final int DEFAULT_SOCKET_TIMEOUT_MILLIS = 30000; public static final int DEFAULT_MAX_RETRY_TIMEOUT_MILLIS = DEFAULT_SOCKET_TIMEOUT_MILLIS; - public static final int DEFAULT_CONNECTION_REQUEST_TIMEOUT_MILLIS = 500; public static final int DEFAULT_MAX_CONN_PER_ROUTE = 10; public static final int DEFAULT_MAX_CONN_TOTAL = 30; @@ -196,8 +195,7 @@ private CloseableHttpAsyncClient createHttpClient() { //default timeouts are all infinite RequestConfig.Builder requestConfigBuilder = RequestConfig.custom() .setConnectTimeout(DEFAULT_CONNECT_TIMEOUT_MILLIS) - .setSocketTimeout(DEFAULT_SOCKET_TIMEOUT_MILLIS) - .setConnectionRequestTimeout(DEFAULT_CONNECTION_REQUEST_TIMEOUT_MILLIS); + .setSocketTimeout(DEFAULT_SOCKET_TIMEOUT_MILLIS); if (requestConfigCallback != null) { requestConfigBuilder = requestConfigCallback.customizeRequestConfig(requestConfigBuilder); } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java b/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java index 98fcf8421ae6a..b83115a5341dd 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.client; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -27,9 +29,11 @@ import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; import org.apache.http.message.BasicHeader; +import org.apache.http.nio.entity.NStringEntity; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; public class RequestTests extends RestClientTestCase { @@ -99,12 +103,27 @@ public void testSetEntity() { final String endpoint = randomAsciiLettersOfLengthBetween(1, 10); final HttpEntity entity = randomBoolean() ? new StringEntity(randomAsciiLettersOfLengthBetween(1, 100), ContentType.TEXT_PLAIN) : null; - Request request = new Request(method, endpoint); + Request request = new Request(method, endpoint); request.setEntity(entity); assertEquals(entity, request.getEntity()); } + public void testSetJsonEntity() throws IOException { + final String method = randomFrom(new String[] {"GET", "PUT", "POST", "HEAD", "DELETE"}); + final String endpoint = randomAsciiLettersOfLengthBetween(1, 10); + + Request request = new Request(method, endpoint); + assertNull(request.getEntity()); + + final String json = randomAsciiLettersOfLengthBetween(1, 100); + request.setJsonEntity(json); + assertEquals(ContentType.APPLICATION_JSON.toString(), request.getEntity().getContentType().getValue()); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + request.getEntity().writeTo(os); + assertEquals(json, new String(os.toByteArray(), ContentType.APPLICATION_JSON.getCharset())); + } + public void testSetHeaders() { final String method = randomFrom(new String[] {"GET", "PUT", "POST", "HEAD", "DELETE"}); final String endpoint = randomAsciiLettersOfLengthBetween(1, 10); diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java index c9243d3aaf6ce..9657e782bda04 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientBuilderTests.java @@ -177,4 +177,24 @@ private static void assertSetPathPrefixThrows(final String pathPrefix) { } } + /** + * This test verifies that we don't change the default value for the connection request timeout as that causes problems. + * See https://github.com/elastic/elasticsearch/issues/24069 + */ + public void testDefaultConnectionRequestTimeout() throws IOException { + RestClientBuilder builder = RestClient.builder(new HttpHost("localhost", 9200)); + builder.setRequestConfigCallback(new RestClientBuilder.RequestConfigCallback() { + @Override + public RequestConfig.Builder customizeRequestConfig(RequestConfig.Builder requestConfigBuilder) { + RequestConfig requestConfig = requestConfigBuilder.build(); + assertEquals(RequestConfig.DEFAULT.getConnectionRequestTimeout(), requestConfig.getConnectionRequestTimeout()); + //this way we get notified if the default ever changes + assertEquals(-1, requestConfig.getConnectionRequestTimeout()); + return requestConfigBuilder; + } + }); + try (RestClient restClient = builder.build()) { + assertNotNull(restClient); + } + } } diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java index dd23dbe454fa4..3f57e8c391215 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java @@ -32,6 +32,7 @@ import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.nio.entity.NStringEntity; import org.apache.http.util.EntityUtils; import org.elasticsearch.mocksocket.MockHttpServer; import org.junit.AfterClass; @@ -48,6 +49,9 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.client.RestClientTestUtil.getAllStatusCodes; import static org.elasticsearch.client.RestClientTestUtil.getHttpMethods; @@ -159,6 +163,42 @@ public static void stopHttpServers() throws IOException { httpServer = null; } + /** + * Tests sending a bunch of async requests works well (e.g. no TimeoutException from the leased pool) + * See https://github.com/elastic/elasticsearch/issues/24069 + */ + public void testManyAsyncRequests() throws Exception { + int iters = randomIntBetween(500, 1000); + final CountDownLatch latch = new CountDownLatch(iters); + final List exceptions = new CopyOnWriteArrayList<>(); + for (int i = 0; i < iters; i++) { + Request request = new Request("PUT", "/200"); + request.setEntity(new NStringEntity("{}", ContentType.APPLICATION_JSON)); + restClient.performRequestAsync(request, new ResponseListener() { + @Override + public void onSuccess(Response response) { + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + exceptions.add(exception); + latch.countDown(); + } + }); + } + + assertTrue("timeout waiting for requests to be sent", latch.await(10, TimeUnit.SECONDS)); + if (exceptions.isEmpty() == false) { + AssertionError error = new AssertionError("expected no failures but got some. see suppressed for first 10 of [" + + exceptions.size() + "] failures"); + for (Exception exception : exceptions.subList(0, Math.min(10, exceptions.size()))) { + error.addSuppressed(exception); + } + throw error; + } + } + /** * End to end test for headers. We test it explicitly against a real http client as there are different ways * to set/add headers to the {@link org.apache.http.client.HttpClient}. 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 aa89a7d76ab26..5ee97399b34e6 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 @@ -168,10 +168,13 @@ public void onFailure(Exception exception) { request.addParameter("pretty", "true"); //end::rest-client-parameters //tag::rest-client-body - request.setEntity(new StringEntity( + request.setEntity(new NStringEntity( "{\"json\":\"text\"}", ContentType.APPLICATION_JSON)); //end::rest-client-body + //tag::rest-client-body-shorter + request.setJsonEntity("{\"json\":\"text\"}"); + //end::rest-client-body-shorter //tag::rest-client-headers request.setHeaders( new BasicHeader("Accept", "text/plain"), diff --git a/docs/CHANGELOG.asciidoc b/docs/CHANGELOG.asciidoc index 380c7288e8424..5f7ed63cdd8ad 100644 --- a/docs/CHANGELOG.asciidoc +++ b/docs/CHANGELOG.asciidoc @@ -3,7 +3,7 @@ [partintro] -- -// To add a release, copy and paste the template text +// To add a release, copy and paste the template text // and add a link to the new section. Note that release subheads must // be floated and sections cannot be empty. @@ -104,6 +104,10 @@ ones that the user is authorized to access in case field level security is enabl [float] === Bug Fixes +Use date format in `date_range` mapping before fallback to default ({pull}29310[#29310]) + +Fix NPE in 'more_like_this' when field has zero tokens ({pull}30365[#30365]) + Fixed prerelease version of elasticsearch in the `deb` package to sort before GA versions ({pull}29000[#29000]) @@ -117,6 +121,7 @@ Fail snapshot operations early when creating or deleting a snapshot on a reposit written to by an older Elasticsearch after writing to it with a newer Elasticsearch version. ({pull}30140[#30140]) Fix NPE when CumulativeSum agg encounters null value/empty bucket ({pull}29641[#29641]) +Do not fail snapshot when deleting a missing snapshotted file ({pull}30332[#30332]) //[float] //=== Regressions @@ -136,8 +141,11 @@ coming[6.4.0] //[float] //=== Breaking Java Changes -//[float] -//=== Deprecations +[float] +=== Deprecations + +Deprecated multi-argument versions of the request methods in the RestClient. +Prefer the "Request" object flavored methods. ({pull}30315[#30315]) [float] === New Features @@ -154,8 +162,12 @@ analysis module. ({pull}30397[#30397]) {ref-64}/breaking_64_api_changes.html#copy-source-settings-on-resize[Allow copying source settings on index resize operations] ({pull}30255[#30255]) -Added new "Request" object flavored request methods. Prefer these instead of the -multi-argument versions. ({pull}29623[#29623]) +Added new "Request" object flavored request methods in the RestClient. Prefer +these instead of the multi-argument versions. ({pull}29623[#29623]) + +Added `setJsonEntity` to `Request` object so it is marginally easier to send JSON. ({pull}30447[#30447]) +Watcher HTTP client used in watches now allows more parallel connections to the +same endpoint and evicts long running connections. ({pull}30130[#30130]) The cluster state listener to decide if watcher should be stopped/started/paused now runs far less code in an executor but is more @@ -163,9 +175,18 @@ synchronous and predictable. Also the trigger engine thread is only started on data nodes. And the Execute Watch API can be triggered regardless is watcher is started or stopped. ({pull}30118[#30118]) +Added put index template API to the high level rest client ({pull}30400[#30400]) + +Add ability to filter coordinating-only nodes when interacting with cluster +APIs. ({pull}30313[#30313]) + [float] === Bug Fixes +Use date format in `date_range` mapping before fallback to default ({pull}29310[#29310]) + +Fix NPE in 'more_like_this' when field has zero tokens ({pull}30365[#30365]) + Do not ignore request analysis/similarity settings on index resize operations when the source index already contains such settings ({pull}30216[#30216]) Fix NPE when CumulativeSum agg encounters null value/empty bucket ({pull}29641[#29641]) @@ -174,10 +195,18 @@ Machine Learning:: * Account for gaps in data counts after job is reopened ({pull}30294[#30294]) +Add validation that geohashes are not empty and don't contain unsupported characters ({pull}30376[#30376]) + Rollup:: * Validate timezone in range queries to ensure they match the selected job when searching ({pull}30338[#30338]) + +Allocation:: + +Auto-expand replicas when adding or removing nodes to prevent shard copies from +being dropped and resynced when a data node rejoins the cluster ({pull}30423[#30423]) + //[float] //=== Regressions diff --git a/docs/README.asciidoc b/docs/README.asciidoc index 1e4a313578529..f0826b4161224 100644 --- a/docs/README.asciidoc +++ b/docs/README.asciidoc @@ -68,6 +68,23 @@ for its modifiers: but rather than the setup defined in `docs/build.gradle` the setup is defined right in the documentation file. +In addition to the standard CONSOLE syntax these snippets can contain blocks +of yaml surrounded by markers like this: + +``` +startyaml + - compare_analyzers: {index: thai_example, first: thai, second: rebuilt_thai} +endyaml +``` + +This allows slightly more expressive testing of the snippets. Since that syntax +is not supported by CONSOLE the usual way to incorporate it is with a +`// TEST[s//]` marker like this: + +``` +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: thai_example, first: thai, second: rebuilt_thai}\nendyaml\n/] +``` + Any place you can use json you can use elements like `$body.path.to.thing` which is replaced on the fly with the contents of the thing at `path.to.thing` in the last response. diff --git a/docs/build.gradle b/docs/build.gradle index e8c406594b29f..6498de0218120 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -60,6 +60,8 @@ buildRestTests.docs = fileTree(projectDir) { exclude 'build.gradle' // That is where the snippets go, not where they come from! exclude 'build' + // Just syntax examples + exclude 'README.asciidoc' } Closure setupTwitter = { String name, int count -> diff --git a/docs/java-rest/high-level/indices/put_template.asciidoc b/docs/java-rest/high-level/indices/put_template.asciidoc new file mode 100644 index 0000000000000..7f0f3a1fee7f5 --- /dev/null +++ b/docs/java-rest/high-level/indices/put_template.asciidoc @@ -0,0 +1,168 @@ +[[java-rest-high-put-template]] +=== Put Template API + +[[java-rest-high-put-template-request]] +==== Put Index Template Request + +A `PutIndexTemplateRequest` specifies the `name` of a template and `patterns` +which controls whether the template should be applied to the new index. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request] +-------------------------------------------------- +<1> The name of the template +<2> The patterns of the template + +==== Settings +The settings of the template will be applied to the new index whose name matches the +template's patterns. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-settings] +-------------------------------------------------- +<1> Settings for this template + +[[java-rest-high-put-template-request-mappings]] +==== Mappings +The mapping of the template will be applied to the new index whose name matches the +template's patterns. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-mappings-json] +-------------------------------------------------- +<1> The type to define +<2> The mapping for this type, provided as a JSON string + +The mapping source can be provided in different ways in addition to the +`String` example shown above: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-mappings-map] +-------------------------------------------------- +<1> Mapping source provided as a `Map` which gets automatically converted +to JSON format + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-mappings-xcontent] +-------------------------------------------------- +<1> Mapping source provided as an `XContentBuilder` object, the Elasticsearch +built-in helpers to generate JSON content + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-mappings-shortcut] +-------------------------------------------------- +<1> Mapping source provided as `Object` key-pairs, which gets converted to +JSON format + +==== Aliases +The aliases of the template will define aliasing to the index whose name matches the +template's patterns. A placeholder `{index}` can be used in an alias of a template. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-aliases] +-------------------------------------------------- +<1> The alias to define +<2> The alias to define with placeholder + +==== Order +In case multiple templates match an index, the orders of matching templates determine +the sequence that settings, mappings, and alias of each matching template is applied. +Templates with lower orders are applied first, and higher orders override them. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-order] +-------------------------------------------------- +<1> The order of the template + +==== Version +A template can optionally specify a version number which can be used to simplify template +management by external systems. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-version] +-------------------------------------------------- +<1> The version number of the template + +==== Providing the whole source +The whole source including all of its sections (mappings, settings and aliases) +can also be provided: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-whole-source] +-------------------------------------------------- +<1> The source provided as a JSON string. It can also be provided as a `Map` +or an `XContentBuilder`. + +==== Optional arguments +The following arguments can optionally be provided: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-request-create] +-------------------------------------------------- +<1> To force to only create a new template; do not overwrite the existing template + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-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-put-template-sync]] +==== Synchronous Execution + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-execute] +-------------------------------------------------- + +[[java-rest-high-put-template-async]] +==== Asynchronous Execution + +The asynchronous execution of a put template request requires both the `PutIndexTemplateRequest` +instance and an `ActionListener` instance to be passed to the asynchronous method: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-execute-async] +-------------------------------------------------- +<1> The `PutIndexTemplateRequest` 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 `PutIndexTemplateResponse` looks like: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-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-put-template-response]] +==== Put Index Template Response + +The returned `PutIndexTemplateResponse` allows to retrieve information about the +executed operation as follows: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-template-response] +-------------------------------------------------- +<1> Indicates whether all of the nodes have acknowledged the request diff --git a/docs/java-rest/high-level/snapshot/get_repository.asciidoc b/docs/java-rest/high-level/snapshot/get_repository.asciidoc new file mode 100644 index 0000000000000..af006c66ab087 --- /dev/null +++ b/docs/java-rest/high-level/snapshot/get_repository.asciidoc @@ -0,0 +1,86 @@ +[[java-rest-high-snapshot-get-repository]] +=== Snapshot Get Repository API + +The Snapshot Get Repository API allows to retrieve information about a registered repository. + +[[java-rest-high-snapshot-get-repository-request]] +==== Snapshot Get Repository Request + +A `GetRepositoriesRequest`: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-request] +-------------------------------------------------- + +==== Optional Arguments +The following arguments can optionally be provided: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-request-repositories] +-------------------------------------------------- +<1> Sets the repositories to retrieve + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-request-local] +-------------------------------------------------- +<1> The `local` flag (defaults to `false`) controls whether the repositories need +to be looked up in the local cluster state or in the cluster state held by +the elected master node + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-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-snapshot-get-repository-sync]] +==== Synchronous Execution + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-execute] +-------------------------------------------------- + +[[java-rest-high-snapshot-get-repository-async]] +==== Asynchronous Execution + +The asynchronous execution of a snapshot get repository requires both the +`GetRepositoriesRequest` instance and an `ActionListener` instance to be +passed to the asynchronous method: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-execute-async] +-------------------------------------------------- +<1> The `GetRepositoriesRequest` 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 `GetRepositoriesResponse` looks like: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-execute-listener] +-------------------------------------------------- +<1> Called when the execution is successfully completed. The response is +provided as an argument +<2> Called in case of a failure. The raised exception is provided as an argument + +[[java-rest-high-cluster-get-repository-response]] +==== Snapshot Get Repository Response + +The returned `GetRepositoriesResponse` allows to retrieve information about the +executed operation as follows: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/SnapshotClientDocumentationIT.java[get-repository-response] +-------------------------------------------------- diff --git a/docs/java-rest/high-level/supported-apis.asciidoc b/docs/java-rest/high-level/supported-apis.asciidoc index 4d845e538415f..2dee4643e73eb 100644 --- a/docs/java-rest/high-level/supported-apis.asciidoc +++ b/docs/java-rest/high-level/supported-apis.asciidoc @@ -95,6 +95,7 @@ include::indices/update_aliases.asciidoc[] include::indices/exists_alias.asciidoc[] include::indices/put_settings.asciidoc[] include::indices/get_settings.asciidoc[] +include::indices/put_template.asciidoc[] == Cluster APIs @@ -103,3 +104,11 @@ The Java High Level REST Client supports the following Cluster APIs: * <> include::cluster/put_settings.asciidoc[] + +== Snapshot APIs + +The Java High Level REST Client supports the following Snapshot APIs: + +* <> + +include::snapshot/get_repository.asciidoc[] \ No newline at end of file diff --git a/docs/java-rest/low-level/usage.asciidoc b/docs/java-rest/low-level/usage.asciidoc index 5ffc4332a7681..68367b9a64fdf 100644 --- a/docs/java-rest/low-level/usage.asciidoc +++ b/docs/java-rest/low-level/usage.asciidoc @@ -263,6 +263,14 @@ IMPORTANT: The `ContentType` specified for the `HttpEntity` is important because it will be used to set the `Content-Type` header so that Elasticsearch can properly parse the content. +You can also set it to a `String` which will default to +a `ContentType` of `application/json`. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-body-shorter] +-------------------------------------------------- + And you can set a list of headers to send with the request: ["source","java",subs="attributes,callouts,macros"] diff --git a/docs/plugins/api.asciidoc b/docs/plugins/api.asciidoc index a2fbc5165ac94..4dffb9608821f 100644 --- a/docs/plugins/api.asciidoc +++ b/docs/plugins/api.asciidoc @@ -19,6 +19,9 @@ A number of plugins have been contributed by our community: * https://github.com/YannBrrd/elasticsearch-entity-resolution[Entity Resolution Plugin]: Uses http://github.com/larsga/Duke[Duke] for duplication detection (by Yann Barraud) + +* https://github.com/zentity-io/zentity[Entity Resolution Plugin] (https://zentity.io[zentity]): + Real-time entity resolution with pure Elasticsearch (by Dave Moore) * https://github.com/NLPchina/elasticsearch-sql/[SQL language Plugin]: Allows Elasticsearch to be queried with SQL (by nlpcn) diff --git a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc index d458d377a6077..96822f6ea9c34 100644 --- a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc @@ -63,7 +63,7 @@ POST /sales/_search?size=0 defines a unique count below which counts are expected to be close to accurate. Above this value, counts might become a bit more fuzzy. The maximum supported value is 40000, thresholds above this number will have the same -effect as a threshold of 40000. The default values is +3000+. +effect as a threshold of 40000. The default value is +3000+. ==== Counts are approximate diff --git a/docs/reference/analysis/analyzers/lang-analyzer.asciidoc b/docs/reference/analysis/analyzers/lang-analyzer.asciidoc index cb976601fdcbe..d718a0b2da6ff 100644 --- a/docs/reference/analysis/analyzers/lang-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/lang-analyzer.asciidoc @@ -97,10 +97,11 @@ PUT /arabic_example } }, "analyzer": { - "arabic": { + "rebuilt_arabic": { "tokenizer": "standard", "filter": [ "lowercase", + "decimal_digit", "arabic_stop", "arabic_normalization", "arabic_keywords", @@ -113,6 +114,8 @@ PUT /arabic_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"arabic_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: arabic_example, first: arabic, second: rebuilt_arabic}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -144,7 +147,7 @@ PUT /armenian_example } }, "analyzer": { - "armenian": { + "rebuilt_armenian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -159,6 +162,8 @@ PUT /armenian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"armenian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: armenian_example, first: armenian, second: rebuilt_armenian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -190,7 +195,7 @@ PUT /basque_example } }, "analyzer": { - "basque": { + "rebuilt_basque": { "tokenizer": "standard", "filter": [ "lowercase", @@ -205,6 +210,8 @@ PUT /basque_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"basque_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: basque_example, first: basque, second: rebuilt_basque}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -236,14 +243,15 @@ PUT /bengali_example } }, "analyzer": { - "bengali": { + "rebuilt_bengali": { "tokenizer": "standard", "filter": [ "lowercase", + "decimal_digit", + "bengali_keywords", "indic_normalization", "bengali_normalization", "bengali_stop", - "bengali_keywords", "bengali_stemmer" ] } @@ -253,6 +261,8 @@ PUT /bengali_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"bengali_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: bengali_example, first: bengali, second: rebuilt_bengali}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -284,7 +294,7 @@ PUT /brazilian_example } }, "analyzer": { - "brazilian": { + "rebuilt_brazilian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -299,6 +309,8 @@ PUT /brazilian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"brazilian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: brazilian_example, first: brazilian, second: rebuilt_brazilian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -330,7 +342,7 @@ PUT /bulgarian_example } }, "analyzer": { - "bulgarian": { + "rebuilt_bulgarian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -345,6 +357,8 @@ PUT /bulgarian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"bulgarian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: bulgarian_example, first: bulgarian, second: rebuilt_bulgarian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -380,7 +394,7 @@ PUT /catalan_example } }, "analyzer": { - "catalan": { + "rebuilt_catalan": { "tokenizer": "standard", "filter": [ "catalan_elision", @@ -396,6 +410,8 @@ PUT /catalan_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"catalan_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: catalan_example, first: catalan, second: rebuilt_catalan}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -415,11 +431,17 @@ PUT /cjk_example "filter": { "english_stop": { "type": "stop", - "stopwords": "_english_" <1> + "stopwords": [ <1> + "a", "and", "are", "as", "at", "be", "but", "by", "for", + "if", "in", "into", "is", "it", "no", "not", "of", "on", + "or", "s", "such", "t", "that", "the", "their", "then", + "there", "these", "they", "this", "to", "was", "will", + "with", "www" + ] } }, "analyzer": { - "cjk": { + "rebuilt_cjk": { "tokenizer": "standard", "filter": [ "cjk_width", @@ -434,8 +456,12 @@ PUT /cjk_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"cjk_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: cjk_example, first: cjk, second: rebuilt_cjk}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` - or `stopwords_path` parameters. + or `stopwords_path` parameters. The default stop words are + *almost* the same as the `_english_` set, but not exactly + the same. [[czech-analyzer]] ===== `czech` analyzer @@ -463,7 +489,7 @@ PUT /czech_example } }, "analyzer": { - "czech": { + "rebuilt_czech": { "tokenizer": "standard", "filter": [ "lowercase", @@ -478,6 +504,8 @@ PUT /czech_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"czech_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: czech_example, first: czech, second: rebuilt_czech}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -509,7 +537,7 @@ PUT /danish_example } }, "analyzer": { - "danish": { + "rebuilt_danish": { "tokenizer": "standard", "filter": [ "lowercase", @@ -524,6 +552,8 @@ PUT /danish_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"danish_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: danish_example, first: danish, second: rebuilt_danish}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -564,7 +594,7 @@ PUT /dutch_example } }, "analyzer": { - "dutch": { + "rebuilt_dutch": { "tokenizer": "standard", "filter": [ "lowercase", @@ -580,6 +610,8 @@ PUT /dutch_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"dutch_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: dutch_example, first: dutch, second: rebuilt_dutch}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -615,7 +647,7 @@ PUT /english_example } }, "analyzer": { - "english": { + "rebuilt_english": { "tokenizer": "standard", "filter": [ "english_possessive_stemmer", @@ -631,6 +663,8 @@ PUT /english_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"english_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: english_example, first: english, second: rebuilt_english}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -662,7 +696,7 @@ PUT /finnish_example } }, "analyzer": { - "finnish": { + "rebuilt_finnish": { "tokenizer": "standard", "filter": [ "lowercase", @@ -677,6 +711,8 @@ PUT /finnish_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"finnish_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: finnish_example, first: finnish, second: rebuilt_finnish}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -717,7 +753,7 @@ PUT /french_example } }, "analyzer": { - "french": { + "rebuilt_french": { "tokenizer": "standard", "filter": [ "french_elision", @@ -733,6 +769,8 @@ PUT /french_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"french_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: french_example, first: french, second: rebuilt_french}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -764,7 +802,7 @@ PUT /galician_example } }, "analyzer": { - "galician": { + "rebuilt_galician": { "tokenizer": "standard", "filter": [ "lowercase", @@ -779,6 +817,8 @@ PUT /galician_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"galician_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: galician_example, first: galician, second: rebuilt_galician}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -810,7 +850,7 @@ PUT /german_example } }, "analyzer": { - "german": { + "rebuilt_german": { "tokenizer": "standard", "filter": [ "lowercase", @@ -826,6 +866,8 @@ PUT /german_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"german_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: german_example, first: german, second: rebuilt_german}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -861,7 +903,7 @@ PUT /greek_example } }, "analyzer": { - "greek": { + "rebuilt_greek": { "tokenizer": "standard", "filter": [ "greek_lowercase", @@ -876,6 +918,8 @@ PUT /greek_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"greek_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: greek_example, first: greek, second: rebuilt_greek}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -907,14 +951,15 @@ PUT /hindi_example } }, "analyzer": { - "hindi": { + "rebuilt_hindi": { "tokenizer": "standard", "filter": [ "lowercase", + "decimal_digit", + "hindi_keywords", "indic_normalization", "hindi_normalization", "hindi_stop", - "hindi_keywords", "hindi_stemmer" ] } @@ -924,6 +969,8 @@ PUT /hindi_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"hindi_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: hindi_example, first: hindi, second: rebuilt_hindi}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -955,7 +1002,7 @@ PUT /hungarian_example } }, "analyzer": { - "hungarian": { + "rebuilt_hungarian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -970,6 +1017,8 @@ PUT /hungarian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"hungarian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: hungarian_example, first: hungarian, second: rebuilt_hungarian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1002,7 +1051,7 @@ PUT /indonesian_example } }, "analyzer": { - "indonesian": { + "rebuilt_indonesian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1017,6 +1066,8 @@ PUT /indonesian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"indonesian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: indonesian_example, first: indonesian, second: rebuilt_indonesian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1034,9 +1085,15 @@ PUT /irish_example "settings": { "analysis": { "filter": { + "irish_hyphenation": { + "type": "stop", + "stopwords": [ "h", "n", "t" ], + "ignore_case": true + }, "irish_elision": { "type": "elision", - "articles": [ "h", "n", "t" ] + "articles": [ "d", "m", "b" ], + "articles_case": true }, "irish_stop": { "type": "stop", @@ -1056,12 +1113,13 @@ PUT /irish_example } }, "analyzer": { - "irish": { + "rebuilt_irish": { "tokenizer": "standard", "filter": [ - "irish_stop", + "irish_hyphenation", "irish_elision", "irish_lowercase", + "irish_stop", "irish_keywords", "irish_stemmer" ] @@ -1072,6 +1130,8 @@ PUT /irish_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"irish_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: irish_example, first: irish, second: rebuilt_irish}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1112,7 +1172,7 @@ PUT /italian_example } }, "analyzer": { - "italian": { + "rebuilt_italian": { "tokenizer": "standard", "filter": [ "italian_elision", @@ -1128,6 +1188,8 @@ PUT /italian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"italian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: italian_example, first: italian, second: rebuilt_italian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1159,7 +1221,7 @@ PUT /latvian_example } }, "analyzer": { - "latvian": { + "rebuilt_latvian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1174,6 +1236,8 @@ PUT /latvian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"latvian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: latvian_example, first: latvian, second: rebuilt_latvian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1205,7 +1269,7 @@ PUT /lithuanian_example } }, "analyzer": { - "lithuanian": { + "rebuilt_lithuanian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1220,6 +1284,8 @@ PUT /lithuanian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"lithuanian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: lithuanian_example, first: lithuanian, second: rebuilt_lithuanian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1251,7 +1317,7 @@ PUT /norwegian_example } }, "analyzer": { - "norwegian": { + "rebuilt_norwegian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1266,6 +1332,8 @@ PUT /norwegian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"norwegian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: norwegian_example, first: norwegian, second: rebuilt_norwegian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1295,11 +1363,12 @@ PUT /persian_example } }, "analyzer": { - "persian": { + "rebuilt_persian": { "tokenizer": "standard", "char_filter": [ "zero_width_spaces" ], "filter": [ "lowercase", + "decimal_digit", "arabic_normalization", "persian_normalization", "persian_stop" @@ -1311,6 +1380,7 @@ PUT /persian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: persian_example, first: persian, second: rebuilt_persian}\nendyaml\n/] <1> Replaces zero-width non-joiners with an ASCII space. <2> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. @@ -1341,7 +1411,7 @@ PUT /portuguese_example } }, "analyzer": { - "portuguese": { + "rebuilt_portuguese": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1356,6 +1426,8 @@ PUT /portuguese_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"portuguese_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: portuguese_example, first: portuguese, second: rebuilt_portuguese}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1387,7 +1459,7 @@ PUT /romanian_example } }, "analyzer": { - "romanian": { + "rebuilt_romanian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1402,6 +1474,8 @@ PUT /romanian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"romanian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: romanian_example, first: romanian, second: rebuilt_romanian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1434,7 +1508,7 @@ PUT /russian_example } }, "analyzer": { - "russian": { + "rebuilt_russian": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1449,6 +1523,8 @@ PUT /russian_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"russian_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: russian_example, first: russian, second: rebuilt_russian}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1480,11 +1556,12 @@ PUT /sorani_example } }, "analyzer": { - "sorani": { + "rebuilt_sorani": { "tokenizer": "standard", "filter": [ "sorani_normalization", "lowercase", + "decimal_digit", "sorani_stop", "sorani_keywords", "sorani_stemmer" @@ -1496,6 +1573,8 @@ PUT /sorani_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"sorani_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: sorani_example, first: sorani, second: rebuilt_sorani}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1527,7 +1606,7 @@ PUT /spanish_example } }, "analyzer": { - "spanish": { + "rebuilt_spanish": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1542,6 +1621,8 @@ PUT /spanish_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"spanish_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: spanish_example, first: spanish, second: rebuilt_spanish}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1573,7 +1654,7 @@ PUT /swedish_example } }, "analyzer": { - "swedish": { + "rebuilt_swedish": { "tokenizer": "standard", "filter": [ "lowercase", @@ -1588,6 +1669,8 @@ PUT /swedish_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"swedish_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: swedish_example, first: swedish, second: rebuilt_swedish}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1623,7 +1706,7 @@ PUT /turkish_example } }, "analyzer": { - "turkish": { + "rebuilt_turkish": { "tokenizer": "standard", "filter": [ "apostrophe", @@ -1639,6 +1722,8 @@ PUT /turkish_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"turkish_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: turkish_example, first: turkish, second: rebuilt_turkish}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. <2> This filter should be removed unless there are words which should @@ -1662,10 +1747,11 @@ PUT /thai_example } }, "analyzer": { - "thai": { + "rebuilt_thai": { "tokenizer": "thai", "filter": [ "lowercase", + "decimal_digit", "thai_stop" ] } @@ -1675,5 +1761,7 @@ PUT /thai_example } ---------------------------------------------------- // CONSOLE +// TEST[s/"thai_keywords",//] +// TEST[s/\n$/\nstartyaml\n - compare_analyzers: {index: thai_example, first: thai, second: rebuilt_thai}\nendyaml\n/] <1> The default stopwords can be overridden with the `stopwords` or `stopwords_path` parameters. diff --git a/docs/reference/cluster/tasks.asciidoc b/docs/reference/cluster/tasks.asciidoc index b3457953f46e5..2e59da4222436 100644 --- a/docs/reference/cluster/tasks.asciidoc +++ b/docs/reference/cluster/tasks.asciidoc @@ -64,9 +64,10 @@ It is also possible to retrieve information for a particular task: [source,js] -------------------------------------------------- -GET _tasks/task_id:1 <1> +GET _tasks/task_id <1> -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] // TEST[catch:missing] <1> This will return a 404 if the task isn't found. @@ -75,9 +76,10 @@ Or to retrieve all children of a particular task: [source,js] -------------------------------------------------- -GET _tasks?parent_task_id=parentTaskId:1 <1> +GET _tasks?parent_task_id=parent_task_id <1> -------------------------------------------------- // CONSOLE +// TEST[s/=parent_task_id/=node_id:1/] <1> This won't return a 404 if the parent isn't found. diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index f9919483e5a47..6edc453903d66 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -357,9 +357,10 @@ With the task id you can look up the task directly: [source,js] -------------------------------------------------- -GET /_tasks/taskId:1 +GET /_tasks/task_id -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] // TEST[catch:missing] The advantage of this API is that it integrates with `wait_for_completion=false` @@ -378,8 +379,9 @@ Any Delete By Query can be canceled using the <>: [source,js] -------------------------------------------------- -POST _tasks/task_id:1/_cancel +POST _tasks/task_id/_cancel -------------------------------------------------- +// TEST[s/task_id/node_id:1/] // CONSOLE The `task_id` can be found using the tasks API above. @@ -397,8 +399,9 @@ using the `_rethrottle` API: [source,js] -------------------------------------------------- -POST _delete_by_query/task_id:1/_rethrottle?requests_per_second=-1 +POST _delete_by_query/task_id/_rethrottle?requests_per_second=-1 -------------------------------------------------- +// TEST[s/task_id/node_id:1/] // CONSOLE The `task_id` can be found using the tasks API above. diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index e8283abfc2ef0..b4205311dfe2d 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -740,9 +740,10 @@ With the task id you can look up the task directly: [source,js] -------------------------------------------------- -GET /_tasks/taskId:1 +GET /_tasks/task_id -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] // TEST[catch:missing] The advantage of this API is that it integrates with `wait_for_completion=false` @@ -761,9 +762,10 @@ Any Reindex can be canceled using the <>: [source,js] -------------------------------------------------- -POST _tasks/task_id:1/_cancel +POST _tasks/task_id/_cancel -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] The `task_id` can be found using the Tasks API. @@ -780,9 +782,10 @@ the `_rethrottle` API: [source,js] -------------------------------------------------- -POST _reindex/task_id:1/_rethrottle?requests_per_second=-1 +POST _reindex/task_id/_rethrottle?requests_per_second=-1 -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] The `task_id` can be found using the Tasks API above. diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index 1d81e4a44ff24..6e7cfbd2b794b 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -415,9 +415,10 @@ With the task id you can look up the task directly: [source,js] -------------------------------------------------- -GET /_tasks/taskId:1 +GET /_tasks/task_id -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] // TEST[catch:missing] The advantage of this API is that it integrates with `wait_for_completion=false` @@ -436,9 +437,10 @@ Any Update By Query can be canceled using the <>: [source,js] -------------------------------------------------- -POST _tasks/task_id:1/_cancel +POST _tasks/task_id/_cancel -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] The `task_id` can be found using the tasks API above. @@ -455,9 +457,10 @@ using the `_rethrottle` API: [source,js] -------------------------------------------------- -POST _update_by_query/task_id:1/_rethrottle?requests_per_second=-1 +POST _update_by_query/task_id/_rethrottle?requests_per_second=-1 -------------------------------------------------- // CONSOLE +// TEST[s/task_id/node_id:1/] The `task_id` can be found using the tasks API above. diff --git a/docs/reference/indices/forcemerge.asciidoc b/docs/reference/indices/forcemerge.asciidoc index 26baf214176d1..57fe746d59bb2 100644 --- a/docs/reference/indices/forcemerge.asciidoc +++ b/docs/reference/indices/forcemerge.asciidoc @@ -10,6 +10,11 @@ This call will block until the merge is complete. If the http connection is lost, the request will continue in the background, and any new requests will block until the previous force merge is complete. +WARNING: Force merge should only be called against *read-only indices*. Running +force merge against a read-write index can cause very large segments to be produced +(>5Gb per segment), and the merge policy will never consider it for merging again until +it mostly consists of deleted docs. This can cause very large segments to remain in the shards. + [source,js] -------------------------------------------------- POST /twitter/_forcemerge @@ -38,6 +43,13 @@ deletes. Defaults to `false`. Note that this won't override the `flush`:: Should a flush be performed after the forced merge. Defaults to `true`. +[source,js] +-------------------------------------------------- +POST /kimchy/_forcemerge?only_expunge_deletes=false&max_num_segments=100&flush=true +-------------------------------------------------- +// CONSOLE +// TEST[s/^/PUT kimchy\n/] + [float] [[forcemerge-multi-index]] === Multi Index diff --git a/docs/reference/search/request/preference.asciidoc b/docs/reference/search/request/preference.asciidoc index dbd9055ff8c86..4fd801c5f76e3 100644 --- a/docs/reference/search/request/preference.asciidoc +++ b/docs/reference/search/request/preference.asciidoc @@ -2,7 +2,8 @@ === Preference Controls a `preference` of which shard copies on which to execute the -search. By default, the operation is randomized among the available shard copies. +search. By default, the operation is randomized among the available shard +copies, unless allocation awareness is used. The `preference` is a query string parameter which can be set to: diff --git a/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java b/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java index 46e448fa54da9..a36df9987e7de 100644 --- a/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java +++ b/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java @@ -20,18 +20,39 @@ package org.elasticsearch.smoketest; import org.apache.http.HttpHost; +import org.apache.lucene.util.BytesRef; + import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import org.elasticsearch.Version; import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentLocation; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.test.rest.yaml.ClientYamlDocsTestClient; import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ClientYamlTestClient; +import org.elasticsearch.test.rest.yaml.ClientYamlTestExecutionContext; +import org.elasticsearch.test.rest.yaml.ClientYamlTestResponse; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestSpec; +import org.elasticsearch.test.rest.yaml.section.ExecutableSection; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; public class DocsClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { @@ -41,7 +62,12 @@ public DocsClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandi @ParametersFactory public static Iterable parameters() throws Exception { - return ESClientYamlSuiteTestCase.createParameters(); + List entries = new ArrayList<>(ExecutableSection.DEFAULT_EXECUTABLE_CONTEXTS.size() + 1); + entries.addAll(ExecutableSection.DEFAULT_EXECUTABLE_CONTEXTS); + entries.add(new NamedXContentRegistry.Entry(ExecutableSection.class, + new ParseField("compare_analyzers"), CompareAnalyzers::parse)); + NamedXContentRegistry executeableSectionRegistry = new NamedXContentRegistry(entries); + return ESClientYamlSuiteTestCase.createParameters(executeableSectionRegistry); } @Override @@ -64,5 +90,117 @@ protected ClientYamlTestClient initClientYamlTestClient(ClientYamlSuiteRestSpec List hosts, Version esVersion) throws IOException { return new ClientYamlDocsTestClient(restSpec, restClient, hosts, esVersion); } -} + /** + * Compares the the results of running two analyzers against many random + * strings. The goal is to figure out if two anlayzers are "the same" by + * comparing their results. This is far from perfect but should be fairly + * accurate, especially for gross things like missing {@code decimal_digit} + * token filters, and should be fairly fast because it compares a fairly + * small number of tokens. + */ + private static class CompareAnalyzers implements ExecutableSection { + private static ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("test_analyzer", false, (a, location) -> { + String index = (String) a[0]; + String first = (String) a[1]; + String second = (String) a[2]; + return new CompareAnalyzers(location, index, first, second); + }); + static { + PARSER.declareString(constructorArg(), new ParseField("index")); + PARSER.declareString(constructorArg(), new ParseField("first")); + PARSER.declareString(constructorArg(), new ParseField("second")); + } + private static CompareAnalyzers parse(XContentParser parser) throws IOException { + XContentLocation location = parser.getTokenLocation(); + CompareAnalyzers section = PARSER.parse(parser, location); + assert parser.currentToken() == Token.END_OBJECT : "End of object required"; + parser.nextToken(); // throw out the END_OBJECT to conform with other ExecutableSections + return section; + } + + private final XContentLocation location; + private final String index; + private final String first; + private final String second; + + private CompareAnalyzers(XContentLocation location, String index, String first, String second) { + this.location = location; + this.index = index; + this.first = first; + this.second = second; + } + + @Override + public XContentLocation getLocation() { + return location; + } + + @Override + public void execute(ClientYamlTestExecutionContext executionContext) throws IOException { + int size = 100; + int maxLength = 15; + List testText = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + /** + * Build a string with a few unicode sequences separated by + * spaces. The unicode sequences aren't going to be of the same + * code page which is a shame because it makes the entire + * string less realistic. But this still provides a fairly + * nice string to compare. + */ + int spaces = between(0, 5); + StringBuilder b = new StringBuilder((spaces + 1) * maxLength); + b.append(randomRealisticUnicodeOfCodepointLengthBetween(1, maxLength)); + for (int t = 0; t < spaces; t++) { + b.append(' '); + b.append(randomRealisticUnicodeOfCodepointLengthBetween(1, maxLength)); + } + testText.add(b.toString() + // Don't look up stashed values + .replace("$", "\\$")); + } + Map body = new HashMap<>(2); + body.put("analyzer", first); + body.put("text", testText); + ClientYamlTestResponse response = executionContext.callApi("indices.analyze", singletonMap("index", index), + singletonList(body), emptyMap()); + Iterator firstTokens = ((List) response.evaluate("tokens")).iterator(); + body.put("analyzer", second); + response = executionContext.callApi("indices.analyze", singletonMap("index", index), + singletonList(body), emptyMap()); + Iterator secondTokens = ((List) response.evaluate("tokens")).iterator(); + + Object previousFirst = null; + Object previousSecond = null; + while (firstTokens.hasNext()) { + if (false == secondTokens.hasNext()) { + fail(second + " has fewer tokens than " + first + ". " + + first + " has [" + firstTokens.next() + "] but " + second + " is out of tokens. " + + first + "'s last token was [" + previousFirst + "] and " + + second + "'s last token was' [" + previousSecond + "]"); + } + Map firstToken = (Map) firstTokens.next(); + Map secondToken = (Map) secondTokens.next(); + String firstText = (String) firstToken.get("token"); + String secondText = (String) secondToken.get("token"); + // Check the text and produce an error message with the utf8 sequence if they don't match. + if (false == secondText.equals(firstText)) { + fail("text differs: " + first + " was [" + firstText + "] but " + second + " was [" + secondText + + "]. In utf8 those are\n" + new BytesRef(firstText) + " and\n" + new BytesRef(secondText)); + } + // Now check the whole map just in case the text matches but something else differs + assertEquals(firstToken, secondToken); + previousFirst = firstToken; + previousSecond = secondToken; + } + if (secondTokens.hasNext()) { + fail(second + " has more tokens than " + first + ". " + + second + " has [" + secondTokens.next() + "] but " + first + " is out of tokens. " + + first + "'s last token was [" + previousFirst + "] and " + + second + "'s last token was' [" + previousSecond + "]"); + } + } + } +} diff --git a/gradle.properties b/gradle.properties index 2511c740bb5b1..a8a309f10678f 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ org.gradle.daemon=false -org.gradle.jvmargs=-Xmx1536m +org.gradle.jvmargs=-Xmx1792m diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java index 8350d86615008..5bd83b6c19a2e 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java @@ -32,7 +32,6 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.Retry; -import org.elasticsearch.index.reindex.ScrollableHitSource.SearchFailure; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.ParentTaskAssigningClient; @@ -41,7 +40,6 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.IndexFieldMapper; @@ -49,6 +47,7 @@ import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.TypeFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; +import org.elasticsearch.index.reindex.ScrollableHitSource.SearchFailure; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService; @@ -75,8 +74,8 @@ import static java.util.Collections.emptyList; import static java.util.Collections.unmodifiableList; import static org.elasticsearch.action.bulk.BackoffPolicy.exponentialBackoff; -import static org.elasticsearch.index.reindex.AbstractBulkByScrollRequest.SIZE_ALL_MATCHES; import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; +import static org.elasticsearch.index.reindex.AbstractBulkByScrollRequest.SIZE_ALL_MATCHES; import static org.elasticsearch.rest.RestStatus.CONFLICT; import static org.elasticsearch.search.sort.SortBuilders.fieldSort; @@ -139,7 +138,7 @@ public AbstractAsyncBulkByScrollAction(BulkByScrollTask task, Logger logger, Par this.mainRequest = mainRequest; this.listener = listener; BackoffPolicy backoffPolicy = buildBackoffPolicy(); - bulkRetry = new Retry(EsRejectedExecutionException.class, BackoffPolicy.wrap(backoffPolicy, worker::countBulkRetry), threadPool); + bulkRetry = new Retry(BackoffPolicy.wrap(backoffPolicy, worker::countBulkRetry), threadPool); scrollSource = buildScrollableResultSource(backoffPolicy); scriptApplier = Objects.requireNonNull(buildScriptApplier(), "script applier must not be null"); /* diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java index 063a0ad31f38e..e8e3760882eea 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java @@ -19,14 +19,12 @@ package org.elasticsearch.index.reindex.remote; -import org.apache.http.HttpEntity; -import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.ContentType; -import org.apache.http.entity.StringEntity; -import org.apache.lucene.util.BytesRef; +import org.apache.http.nio.entity.NStringEntity; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.Request; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.unit.TimeValue; @@ -40,33 +38,27 @@ import org.elasticsearch.search.sort.SortBuilder; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import static java.util.Collections.singletonMap; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; /** * Builds requests for remote version of Elasticsearch. Note that unlike most of the * rest of Elasticsearch this file needs to be compatible with very old versions of - * Elasticsearch. Thus is often uses identifiers for versions like {@code 2000099} + * Elasticsearch. Thus it often uses identifiers for versions like {@code 2000099} * for {@code 2.0.0-alpha1}. Do not drop support for features from this file just * because the version constants have been removed. */ final class RemoteRequestBuilders { private RemoteRequestBuilders() {} - static String initialSearchPath(SearchRequest searchRequest) { + static Request initialSearch(SearchRequest searchRequest, BytesReference query, Version remoteVersion) { // It is nasty to build paths with StringBuilder but we'll be careful.... StringBuilder path = new StringBuilder("/"); addIndexesOrTypes(path, "Index", searchRequest.indices()); addIndexesOrTypes(path, "Type", searchRequest.types()); path.append("_search"); - return path.toString(); - } + Request request = new Request("POST", path.toString()); - static Map initialSearchParams(SearchRequest searchRequest, Version remoteVersion) { - Map params = new HashMap<>(); if (searchRequest.scroll() != null) { TimeValue keepAlive = searchRequest.scroll().keepAlive(); if (remoteVersion.before(Version.V_5_0_0)) { @@ -75,16 +67,16 @@ static Map initialSearchParams(SearchRequest searchRequest, Vers * timeout seems safer than less. */ keepAlive = timeValueMillis((long) Math.ceil(keepAlive.millisFrac())); } - params.put("scroll", keepAlive.getStringRep()); + request.addParameter("scroll", keepAlive.getStringRep()); } - params.put("size", Integer.toString(searchRequest.source().size())); + request.addParameter("size", Integer.toString(searchRequest.source().size())); if (searchRequest.source().version() == null || searchRequest.source().version() == true) { /* * Passing `null` here just add the `version` request parameter * without any value. This way of requesting the version works * for all supported versions of Elasticsearch. */ - params.put("version", null); + request.addParameter("version", null); } if (searchRequest.source().sorts() != null) { boolean useScan = false; @@ -101,13 +93,13 @@ static Map initialSearchParams(SearchRequest searchRequest, Vers } } if (useScan) { - params.put("search_type", "scan"); + request.addParameter("search_type", "scan"); } else { StringBuilder sorts = new StringBuilder(sortToUri(searchRequest.source().sorts().get(0))); for (int i = 1; i < searchRequest.source().sorts().size(); i++) { sorts.append(',').append(sortToUri(searchRequest.source().sorts().get(i))); } - params.put("sort", sorts.toString()); + request.addParameter("sort", sorts.toString()); } } if (remoteVersion.before(Version.fromId(2000099))) { @@ -126,12 +118,9 @@ static Map initialSearchParams(SearchRequest searchRequest, Vers fields.append(',').append(searchRequest.source().storedFields().fieldNames().get(i)); } String storedFieldsParamName = remoteVersion.before(Version.V_5_0_0_alpha4) ? "fields" : "stored_fields"; - params.put(storedFieldsParamName, fields.toString()); + request.addParameter(storedFieldsParamName, fields.toString()); } - return params; - } - static HttpEntity initialSearchEntity(SearchRequest searchRequest, BytesReference query, Version remoteVersion) { // EMPTY is safe here because we're not calling namedObject try (XContentBuilder entity = JsonXContent.contentBuilder(); XContentParser queryParser = XContentHelper @@ -139,7 +128,8 @@ static HttpEntity initialSearchEntity(SearchRequest searchRequest, BytesReferenc entity.startObject(); entity.field("query"); { - /* We're intentionally a bit paranoid here - copying the query as xcontent rather than writing a raw field. We don't want + /* We're intentionally a bit paranoid here - copying the query + * as xcontent rather than writing a raw field. We don't want * poorly written queries to escape. Ever. */ entity.copyCurrentStructure(queryParser); XContentParser.Token shouldBeEof = queryParser.nextToken(); @@ -159,11 +149,11 @@ static HttpEntity initialSearchEntity(SearchRequest searchRequest, BytesReferenc } entity.endObject(); - BytesRef bytes = BytesReference.bytes(entity).toBytesRef(); - return new ByteArrayEntity(bytes.bytes, bytes.offset, bytes.length, ContentType.APPLICATION_JSON); + request.setJsonEntity(Strings.toString(entity)); } catch (IOException e) { throw new ElasticsearchException("unexpected error building entity", e); } + return request; } private static void addIndexesOrTypes(StringBuilder path, String name, String[] indicesOrTypes) { @@ -193,45 +183,50 @@ private static String sortToUri(SortBuilder sort) { throw new IllegalArgumentException("Unsupported sort [" + sort + "]"); } - static String scrollPath() { - return "/_search/scroll"; - } + static Request scroll(String scroll, TimeValue keepAlive, Version remoteVersion) { + Request request = new Request("POST", "/_search/scroll"); - static Map scrollParams(TimeValue keepAlive, Version remoteVersion) { if (remoteVersion.before(Version.V_5_0_0)) { /* Versions of Elasticsearch before 5.0 couldn't parse nanos or micros * so we toss out that resolution, rounding up so we shouldn't end up * with 0s. */ keepAlive = timeValueMillis((long) Math.ceil(keepAlive.millisFrac())); } - return singletonMap("scroll", keepAlive.getStringRep()); - } + request.addParameter("scroll", keepAlive.getStringRep()); - static HttpEntity scrollEntity(String scroll, Version remoteVersion) { if (remoteVersion.before(Version.fromId(2000099))) { // Versions before 2.0.0 extract the plain scroll_id from the body - return new StringEntity(scroll, ContentType.TEXT_PLAIN); + request.setEntity(new NStringEntity(scroll, ContentType.TEXT_PLAIN)); + return request; } + try (XContentBuilder entity = JsonXContent.contentBuilder()) { - return new StringEntity(Strings.toString(entity.startObject() - .field("scroll_id", scroll) - .endObject()), ContentType.APPLICATION_JSON); + entity.startObject() + .field("scroll_id", scroll) + .endObject(); + request.setJsonEntity(Strings.toString(entity)); } catch (IOException e) { throw new ElasticsearchException("failed to build scroll entity", e); } + return request; } - static HttpEntity clearScrollEntity(String scroll, Version remoteVersion) { + static Request clearScroll(String scroll, Version remoteVersion) { + Request request = new Request("DELETE", "/_search/scroll"); + if (remoteVersion.before(Version.fromId(2000099))) { // Versions before 2.0.0 extract the plain scroll_id from the body - return new StringEntity(scroll, ContentType.TEXT_PLAIN); + request.setEntity(new NStringEntity(scroll, ContentType.TEXT_PLAIN)); + return request; } try (XContentBuilder entity = JsonXContent.contentBuilder()) { - return new StringEntity(Strings.toString(entity.startObject() - .array("scroll_id", scroll) - .endObject()), ContentType.APPLICATION_JSON); + entity.startObject() + .array("scroll_id", scroll) + .endObject(); + request.setJsonEntity(Strings.toString(entity)); } catch (IOException e) { throw new ElasticsearchException("failed to build clear scroll entity", e); } + return request; } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java index 566c97c61c455..9264cdde30c75 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteScrollableHitSource.java @@ -30,22 +30,22 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.Version; import org.elasticsearch.action.bulk.BackoffPolicy; -import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; -import org.elasticsearch.common.xcontent.XContentParseException; import org.elasticsearch.index.reindex.ScrollableHitSource; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.Request; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.ResponseListener; import org.elasticsearch.client.RestClient; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentParseException; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; @@ -53,20 +53,11 @@ import java.io.IOException; import java.io.InputStream; import java.util.Iterator; -import java.util.Map; import java.util.function.BiFunction; import java.util.function.Consumer; -import static java.util.Collections.emptyMap; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.clearScrollEntity; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearchEntity; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearchParams; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearchPath; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.scrollEntity; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.scrollParams; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.scrollPath; import static org.elasticsearch.index.reindex.remote.RemoteResponseParsers.MAIN_ACTION_PARSER; import static org.elasticsearch.index.reindex.remote.RemoteResponseParsers.RESPONSE_PARSER; @@ -88,13 +79,13 @@ public RemoteScrollableHitSource(Logger logger, BackoffPolicy backoffPolicy, Thr protected void doStart(Consumer onResponse) { lookupRemoteVersion(version -> { remoteVersion = version; - execute("POST", initialSearchPath(searchRequest), initialSearchParams(searchRequest, version), - initialSearchEntity(searchRequest, query, remoteVersion), RESPONSE_PARSER, r -> onStartResponse(onResponse, r)); + execute(RemoteRequestBuilders.initialSearch(searchRequest, query, remoteVersion), + RESPONSE_PARSER, r -> onStartResponse(onResponse, r)); }); } void lookupRemoteVersion(Consumer onVersion) { - execute("GET", "", emptyMap(), null, MAIN_ACTION_PARSER, onVersion); + execute(new Request("GET", ""), MAIN_ACTION_PARSER, onVersion); } private void onStartResponse(Consumer onResponse, Response response) { @@ -108,15 +99,13 @@ private void onStartResponse(Consumer onResponse, Response res @Override protected void doStartNextScroll(String scrollId, TimeValue extraKeepAlive, Consumer onResponse) { - Map scrollParams = scrollParams( - timeValueNanos(searchRequest.scroll().keepAlive().nanos() + extraKeepAlive.nanos()), - remoteVersion); - execute("POST", scrollPath(), scrollParams, scrollEntity(scrollId, remoteVersion), RESPONSE_PARSER, onResponse); + TimeValue keepAlive = timeValueNanos(searchRequest.scroll().keepAlive().nanos() + extraKeepAlive.nanos()); + execute(RemoteRequestBuilders.scroll(scrollId, keepAlive, remoteVersion), RESPONSE_PARSER, onResponse); } @Override protected void clearScroll(String scrollId, Runnable onCompletion) { - client.performRequestAsync("DELETE", scrollPath(), emptyMap(), clearScrollEntity(scrollId, remoteVersion), new ResponseListener() { + client.performRequestAsync(RemoteRequestBuilders.clearScroll(scrollId, remoteVersion), new ResponseListener() { @Override public void onSuccess(org.elasticsearch.client.Response response) { logger.debug("Successfully cleared [{}]", scrollId); @@ -162,7 +151,7 @@ protected void cleanup(Runnable onCompletion) { }); } - private void execute(String method, String uri, Map params, HttpEntity entity, + private void execute(Request request, BiFunction parser, Consumer listener) { // Preserve the thread context so headers survive after the call java.util.function.Supplier contextSupplier = threadPool.getThreadContext().newRestorableContext(true); @@ -171,7 +160,7 @@ class RetryHelper extends AbstractRunnable { @Override protected void doRun() throws Exception { - client.performRequestAsync(method, uri, params, entity, new ResponseListener() { + client.performRequestAsync(request, new ResponseListener() { @Override public void onSuccess(org.elasticsearch.client.Response response) { // Restore the thread context to get the precious headers diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java index 298b0604cb0f4..aea720aeb21e2 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java @@ -186,7 +186,7 @@ private void testCase( bulk.add(client().prepareIndex("source", "test").setSource("foo", "bar " + i)); } - Retry retry = new Retry(EsRejectedExecutionException.class, BackoffPolicy.exponentialBackoff(), client().threadPool()); + Retry retry = new Retry(BackoffPolicy.exponentialBackoff(), client().threadPool()); BulkResponse initialBulkResponse = retry.withBackoff(client()::bulk, bulk.request(), client().settings()).actionGet(); assertFalse(initialBulkResponse.buildFailureMessage(), initialBulkResponse.hasFailures()); client().admin().indices().prepareRefresh("source").get(); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java index 9cb644162da40..b51525f20e3c2 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java @@ -23,7 +23,9 @@ import org.apache.http.entity.ContentType; import org.elasticsearch.Version; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.Request; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -35,14 +37,12 @@ import java.util.Map; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.clearScrollEntity; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearchEntity; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearchParams; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearchPath; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.scrollEntity; -import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.scrollParams; +import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.clearScroll; +import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.initialSearch; +import static org.elasticsearch.index.reindex.remote.RemoteRequestBuilders.scroll; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.either; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.hasKey; @@ -57,15 +57,17 @@ */ public class RemoteRequestBuildersTests extends ESTestCase { public void testIntialSearchPath() { - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); + Version remoteVersion = Version.fromId(between(0, Version.CURRENT.id)); + BytesReference query = new BytesArray("{}"); - assertEquals("/_search", initialSearchPath(searchRequest)); + SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); + assertEquals("/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("a"); searchRequest.types("b"); - assertEquals("/a/b/_search", initialSearchPath(searchRequest)); + assertEquals("/a/b/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("a", "b"); searchRequest.types("c", "d"); - assertEquals("/a,b/c,d/_search", initialSearchPath(searchRequest)); + assertEquals("/a,b/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("cat,"); expectBadStartRequest(searchRequest, "Index", ",", "cat,"); @@ -96,63 +98,70 @@ public void testIntialSearchPath() { } private void expectBadStartRequest(SearchRequest searchRequest, String type, String bad, String failed) { - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> initialSearchPath(searchRequest)); + Version remoteVersion = Version.fromId(between(0, Version.CURRENT.id)); + BytesReference query = new BytesArray("{}"); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> initialSearch(searchRequest, query, remoteVersion)); assertEquals(type + " containing [" + bad + "] not supported but got [" + failed + "]", e.getMessage()); } public void testInitialSearchParamsSort() { + BytesReference query = new BytesArray("{}"); SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); // Test sort:_doc for versions that support it. Version remoteVersion = Version.fromId(between(2010099, Version.CURRENT.id)); searchRequest.source().sort("_doc"); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("sort", "_doc:asc")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), hasEntry("sort", "_doc:asc")); // Test search_type scan for versions that don't support sort:_doc. remoteVersion = Version.fromId(between(0, 2010099 - 1)); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("search_type", "scan")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), hasEntry("search_type", "scan")); // Test sorting by some field. Version doesn't matter. remoteVersion = Version.fromId(between(0, Version.CURRENT.id)); searchRequest.source().sorts().clear(); searchRequest.source().sort("foo"); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("sort", "foo:asc")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), hasEntry("sort", "foo:asc")); } public void testInitialSearchParamsFields() { + BytesReference query = new BytesArray("{}"); SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); // Test request without any fields Version remoteVersion = Version.fromId(between(2000099, Version.CURRENT.id)); - assertThat(initialSearchParams(searchRequest, remoteVersion), + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), not(either(hasKey("stored_fields")).or(hasKey("fields")))); // Test stored_fields for versions that support it searchRequest = new SearchRequest().source(new SearchSourceBuilder()); searchRequest.source().storedField("_source").storedField("_id"); remoteVersion = Version.fromId(between(Version.V_5_0_0_alpha4_ID, Version.CURRENT.id)); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("stored_fields", "_source,_id")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), hasEntry("stored_fields", "_source,_id")); // Test fields for versions that support it searchRequest = new SearchRequest().source(new SearchSourceBuilder()); searchRequest.source().storedField("_source").storedField("_id"); remoteVersion = Version.fromId(between(2000099, Version.V_5_0_0_alpha4_ID - 1)); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("fields", "_source,_id")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), hasEntry("fields", "_source,_id")); // Test extra fields for versions that need it searchRequest = new SearchRequest().source(new SearchSourceBuilder()); searchRequest.source().storedField("_source").storedField("_id"); remoteVersion = Version.fromId(between(0, 2000099 - 1)); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("fields", "_source,_id,_parent,_routing,_ttl")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), + hasEntry("fields", "_source,_id,_parent,_routing,_ttl")); // But only versions before 1.0 force _source to be in the list searchRequest = new SearchRequest().source(new SearchSourceBuilder()); searchRequest.source().storedField("_id"); remoteVersion = Version.fromId(between(1000099, 2000099 - 1)); - assertThat(initialSearchParams(searchRequest, remoteVersion), hasEntry("fields", "_id,_parent,_routing,_ttl")); + assertThat(initialSearch(searchRequest, query, remoteVersion).getParameters(), + hasEntry("fields", "_id,_parent,_routing,_ttl")); } public void testInitialSearchParamsMisc() { + BytesReference query = new BytesArray("{}"); SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); Version remoteVersion = Version.fromId(between(0, Version.CURRENT.id)); @@ -169,7 +178,7 @@ public void testInitialSearchParamsMisc() { searchRequest.source().version(fetchVersion); } - Map params = initialSearchParams(searchRequest, remoteVersion); + Map params = initialSearch(searchRequest, query, remoteVersion).getParameters(); if (scroll == null) { assertThat(params, not(hasKey("scroll"))); @@ -199,7 +208,7 @@ public void testInitialSearchEntity() throws IOException { SearchRequest searchRequest = new SearchRequest(); searchRequest.source(new SearchSourceBuilder()); String query = "{\"match_all\":{}}"; - HttpEntity entity = initialSearchEntity(searchRequest, new BytesArray(query), remoteVersion); + HttpEntity entity = initialSearch(searchRequest, new BytesArray(query), remoteVersion).getEntity(); assertEquals(ContentType.APPLICATION_JSON.toString(), entity.getContentType().getValue()); if (remoteVersion.onOrAfter(Version.fromId(1000099))) { assertEquals("{\"query\":" + query + ",\"_source\":true}", @@ -211,48 +220,51 @@ public void testInitialSearchEntity() throws IOException { // Source filtering is included if set up searchRequest.source().fetchSource(new String[] {"in1", "in2"}, new String[] {"out"}); - entity = initialSearchEntity(searchRequest, new BytesArray(query), remoteVersion); + entity = initialSearch(searchRequest, new BytesArray(query), remoteVersion).getEntity(); assertEquals(ContentType.APPLICATION_JSON.toString(), entity.getContentType().getValue()); assertEquals("{\"query\":" + query + ",\"_source\":{\"includes\":[\"in1\",\"in2\"],\"excludes\":[\"out\"]}}", Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); // Invalid XContent fails RuntimeException e = expectThrows(RuntimeException.class, - () -> initialSearchEntity(searchRequest, new BytesArray("{}, \"trailing\": {}"), remoteVersion)); + () -> initialSearch(searchRequest, new BytesArray("{}, \"trailing\": {}"), remoteVersion)); assertThat(e.getCause().getMessage(), containsString("Unexpected character (',' (code 44))")); - e = expectThrows(RuntimeException.class, () -> initialSearchEntity(searchRequest, new BytesArray("{"), remoteVersion)); + e = expectThrows(RuntimeException.class, () -> initialSearch(searchRequest, new BytesArray("{"), remoteVersion)); assertThat(e.getCause().getMessage(), containsString("Unexpected end-of-input")); } public void testScrollParams() { + String scroll = randomAlphaOfLength(30); Version remoteVersion = Version.fromId(between(0, Version.CURRENT.id)); - TimeValue scroll = TimeValue.parseTimeValue(randomPositiveTimeValue(), "test"); - assertScroll(remoteVersion, scrollParams(scroll, remoteVersion), scroll); + TimeValue keepAlive = TimeValue.parseTimeValue(randomPositiveTimeValue(), "test"); + assertScroll(remoteVersion, scroll(scroll, keepAlive, remoteVersion).getParameters(), keepAlive); } public void testScrollEntity() throws IOException { String scroll = randomAlphaOfLength(30); - HttpEntity entity = scrollEntity(scroll, Version.V_5_0_0); + HttpEntity entity = scroll(scroll, timeValueMillis(between(1, 1000)), Version.V_5_0_0).getEntity(); assertEquals(ContentType.APPLICATION_JSON.toString(), entity.getContentType().getValue()); assertThat(Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8)), containsString("\"" + scroll + "\"")); // Test with version < 2.0.0 - entity = scrollEntity(scroll, Version.fromId(1070499)); + entity = scroll(scroll, timeValueMillis(between(1, 1000)), Version.fromId(1070499)).getEntity(); assertEquals(ContentType.TEXT_PLAIN.toString(), entity.getContentType().getValue()); assertEquals(scroll, Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); } - public void testClearScrollEntity() throws IOException { + public void testClearScroll() throws IOException { String scroll = randomAlphaOfLength(30); - HttpEntity entity = clearScrollEntity(scroll, Version.V_5_0_0); - assertEquals(ContentType.APPLICATION_JSON.toString(), entity.getContentType().getValue()); - assertThat(Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8)), + Request request = clearScroll(scroll, Version.V_5_0_0); + assertEquals(ContentType.APPLICATION_JSON.toString(), request.getEntity().getContentType().getValue()); + assertThat(Streams.copyToString(new InputStreamReader(request.getEntity().getContent(), StandardCharsets.UTF_8)), containsString("\"" + scroll + "\"")); + assertThat(request.getParameters().keySet(), empty()); // Test with version < 2.0.0 - entity = clearScrollEntity(scroll, Version.fromId(1070499)); - assertEquals(ContentType.TEXT_PLAIN.toString(), entity.getContentType().getValue()); - assertEquals(scroll, Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8))); + request = clearScroll(scroll, Version.fromId(1070499)); + assertEquals(ContentType.TEXT_PLAIN.toString(), request.getEntity().getContentType().getValue()); + assertEquals(scroll, Streams.copyToString(new InputStreamReader(request.getEntity().getContent(), StandardCharsets.UTF_8))); + assertThat(request.getParameters().keySet(), empty()); } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_settings/11_reset.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_settings/11_reset.yml index bc2dace0e1871..d7bd87cc73a82 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_settings/11_reset.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_settings/11_reset.yml @@ -23,10 +23,15 @@ Test reset index settings: indices.get_settings: flat_settings: false - is_false: test-index.settings.index\.refresh_interval - - do: - indices.get_settings: - include_defaults: true - flat_settings: true - index: test-index - - match: - test-index.defaults.index\.refresh_interval: "1s" + +# Disabled until https://github.com/elastic/elasticsearch/pull/29229 is back-ported +# That PR changed the execution path of index settings default to be on the master +# until the PR is back-ported the old master will not return default settings. +# +# - do: +# indices.get_settings: +# include_defaults: true +# flat_settings: true +# index: test-index +# - match: +# test-index.defaults.index\.refresh_interval: "1s" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml index ee40cc36347b9..9e32f98831dde 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml @@ -103,9 +103,13 @@ setup: --- "Split from 1 to N": +# - skip: +# version: " - 6.99.99" +# reason: Added in 7.0.0 +# uncomment once AwaitsFix is resolved - skip: - version: " - 6.99.99" - reason: Added in 7.0.0 + version: "all" + reason: "AwaitsFix'ing, see https://github.com/elastic/elasticsearch/issues/30503" - do: indices.create: index: source_one_shard diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml index 38f36c405a1cb..69b505097f2ec 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml @@ -1,8 +1,12 @@ --- "Split index ignores target template mapping": +# - skip: +# version: " - 6.0.99" +# reason: Added in 6.1.0 +# uncomment once AwaitsFix is resolved - skip: - version: " - 6.0.99" - reason: Added in 6.1.0 + version: "all" + reason: "AwaitsFix'ing, see https://github.com/elastic/elasticsearch/issues/30503" # create index - do: diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java index a71f9fd8fd7cc..c1d1420eef8c8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/GetRepositoriesResponse.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.admin.cluster.repositories.get; +import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.cluster.metadata.RepositoriesMetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; @@ -26,12 +27,15 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; + /** * Get repositories response */ @@ -74,4 +78,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); return builder; } + + public static GetRepositoriesResponse fromXContent(XContentParser parser) throws IOException { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); + return new GetRepositoriesResponse(RepositoriesMetaData.fromXContent(parser)); + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java index 9f401b6312c46..ebc085ef47e60 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsRequest.java @@ -54,6 +54,21 @@ public IndicesStatsRequest clear() { return this; } + /** + * Returns the underlying stats flags. + */ + public CommonStatsFlags flags() { + return flags; + } + + /** + * Sets the underlying stats flags. + */ + public IndicesStatsRequest flags(CommonStatsFlags flags) { + this.flags = flags; + return this; + } + /** * Document types to return stats for. Mainly affects {@link #indexing(boolean)} when * enabled, returning specific indexing stats for those types. diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java index eeefe793db701..6f2aaa063011f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java @@ -99,65 +99,8 @@ protected ShardStats shardOperation(IndicesStatsRequest request, ShardRouting sh throw new ShardNotFoundException(indexShard.shardId()); } - CommonStatsFlags flags = new CommonStatsFlags().clear(); - - if (request.docs()) { - flags.set(CommonStatsFlags.Flag.Docs); - } - if (request.store()) { - flags.set(CommonStatsFlags.Flag.Store); - } - if (request.indexing()) { - flags.set(CommonStatsFlags.Flag.Indexing); - flags.types(request.types()); - } - if (request.get()) { - flags.set(CommonStatsFlags.Flag.Get); - } - if (request.search()) { - flags.set(CommonStatsFlags.Flag.Search); - flags.groups(request.groups()); - } - if (request.merge()) { - flags.set(CommonStatsFlags.Flag.Merge); - } - if (request.refresh()) { - flags.set(CommonStatsFlags.Flag.Refresh); - } - if (request.flush()) { - flags.set(CommonStatsFlags.Flag.Flush); - } - if (request.warmer()) { - flags.set(CommonStatsFlags.Flag.Warmer); - } - if (request.queryCache()) { - flags.set(CommonStatsFlags.Flag.QueryCache); - } - if (request.fieldData()) { - flags.set(CommonStatsFlags.Flag.FieldData); - flags.fieldDataFields(request.fieldDataFields()); - } - if (request.segments()) { - flags.set(CommonStatsFlags.Flag.Segments); - flags.includeSegmentFileSizes(request.includeSegmentFileSizes()); - } - if (request.completion()) { - flags.set(CommonStatsFlags.Flag.Completion); - flags.completionDataFields(request.completionFields()); - } - if (request.translog()) { - flags.set(CommonStatsFlags.Flag.Translog); - } - if (request.requestCache()) { - flags.set(CommonStatsFlags.Flag.RequestCache); - } - if (request.recovery()) { - flags.set(CommonStatsFlags.Flag.Recovery); - } - - return new ShardStats( - indexShard.routingEntry(), - indexShard.shardPath(), - new CommonStats(indicesService.getIndicesQueryCache(), indexShard, flags), indexShard.commitStats(), indexShard.seqNoStats()); + CommonStats commonStats = new CommonStats(indicesService.getIndicesQueryCache(), indexShard, request.flags()); + return new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), commonStats, + indexShard.commitStats(), indexShard.seqNoStats()); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index 8cd1fac6f6fd1..b018e24a565b8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; @@ -58,14 +59,14 @@ import java.util.stream.Collectors; import static org.elasticsearch.action.ValidateActions.addValidationError; +import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS; import static org.elasticsearch.common.settings.Settings.readSettingsFromStream; import static org.elasticsearch.common.settings.Settings.writeSettingsToStream; -import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS; /** * A request to create an index template. */ -public class PutIndexTemplateRequest extends MasterNodeRequest implements IndicesRequest { +public class PutIndexTemplateRequest extends MasterNodeRequest implements IndicesRequest, ToXContent { private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(PutIndexTemplateRequest.class)); @@ -539,4 +540,34 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeOptionalVInt(version); } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (customs.isEmpty() == false) { + throw new IllegalArgumentException("Custom data type is no longer supported in index template [" + customs + "]"); + } + builder.field("index_patterns", indexPatterns); + builder.field("order", order); + if (version != null) { + builder.field("version", version); + } + + builder.startObject("settings"); + settings.toXContent(builder, params); + builder.endObject(); + + builder.startObject("mappings"); + for (Map.Entry entry : mappings.entrySet()) { + Map mapping = XContentHelper.convertToMap(new BytesArray(entry.getValue()), false).v2(); + builder.field(entry.getKey(), mapping); + } + builder.endObject(); + + builder.startObject("aliases"); + for (Alias alias : aliases) { + alias.toXContent(builder, params); + } + builder.endObject(); + return builder; + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateResponse.java index bf6e05a6c7b43..6c8a5291b12d5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateResponse.java @@ -21,6 +21,8 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; @@ -47,4 +49,14 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); writeAcknowledged(out); } + + private static final ConstructingObjectParser PARSER; + static { + PARSER = new ConstructingObjectParser<>("put_index_template", true, args -> new PutIndexTemplateResponse((boolean) args[0])); + declareAcknowledgedField(PARSER); + } + + public static PutIndexTemplateResponse fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java index 5be321734b5db..c90a7428268cd 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java @@ -184,7 +184,7 @@ protected ValidateQueryResponse newResponse(ValidateQueryRequest request, Atomic } @Override - protected ShardValidateQueryResponse shardOperation(ShardValidateQueryRequest request) throws IOException { + protected ShardValidateQueryResponse shardOperation(ShardValidateQueryRequest request, Task task) throws IOException { boolean valid; String explanation = null; String error = null; diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestHandler.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestHandler.java index adb1d32161fe1..d02173ca370bd 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestHandler.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestHandler.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.threadpool.Scheduler; import java.util.concurrent.CountDownLatch; @@ -49,7 +48,7 @@ public final class BulkRequestHandler { this.consumer = consumer; this.listener = listener; this.concurrentRequests = concurrentRequests; - this.retry = new Retry(EsRejectedExecutionException.class, backoffPolicy, scheduler); + this.retry = new Retry(backoffPolicy, scheduler); this.semaphore = new Semaphore(concurrentRequests > 0 ? concurrentRequests : 1); } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/Retry.java b/server/src/main/java/org/elasticsearch/action/bulk/Retry.java index 9985d23b9badb..75a1a2d5f8daa 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/Retry.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/Retry.java @@ -19,13 +19,13 @@ package org.elasticsearch.action.bulk; import org.apache.logging.log4j.Logger; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.FutureUtils; +import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool; @@ -40,12 +40,10 @@ * Encapsulates synchronous and asynchronous retry logic. */ public class Retry { - private final Class retryOnThrowable; private final BackoffPolicy backoffPolicy; private final Scheduler scheduler; - public Retry(Class retryOnThrowable, BackoffPolicy backoffPolicy, Scheduler scheduler) { - this.retryOnThrowable = retryOnThrowable; + public Retry(BackoffPolicy backoffPolicy, Scheduler scheduler) { this.backoffPolicy = backoffPolicy; this.scheduler = scheduler; } @@ -60,7 +58,7 @@ public Retry(Class retryOnThrowable, BackoffPolicy backoffP */ public void withBackoff(BiConsumer> consumer, BulkRequest bulkRequest, ActionListener listener, Settings settings) { - RetryHandler r = new RetryHandler(retryOnThrowable, backoffPolicy, consumer, listener, settings, scheduler); + RetryHandler r = new RetryHandler(backoffPolicy, consumer, listener, settings, scheduler); r.execute(bulkRequest); } @@ -81,12 +79,13 @@ public PlainActionFuture withBackoff(BiConsumer { + private static final RestStatus RETRY_STATUS = RestStatus.TOO_MANY_REQUESTS; + private final Logger logger; private final Scheduler scheduler; private final BiConsumer> consumer; private final ActionListener listener; private final Iterator backoff; - private final Class retryOnThrowable; // Access only when holding a client-side lock, see also #addResponses() private final List responses = new ArrayList<>(); private final long startTimestampNanos; @@ -95,10 +94,8 @@ static class RetryHandler implements ActionListener { private volatile BulkRequest currentBulkRequest; private volatile ScheduledFuture scheduledRequestFuture; - RetryHandler(Class retryOnThrowable, BackoffPolicy backoffPolicy, - BiConsumer> consumer, ActionListener listener, - Settings settings, Scheduler scheduler) { - this.retryOnThrowable = retryOnThrowable; + RetryHandler(BackoffPolicy backoffPolicy, BiConsumer> consumer, + ActionListener listener, Settings settings, Scheduler scheduler) { this.backoff = backoffPolicy.iterator(); this.consumer = consumer; this.listener = listener; @@ -160,9 +157,8 @@ private boolean canRetry(BulkResponse bulkItemResponses) { } for (BulkItemResponse bulkItemResponse : bulkItemResponses) { if (bulkItemResponse.isFailed()) { - final Throwable cause = bulkItemResponse.getFailure().getCause(); - final Throwable rootCause = ExceptionsHelper.unwrapCause(cause); - if (!rootCause.getClass().equals(retryOnThrowable)) { + final RestStatus status = bulkItemResponse.status(); + if (status != RETRY_STATUS) { return false; } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 57ecb250f7caf..7201b878f9800 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -543,6 +543,14 @@ private static Engine.Result performOpOnReplica(DocWriteResponse primaryResponse + docWriteRequest.opType().getLowercase()); } if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) { + // Even though the primary waits on all nodes to ack the mapping changes to the master + // (see MappingUpdatedAction.updateMappingOnMaster) we still need to protect against missing mappings + // and wait for them. The reason is concurrent requests. Request r1 which has new field f triggers a + // mapping update. Assume that that update is first applied on the primary, and only later on the replica + // (it’s happening concurrently). Request r2, which now arrives on the primary and which also has the new + // field f might see the updated mapping (on the primary), and will therefore proceed to be replicated + // to the replica. When it arrives on the replica, there’s no guarantee that the replica has already + // applied the new mapping, so there is no other option than to wait. throw new TransportReplicationAction.RetryOnReplicaException(replica.shardId(), "Mappings are not available on the replica yet, triggered update: " + result.getRequiredMappingUpdate()); } diff --git a/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java b/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java index 0961ab74c4703..60eaa19eaff63 100644 --- a/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java @@ -84,11 +84,7 @@ protected final void doExecute(Request request, ActionListener listene protected abstract ShardResponse newShardResponse(); - protected abstract ShardResponse shardOperation(ShardRequest request) throws IOException; - - protected ShardResponse shardOperation(ShardRequest request, Task task) throws IOException { - return shardOperation(request); - } + protected abstract ShardResponse shardOperation(ShardRequest request, Task task) throws IOException; /** * Determines the shards this operation will be executed on. The operation is executed once per shard iterator, typically @@ -284,7 +280,7 @@ class ShardTransportHandler implements TransportRequestHandler { @Override public void messageReceived(ShardRequest request, TransportChannel channel, Task task) throws Exception { - channel.sendResponse(shardOperation(request)); + channel.sendResponse(shardOperation(request, task)); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsWriter.java b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsWriter.java index 06eea6367edca..8a54406c1f9cb 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsWriter.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TermVectorsWriter.java @@ -70,6 +70,10 @@ void setFields(Fields termVectorsByField, Set selectedFields, EnumSet SETTING = new Setting<>(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, "false", (value) -> { + + private static final AutoExpandReplicas FALSE_INSTANCE = new AutoExpandReplicas(0, 0, false); + + public static final Setting SETTING = new Setting<>(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, "false", + AutoExpandReplicas::parse, Property.Dynamic, Property.IndexScope); + + private static AutoExpandReplicas parse(String value) { final int min; final int max; if (Booleans.isFalse(value)) { - return new AutoExpandReplicas(0, 0, false); + return FALSE_INSTANCE; } final int dash = value.indexOf('-'); if (-1 == dash) { @@ -57,7 +70,7 @@ final class AutoExpandReplicas { } } return new AutoExpandReplicas(min, max, true); - }, Property.Dynamic, Property.IndexScope); + } private final int minReplicas; private final int maxReplicas; @@ -80,6 +93,24 @@ int getMaxReplicas(int numDataNodes) { return Math.min(maxReplicas, numDataNodes-1); } + Optional getDesiredNumberOfReplicas(int numDataNodes) { + if (enabled) { + final int min = getMinReplicas(); + final int max = getMaxReplicas(numDataNodes); + int numberOfReplicas = numDataNodes - 1; + if (numberOfReplicas < min) { + numberOfReplicas = min; + } else if (numberOfReplicas > max) { + numberOfReplicas = max; + } + + if (numberOfReplicas >= min && numberOfReplicas <= max) { + return Optional.of(numberOfReplicas); + } + } + return Optional.empty(); + } + @Override public String toString() { return enabled ? minReplicas + "-" + maxReplicas : "false"; @@ -88,6 +119,31 @@ public String toString() { boolean isEnabled() { return enabled; } + + /** + * Checks if the are replicas with the auto-expand feature that need to be adapted. + * Returns a map of updates, which maps the indices to be updated to the desired number of replicas. + * The map has the desired number of replicas as key and the indices to update as value, as this allows the result + * of this method to be directly applied to RoutingTable.Builder#updateNumberOfReplicas. + */ + public static Map> getAutoExpandReplicaChanges(MetaData metaData, DiscoveryNodes discoveryNodes) { + // used for translating "all" to a number + final int dataNodeCount = discoveryNodes.getDataNodes().size(); + + Map> nrReplicasChanged = new HashMap<>(); + + for (final IndexMetaData indexMetaData : metaData) { + if (indexMetaData.getState() != IndexMetaData.State.CLOSE) { + AutoExpandReplicas autoExpandReplicas = SETTING.get(indexMetaData.getSettings()); + autoExpandReplicas.getDesiredNumberOfReplicas(dataNodeCount).ifPresent(numberOfReplicas -> { + if (numberOfReplicas != indexMetaData.getNumberOfReplicas()) { + nrReplicasChanged.computeIfAbsent(numberOfReplicas, ArrayList::new).add(indexMetaData.getIndex().getName()); + } + }); + } + } + return nrReplicasChanged; + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java index 59c38be50e86f..ce5ad12a53d6a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java @@ -25,9 +25,7 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeSettingsClusterStateUpdateRequest; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; -import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -42,16 +40,12 @@ import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; @@ -61,7 +55,7 @@ /** * Service responsible for submitting update index settings requests */ -public class MetaDataUpdateSettingsService extends AbstractComponent implements ClusterStateListener { +public class MetaDataUpdateSettingsService extends AbstractComponent { private final ClusterService clusterService; @@ -77,87 +71,11 @@ public MetaDataUpdateSettingsService(Settings settings, ClusterService clusterSe super(settings); this.clusterService = clusterService; this.threadPool = threadPool; - this.clusterService.addListener(this); this.allocationService = allocationService; this.indexScopedSettings = indexScopedSettings; this.indicesService = indicesService; } - @Override - public void clusterChanged(ClusterChangedEvent event) { - // update an index with number of replicas based on data nodes if possible - if (!event.state().nodes().isLocalNodeElectedMaster()) { - return; - } - // we will want to know this for translating "all" to a number - final int dataNodeCount = event.state().nodes().getDataNodes().size(); - - Map> nrReplicasChanged = new HashMap<>(); - // we need to do this each time in case it was changed by update settings - for (final IndexMetaData indexMetaData : event.state().metaData()) { - AutoExpandReplicas autoExpandReplicas = IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING.get(indexMetaData.getSettings()); - if (autoExpandReplicas.isEnabled()) { - /* - * we have to expand the number of replicas for this index to at least min and at most max nodes here - * so we are bumping it up if we have to or reduce it depending on min/max and the number of datanodes. - * If we change the number of replicas we just let the shard allocator do it's thing once we updated it - * since it goes through the index metadata to figure out if something needs to be done anyway. Do do that - * we issue a cluster settings update command below and kicks off a reroute. - */ - final int min = autoExpandReplicas.getMinReplicas(); - final int max = autoExpandReplicas.getMaxReplicas(dataNodeCount); - int numberOfReplicas = dataNodeCount - 1; - if (numberOfReplicas < min) { - numberOfReplicas = min; - } else if (numberOfReplicas > max) { - numberOfReplicas = max; - } - // same value, nothing to do there - if (numberOfReplicas == indexMetaData.getNumberOfReplicas()) { - continue; - } - - if (numberOfReplicas >= min && numberOfReplicas <= max) { - - if (!nrReplicasChanged.containsKey(numberOfReplicas)) { - nrReplicasChanged.put(numberOfReplicas, new ArrayList<>()); - } - - nrReplicasChanged.get(numberOfReplicas).add(indexMetaData.getIndex()); - } - } - } - - if (nrReplicasChanged.size() > 0) { - // update settings and kick of a reroute (implicit) for them to take effect - for (final Integer fNumberOfReplicas : nrReplicasChanged.keySet()) { - Settings settings = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, fNumberOfReplicas).build(); - final List indices = nrReplicasChanged.get(fNumberOfReplicas); - - UpdateSettingsClusterStateUpdateRequest updateRequest = new UpdateSettingsClusterStateUpdateRequest() - .indices(indices.toArray(new Index[indices.size()])).settings(settings) - .ackTimeout(TimeValue.timeValueMillis(0)) //no need to wait for ack here - .masterNodeTimeout(TimeValue.timeValueMinutes(10)); - - updateSettings(updateRequest, new ActionListener() { - @Override - public void onResponse(ClusterStateUpdateResponse response) { - for (Index index : indices) { - logger.info("{} auto expanded replicas to [{}]", index, fNumberOfReplicas); - } - } - - @Override - public void onFailure(Exception t) { - for (Index index : indices) { - logger.warn("{} fail to auto expand replicas to [{}]", index, fNumberOfReplicas); - } - } - }); - } - } - } - public void updateSettings(final UpdateSettingsClusterStateUpdateRequest request, final ActionListener listener) { final Settings normalizedSettings = Settings.builder().put(request.settings()).normalizePrefix(IndexMetaData.INDEX_SETTING_PREFIX).build(); Settings.Builder settingsForClosedIndices = Settings.builder(); diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index f6f18540825cc..afab4af320416 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -44,6 +44,8 @@ */ public class DiscoveryNode implements Writeable, ToXContentFragment { + static final String COORDINATING_ONLY = "coordinating_only"; + public static boolean nodeRequiresLocalStorage(Settings settings) { boolean localStorageEnable = Node.NODE_LOCAL_STORAGE_SETTING.get(settings); if (localStorageEnable == false && diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index c657af38703a5..2e0e63186917b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -148,6 +148,19 @@ public ImmutableOpenMap getMasterAndDataNodes() { return nodes.build(); } + /** + * Get a {@link Map} of the coordinating only nodes (nodes which are neither master, nor data, nor ingest nodes) arranged by their ids + * + * @return {@link Map} of the coordinating only nodes arranged by their ids + */ + public ImmutableOpenMap getCoordinatingOnlyNodes() { + ImmutableOpenMap.Builder nodes = ImmutableOpenMap.builder(this.nodes); + nodes.removeAll(masterNodes.keys()); + nodes.removeAll(dataNodes.keys()); + nodes.removeAll(ingestNodes.keys()); + return nodes.build(); + } + /** * Get a node by its id * @@ -297,7 +310,7 @@ public DiscoveryNode resolveNode(String node) { * - "_local" or "_master" for the relevant nodes * - a node id * - a wild card pattern that will be matched against node names - * - a "attr:value" pattern, where attr can be a node role (master, data, ingest etc.) in which case the value can be true of false + * - a "attr:value" pattern, where attr can be a node role (master, data, ingest etc.) in which case the value can be true or false, * or a generic node attribute name in which case value will be treated as a wildcard and matched against the node attribute values. */ public String[] resolveNodes(String... nodes) { @@ -349,6 +362,12 @@ public String[] resolveNodes(String... nodes) { } else { resolvedNodesIds.removeAll(ingestNodes.keys()); } + } else if (DiscoveryNode.COORDINATING_ONLY.equals(matchAttrName)) { + if (Booleans.parseBoolean(matchAttrValue, true)) { + resolvedNodesIds.addAll(getCoordinatingOnlyNodes().keys()); + } else { + resolvedNodesIds.removeAll(getCoordinatingOnlyNodes().keys()); + } } else { for (DiscoveryNode node : this) { for (Map.Entry entry : node.getAttributes().entrySet()) { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index e6032c52585ec..deb10b83b5a5d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterStateHealth; +import org.elasticsearch.cluster.metadata.AutoExpandReplicas; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.routing.RoutingNode; @@ -46,6 +47,7 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; @@ -206,11 +208,12 @@ public ClusterState applyFailedShards(final ClusterState clusterState, final Lis * unassigned an shards that are associated with nodes that are no longer part of the cluster, potentially promoting replicas * if needed. */ - public ClusterState deassociateDeadNodes(final ClusterState clusterState, boolean reroute, String reason) { - RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); + public ClusterState deassociateDeadNodes(ClusterState clusterState, boolean reroute, String reason) { + ClusterState fixedClusterState = adaptAutoExpandReplicas(clusterState); + RoutingNodes routingNodes = getMutableRoutingNodes(fixedClusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); - RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, + RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, fixedClusterState, clusterInfoService.getClusterInfo(), currentNanoTime()); // first, clear from the shards any node id they used to belong to that is now dead @@ -220,12 +223,40 @@ public ClusterState deassociateDeadNodes(final ClusterState clusterState, boolea reroute(allocation); } - if (allocation.routingNodesChanged() == false) { + if (fixedClusterState == clusterState && allocation.routingNodesChanged() == false) { return clusterState; } return buildResultAndLogHealthChange(clusterState, allocation, reason); } + /** + * Checks if the are replicas with the auto-expand feature that need to be adapted. + * Returns an updated cluster state if changes were necessary, or the identical cluster if no changes were required. + */ + private ClusterState adaptAutoExpandReplicas(ClusterState clusterState) { + final Map> autoExpandReplicaChanges = + AutoExpandReplicas.getAutoExpandReplicaChanges(clusterState.metaData(), clusterState.nodes()); + if (autoExpandReplicaChanges.isEmpty()) { + return clusterState; + } else { + final RoutingTable.Builder routingTableBuilder = RoutingTable.builder(clusterState.routingTable()); + final MetaData.Builder metaDataBuilder = MetaData.builder(clusterState.metaData()); + for (Map.Entry> entry : autoExpandReplicaChanges.entrySet()) { + final int numberOfReplicas = entry.getKey(); + final String[] indices = entry.getValue().toArray(new String[entry.getValue().size()]); + // we do *not* update the in sync allocation ids as they will be removed upon the first index + // operation which make these copies stale + routingTableBuilder.updateNumberOfReplicas(numberOfReplicas, indices); + metaDataBuilder.updateNumberOfReplicas(numberOfReplicas, indices); + logger.info("updating number_of_replicas to [{}] for indices {}", numberOfReplicas, indices); + } + final ClusterState fixedState = ClusterState.builder(clusterState).routingTable(routingTableBuilder.build()) + .metaData(metaDataBuilder).build(); + assert AutoExpandReplicas.getAutoExpandReplicaChanges(fixedState.metaData(), fixedState.nodes()).isEmpty(); + return fixedState; + } + } + /** * Removes delay markers from unassigned shards based on current time stamp. */ @@ -301,6 +332,7 @@ public CommandsResult reroute(final ClusterState clusterState, AllocationCommand if (retryFailed) { resetFailedAllocationCounter(allocation); } + reroute(allocation); return new CommandsResult(explanations, buildResultAndLogHealthChange(clusterState, allocation, "reroute commands")); } @@ -320,15 +352,17 @@ public ClusterState reroute(ClusterState clusterState, String reason) { *

* If the same instance of ClusterState is returned, then no change has been made. */ - protected ClusterState reroute(final ClusterState clusterState, String reason, boolean debug) { - RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); + protected ClusterState reroute(ClusterState clusterState, String reason, boolean debug) { + ClusterState fixedClusterState = adaptAutoExpandReplicas(clusterState); + + RoutingNodes routingNodes = getMutableRoutingNodes(fixedClusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards routingNodes.unassigned().shuffle(); - RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, + RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, fixedClusterState, clusterInfoService.getClusterInfo(), currentNanoTime()); allocation.debugDecision(debug); reroute(allocation); - if (allocation.routingNodesChanged() == false) { + if (fixedClusterState == clusterState && allocation.routingNodesChanged() == false) { return clusterState; } return buildResultAndLogHealthChange(clusterState, allocation, reason); @@ -353,6 +387,8 @@ private boolean hasDeadNodes(RoutingAllocation allocation) { private void reroute(RoutingAllocation allocation) { assert hasDeadNodes(allocation) == false : "dead nodes should be explicitly cleaned up. See deassociateDeadNodes"; + assert AutoExpandReplicas.getAutoExpandReplicaChanges(allocation.metaData(), allocation.nodes()).isEmpty() : + "auto-expand replicas out of sync with number of nodes in the cluster"; // now allocate all the unassigned to available nodes if (allocation.routingNodes().unassigned().size() > 0) { diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index a04c75941e7b6..6b9992e7e4c3a 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -75,7 +75,8 @@ public interface BlobContainer { void writeBlob(String blobName, InputStream inputStream, long blobSize) throws IOException; /** - * Deletes a blob with giving name, if the blob exists. If the blob does not exist, this method throws an IOException. + * Deletes a blob with giving name, if the blob exists. If the blob does not exist, + * this method throws a NoSuchFileException. * * @param blobName * The name of the blob to delete. @@ -84,6 +85,21 @@ public interface BlobContainer { */ void deleteBlob(String blobName) throws IOException; + /** + * Deletes a blob with giving name, ignoring if the blob does not exist. + * + * @param blobName + * The name of the blob to delete. + * @throws IOException if the blob exists but could not be deleted. + */ + default void deleteBlobIgnoringIfNotExists(String blobName) throws IOException { + try { + deleteBlob(blobName); + } catch (final NoSuchFileException ignored) { + // This exception is ignored + } + } + /** * Lists all blobs in the container. * diff --git a/server/src/main/java/org/elasticsearch/common/cache/Cache.java b/server/src/main/java/org/elasticsearch/common/cache/Cache.java index 620612619104b..0db4d718709d8 100644 --- a/server/src/main/java/org/elasticsearch/common/cache/Cache.java +++ b/server/src/main/java/org/elasticsearch/common/cache/Cache.java @@ -206,34 +206,33 @@ private static class CacheSegment { */ Entry get(K key, long now, Predicate> isExpired, Consumer> onExpiration) { CompletableFuture> future; - Entry entry = null; try (ReleasableLock ignored = readLock.acquire()) { future = map.get(key); } if (future != null) { + Entry entry; try { - entry = future.handle((ok, ex) -> { - if (ok != null && !isExpired.test(ok)) { - segmentStats.hit(); - ok.accessTime = now; - return ok; - } else { - segmentStats.miss(); - if (ok != null) { - assert isExpired.test(ok); - onExpiration.accept(ok); - } - return null; - } - }).get(); - } catch (ExecutionException | InterruptedException e) { + entry = future.get(); + } catch (ExecutionException e) { + assert future.isCompletedExceptionally(); + segmentStats.miss(); + return null; + } catch (InterruptedException e) { throw new IllegalStateException(e); } - } - else { + if (isExpired.test(entry)) { + segmentStats.miss(); + onExpiration.accept(entry); + return null; + } else { + segmentStats.hit(); + entry.accessTime = now; + return entry; + } + } else { segmentStats.miss(); + return null; } - return entry; } /** @@ -269,30 +268,18 @@ Tuple, Entry> put(K key, V value, long now) { /** * remove an entry from the segment * - * @param key the key of the entry to remove from the cache - * @return the removed entry if there was one, otherwise null + * @param key the key of the entry to remove from the cache + * @param onRemoval a callback for the removed entry */ - Entry remove(K key) { + void remove(K key, Consumer>> onRemoval) { CompletableFuture> future; - Entry entry = null; try (ReleasableLock ignored = writeLock.acquire()) { future = map.remove(key); } if (future != null) { - try { - entry = future.handle((ok, ex) -> { - if (ok != null) { - segmentStats.eviction(); - return ok; - } else { - return null; - } - }).get(); - } catch (ExecutionException | InterruptedException e) { - throw new IllegalStateException(e); - } + segmentStats.eviction(); + onRemoval.accept(future); } - return entry; } private static class SegmentStats { @@ -476,12 +463,18 @@ private void put(K key, V value, long now) { */ public void invalidate(K key) { CacheSegment segment = getCacheSegment(key); - Entry entry = segment.remove(key); - if (entry != null) { - try (ReleasableLock ignored = lruLock.acquire()) { - delete(entry, RemovalNotification.RemovalReason.INVALIDATED); + segment.remove(key, f -> { + try { + Entry entry = f.get(); + try (ReleasableLock ignored = lruLock.acquire()) { + delete(entry, RemovalNotification.RemovalReason.INVALIDATED); + } + } catch (ExecutionException e) { + // ok + } catch (InterruptedException e) { + throw new IllegalStateException(e); } - } + }); } /** @@ -632,7 +625,7 @@ public void remove() { Entry entry = current; if (entry != null) { CacheSegment segment = getCacheSegment(entry.key); - segment.remove(entry.key); + segment.remove(entry.key, f -> {}); try (ReleasableLock ignored = lruLock.acquire()) { current = null; delete(entry, RemovalNotification.RemovalReason.INVALIDATED); @@ -717,7 +710,7 @@ private void evictEntry(Entry entry) { CacheSegment segment = getCacheSegment(entry.key); if (segment != null) { - segment.remove(entry.key); + segment.remove(entry.key, f -> {}); } delete(entry, RemovalNotification.RemovalReason.EVICTED); } diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java b/server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java index d2ca936740e27..0ee8d095f49a6 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java @@ -171,11 +171,17 @@ public static final String stringEncodeFromMortonLong(long hashedVal, final int * Encode to a morton long value from a given geohash string */ public static final long mortonEncode(final String hash) { + if (hash.isEmpty()) { + throw new IllegalArgumentException("empty geohash"); + } int level = 11; long b; long l = 0L; for(char c : hash.toCharArray()) { b = (long)(BASE_32_STRING.indexOf(c)); + if (b < 0) { + throw new IllegalArgumentException("unsupported symbol [" + c + "] in geohash [" + hash + "]"); + } l |= (b<<((level--*5) + MORTON_OFFSET)); if (level < 0) { // We cannot handle more than 12 levels diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java b/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java index e43c9e9a8e3cc..8a0c3efa5afd9 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.common.Strings; import java.io.IOException; import java.util.Arrays; @@ -126,7 +125,12 @@ public GeoPoint resetFromIndexableField(IndexableField field) { } public GeoPoint resetFromGeoHash(String geohash) { - final long hash = mortonEncode(geohash); + final long hash; + try { + hash = mortonEncode(geohash); + } catch (IllegalArgumentException ex) { + throw new ElasticsearchParseException(ex.getMessage(), ex); + } return this.reset(GeoHashUtils.decodeLatitude(hash), GeoHashUtils.decodeLongitude(hash)); } diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index e36497d09164f..e59fc8ad51385 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -58,9 +58,7 @@ public class NodeJoinController extends AbstractComponent { private final MasterService masterService; - private final AllocationService allocationService; - private final ElectMasterService electMaster; - private final JoinTaskExecutor joinTaskExecutor = new JoinTaskExecutor(); + private final JoinTaskExecutor joinTaskExecutor; // this is set while trying to become a master // mutation should be done under lock @@ -71,8 +69,7 @@ public NodeJoinController(MasterService masterService, AllocationService allocat Settings settings) { super(settings); this.masterService = masterService; - this.allocationService = allocationService; - this.electMaster = electMaster; + joinTaskExecutor = new JoinTaskExecutor(allocationService, electMaster, logger); } /** @@ -404,7 +401,20 @@ public String toString() { } }; - class JoinTaskExecutor implements ClusterStateTaskExecutor { + // visible for testing + public static class JoinTaskExecutor implements ClusterStateTaskExecutor { + + private final AllocationService allocationService; + + private final ElectMasterService electMasterService; + + private final Logger logger; + + public JoinTaskExecutor(AllocationService allocationService, ElectMasterService electMasterService, Logger logger) { + this.allocationService = allocationService; + this.electMasterService = electMasterService; + this.logger = logger; + } @Override public ClusterTasksResult execute(ClusterState currentState, List joiningNodes) throws Exception { @@ -512,7 +522,7 @@ public boolean runOnlyOnMaster() { @Override public void clusterStatePublished(ClusterChangedEvent event) { - NodeJoinController.this.electMaster.logMinimumMasterNodesWarningIfNecessary(event.previousState(), event.state()); + electMasterService.logMinimumMasterNodesWarningIfNecessary(event.previousState(), event.state()); } } } diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 79ba587974398..4621e6769e962 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -558,19 +558,19 @@ void setCommittedState(ClusterState clusterState) { } // visible for testing - static class NodeRemovalClusterStateTaskExecutor implements ClusterStateTaskExecutor, ClusterStateTaskListener { + public static class NodeRemovalClusterStateTaskExecutor implements ClusterStateTaskExecutor, ClusterStateTaskListener { private final AllocationService allocationService; private final ElectMasterService electMasterService; private final Consumer rejoin; private final Logger logger; - static class Task { + public static class Task { private final DiscoveryNode node; private final String reason; - Task(final DiscoveryNode node, final String reason) { + public Task(final DiscoveryNode node, final String reason) { this.node = node; this.reason = reason; } @@ -589,7 +589,7 @@ public String toString() { } } - NodeRemovalClusterStateTaskExecutor( + public NodeRemovalClusterStateTaskExecutor( final AllocationService allocationService, final ElectMasterService electMasterService, final Consumer rejoin, diff --git a/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java index 2ea31f67e2908..551f7c18c1c93 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java @@ -299,14 +299,7 @@ public Mapper parse(ParseContext context) throws IOException { if (token == XContentParser.Token.START_ARRAY) { // its an array of array of lon/lat [ [1.2, 1.3], [1.4, 1.5] ] while (token != XContentParser.Token.END_ARRAY) { - try { - parse(context, GeoUtils.parseGeoPoint(context.parser(), sparse)); - } catch (ElasticsearchParseException e) { - if (ignoreMalformed.value() == false) { - throw e; - } - context.addIgnoredField(fieldType.name()); - } + parseGeoPointIgnoringMalformed(context, sparse); token = context.parser().nextToken(); } } else { @@ -326,35 +319,22 @@ public Mapper parse(ParseContext context) throws IOException { } else { while (token != XContentParser.Token.END_ARRAY) { if (token == XContentParser.Token.VALUE_STRING) { - parse(context, sparse.resetFromString(context.parser().text(), ignoreZValue.value())); + parseGeoPointStringIgnoringMalformed(context, sparse); } else { - try { - parse(context, GeoUtils.parseGeoPoint(context.parser(), sparse)); - } catch (ElasticsearchParseException e) { - if (ignoreMalformed.value() == false) { - throw e; - } - } + parseGeoPointIgnoringMalformed(context, sparse); } token = context.parser().nextToken(); } } } } else if (token == XContentParser.Token.VALUE_STRING) { - parse(context, sparse.resetFromString(context.parser().text(), ignoreZValue.value())); + parseGeoPointStringIgnoringMalformed(context, sparse); } else if (token == XContentParser.Token.VALUE_NULL) { if (fieldType.nullValue() != null) { parse(context, (GeoPoint) fieldType.nullValue()); } } else { - try { - parse(context, GeoUtils.parseGeoPoint(context.parser(), sparse)); - } catch (ElasticsearchParseException e) { - if (ignoreMalformed.value() == false) { - throw e; - } - context.addIgnoredField(fieldType.name()); - } + parseGeoPointIgnoringMalformed(context, sparse); } } @@ -362,6 +342,34 @@ public Mapper parse(ParseContext context) throws IOException { return null; } + /** + * Parses geopoint represented as an object or an array, ignores malformed geopoints if needed + */ + private void parseGeoPointIgnoringMalformed(ParseContext context, GeoPoint sparse) throws IOException { + try { + parse(context, GeoUtils.parseGeoPoint(context.parser(), sparse)); + } catch (ElasticsearchParseException e) { + if (ignoreMalformed.value() == false) { + throw e; + } + context.addIgnoredField(fieldType.name()); + } + } + + /** + * Parses geopoint represented as a string and ignores malformed geopoints if needed + */ + private void parseGeoPointStringIgnoringMalformed(ParseContext context, GeoPoint sparse) throws IOException { + try { + parse(context, sparse.resetFromString(context.parser().text(), ignoreZValue.value())); + } catch (ElasticsearchParseException e) { + if (ignoreMalformed.value() == false) { + throw e; + } + context.addIgnoredField(fieldType.name()); + } + } + @Override protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { super.doXContentBody(builder, includeDefaults, params); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java index 4a954f1b99fb0..4c356c3a5592d 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java @@ -286,6 +286,9 @@ public Query termQuery(Object value, QueryShardContext context) { public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, ShapeRelation relation, DateTimeZone timeZone, DateMathParser parser, QueryShardContext context) { failIfNotIndexed(); + if (parser == null) { + parser = dateMathParser(); + } return rangeType.rangeQuery(name(), hasDocValues(), lowerTerm, upperTerm, includeLower, includeUpper, relation, timeZone, parser, context); } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java index d25b1eb04866d..34b07932e48ff 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java @@ -102,13 +102,6 @@ private BlobStoreIndexShardSnapshots(Map files, List snapshots, int fileListGeneration, Map blobs) { - BlobStoreIndexShardSnapshots newSnapshots = new BlobStoreIndexShardSnapshots(snapshots); - // delete old index files first - for (String blobName : blobs.keySet()) { - if (indexShardSnapshotsFormat.isTempBlobName(blobName) || blobName.startsWith(SNAPSHOT_INDEX_PREFIX)) { - try { - blobContainer.deleteBlob(blobName); - } catch (IOException e) { - // We cannot delete index file - this is fatal, we cannot continue, otherwise we might end up - // with references to non-existing files - throw new IndexShardSnapshotFailedException(shardId, "error deleting index file [" - + blobName + "] during cleanup", e); - } + protected void finalize(final List snapshots, + final int fileListGeneration, + final Map blobs, + final String reason) { + final String indexGeneration = Integer.toString(fileListGeneration); + final String currentIndexGen = indexShardSnapshotsFormat.blobName(indexGeneration); + + final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(snapshots); + try { + // If we deleted all snapshots, we don't need to create a new index file + if (snapshots.size() > 0) { + indexShardSnapshotsFormat.writeAtomic(updatedSnapshots, blobContainer, indexGeneration); } - } - // now go over all the blobs, and if they don't exist in a snapshot, delete them - for (String blobName : blobs.keySet()) { - // delete unused files - if (blobName.startsWith(DATA_BLOB_PREFIX)) { - if (newSnapshots.findNameFile(BlobStoreIndexShardSnapshot.FileInfo.canonicalName(blobName)) == null) { + // Delete old index files + for (final String blobName : blobs.keySet()) { + if (indexShardSnapshotsFormat.isTempBlobName(blobName) || blobName.startsWith(SNAPSHOT_INDEX_PREFIX)) { try { - blobContainer.deleteBlob(blobName); + blobContainer.deleteBlobIgnoringIfNotExists(blobName); } catch (IOException e) { - // TODO: don't catch and let the user handle it? - logger.debug(() -> new ParameterizedMessage("[{}] [{}] error deleting blob [{}] during cleanup", snapshotId, shardId, blobName), e); + logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete index blob [{}] during finalization", + snapshotId, shardId, blobName), e); + throw e; } } } - } - // If we deleted all snapshots - we don't need to create the index file - if (snapshots.size() > 0) { - try { - indexShardSnapshotsFormat.writeAtomic(newSnapshots, blobContainer, Integer.toString(fileListGeneration)); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to write file list", e); + // Delete all blobs that don't exist in a snapshot + for (final String blobName : blobs.keySet()) { + if (blobName.startsWith(DATA_BLOB_PREFIX) && (updatedSnapshots.findNameFile(canonicalName(blobName)) == null)) { + try { + blobContainer.deleteBlobIgnoringIfNotExists(blobName); + } catch (IOException e) { + logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete data blob [{}] during finalization", + snapshotId, shardId, blobName), e); + } + } } + } catch (IOException e) { + String message = "Failed to finalize " + reason + " with shard index [" + currentIndexGen + "]"; + throw new IndexShardSnapshotFailedException(shardId, message, e); } } @@ -1003,7 +1007,7 @@ protected long findLatestFileNameGeneration(Map blobs) { if (!name.startsWith(DATA_BLOB_PREFIX)) { continue; } - name = BlobStoreIndexShardSnapshot.FileInfo.canonicalName(name); + name = canonicalName(name); try { long currentGen = Long.parseLong(name.substring(DATA_BLOB_PREFIX.length()), Character.MAX_RADIX); if (currentGen > generation) { @@ -1217,7 +1221,7 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { newSnapshotsList.add(point); } // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index - finalize(newSnapshotsList, fileListGeneration + 1, blobs); + finalize(newSnapshotsList, fileListGeneration + 1, blobs, "snapshot creation [" + snapshotId + "]"); snapshotStatus.moveToDone(System.currentTimeMillis()); } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java index fd70b7461ec67..715da019e6df4 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.rest.action.admin.indices; +import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; +import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; @@ -57,23 +59,10 @@ public String getName() { static final Map> METRICS; static { - final Map> metrics = new HashMap<>(); - metrics.put("docs", r -> r.docs(true)); - metrics.put("store", r -> r.store(true)); - metrics.put("indexing", r -> r.indexing(true)); - metrics.put("search", r -> r.search(true)); - metrics.put("get", r -> r.get(true)); - metrics.put("merge", r -> r.merge(true)); - metrics.put("refresh", r -> r.refresh(true)); - metrics.put("flush", r -> r.flush(true)); - metrics.put("warmer", r -> r.warmer(true)); - metrics.put("query_cache", r -> r.queryCache(true)); - metrics.put("segments", r -> r.segments(true)); - metrics.put("fielddata", r -> r.fieldData(true)); - metrics.put("completion", r -> r.completion(true)); - metrics.put("request_cache", r -> r.requestCache(true)); - metrics.put("recovery", r -> r.recovery(true)); - metrics.put("translog", r -> r.translog(true)); + Map> metrics = new HashMap<>(); + for (Flag flag : CommonStatsFlags.Flag.values()) { + metrics.put(flag.getRestName(), m -> m.flags().set(flag, true)); + } METRICS = Collections.unmodifiableMap(metrics); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/create/ShrinkIndexIT.java b/server/src/test/java/org/elasticsearch/action/admin/indices/create/ShrinkIndexIT.java index e48f151081f62..e4bb197f80ace 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/create/ShrinkIndexIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/create/ShrinkIndexIT.java @@ -83,6 +83,7 @@ protected Collection> nodePlugins() { return Arrays.asList(InternalSettingsPlugin.class); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/30416") public void testCreateShrinkIndexToN() { int[][] possibleShardSplits = new int[][] {{8,4,2}, {9, 3, 1}, {4, 2, 1}, {15,5,1}}; int[] shardSplits = randomFrom(possibleShardSplits); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/create/SplitIndexIT.java b/server/src/test/java/org/elasticsearch/action/admin/indices/create/SplitIndexIT.java index fe6e980ab4259..a7f7ed6f52546 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/create/SplitIndexIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/create/SplitIndexIT.java @@ -24,6 +24,7 @@ import org.apache.lucene.search.SortedSetSelector; import org.apache.lucene.search.SortedSetSortField; import org.apache.lucene.search.join.ScoreMode; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; @@ -80,6 +81,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/30416") public class SplitIndexIT extends ESIntegTestCase { @Override diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequestTests.java index 72cbe2bd9ecab..294213452596f 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequestTests.java @@ -20,10 +20,15 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.yaml.YamlXContent; @@ -35,6 +40,7 @@ import java.util.Collections; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.core.Is.is; @@ -131,4 +137,52 @@ public void testValidateErrorMessage() throws Exception { assertThat(noError, is(nullValue())); } + private PutIndexTemplateRequest randomPutIndexTemplateRequest() throws IOException { + PutIndexTemplateRequest request = new PutIndexTemplateRequest(); + request.name("test"); + if (randomBoolean()){ + request.version(randomInt()); + } + if (randomBoolean()){ + request.order(randomInt()); + } + request.patterns(Arrays.asList(generateRandomStringArray(20, 100, false, false))); + int numAlias = between(0, 5); + for (int i = 0; i < numAlias; i++) { + Alias alias = new Alias(randomRealisticUnicodeOfLengthBetween(1, 10)); + if (randomBoolean()) { + alias.indexRouting(randomRealisticUnicodeOfLengthBetween(1, 10)); + } + if (randomBoolean()) { + alias.searchRouting(randomRealisticUnicodeOfLengthBetween(1, 10)); + } + request.alias(alias); + } + if (randomBoolean()) { + request.mapping("doc", XContentFactory.jsonBuilder().startObject() + .startObject("doc").startObject("properties") + .startObject("field-" + randomInt()).field("type", randomFrom("keyword", "text")).endObject() + .endObject().endObject().endObject()); + } + if (randomBoolean()){ + request.settings(Settings.builder().put("setting1", randomLong()).put("setting2", randomTimeValue()).build()); + } + return request; + } + + public void testFromToXContentPutTemplateRequest() throws Exception { + for (int i = 0; i < 10; i++) { + PutIndexTemplateRequest expected = randomPutIndexTemplateRequest(); + XContentType xContentType = randomFrom(XContentType.values()); + BytesReference shuffled = toShuffledXContent(expected, xContentType, ToXContent.EMPTY_PARAMS, randomBoolean()); + PutIndexTemplateRequest parsed = new PutIndexTemplateRequest().source(shuffled, xContentType); + assertNotSame(expected, parsed); + assertThat(parsed.version(), equalTo(expected.version())); + assertThat(parsed.order(), equalTo(expected.order())); + assertThat(parsed.patterns(), equalTo(expected.patterns())); + assertThat(parsed.aliases(), equalTo(expected.aliases())); + assertThat(parsed.mappings(), equalTo(expected.mappings())); + assertThat(parsed.settings(), equalTo(expected.settings())); + } + } } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index 4b96f3d17543c..f1731083ae376 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -18,15 +18,13 @@ */ package org.elasticsearch.action.bulk; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESIntegTestCase; -import org.hamcrest.Matcher; import java.util.Collections; import java.util.Iterator; @@ -38,6 +36,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE, numDataNodes = 2) @@ -108,26 +107,28 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) assertThat(responses.size(), equalTo(numberOfAsyncOps)); // validate all responses + boolean rejectedAfterAllRetries = false; for (Object response : responses) { if (response instanceof BulkResponse) { BulkResponse bulkResponse = (BulkResponse) response; for (BulkItemResponse bulkItemResponse : bulkResponse.getItems()) { if (bulkItemResponse.isFailed()) { BulkItemResponse.Failure failure = bulkItemResponse.getFailure(); - Throwable rootCause = ExceptionsHelper.unwrapCause(failure.getCause()); - if (rootCause instanceof EsRejectedExecutionException) { + if (failure.getStatus() == RestStatus.TOO_MANY_REQUESTS) { if (rejectedExecutionExpected == false) { Iterator backoffState = internalPolicy.backoffStateFor(bulkResponse); assertNotNull("backoffState is null (indicates a bulk request got rejected without retry)", backoffState); if (backoffState.hasNext()) { // we're not expecting that we overwhelmed it even once when we maxed out the number of retries - throw new AssertionError("Got rejected although backoff policy would allow more retries", rootCause); + throw new AssertionError("Got rejected although backoff policy would allow more retries", + failure.getCause()); } else { + rejectedAfterAllRetries = true; logger.debug("We maxed out the number of bulk retries and got rejected (this is ok)."); } } } else { - throw new AssertionError("Unexpected failure", rootCause); + throw new AssertionError("Unexpected failure status: " + failure.getStatus()); } } } @@ -140,18 +141,20 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) client().admin().indices().refresh(new RefreshRequest()).get(); - // validate we did not create any duplicates due to retries - Matcher searchResultCount; - // it is ok if we lost some index operations to rejected executions (which is possible even when backing off (although less likely) - searchResultCount = lessThanOrEqualTo((long) numberOfAsyncOps); - SearchResponse results = client() .prepareSearch(INDEX_NAME) .setTypes(TYPE_NAME) .setQuery(QueryBuilders.matchAllQuery()) .setSize(0) .get(); - assertThat(results.getHits().getTotalHits(), searchResultCount); + + if (rejectedExecutionExpected) { + assertThat((int) results.getHits().getTotalHits(), lessThanOrEqualTo(numberOfAsyncOps)); + } else if (rejectedAfterAllRetries) { + assertThat((int) results.getHits().getTotalHits(), lessThan(numberOfAsyncOps)); + } else { + assertThat((int) results.getHits().getTotalHits(), equalTo(numberOfAsyncOps)); + } } private static void indexDocs(BulkProcessor processor, int numDocs) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/RetryTests.java b/server/src/test/java/org/elasticsearch/action/bulk/RetryTests.java index 136097a292668..320f11ff6d04f 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/RetryTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/RetryTests.java @@ -84,7 +84,7 @@ public void testRetryBacksOff() throws Exception { BackoffPolicy backoff = BackoffPolicy.constantBackoff(DELAY, CALLS_TO_FAIL); BulkRequest bulkRequest = createBulkRequest(); - BulkResponse response = new Retry(EsRejectedExecutionException.class, backoff, bulkClient.threadPool()) + BulkResponse response = new Retry(backoff, bulkClient.threadPool()) .withBackoff(bulkClient::bulk, bulkRequest, bulkClient.settings()) .actionGet(); @@ -96,7 +96,7 @@ public void testRetryFailsAfterBackoff() throws Exception { BackoffPolicy backoff = BackoffPolicy.constantBackoff(DELAY, CALLS_TO_FAIL - 1); BulkRequest bulkRequest = createBulkRequest(); - BulkResponse response = new Retry(EsRejectedExecutionException.class, backoff, bulkClient.threadPool()) + BulkResponse response = new Retry(backoff, bulkClient.threadPool()) .withBackoff(bulkClient::bulk, bulkRequest, bulkClient.settings()) .actionGet(); @@ -109,7 +109,7 @@ public void testRetryWithListenerBacksOff() throws Exception { AssertingListener listener = new AssertingListener(); BulkRequest bulkRequest = createBulkRequest(); - Retry retry = new Retry(EsRejectedExecutionException.class, backoff, bulkClient.threadPool()); + Retry retry = new Retry(backoff, bulkClient.threadPool()); retry.withBackoff(bulkClient::bulk, bulkRequest, listener, bulkClient.settings()); listener.awaitCallbacksCalled(); @@ -124,7 +124,7 @@ public void testRetryWithListenerFailsAfterBacksOff() throws Exception { AssertingListener listener = new AssertingListener(); BulkRequest bulkRequest = createBulkRequest(); - Retry retry = new Retry(EsRejectedExecutionException.class, backoff, bulkClient.threadPool()); + Retry retry = new Retry(backoff, bulkClient.threadPool()); retry.withBackoff(bulkClient::bulk, bulkRequest, listener, bulkClient.settings()); listener.awaitCallbacksCalled(); diff --git a/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index 80f2401ed7735..cf0b1eebe983a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -102,9 +102,17 @@ public void testCoordinatorOnlyNodes() { .map(DiscoveryNode::getId) .toArray(String[]::new); - assertThat( - discoveryNodes.resolveNodes("_all", "data:false", "ingest:false", "master:false"), - arrayContainingInAnyOrder(coordinatorOnlyNodes)); + final String[] nonCoordinatorOnlyNodes = + StreamSupport.stream(discoveryNodes.getNodes().values().spliterator(), false) + .map(n -> n.value) + .filter(n -> n.isMasterNode() || n.isDataNode() || n.isIngestNode()) + .map(DiscoveryNode::getId) + .toArray(String[]::new); + + assertThat(discoveryNodes.resolveNodes("coordinating_only:true"), arrayContainingInAnyOrder(coordinatorOnlyNodes)); + assertThat(discoveryNodes.resolveNodes("_all", "data:false", "ingest:false", "master:false"), + arrayContainingInAnyOrder(coordinatorOnlyNodes)); + assertThat(discoveryNodes.resolveNodes("_all", "coordinating_only:false"), arrayContainingInAnyOrder(nonCoordinatorOnlyNodes)); } public void testResolveNodesIds() { @@ -275,6 +283,13 @@ Set matchingNodeIds(DiscoveryNodes nodes) { nodes.getIngestNodes().keysIt().forEachRemaining(ids::add); return ids; } + }, COORDINATING_ONLY(DiscoveryNode.COORDINATING_ONLY + ":true") { + @Override + Set matchingNodeIds(DiscoveryNodes nodes) { + Set ids = new HashSet<>(); + nodes.getCoordinatingOnlyNodes().keysIt().forEachRemaining(ids::add); + return ids; + } }, CUSTOM_ATTRIBUTE("attr:value") { @Override Set matchingNodeIds(DiscoveryNodes nodes) { diff --git a/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java b/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java index 773585cc3b494..fe64fd16af68c 100644 --- a/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java +++ b/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.common.cache; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -343,6 +344,37 @@ protected long now() { assertEquals(numberOfEntries, cache.stats().getEvictions()); } + public void testComputeIfAbsentDeadlock() throws BrokenBarrierException, InterruptedException { + final int numberOfThreads = randomIntBetween(2, 32); + final Cache cache = + CacheBuilder.builder().setExpireAfterAccess(TimeValue.timeValueNanos(1)).build(); + + final CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); + for (int i = 0; i < numberOfThreads; i++) { + final Thread thread = new Thread(() -> { + try { + barrier.await(); + for (int j = 0; j < numberOfEntries; j++) { + try { + cache.computeIfAbsent(0, k -> Integer.toString(k)); + } catch (final ExecutionException e) { + throw new AssertionError(e); + } + } + barrier.await(); + } catch (final BrokenBarrierException | InterruptedException e) { + throw new AssertionError(e); + } + }); + thread.start(); + } + + // wait for all threads to be ready + barrier.await(); + // wait for all threads to finish + barrier.await(); + } + // randomly promote some entries, step the clock forward, then check that the promoted entries remain and the // non-promoted entries were removed public void testPromotion() { diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java index 2726380b7e3bc..1ab67b058f115 100644 --- a/server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java +++ b/server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.common.geo; import org.apache.lucene.geo.Rectangle; +import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.test.ESTestCase; /** @@ -95,7 +96,17 @@ public void testLongGeohashes() { Rectangle expectedBbox = GeoHashUtils.bbox(geohash); Rectangle actualBbox = GeoHashUtils.bbox(extendedGeohash); assertEquals("Additional data points above 12 should be ignored [" + extendedGeohash + "]" , expectedBbox, actualBbox); - } } + + public void testInvalidGeohashes() { + IllegalArgumentException ex; + + ex = expectThrows(IllegalArgumentException.class, () -> GeoHashUtils.mortonEncode("55.5")); + assertEquals("unsupported symbol [.] in geohash [55.5]", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> GeoHashUtils.mortonEncode("")); + assertEquals("empty geohash", ex.getMessage()); + } + } diff --git a/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java index 2998ec8a6ba66..fab38a2b73b4a 100644 --- a/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -61,10 +60,13 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static org.elasticsearch.action.DocWriteResponse.Result.CREATED; +import static org.elasticsearch.action.DocWriteResponse.Result.UPDATED; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; /** @@ -135,7 +137,7 @@ public void testAckedIndexing() throws Exception { .setSource("{}", XContentType.JSON) .setTimeout(timeout) .get(timeout); - assertEquals(DocWriteResponse.Result.CREATED, response.getResult()); + assertThat(response.getResult(), isOneOf(CREATED, UPDATED)); ackedDocs.put(id, node); logger.trace("[{}] indexed id [{}] through node [{}], response [{}]", name, id, node, response); } catch (ElasticsearchException e) { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java index 0de90631a14b3..facafaf180ec2 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java @@ -49,6 +49,7 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { @@ -398,4 +399,50 @@ public void testNullValue() throws Exception { assertThat(defaultValue, not(equalTo(doc.rootDoc().getField("location").binaryValue()))); } + public void testInvalidGeohashIgnored() throws Exception { + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties") + .startObject("location") + .field("type", "geo_point") + .field("ignore_malformed", "true") + .endObject() + .endObject().endObject().endObject()); + + DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser() + .parse("type", new CompressedXContent(mapping)); + + ParsedDocument doc = defaultMapper.parse(SourceToParse.source("test", "type", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .field("location", "1234.333") + .endObject()), + XContentType.JSON)); + + assertThat(doc.rootDoc().getField("location"), nullValue()); + } + + + public void testInvalidGeohashNotIgnored() throws Exception { + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties") + .startObject("location") + .field("type", "geo_point") + .endObject() + .endObject().endObject().endObject()); + + DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser() + .parse("type", new CompressedXContent(mapping)); + + MapperParsingException ex = expectThrows(MapperParsingException.class, + () -> defaultMapper.parse(SourceToParse.source("test", "type", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .field("location", "1234.333") + .endObject()), + XContentType.JSON))); + + assertThat(ex.getMessage(), equalTo("failed to parse")); + assertThat(ex.getRootCause().getMessage(), equalTo("unsupported symbol [.] in geohash [1234.333]")); + } + } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java index f218f2a5db7e4..6ecd61275fe96 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.mapper; -import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.lucene.document.DoubleRange; import org.apache.lucene.document.FloatRange; import org.apache.lucene.document.InetAddressPoint; @@ -31,13 +30,16 @@ import org.apache.lucene.search.IndexOrDocValuesQuery; import org.apache.lucene.search.Query; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.geo.ShapeRelation; +import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.RangeFieldMapper.RangeFieldType; import org.elasticsearch.index.mapper.RangeFieldMapper.RangeType; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.test.IndexSettingsModule; @@ -55,42 +57,38 @@ public class RangeFieldTypeTests extends FieldTypeTestCase { @Before public void setupProperties() { - type = RandomPicks.randomFrom(random(), RangeType.values()); + type = randomFrom(RangeType.values()); nowInMillis = randomNonNegativeLong(); if (type == RangeType.DATE) { addModifier(new Modifier("format", true) { @Override public void modify(MappedFieldType ft) { - ((RangeFieldMapper.RangeFieldType) ft).setDateTimeFormatter(Joda.forPattern("basic_week_date", Locale.ROOT)); + ((RangeFieldType) ft).setDateTimeFormatter(Joda.forPattern("basic_week_date", Locale.ROOT)); } }); addModifier(new Modifier("locale", true) { @Override public void modify(MappedFieldType ft) { - ((RangeFieldMapper.RangeFieldType) ft).setDateTimeFormatter(Joda.forPattern("date_optional_time", Locale.CANADA)); + ((RangeFieldType) ft).setDateTimeFormatter(Joda.forPattern("date_optional_time", Locale.CANADA)); } }); } } @Override - protected RangeFieldMapper.RangeFieldType createDefaultFieldType() { - return new RangeFieldMapper.RangeFieldType(type); + protected RangeFieldType createDefaultFieldType() { + return new RangeFieldType(type); } public void testRangeQuery() throws Exception { - Settings indexSettings = Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); - IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); - QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), - writableRegistry(), null, null, () -> nowInMillis, null); - RangeFieldMapper.RangeFieldType ft = new RangeFieldMapper.RangeFieldType(type); + QueryShardContext context = createContext(); + RangeFieldType ft = new RangeFieldType(type); ft.setName(FIELDNAME); ft.setIndexOptions(IndexOptions.DOCS); - ShapeRelation relation = RandomPicks.randomFrom(random(), ShapeRelation.values()); - boolean includeLower = random().nextBoolean(); - boolean includeUpper = random().nextBoolean(); + ShapeRelation relation = randomFrom(ShapeRelation.values()); + boolean includeLower = randomBoolean(); + boolean includeUpper = randomBoolean(); Object from = nextFrom(); Object to = nextTo(from); @@ -98,6 +96,41 @@ public void testRangeQuery() throws Exception { ft.rangeQuery(from, to, includeLower, includeUpper, relation, null, null, context)); } + private QueryShardContext createContext() { + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); + return new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), + writableRegistry(), null, null, () -> nowInMillis, null); + } + + public void testDateRangeQueryUsingMappingFormat() { + QueryShardContext context = createContext(); + RangeFieldType fieldType = new RangeFieldType(RangeType.DATE); + fieldType.setName(FIELDNAME); + fieldType.setIndexOptions(IndexOptions.DOCS); + fieldType.setHasDocValues(false); + ShapeRelation relation = randomFrom(ShapeRelation.values()); + + // dates will break the default format + final String from = "2016-15-06T15:29:50+08:00"; + final String to = "2016-16-06T15:29:50+08:00"; + + ElasticsearchParseException ex = expectThrows(ElasticsearchParseException.class, + () -> fieldType.rangeQuery(from, to, true, true, relation, null, null, context)); + assertEquals("failed to parse date field [2016-15-06T15:29:50+08:00] with format [strict_date_optional_time||epoch_millis]", + ex.getMessage()); + + // setting mapping format which is compatible with those dates + final FormatDateTimeFormatter formatter = Joda.forPattern("yyyy-dd-MM'T'HH:mm:ssZZ"); + assertEquals(1465975790000L, formatter.parser().parseMillis(from)); + assertEquals(1466062190000L, formatter.parser().parseMillis(to)); + + fieldType.setDateTimeFormatter(formatter); + final Query query = fieldType.rangeQuery(from, to, true, true, relation, null, null, context); + assertEquals("field:", query.toString()); + } + private Query getExpectedRangeQuery(ShapeRelation relation, Object from, Object to, boolean includeLower, boolean includeUpper) { switch (type) { case DATE: @@ -277,14 +310,10 @@ public void testParseIp() { assertEquals(InetAddresses.forString("::1"), RangeFieldMapper.RangeType.IP.parse(new BytesRef("::1"), randomBoolean())); } - public void testTermQuery() throws Exception, IllegalArgumentException { + public void testTermQuery() throws Exception { // See https://github.com/elastic/elasticsearch/issues/25950 - Settings indexSettings = Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); - IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); - QueryShardContext context = new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), - writableRegistry(), null, null, () -> nowInMillis, null); - RangeFieldMapper.RangeFieldType ft = new RangeFieldMapper.RangeFieldType(type); + QueryShardContext context = createContext(); + RangeFieldType ft = new RangeFieldType(type); ft.setName(FIELDNAME); ft.setIndexOptions(IndexOptions.DOCS); diff --git a/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java b/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java index f3d109868ef14..4b580aa6a2467 100644 --- a/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java +++ b/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java @@ -175,6 +175,19 @@ public void testInvalidField() throws IOException { assertThat(e.getMessage(), is("field must be either [lat], [lon] or [geohash]")); } + public void testInvalidGeoHash() throws IOException { + XContentBuilder content = JsonXContent.contentBuilder(); + content.startObject(); + content.field("geohash", "!!!!"); + content.endObject(); + + XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); + parser.nextToken(); + + Exception e = expectThrows(ElasticsearchParseException.class, () -> GeoUtils.parseGeoPoint(parser)); + assertThat(e.getMessage(), is("unsupported symbol [!] in geohash [!!!!]")); + } + private XContentParser objectLatLon(double lat, double lon) throws IOException { XContentBuilder content = JsonXContent.contentBuilder(); content.startObject(); diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index 8af19aa9ac1e4..9e8638af2491e 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -72,6 +72,9 @@ import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.discovery.zen.ElectMasterService; +import org.elasticsearch.discovery.zen.NodeJoinController; +import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.index.IndexService; @@ -117,6 +120,9 @@ public class ClusterStateChanges extends AbstractComponent { private final TransportClusterRerouteAction transportClusterRerouteAction; private final TransportCreateIndexAction transportCreateIndexAction; + private final ZenDiscovery.NodeRemovalClusterStateTaskExecutor nodeRemovalExecutor; + private final NodeJoinController.JoinTaskExecutor joinTaskExecutor; + public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool threadPool) { super(Settings.builder().put(PATH_HOME_SETTING.getKey(), "dummy").build()); @@ -191,6 +197,11 @@ allocationService, new AliasValidator(settings), environment, transportService, clusterService, threadPool, allocationService, actionFilters, indexNameExpressionResolver); transportCreateIndexAction = new TransportCreateIndexAction(settings, transportService, clusterService, threadPool, createIndexService, actionFilters, indexNameExpressionResolver); + + ElectMasterService electMasterService = new ElectMasterService(settings); + nodeRemovalExecutor = new ZenDiscovery.NodeRemovalClusterStateTaskExecutor(allocationService, electMasterService, + s -> { throw new AssertionError("rejoin not implemented"); }, logger); + joinTaskExecutor = new NodeJoinController.JoinTaskExecutor(allocationService, electMasterService, logger); } public ClusterState createIndex(ClusterState state, CreateIndexRequest request) { @@ -217,8 +228,13 @@ public ClusterState reroute(ClusterState state, ClusterRerouteRequest request) { return execute(transportClusterRerouteAction, request, state); } - public ClusterState deassociateDeadNodes(ClusterState clusterState, boolean reroute, String reason) { - return allocationService.deassociateDeadNodes(clusterState, reroute, reason); + public ClusterState addNodes(ClusterState clusterState, List nodes) { + return runTasks(joinTaskExecutor, clusterState, nodes); + } + + public ClusterState removeNodes(ClusterState clusterState, List nodes) { + return runTasks(nodeRemovalExecutor, clusterState, nodes.stream() + .map(n -> new ZenDiscovery.NodeRemovalClusterStateTaskExecutor.Task(n, "dummy reason")).collect(Collectors.toList())); } public ClusterState applyFailedShards(ClusterState clusterState, List failedShards) { diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index 6079a9104d3db..4a496167c80c1 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -70,6 +70,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; @@ -258,8 +259,14 @@ public ClusterState randomlyUpdateClusterState(ClusterState state, } String name = "index_" + randomAlphaOfLength(15).toLowerCase(Locale.ROOT); Settings.Builder settingsBuilder = Settings.builder() - .put(SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 3)) - .put(SETTING_NUMBER_OF_REPLICAS, randomInt(2)); + .put(SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 3)); + if (randomBoolean()) { + int min = randomInt(2); + int max = min + randomInt(3); + settingsBuilder.put(SETTING_AUTO_EXPAND_REPLICAS, randomBoolean() ? min + "-" + max : min + "-all"); + } else { + settingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, randomInt(2)); + } CreateIndexRequest request = new CreateIndexRequest(name, settingsBuilder.build()).waitForActiveShards(ActiveShardCount.NONE); state = cluster.createIndex(state, request); assertTrue(state.metaData().hasIndex(name)); @@ -345,9 +352,7 @@ public ClusterState randomlyUpdateClusterState(ClusterState state, if (randomBoolean()) { // add node if (state.nodes().getSize() < 10) { - DiscoveryNodes newNodes = DiscoveryNodes.builder(state.nodes()).add(createNode()).build(); - state = ClusterState.builder(state).nodes(newNodes).build(); - state = cluster.reroute(state, new ClusterRerouteRequest()); // always reroute after node leave + state = cluster.addNodes(state, Collections.singletonList(createNode())); updateNodes(state, clusterStateServiceMap, indicesServiceSupplier); } } else { @@ -355,16 +360,12 @@ public ClusterState randomlyUpdateClusterState(ClusterState state, if (state.nodes().getDataNodes().size() > 3) { DiscoveryNode discoveryNode = randomFrom(state.nodes().getNodes().values().toArray(DiscoveryNode.class)); if (discoveryNode.equals(state.nodes().getMasterNode()) == false) { - DiscoveryNodes newNodes = DiscoveryNodes.builder(state.nodes()).remove(discoveryNode.getId()).build(); - state = ClusterState.builder(state).nodes(newNodes).build(); - state = cluster.deassociateDeadNodes(state, true, "removed and added a node"); + state = cluster.removeNodes(state, Collections.singletonList(discoveryNode)); updateNodes(state, clusterStateServiceMap, indicesServiceSupplier); } if (randomBoolean()) { // and add it back - DiscoveryNodes newNodes = DiscoveryNodes.builder(state.nodes()).add(discoveryNode).build(); - state = ClusterState.builder(state).nodes(newNodes).build(); - state = cluster.reroute(state, new ClusterRerouteRequest()); + state = cluster.addNodes(state, Collections.singletonList(discoveryNode)); updateNodes(state, clusterStateServiceMap, indicesServiceSupplier); } } diff --git a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java index 3487de255e164..21c043aa65616 100644 --- a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java +++ b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java @@ -91,6 +91,36 @@ public void testSimpleMoreLikeThis() throws Exception { assertHitCount(response, 1L); } + //Issue #30148 + public void testMoreLikeThisForZeroTokensInOneOfTheAnalyzedFields() throws Exception { + CreateIndexRequestBuilder createIndexRequestBuilder = prepareCreate("test") + .addMapping("type", jsonBuilder() + .startObject().startObject("type") + .startObject("properties") + .startObject("myField").field("type", "text").endObject() + .startObject("empty").field("type", "text").endObject() + .endObject() + .endObject().endObject()); + + assertAcked(createIndexRequestBuilder); + + ensureGreen(); + + client().index(indexRequest("test").type("type").id("1").source(jsonBuilder().startObject() + .field("myField", "and_foo").field("empty", "").endObject())).actionGet(); + client().index(indexRequest("test").type("type").id("2").source(jsonBuilder().startObject() + .field("myField", "and_foo").field("empty", "").endObject())).actionGet(); + + client().admin().indices().refresh(refreshRequest()).actionGet(); + + SearchResponse searchResponse = client().prepareSearch().setQuery( + moreLikeThisQuery(new String[]{"myField", "empty"}, null, new Item[]{new Item("test", "type", "1")}) + .minTermFreq(1).minDocFreq(1) + ).get(); + + assertHitCount(searchResponse, 1L); + } + public void testSimpleMoreLikeOnLongField() throws Exception { logger.info("Creating index test"); assertAcked(prepareCreate("test") diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java b/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java index 18be4e9437770..8aff12edc8a53 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java @@ -29,13 +29,14 @@ import java.io.IOException; import java.io.InputStream; +import java.nio.file.NoSuchFileException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import static org.elasticsearch.repositories.ESBlobStoreTestCase.writeRandomBlob; import static org.elasticsearch.repositories.ESBlobStoreTestCase.randomBytes; import static org.elasticsearch.repositories.ESBlobStoreTestCase.readBlobFully; +import static org.elasticsearch.repositories.ESBlobStoreTestCase.writeRandomBlob; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; @@ -116,7 +117,7 @@ public void testDeleteBlob() throws IOException { try (BlobStore store = newBlobStore()) { final String blobName = "foobar"; final BlobContainer container = store.blobContainer(new BlobPath()); - expectThrows(IOException.class, () -> container.deleteBlob(blobName)); + expectThrows(NoSuchFileException.class, () -> container.deleteBlob(blobName)); byte[] data = randomBytes(randomIntBetween(10, scaledRandomIntBetween(1024, 1 << 16))); final BytesArray bytesArray = new BytesArray(data); @@ -124,7 +125,19 @@ public void testDeleteBlob() throws IOException { container.deleteBlob(blobName); // should not raise // blob deleted, so should raise again - expectThrows(IOException.class, () -> container.deleteBlob(blobName)); + expectThrows(NoSuchFileException.class, () -> container.deleteBlob(blobName)); + } + } + + public void testDeleteBlobIgnoringIfNotExists() throws IOException { + try (BlobStore store = newBlobStore()) { + BlobPath blobPath = new BlobPath(); + if (randomBoolean()) { + blobPath = blobPath.add(randomAlphaOfLengthBetween(1, 10)); + } + + final BlobContainer container = store.blobContainer(blobPath); + container.deleteBlobIgnoringIfNotExists("does_not_exist"); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java index f5e834aa90c69..16aa9c428d7e9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestClient.java @@ -121,7 +121,7 @@ public ClientYamlTestResponse callApi(String apiName, Map params } String contentType = entity.getContentType().getValue(); //randomly test the GET with source param instead of GET/POST with body - if (sendBodyAsSourceParam(supportedMethods, contentType)) { + if (sendBodyAsSourceParam(supportedMethods, contentType, entity.getContentLength())) { logger.debug("sending the request body as source param with GET method"); queryStringParams.put("source", EntityUtils.toString(entity)); queryStringParams.put("source_content_type", contentType); @@ -177,14 +177,25 @@ public ClientYamlTestResponse callApi(String apiName, Map params } } - private static boolean sendBodyAsSourceParam(List supportedMethods, String contentType) { - if (supportedMethods.contains(HttpGet.METHOD_NAME)) { - if (contentType.startsWith(ContentType.APPLICATION_JSON.getMimeType()) || - contentType.startsWith(YAML_CONTENT_TYPE.getMimeType())) { - return RandomizedTest.rarely(); - } + private static boolean sendBodyAsSourceParam(List supportedMethods, String contentType, long contentLength) { + if (false == supportedMethods.contains(HttpGet.METHOD_NAME)) { + // The API doesn't claim to support GET anyway + return false; + } + if (contentLength < 0) { + // Negative length means "unknown" or "huge" in this case. Either way we can't send it as a parameter + return false; + } + if (contentLength > 2000) { + // Long bodies won't fit in the parameter and will cause a too_long_frame_exception + return false; + } + if (false == contentType.startsWith(ContentType.APPLICATION_JSON.getMimeType()) + && false == contentType.startsWith(YAML_CONTENT_TYPE.getMimeType())) { + // We can only encode JSON or YAML this way. + return false; } - return false; + return RandomizedTest.rarely(); } private ClientYamlSuiteRestApi restApi(String apiName) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java index 927f9b46c966a..950bb14eed9af 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.PathUtils; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestApi; import org.elasticsearch.test.rest.yaml.restspec.ClientYamlSuiteRestSpec; @@ -143,7 +144,19 @@ protected ClientYamlTestClient initClientYamlTestClient(ClientYamlSuiteRestSpec return new ClientYamlTestClient(restSpec, restClient, hosts, esVersion); } + /** + * Create parameters for this parameterized test. Uses the + * {@link ExecutableSection#XCONTENT_REGISTRY list} of executable sections + * defined in {@link ExecutableSection}. + */ public static Iterable createParameters() throws Exception { + return createParameters(ExecutableSection.XCONTENT_REGISTRY); + } + + /** + * Create parameters for this parameterized test. + */ + public static Iterable createParameters(NamedXContentRegistry executeableSectionRegistry) throws Exception { String[] paths = resolvePathsProperty(REST_TESTS_SUITE, ""); // default to all tests under the test root List tests = new ArrayList<>(); Map> yamlSuites = loadSuites(paths); @@ -151,7 +164,7 @@ public static Iterable createParameters() throws Exception { for (String api : yamlSuites.keySet()) { List yamlFiles = new ArrayList<>(yamlSuites.get(api)); for (Path yamlFile : yamlFiles) { - ClientYamlTestSuite restTestSuite = ClientYamlTestSuite.parse(api, yamlFile); + ClientYamlTestSuite restTestSuite = ClientYamlTestSuite.parse(executeableSectionRegistry, api, yamlFile); for (ClientYamlTestSection testSection : restTestSuite.getTestSections()) { tests.add(new Object[]{ new ClientYamlTestCandidate(restTestSuite, testSection) }); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSuite.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSuite.java index 72c83f632efb0..b9988128b02a4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSuite.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ClientYamlTestSuite.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.yaml.YamlXContent; @@ -40,7 +41,7 @@ * Supports a setup section and multiple test sections. */ public class ClientYamlTestSuite { - public static ClientYamlTestSuite parse(String api, Path file) throws IOException { + public static ClientYamlTestSuite parse(NamedXContentRegistry executeableSectionRegistry, String api, Path file) throws IOException { if (!Files.isRegularFile(file)) { throw new IllegalArgumentException(file.toAbsolutePath() + " is not a file"); } @@ -64,7 +65,7 @@ public static ClientYamlTestSuite parse(String api, Path file) throws IOExceptio } } - try (XContentParser parser = YamlXContent.yamlXContent.createParser(ExecutableSection.XCONTENT_REGISTRY, + try (XContentParser parser = YamlXContent.yamlXContent.createParser(executeableSectionRegistry, LoggingDeprecationHandler.INSTANCE, Files.newInputStream(file))) { return parse(api, filename, parser); } catch(Exception e) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java index 827457f4c2ae2..ce5ea1c1cde06 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java @@ -26,15 +26,18 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; + +import static java.util.Collections.unmodifiableList; /** * Represents a test fragment that can be executed (e.g. api call, assertion) */ public interface ExecutableSection { /** - * {@link NamedXContentRegistry} needed in the {@link XContentParser} before calling {@link ExecutableSection#parse(XContentParser)}. + * Default list of {@link ExecutableSection}s available for tests. */ - NamedXContentRegistry XCONTENT_REGISTRY = new NamedXContentRegistry(Arrays.asList( + List DEFAULT_EXECUTABLE_CONTEXTS = unmodifiableList(Arrays.asList( new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("do"), DoSection::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("set"), SetSection::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("match"), MatchAssertion::parse), @@ -46,6 +49,12 @@ public interface ExecutableSection { new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("lte"), LessThanOrEqualToAssertion::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("length"), LengthAssertion::parse))); + /** + * {@link NamedXContentRegistry} that parses the default list of + * {@link ExecutableSection}s available for tests. + */ + NamedXContentRegistry XCONTENT_REGISTRY = new NamedXContentRegistry(DEFAULT_EXECUTABLE_CONTEXTS); + static ExecutableSection parse(XContentParser parser) throws IOException { ParserUtils.advanceToFieldName(parser); String section = parser.currentName(); @@ -60,7 +69,7 @@ static ExecutableSection parse(XContentParser parser) throws IOException { } /** - * Get the location in the test that this was defined. + * Get the location in the test that this was defined. */ XContentLocation getLocation(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/tasks/MockTaskManager.java b/test/framework/src/main/java/org/elasticsearch/test/tasks/MockTaskManager.java index dec204537b917..41cdaefe03575 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/tasks/MockTaskManager.java +++ b/test/framework/src/main/java/org/elasticsearch/test/tasks/MockTaskManager.java @@ -57,7 +57,7 @@ public Task register(String type, String action, TaskAwareRequest request) { } catch (Exception e) { logger.warn( (Supplier) () -> new ParameterizedMessage( - "failed to notify task manager listener about unregistering the task with id {}", + "failed to notify task manager listener about registering the task with id {}", task.getId()), e); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java index 0becdffb7ea7d..44fd61e1693ad 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.http.HttpServerTransport; @@ -38,6 +39,7 @@ import org.elasticsearch.ingest.Processor; import org.elasticsearch.license.LicenseService; import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.persistent.PersistentTasksExecutor; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.AnalysisPlugin; import org.elasticsearch.plugins.ClusterPlugin; @@ -57,9 +59,9 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.watcher.ResourceWatcherService; -import org.elasticsearch.persistent.PersistentTasksExecutor; import org.elasticsearch.xpack.core.ssl.SSLService; +import java.io.IOException; import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; @@ -391,6 +393,11 @@ public List> getPersistentTasksExecutor(ClusterServic .collect(toList()); } + @Override + public void close() throws IOException { + IOUtils.close(plugins); + } + private List filterPlugins(Class type) { return plugins.stream().filter(x -> type.isAssignableFrom(x.getClass())).map(p -> ((T)p)) .collect(Collectors.toList()); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java index 4e9c6d1e5d754..a448cd9c65c17 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java @@ -197,7 +197,7 @@ import org.elasticsearch.xpack.security.rest.action.user.RestHasPrivilegesAction; import org.elasticsearch.xpack.security.rest.action.user.RestPutUserAction; import org.elasticsearch.xpack.security.rest.action.user.RestSetEnabledAction; -import org.elasticsearch.xpack.security.support.IndexLifecycleManager; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor; import org.elasticsearch.xpack.security.transport.filter.IPFilter; import org.elasticsearch.xpack.security.transport.netty4.SecurityNetty4HttpServerTransport; @@ -233,7 +233,7 @@ import static org.elasticsearch.xpack.core.XPackSettings.HTTP_SSL_ENABLED; import static org.elasticsearch.xpack.core.security.SecurityLifecycleServiceField.SECURITY_TEMPLATE_NAME; import static org.elasticsearch.xpack.security.SecurityLifecycleService.SECURITY_INDEX_NAME; -import static org.elasticsearch.xpack.security.support.IndexLifecycleManager.INTERNAL_INDEX_FORMAT; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_INDEX_FORMAT; public class Security extends Plugin implements ActionPlugin, IngestPlugin, NetworkPlugin, ClusterPlugin, DiscoveryPlugin, MapperPlugin, ExtensiblePlugin { @@ -424,8 +424,8 @@ Collection createComponents(Client client, ThreadPool threadPool, Cluste components.add(realms); components.add(reservedRealm); - securityLifecycleService.addSecurityIndexHealthChangeListener(nativeRoleMappingStore::onSecurityIndexHealthChange); - securityLifecycleService.addSecurityIndexOutOfDateListener(nativeRoleMappingStore::onSecurityIndexOutOfDateChange); + securityLifecycleService.securityIndex().addIndexHealthChangeListener(nativeRoleMappingStore::onSecurityIndexHealthChange); + securityLifecycleService.securityIndex().addIndexOutOfDateListener(nativeRoleMappingStore::onSecurityIndexOutOfDateChange); AuthenticationFailureHandler failureHandler = null; String extensionName = null; @@ -458,8 +458,8 @@ Collection createComponents(Client client, ThreadPool threadPool, Cluste } final CompositeRolesStore allRolesStore = new CompositeRolesStore(settings, fileRolesStore, nativeRolesStore, reservedRolesStore, rolesProviders, threadPool.getThreadContext(), getLicenseState()); - securityLifecycleService.addSecurityIndexHealthChangeListener(allRolesStore::onSecurityIndexHealthChange); - securityLifecycleService.addSecurityIndexOutOfDateListener(allRolesStore::onSecurityIndexOutOfDateChange); + securityLifecycleService.securityIndex().addIndexHealthChangeListener(allRolesStore::onSecurityIndexHealthChange); + securityLifecycleService.securityIndex().addIndexOutOfDateListener(allRolesStore::onSecurityIndexOutOfDateChange); // to keep things simple, just invalidate all cached entries on license change. this happens so rarely that the impact should be // minimal getLicenseState().addListener(allRolesStore::invalidateAll); @@ -886,7 +886,7 @@ public UnaryOperator> getIndexTemplateMetaDat templates.remove(SECURITY_TEMPLATE_NAME); final XContent xContent = XContentFactory.xContent(XContentType.JSON); final byte[] auditTemplate = TemplateUtils.loadTemplate("/" + IndexAuditTrail.INDEX_TEMPLATE_NAME + ".json", - Version.CURRENT.toString(), IndexLifecycleManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); + Version.CURRENT.toString(), SecurityIndexManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); try (XContentParser parser = xContent .createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, auditTemplate)) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityLifecycleService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityLifecycleService.java index fd9bf875b3465..099c9cc625b14 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityLifecycleService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityLifecycleService.java @@ -22,7 +22,7 @@ import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.security.audit.index.IndexAuditTrail; -import org.elasticsearch.xpack.security.support.IndexLifecycleManager; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import java.util.Arrays; import java.util.Collections; @@ -46,7 +46,7 @@ */ public class SecurityLifecycleService extends AbstractComponent implements ClusterStateListener { - public static final String INTERNAL_SECURITY_INDEX = IndexLifecycleManager.INTERNAL_SECURITY_INDEX; + public static final String INTERNAL_SECURITY_INDEX = SecurityIndexManager.INTERNAL_SECURITY_INDEX; public static final String SECURITY_INDEX_NAME = ".security"; private static final Version MIN_READ_VERSION = Version.V_5_0_0; @@ -55,7 +55,7 @@ public class SecurityLifecycleService extends AbstractComponent implements Clust private final ThreadPool threadPool; private final IndexAuditTrail indexAuditTrail; - private final IndexLifecycleManager securityIndex; + private final SecurityIndexManager securityIndex; public SecurityLifecycleService(Settings settings, ClusterService clusterService, ThreadPool threadPool, Client client, @@ -64,7 +64,7 @@ public SecurityLifecycleService(Settings settings, ClusterService clusterService this.settings = settings; this.threadPool = threadPool; this.indexAuditTrail = indexAuditTrail; - this.securityIndex = new IndexLifecycleManager(settings, client, SECURITY_INDEX_NAME); + this.securityIndex = new SecurityIndexManager(settings, client, SECURITY_INDEX_NAME); clusterService.addListener(this); clusterService.addLifecycleListener(new LifecycleListener() { @Override @@ -110,69 +110,10 @@ public void doRun() { } } - IndexLifecycleManager securityIndex() { + public SecurityIndexManager securityIndex() { return securityIndex; } - /** - * Returns {@code true} if the security index exists - */ - public boolean isSecurityIndexExisting() { - return securityIndex.indexExists(); - } - - /** - * Returns true if the security index does not exist or it exists and has the current - * value for the index.format index setting - */ - public boolean isSecurityIndexUpToDate() { - return securityIndex.isIndexUpToDate(); - } - - /** - * Returns true if the security index exists and all primary shards are active - */ - public boolean isSecurityIndexAvailable() { - return securityIndex.isAvailable(); - } - - /** - * Returns true if the security index does not exist or the mappings are up to date - * based on the version in the _meta field - */ - public boolean isSecurityIndexMappingUpToDate() { - return securityIndex().isMappingUpToDate(); - } - - /** - * Test whether the effective (active) version of the security mapping meets the - * requiredVersion. - * - * @return true if the effective version passes the predicate, or the security - * mapping does not exist (null version). Otherwise, false. - */ - public boolean checkSecurityMappingVersion(Predicate requiredVersion) { - return securityIndex.checkMappingVersion(requiredVersion); - } - - /** - * Adds a listener which will be notified when the security index health changes. The previous and - * current health will be provided to the listener so that the listener can determine if any action - * needs to be taken. - */ - public void addSecurityIndexHealthChangeListener(BiConsumer listener) { - securityIndex.addIndexHealthChangeListener(listener); - } - - /** - * Adds a listener which will be notified when the security index out of date value changes. The previous and - * current value will be provided to the listener so that the listener can determine if any action - * needs to be taken. - */ - void addSecurityIndexOutOfDateListener(BiConsumer listener) { - securityIndex.addIndexOutOfDateListener(listener); - } - // this is called in a lifecycle listener beforeStop on the cluster service private void close() { if (indexAuditTrail != null) { @@ -193,29 +134,13 @@ static boolean securityIndexMappingUpToDate(ClusterState clusterState, Logger lo } private static boolean checkMappingVersions(ClusterState clusterState, Logger logger, Predicate versionPredicate) { - return IndexLifecycleManager.checkIndexMappingVersionMatches(SECURITY_INDEX_NAME, clusterState, logger, versionPredicate); + return SecurityIndexManager.checkIndexMappingVersionMatches(SECURITY_INDEX_NAME, clusterState, logger, versionPredicate); } public static List indexNames() { return Collections.unmodifiableList(Arrays.asList(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)); } - /** - * Prepares the security index by creating it if it doesn't exist or updating the mappings if the mappings are - * out of date. After any tasks have been executed, the runnable is then executed. - */ - public void prepareIndexIfNeededThenExecute(final Consumer consumer, final Runnable andThen) { - securityIndex.prepareIndexIfNeededThenExecute(consumer, andThen); - } - - /** - * Checks if the security index is out of date with the current version. If the index does not exist - * we treat the index as up to date as we expect it to be created with the current format. - */ - public boolean isSecurityIndexOutOfDate() { - return securityIndex.isIndexUpToDate() == false; - } - /** * Is the move from {@code previousHealth} to {@code currentHealth} a move from an unhealthy ("RED") index state to a healthy * ("non-RED") state. diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrail.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrail.java index e16a09c8a2a44..590c2bc5ecd4e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrail.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrail.java @@ -57,7 +57,7 @@ import org.elasticsearch.xpack.security.audit.AuditLevel; import org.elasticsearch.xpack.security.audit.AuditTrail; import org.elasticsearch.xpack.security.rest.RemoteHostHeader; -import org.elasticsearch.xpack.security.support.IndexLifecycleManager; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.security.transport.filter.SecurityIpFilterRule; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -105,7 +105,7 @@ import static org.elasticsearch.xpack.security.audit.AuditUtil.indices; import static org.elasticsearch.xpack.security.audit.AuditUtil.restRequestContent; import static org.elasticsearch.xpack.security.audit.index.IndexNameResolver.resolve; -import static org.elasticsearch.xpack.security.support.IndexLifecycleManager.SECURITY_VERSION_STRING; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_VERSION_STRING; /** * Audit trail implementation that writes events into an index. @@ -1001,7 +1001,7 @@ private void putTemplate(Settings customSettings, Consumer consumer) private PutIndexTemplateRequest getPutIndexTemplateRequest(Settings customSettings) { final byte[] template = TemplateUtils.loadTemplate("/" + INDEX_TEMPLATE_NAME + ".json", - Version.CURRENT.toString(), IndexLifecycleManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); + Version.CURRENT.toString(), SecurityIndexManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); final PutIndexTemplateRequest request = new PutIndexTemplateRequest(INDEX_TEMPLATE_NAME).source(template, XContentType.JSON); if (customSettings != null && customSettings.names().size() > 0) { Settings updatedSettings = Settings.builder() diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java index 017f4a6e04990..6e97071cea994 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java @@ -96,7 +96,7 @@ public static Map getFactories(ThreadPool threadPool, Res map.put(FileRealmSettings.TYPE, config -> new FileRealm(config, resourceWatcherService)); map.put(NativeRealmSettings.TYPE, config -> { final NativeRealm nativeRealm = new NativeRealm(config, nativeUsersStore); - securityLifecycleService.addSecurityIndexHealthChangeListener(nativeRealm::onSecurityIndexHealthChange); + securityLifecycleService.securityIndex().addIndexHealthChangeListener(nativeRealm::onSecurityIndexHealthChange); return nativeRealm; }); map.put(LdapRealmSettings.AD_TYPE, config -> new LdapRealm(LdapRealmSettings.AD_TYPE, config, sslService, diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java index 305c6caeba649..28098faa50ea6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java @@ -250,7 +250,7 @@ public void createUserToken(Authentication authentication, Authentication origin .setSource(builder) .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) .request(); - lifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client, SECURITY_ORIGIN, IndexAction.INSTANCE, request, ActionListener.wrap(indexResponse -> listener.onResponse(new Tuple<>(userToken, refreshToken)), listener::onFailure)) @@ -354,7 +354,7 @@ void decodeToken(String token, ActionListener listener) throws IOExce if (version.onOrAfter(Version.V_6_2_0)) { // we only have the id and need to get the token from the doc! decryptTokenId(in, cipher, version, ActionListener.wrap(tokenId -> - lifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { final GetRequest getRequest = client.prepareGet(SecurityLifecycleService.SECURITY_INDEX_NAME, TYPE, getTokenDocumentId(tokenId)).request(); @@ -524,7 +524,7 @@ private void indexBwcInvalidation(UserToken userToken, ActionListener l .request(); final String tokenDocId = getTokenDocumentId(userToken); final Version version = userToken.getVersion(); - lifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, indexRequest, ActionListener.wrap(indexResponse -> { ActionListener wrappedListener = @@ -566,7 +566,7 @@ private void indexInvalidation(String tokenDocId, Version version, ActionListene .setVersion(documentVersion) .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) .request(); - lifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, request, ActionListener.wrap(updateResponse -> { if (updateResponse.getGetResult() != null @@ -665,7 +665,7 @@ private void findTokenFromRefreshToken(String refreshToken, ActionListener + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, request, ActionListener.wrap(searchResponse -> { if (searchResponse.isTimedOut()) { @@ -847,7 +847,7 @@ public void findActiveTokensForRealm(String realmName, ActionListener supplier = client.threadPool().getThreadContext().newRestorableContext(false); - lifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> ScrollHelper.fetchAllByEntity(client, request, new ContextPreservingActionListener<>(supplier, listener), this::parseHit)); } @@ -914,11 +914,11 @@ private void ensureEnabled() { * have been explicitly cleared. */ private void checkIfTokenIsRevoked(UserToken userToken, ActionListener listener) { - if (lifecycleService.isSecurityIndexExisting() == false) { + if (lifecycleService.securityIndex().indexExists() == false) { // index doesn't exist so the token is considered valid. listener.onResponse(userToken); } else { - lifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + lifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { MultiGetRequest mGetRequest = client.prepareMultiGet() .add(SecurityLifecycleService.SECURITY_INDEX_NAME, TYPE, getInvalidatedTokenDocumentId(userToken)) .add(SecurityLifecycleService.SECURITY_INDEX_NAME, TYPE, getTokenDocumentId(userToken)) @@ -989,7 +989,7 @@ private Instant getExpirationTime(Instant now) { } private void maybeStartTokenRemover() { - if (lifecycleService.isSecurityIndexAvailable()) { + if (lifecycleService.securityIndex().isAvailable()) { if (client.threadPool().relativeTimeInMillis() - lastExpirationRunMs > deleteInterval.getMillis()) { expiredTokenRemover.submit(client.threadPool()); lastExpirationRunMs = client.threadPool().relativeTimeInMillis(); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java index d4d71523fea50..381053d9633d7 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java @@ -114,7 +114,7 @@ public void getUsers(String[] userNames, final ActionListener> } }; - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { // TODO remove this short circuiting and fix tests that fail without this! listener.onResponse(Collections.emptyList()); } else if (userNames.length == 1) { // optimization for single user lookup @@ -123,7 +123,7 @@ public void getUsers(String[] userNames, final ActionListener> (uap) -> listener.onResponse(uap == null ? Collections.emptyList() : Collections.singletonList(uap.user())), handleException)); } else { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { final QueryBuilder query; if (userNames == null || userNames.length == 0) { query = QueryBuilders.termQuery(Fields.TYPE.getPreferredName(), USER_DOC_TYPE); @@ -154,11 +154,11 @@ public void getUsers(String[] userNames, final ActionListener> * Async method to retrieve a user and their password */ private void getUserAndPassword(final String user, final ActionListener listener) { - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { // TODO remove this short circuiting and fix tests that fail without this! listener.onResponse(null); } else { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareGet(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(USER_DOC_TYPE, user)).request(), @@ -199,7 +199,7 @@ public void changePassword(final ChangePasswordRequest request, final ActionList docType = USER_DOC_TYPE; } - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareUpdate(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(docType, username)) .setDoc(Requests.INDEX_CONTENT_TYPE, Fields.PASSWORD.getPreferredName(), @@ -237,7 +237,7 @@ public void onFailure(Exception e) { * has been indexed */ private void createReservedUser(String username, char[] passwordHash, RefreshPolicy refresh, ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareIndex(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(RESERVED_USER_TYPE, username)) @@ -279,7 +279,7 @@ public void putUser(final PutUserRequest request, final ActionListener private void updateUserWithoutPassword(final PutUserRequest putUserRequest, final ActionListener listener) { assert putUserRequest.passwordHash() == null; // We must have an existing document - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareUpdate(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(USER_DOC_TYPE, putUserRequest.username())) @@ -322,7 +322,7 @@ public void onFailure(Exception e) { private void indexUser(final PutUserRequest putUserRequest, final ActionListener listener) { assert putUserRequest.passwordHash() != null; - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareIndex(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(USER_DOC_TYPE, putUserRequest.username())) @@ -366,7 +366,7 @@ public void setEnabled(final String username, final boolean enabled, final Refre private void setRegularUserEnabled(final String username, final boolean enabled, final RefreshPolicy refreshPolicy, final ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareUpdate(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(USER_DOC_TYPE, username)) @@ -401,7 +401,7 @@ public void onFailure(Exception e) { private void setReservedUserEnabled(final String username, final boolean enabled, final RefreshPolicy refreshPolicy, boolean clearCache, final ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareUpdate(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(RESERVED_USER_TYPE, username)) @@ -431,7 +431,7 @@ public void onFailure(Exception e) { } public void deleteUser(final DeleteUserRequest deleteUserRequest, final ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { DeleteRequest request = client.prepareDelete(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(USER_DOC_TYPE, deleteUserRequest.username())).request(); request.setRefreshPolicy(deleteUserRequest.getRefreshPolicy()); @@ -470,11 +470,11 @@ void verifyPassword(String username, final SecureString password, ActionListener } void getReservedUserInfo(String username, ActionListener listener) { - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { // TODO remove this short circuiting and fix tests that fail without this! listener.onResponse(null); } else { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareGet(SECURITY_INDEX_NAME, INDEX_TYPE, getIdForUser(RESERVED_USER_TYPE, username)).request(), @@ -514,7 +514,7 @@ public void onFailure(Exception e) { } void getAllReservedUserInfo(ActionListener> listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareSearch(SECURITY_INDEX_NAME) .setQuery(QueryBuilders.termQuery(Fields.TYPE.getPreferredName(), RESERVED_USER_TYPE)) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java index 601942b694a76..199a1c1968408 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java @@ -191,7 +191,7 @@ private void getUserInfo(final String username, ActionListener if (userIsDefinedForCurrentSecurityMapping(username) == false) { logger.debug("Marking user [{}] as disabled because the security mapping is not at the required version", username); listener.onResponse(DISABLED_DEFAULT_USER_INFO.deepClone()); - } else if (securityLifecycleService.isSecurityIndexExisting() == false) { + } else if (securityLifecycleService.securityIndex().indexExists() == false) { listener.onResponse(getDefaultUserInfo(username)); } else { nativeUsersStore.getReservedUserInfo(username, ActionListener.wrap((userInfo) -> { @@ -218,7 +218,7 @@ private ReservedUserInfo getDefaultUserInfo(String username) { private boolean userIsDefinedForCurrentSecurityMapping(String username) { final Version requiredVersion = getDefinedVersion(username); - return securityLifecycleService.checkSecurityMappingVersion(requiredVersion::onOrBefore); + return securityLifecycleService.securityIndex().checkMappingVersion(requiredVersion::onOrBefore); } private Version getDefinedVersion(String username) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java index 0fcaf297c0f59..bd26d778c0d6f 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java @@ -120,7 +120,7 @@ private String getIdForName(String name) { * package private for unit testing */ void loadMappings(ActionListener> listener) { - if (securityLifecycleService.isSecurityIndexOutOfDate()) { + if (securityLifecycleService.securityIndex().isIndexUpToDate() == false) { listener.onFailure(new IllegalStateException( "Security index is not on the current version - the native realm will not be operational until " + "the upgrade API is run on the security index")); @@ -176,7 +176,7 @@ public void deleteRoleMapping(DeleteRoleMappingRequest request, ActionListener void modifyMapping(String name, CheckedBiConsumer, Exception> inner, Request request, ActionListener listener) { - if (securityLifecycleService.isSecurityIndexOutOfDate()) { + if (securityLifecycleService.securityIndex().isIndexUpToDate() == false) { listener.onFailure(new IllegalStateException( "Security index is not on the current version - the native realm will not be operational until " + "the upgrade API is run on the security index")); @@ -192,7 +192,7 @@ private void modifyMapping(String name, CheckedBiConsumer listener) { final ExpressionRoleMapping mapping = request.getMapping(); - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { final XContentBuilder xContentBuilder; try { xContentBuilder = mapping.toXContent(jsonBuilder(), ToXContent.EMPTY_PARAMS, true); @@ -222,7 +222,7 @@ public void onFailure(Exception e) { } private void innerDeleteMapping(DeleteRoleMappingRequest request, ActionListener listener) throws IOException { - if (securityLifecycleService.isSecurityIndexOutOfDate()) { + if (securityLifecycleService.securityIndex().isIndexUpToDate() == false) { listener.onFailure(new IllegalStateException( "Security index is not on the current version - the native realm will not be operational until " + "the upgrade API is run on the security index")); @@ -276,16 +276,16 @@ public void onFailure(Exception e) { } private void getMappings(ActionListener> listener) { - if (securityLifecycleService.isSecurityIndexAvailable()) { + if (securityLifecycleService.securityIndex().isAvailable()) { loadMappings(listener); } else { logger.info("The security index is not yet available - no role mappings can be loaded"); if (logger.isDebugEnabled()) { logger.debug("Security Index [{}] [exists: {}] [available: {}] [mapping up to date: {}]", SECURITY_INDEX_NAME, - securityLifecycleService.isSecurityIndexExisting(), - securityLifecycleService.isSecurityIndexAvailable(), - securityLifecycleService.isSecurityIndexMappingUpToDate() + securityLifecycleService.securityIndex().indexExists(), + securityLifecycleService.securityIndex().isAvailable(), + securityLifecycleService.securityIndex().isMappingUpToDate() ); } listener.onResponse(Collections.emptyList()); @@ -302,7 +302,7 @@ private void getMappings(ActionListener> listener) { * */ public void usageStats(ActionListener> listener) { - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { reportStats(listener, Collections.emptyList()); } else { getMappings(ActionListener.wrap(mappings -> reportStats(listener, mappings), listener::onFailure)); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java index 4f0bb5b2e3c32..834a70b9e0304 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java @@ -100,7 +100,7 @@ public NativeRolesStore(Settings settings, Client client, XPackLicenseState lice * Retrieve a list of roles, if rolesToGet is null or empty, fetch all roles */ public void getRoleDescriptors(String[] names, final ActionListener> listener) { - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { // TODO remove this short circuiting and fix tests that fail without this! listener.onResponse(Collections.emptyList()); } else if (names != null && names.length == 1) { @@ -108,7 +108,7 @@ public void getRoleDescriptors(String[] names, final ActionListener { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { QueryBuilder query; if (names == null || names.length == 0) { query = QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE); @@ -133,7 +133,7 @@ public void getRoleDescriptors(String[] names, final ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { DeleteRequest request = client.prepareDelete(SecurityLifecycleService.SECURITY_INDEX_NAME, ROLE_DOC_TYPE, getIdForUser(deleteRoleRequest.name())).request(); request.setRefreshPolicy(deleteRoleRequest.getRefreshPolicy()); @@ -166,7 +166,7 @@ public void putRole(final PutRoleRequest request, final RoleDescriptor role, fin // pkg-private for testing void innerPutRole(final PutRoleRequest request, final RoleDescriptor role, final ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> { final XContentBuilder xContentBuilder; try { xContentBuilder = role.toXContent(jsonBuilder(), ToXContent.EMPTY_PARAMS, true); @@ -197,13 +197,13 @@ public void onFailure(Exception e) { public void usageStats(ActionListener> listener) { Map usageStats = new HashMap<>(); - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { usageStats.put("size", 0L); usageStats.put("fls", false); usageStats.put("dls", false); listener.onResponse(usageStats); } else { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareMultiSearch() .add(client.prepareSearch(SecurityLifecycleService.SECURITY_INDEX_NAME) @@ -259,11 +259,11 @@ public void onFailure(Exception e) { } private void getRoleDescriptor(final String roleId, ActionListener roleActionListener) { - if (securityLifecycleService.isSecurityIndexExisting() == false) { + if (securityLifecycleService.securityIndex().indexExists() == false) { // TODO remove this short circuiting and fix tests that fail without this! roleActionListener.onResponse(null); } else { - securityLifecycleService.prepareIndexIfNeededThenExecute(roleActionListener::onFailure, () -> + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(roleActionListener::onFailure, () -> executeGetRoleRequest(roleId, new ActionListener() { @Override public void onResponse(GetResponse response) { @@ -288,7 +288,7 @@ public void onFailure(Exception e) { } private void executeGetRoleRequest(String role, ActionListener listener) { - securityLifecycleService.prepareIndexIfNeededThenExecute(listener::onFailure, () -> + securityLifecycleService.securityIndex().prepareIndexIfNeededThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareGet(SecurityLifecycleService.SECURITY_INDEX_NAME, ROLE_DOC_TYPE, getIdForUser(role)).request(), diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/IndexLifecycleManager.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java similarity index 98% rename from x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/IndexLifecycleManager.java rename to x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java index e2e278c70820f..bfa02ed17c2c2 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/IndexLifecycleManager.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java @@ -58,7 +58,7 @@ /** * Manages the lifecycle of a single index, its template, mapping and and data upgrades/migrations. */ -public class IndexLifecycleManager extends AbstractComponent { +public class SecurityIndexManager extends AbstractComponent { public static final String INTERNAL_SECURITY_INDEX = ".security-" + IndexUpgradeCheckVersion.UPRADE_VERSION; public static final int INTERNAL_INDEX_FORMAT = 6; @@ -74,7 +74,7 @@ public class IndexLifecycleManager extends AbstractComponent { private volatile State indexState = new State(false, false, false, false, null); - public IndexLifecycleManager(Settings settings, Client client, String indexName) { + public SecurityIndexManager(Settings settings, Client client, String indexName) { super(settings); this.client = client; this.indexName = indexName; @@ -347,7 +347,7 @@ public void onFailure(Exception e) { private Tuple loadMappingAndSettingsSourceFromTemplate() { final byte[] template = TemplateUtils.loadTemplate("/" + SECURITY_TEMPLATE_NAME + ".json", - Version.CURRENT.toString(), IndexLifecycleManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); + Version.CURRENT.toString(), SecurityIndexManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); PutIndexTemplateRequest request = new PutIndexTemplateRequest(SECURITY_TEMPLATE_NAME).source(template, XContentType.JSON); return new Tuple<>(request.mappings().get("doc"), request.settings()); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityLifecycleServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityLifecycleServiceTests.java index af00d4ac616e0..02d99fe0edc92 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityLifecycleServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityLifecycleServiceTests.java @@ -37,7 +37,7 @@ import org.elasticsearch.transport.MockTransportClient; import org.elasticsearch.xpack.core.security.SecurityLifecycleServiceField; import org.elasticsearch.xpack.security.audit.index.IndexAuditTrail; -import org.elasticsearch.xpack.security.support.IndexLifecycleManager; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.security.test.SecurityTestUtils; import org.elasticsearch.xpack.core.template.TemplateUtils; import org.junit.After; @@ -105,10 +105,10 @@ public void testIndexTemplateVersionMatching() throws Exception { ClusterState.Builder clusterStateBuilder = createClusterStateWithTemplate(templateString); final ClusterState clusterState = clusterStateBuilder.build(); - assertTrue(IndexLifecycleManager.checkTemplateExistsAndVersionMatches( + assertTrue(SecurityIndexManager.checkTemplateExistsAndVersionMatches( SecurityLifecycleServiceField.SECURITY_TEMPLATE_NAME, clusterState, logger, Version.V_5_0_0::before)); - assertFalse(IndexLifecycleManager.checkTemplateExistsAndVersionMatches( + assertFalse(SecurityIndexManager.checkTemplateExistsAndVersionMatches( SecurityLifecycleServiceField.SECURITY_TEMPLATE_NAME, clusterState, logger, Version.V_5_0_0::after)); } @@ -126,7 +126,7 @@ public void testMappingVersionMatching() throws IOException { ClusterState.Builder clusterStateBuilder = createClusterStateWithMappingAndTemplate(templateString); securityLifecycleService.clusterChanged(new ClusterChangedEvent("test-event", clusterStateBuilder.build(), EMPTY_CLUSTER_STATE)); - final IndexLifecycleManager securityIndex = securityLifecycleService.securityIndex(); + final SecurityIndexManager securityIndex = securityLifecycleService.securityIndex(); assertTrue(securityIndex.checkMappingVersion(Version.V_5_0_0::before)); assertFalse(securityIndex.checkMappingVersion(Version.V_5_0_0::after)); } @@ -172,7 +172,7 @@ private ClusterState.Builder createClusterStateWithMappingAndTemplate(String sec private static IndexMetaData.Builder createIndexMetadata(String indexName, String templateString) throws IOException { String template = TemplateUtils.loadTemplate(templateString, Version.CURRENT.toString(), - IndexLifecycleManager.TEMPLATE_VERSION_PATTERN); + SecurityIndexManager.TEMPLATE_VERSION_PATTERN); PutIndexTemplateRequest request = new PutIndexTemplateRequest(); request.source(template, XContentType.JSON); IndexMetaData.Builder indexMetaData = IndexMetaData.builder(indexName); @@ -219,7 +219,7 @@ private static IndexTemplateMetaData.Builder getIndexTemplateMetaData( String templateName, String templateString) throws IOException { String template = TemplateUtils.loadTemplate(templateString, Version.CURRENT.toString(), - IndexLifecycleManager.TEMPLATE_VERSION_PATTERN); + SecurityIndexManager.TEMPLATE_VERSION_PATTERN); PutIndexTemplateRequest request = new PutIndexTemplateRequest(); request.source(template, XContentType.JSON); IndexTemplateMetaData.Builder templateBuilder = IndexTemplateMetaData.builder(templateName) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index 7d751a802463e..988f60fe57e4f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -63,7 +63,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_FORMAT_SETTING; import static org.elasticsearch.xpack.security.SecurityLifecycleService.SECURITY_INDEX_NAME; -import static org.elasticsearch.xpack.security.support.IndexLifecycleManager.INTERNAL_INDEX_FORMAT; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_INDEX_FORMAT; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java index a0755f9cd0682..52a2e537d8db5 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java @@ -67,6 +67,7 @@ import org.elasticsearch.xpack.security.authc.saml.SamlRealmTestHelper; import org.elasticsearch.xpack.security.authc.saml.SamlRealmTests; import org.elasticsearch.xpack.security.authc.saml.SamlTestCase; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.After; import org.junit.Before; import org.opensaml.saml.saml2.core.NameID; @@ -161,10 +162,12 @@ void doExecute(Action action, Request request }; final SecurityLifecycleService lifecycleService = mock(SecurityLifecycleService.class); + final SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(lifecycleService.securityIndex()).thenReturn(securityIndex); doAnswer(inv -> { ((Runnable) inv.getArguments()[1]).run(); return null; - }).when(lifecycleService).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); final ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); tokenService = new TokenService(settings, Clock.systemUTC(), client, lifecycleService, clusterService); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java index 50a7a35b7a682..93e6ebf2861cf 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java @@ -56,6 +56,7 @@ import org.elasticsearch.xpack.security.authc.saml.SamlRealmTests; import org.elasticsearch.xpack.security.authc.saml.SamlTestCase; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.After; import org.junit.Before; import org.opensaml.saml.saml2.core.NameID; @@ -173,10 +174,12 @@ public void setup() throws Exception { }).when(client).execute(eq(IndexAction.INSTANCE), any(IndexRequest.class), any(ActionListener.class)); final SecurityLifecycleService lifecycleService = mock(SecurityLifecycleService.class); + final SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(lifecycleService.securityIndex()).thenReturn(securityIndex); doAnswer(inv -> { ((Runnable) inv.getArguments()[1]).run(); return null; - }).when(lifecycleService).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); final ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); tokenService = new TokenService(settings, Clock.systemUTC(), client, lifecycleService, clusterService); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java index b23fccec018dd..02af431f8978b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.security.authc.esnative.NativeUsersStore; import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm; import org.elasticsearch.xpack.security.authc.esnative.ReservedRealmTests; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.Before; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -76,7 +77,9 @@ public void maybeEnableAnonymous() { public void testAnonymousUser() { NativeUsersStore usersStore = mock(NativeUsersStore.class); SecurityLifecycleService securityLifecycleService = mock(SecurityLifecycleService.class); - when(securityLifecycleService.isSecurityIndexAvailable()).thenReturn(true); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(securityLifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); AnonymousUser anonymousUser = new AnonymousUser(settings); ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, anonymousUser, securityLifecycleService, new ThreadContext(Settings.EMPTY)); @@ -146,8 +149,10 @@ public void onFailure(Exception e) { public void testReservedUsersOnly() { NativeUsersStore usersStore = mock(NativeUsersStore.class); SecurityLifecycleService securityLifecycleService = mock(SecurityLifecycleService.class); - when(securityLifecycleService.isSecurityIndexAvailable()).thenReturn(true); - when(securityLifecycleService.checkSecurityMappingVersion(any())).thenReturn(true); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(securityLifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); + when(securityIndex.checkMappingVersion(any())).thenReturn(true); ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); ReservedRealm reservedRealm = @@ -194,7 +199,9 @@ public void testGetAllUsers() { Arrays.asList(new User("jane"), new User("fred")), randomUsers()); NativeUsersStore usersStore = mock(NativeUsersStore.class); SecurityLifecycleService securityLifecycleService = mock(SecurityLifecycleService.class); - when(securityLifecycleService.isSecurityIndexAvailable()).thenReturn(true); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(securityLifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(settings), securityLifecycleService, new ThreadContext(Settings.EMPTY)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java index bab047951e50a..7b26e605207a2 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.xpack.security.authc.esnative.NativeUsersStore; import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm; import org.elasticsearch.xpack.security.authc.esnative.ReservedRealmTests; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -118,7 +119,9 @@ public void onFailure(Exception e) { public void testReservedUser() { NativeUsersStore usersStore = mock(NativeUsersStore.class); SecurityLifecycleService securityLifecycleService = mock(SecurityLifecycleService.class); - when(securityLifecycleService.isSecurityIndexAvailable()).thenReturn(true); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(securityLifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); Settings settings = Settings.builder().put("path.home", createTempDir()).build(); ReservedRealm reservedRealm = new ReservedRealm(TestEnvironment.newEnvironment(settings), settings, usersStore, diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java index 0c75e36fa6c04..41b765cb33322 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java @@ -68,6 +68,7 @@ import org.elasticsearch.xpack.security.audit.AuditTrailService; import org.elasticsearch.xpack.security.authc.AuthenticationService.Authenticator; import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.After; import org.junit.Before; @@ -125,6 +126,7 @@ public class AuthenticationServiceTests extends ESTestCase { private ThreadContext threadContext; private TokenService tokenService; private SecurityLifecycleService lifecycleService; + private SecurityIndexManager securityIndex; private Client client; private InetSocketAddress remoteAddress; @@ -181,11 +183,13 @@ licenseState, threadContext, mock(ReservedRealm.class), Arrays.asList(firstRealm return builder; }).when(client).prepareGet(anyString(), anyString(), anyString()); lifecycleService = mock(SecurityLifecycleService.class); + securityIndex = mock(SecurityIndexManager.class); + when(lifecycleService.securityIndex()).thenReturn(securityIndex); doAnswer(invocationOnMock -> { Runnable runnable = (Runnable) invocationOnMock.getArguments()[1]; runnable.run(); return null; - }).when(lifecycleService).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); tokenService = new TokenService(settings, Clock.systemUTC(), client, lifecycleService, clusterService); service = new AuthenticationService(settings, realms, auditTrail, @@ -924,8 +928,8 @@ public void testInvalidToken() throws Exception { } public void testExpiredToken() throws Exception { - when(lifecycleService.isSecurityIndexAvailable()).thenReturn(true); - when(lifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.isAvailable()).thenReturn(true); + when(lifecycleService.securityIndex().indexExists()).thenReturn(true); User user = new User("_username", "r1"); final Authentication expected = new Authentication(user, new RealmRef("realm", "custom", "node"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); @@ -963,7 +967,7 @@ public void testExpiredToken() throws Exception { doAnswer(invocationOnMock -> { ((Runnable) invocationOnMock.getArguments()[1]).run(); return null; - }).when(lifecycleService).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { threadContext.putHeader("Authorization", "Bearer " + token); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/InternalRealmsTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/InternalRealmsTests.java index 91e8111b54c8d..f0af7a2539e42 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/InternalRealmsTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/InternalRealmsTests.java @@ -18,6 +18,7 @@ import org.elasticsearch.xpack.security.SecurityLifecycleService; import org.elasticsearch.xpack.security.authc.esnative.NativeUsersStore; import org.elasticsearch.xpack.security.authc.support.mapper.NativeRoleMappingStore; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import java.util.Map; import java.util.function.BiConsumer; @@ -30,11 +31,14 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; public class InternalRealmsTests extends ESTestCase { public void testNativeRealmRegistersIndexHealthChangeListener() throws Exception { SecurityLifecycleService lifecycleService = mock(SecurityLifecycleService.class); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(lifecycleService.securityIndex()).thenReturn(securityIndex); Map factories = InternalRealms.getFactories(mock(ThreadPool.class), mock(ResourceWatcherService.class), mock(SSLService.class), mock(NativeUsersStore.class), mock(NativeRoleMappingStore.class), lifecycleService); assertThat(factories, hasEntry(is(NativeRealmSettings.TYPE), any(Realm.Factory.class))); @@ -43,10 +47,10 @@ public void testNativeRealmRegistersIndexHealthChangeListener() throws Exception Settings settings = Settings.builder().put("path.home", createTempDir()).build(); factories.get(NativeRealmSettings.TYPE).create(new RealmConfig("test", Settings.EMPTY, settings, TestEnvironment.newEnvironment(settings), new ThreadContext(settings))); - verify(lifecycleService).addSecurityIndexHealthChangeListener(isA(BiConsumer.class)); + verify(securityIndex).addIndexHealthChangeListener(isA(BiConsumer.class)); factories.get(NativeRealmSettings.TYPE).create(new RealmConfig("test", Settings.EMPTY, settings, TestEnvironment.newEnvironment(settings), new ThreadContext(settings))); - verify(lifecycleService, times(2)).addSecurityIndexHealthChangeListener(isA(BiConsumer.class)); + verify(securityIndex, times(2)).addIndexHealthChangeListener(isA(BiConsumer.class)); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java index 9b40187394122..79a2647997505 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java @@ -51,6 +51,7 @@ import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.core.watcher.watch.ClockMock; import org.elasticsearch.xpack.security.SecurityLifecycleService; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -86,6 +87,7 @@ public class TokenServiceTests extends ESTestCase { private Client client; private SecurityLifecycleService lifecycleService; + private SecurityIndexManager securityIndex; private ClusterService clusterService; private Settings tokenServiceEnabledSettings = Settings.builder() .put(XPackSettings.TOKEN_SERVICE_ENABLED_SETTING.getKey(), true).build(); @@ -131,11 +133,13 @@ public void setupClient() { // setup lifecycle service lifecycleService = mock(SecurityLifecycleService.class); + securityIndex = mock(SecurityIndexManager.class); + when(lifecycleService.securityIndex()).thenReturn(securityIndex); doAnswer(invocationOnMock -> { Runnable runnable = (Runnable) invocationOnMock.getArguments()[1]; runnable.run(); return null; - }).when(lifecycleService).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); this.clusterService = ClusterServiceUtils.createClusterService(threadPool); } @@ -376,7 +380,7 @@ public void testGetTokenWhenKeyCacheHasExpired() throws Exception { } public void testInvalidatedToken() throws Exception { - when(lifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, lifecycleService, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); @@ -563,8 +567,8 @@ public void testIndexNotAvailable() throws Exception { UserToken serialized = future.get(); assertEquals(authentication, serialized.getAuthentication()); - when(lifecycleService.isSecurityIndexAvailable()).thenReturn(false); - when(lifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.isAvailable()).thenReturn(false); + when(securityIndex.indexExists()).thenReturn(true); future = new PlainActionFuture<>(); tokenService.getAndValidateToken(requestContext, future); assertNull(future.get()); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java index 36a49653645e8..2c11411955a0f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java @@ -55,7 +55,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; import static org.elasticsearch.xpack.security.SecurityLifecycleService.SECURITY_INDEX_NAME; -import static org.elasticsearch.xpack.security.support.IndexLifecycleManager.INTERNAL_SECURITY_INDEX; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_SECURITY_INDEX; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java index 51314c6437575..091f6f2ed4571 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.xpack.core.security.user.LogstashSystemUser; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.security.SecurityLifecycleService; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.Before; import java.io.IOException; @@ -236,16 +237,17 @@ private void respondToGetUserRequest(String username, SecureString password, Str private NativeUsersStore startNativeUsersStore() { SecurityLifecycleService securityLifecycleService = mock(SecurityLifecycleService.class); - when(securityLifecycleService.isSecurityIndexAvailable()).thenReturn(true); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); - when(securityLifecycleService.isSecurityIndexMappingUpToDate()).thenReturn(true); - when(securityLifecycleService.isSecurityIndexOutOfDate()).thenReturn(false); - when(securityLifecycleService.isSecurityIndexUpToDate()).thenReturn(true); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(securityLifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); + when(securityIndex.isMappingUpToDate()).thenReturn(true); + when(securityIndex.isIndexUpToDate()).thenReturn(true); doAnswer((i) -> { Runnable action = (Runnable) i.getArguments()[1]; action.run(); return null; - }).when(securityLifecycleService).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); return new NativeUsersStore(Settings.EMPTY, client, securityLifecycleService); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java index 272af679d13ea..024f8f603c928 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.xpack.core.security.user.UsernamesField; import org.elasticsearch.xpack.security.SecurityLifecycleService; import org.elasticsearch.xpack.security.authc.esnative.NativeUsersStore.ReservedUserInfo; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -63,13 +64,16 @@ public class ReservedRealmTests extends ESTestCase { private static final SecureString EMPTY_PASSWORD = new SecureString("".toCharArray()); private NativeUsersStore usersStore; private SecurityLifecycleService securityLifecycleService; + private SecurityIndexManager securityIndex; @Before public void setupMocks() throws Exception { usersStore = mock(NativeUsersStore.class); securityLifecycleService = mock(SecurityLifecycleService.class); - when(securityLifecycleService.isSecurityIndexAvailable()).thenReturn(true); - when(securityLifecycleService.checkSecurityMappingVersion(any())).thenReturn(true); + securityIndex = mock(SecurityIndexManager.class); + when(securityLifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); + when(securityIndex.checkMappingVersion(any())).thenReturn(true); mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); } @@ -90,7 +94,7 @@ public void testAuthenticationDisabled() throws Throwable { Settings settings = Settings.builder().put(XPackSettings.RESERVED_REALM_ENABLED_SETTING.getKey(), false).build(); final boolean securityIndexExists = randomBoolean(); if (securityIndexExists) { - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); } final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, @@ -120,7 +124,7 @@ private void verifySuccessfulAuthentication(boolean enabled) throws Exception { final User expectedUser = randomReservedUser(enabled); final String principal = expectedUser.principal(); final SecureString newPassword = new SecureString("foobar".toCharArray()); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); doAnswer((i) -> { ActionListener callback = (ActionListener) i.getArguments()[1]; callback.onResponse(new ReservedUserInfo(Hasher.BCRYPT.hash(newPassword), enabled, false)); @@ -146,10 +150,10 @@ private void verifySuccessfulAuthentication(boolean enabled) throws Exception { assertEquals(expectedUser, authenticated); assertThat(expectedUser.enabled(), is(enabled)); - verify(securityLifecycleService, times(2)).isSecurityIndexExisting(); + verify(securityIndex, times(2)).indexExists(); verify(usersStore, times(2)).getReservedUserInfo(eq(principal), any(ActionListener.class)); final ArgumentCaptor predicateCaptor = ArgumentCaptor.forClass(Predicate.class); - verify(securityLifecycleService, times(2)).checkSecurityMappingVersion(predicateCaptor.capture()); + verify(securityIndex, times(2)).checkMappingVersion(predicateCaptor.capture()); verifyVersionPredicate(principal, predicateCaptor.getValue()); verifyNoMoreInteractions(usersStore); } @@ -165,10 +169,10 @@ public void testLookup() throws Exception { reservedRealm.doLookupUser(principal, listener); final User user = listener.actionGet(); assertEquals(expectedUser, user); - verify(securityLifecycleService).isSecurityIndexExisting(); + verify(securityIndex).indexExists(); final ArgumentCaptor predicateCaptor = ArgumentCaptor.forClass(Predicate.class); - verify(securityLifecycleService).checkSecurityMappingVersion(predicateCaptor.capture()); + verify(securityIndex).checkMappingVersion(predicateCaptor.capture()); verifyVersionPredicate(principal, predicateCaptor.getValue()); PlainActionFuture future = new PlainActionFuture<>(); @@ -199,7 +203,7 @@ public void testLookupThrows() throws Exception { new AnonymousUser(Settings.EMPTY), securityLifecycleService, new ThreadContext(Settings.EMPTY)); final User expectedUser = randomReservedUser(true); final String principal = expectedUser.principal(); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); final RuntimeException e = new RuntimeException("store threw"); doAnswer((i) -> { ActionListener callback = (ActionListener) i.getArguments()[1]; @@ -212,11 +216,11 @@ public void testLookupThrows() throws Exception { ElasticsearchSecurityException securityException = expectThrows(ElasticsearchSecurityException.class, future::actionGet); assertThat(securityException.getMessage(), containsString("failed to lookup")); - verify(securityLifecycleService).isSecurityIndexExisting(); + verify(securityIndex).indexExists(); verify(usersStore).getReservedUserInfo(eq(principal), any(ActionListener.class)); final ArgumentCaptor predicateCaptor = ArgumentCaptor.forClass(Predicate.class); - verify(securityLifecycleService).checkSecurityMappingVersion(predicateCaptor.capture()); + verify(securityIndex).checkMappingVersion(predicateCaptor.capture()); verifyVersionPredicate(principal, predicateCaptor.getValue()); verifyNoMoreInteractions(usersStore); @@ -269,7 +273,7 @@ public void testGetUsersDisabled() { } public void testFailedAuthentication() throws Exception { - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); SecureString password = new SecureString("password".toCharArray()); char[] hash = Hasher.BCRYPT.hash(password); ReservedUserInfo userInfo = new ReservedUserInfo(hash, true, false); @@ -302,7 +306,7 @@ public void testBootstrapElasticPasswordWorksOnceSecurityIndexExists() throws Ex MockSecureSettings mockSecureSettings = new MockSecureSettings(); mockSecureSettings.setString("bootstrap.password", "foobar"); Settings settings = Settings.builder().setSecureSettings(mockSecureSettings).build(); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(Settings.EMPTY), securityLifecycleService, new ThreadContext(Settings.EMPTY)); @@ -324,7 +328,7 @@ public void testBootstrapElasticPasswordFailsOnceElasticUserExists() throws Exce MockSecureSettings mockSecureSettings = new MockSecureSettings(); mockSecureSettings.setString("bootstrap.password", "foobar"); Settings settings = Settings.builder().setSecureSettings(mockSecureSettings).build(); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(Settings.EMPTY), securityLifecycleService, new ThreadContext(Settings.EMPTY)); @@ -351,7 +355,7 @@ public void testBootstrapElasticPasswordWorksBeforeSecurityIndexExists() throws MockSecureSettings mockSecureSettings = new MockSecureSettings(); mockSecureSettings.setString("bootstrap.password", "foobar"); Settings settings = Settings.builder().setSecureSettings(mockSecureSettings).build(); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(false); + when(securityIndex.indexExists()).thenReturn(false); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(Settings.EMPTY), securityLifecycleService, new ThreadContext(Settings.EMPTY)); @@ -369,7 +373,7 @@ public void testNonElasticUsersCannotUseBootstrapPasswordWhenSecurityIndexExists final String password = randomAlphaOfLengthBetween(8, 24); mockSecureSettings.setString("bootstrap.password", password); Settings settings = Settings.builder().setSecureSettings(mockSecureSettings).build(); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(true); + when(securityIndex.indexExists()).thenReturn(true); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(Settings.EMPTY), securityLifecycleService, new ThreadContext(Settings.EMPTY)); @@ -391,7 +395,7 @@ public void testNonElasticUsersCannotUseBootstrapPasswordWhenSecurityIndexDoesNo final String password = randomAlphaOfLengthBetween(8, 24); mockSecureSettings.setString("bootstrap.password", password); Settings settings = Settings.builder().setSecureSettings(mockSecureSettings).build(); - when(securityLifecycleService.isSecurityIndexExisting()).thenReturn(false); + when(securityIndex.indexExists()).thenReturn(false); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(Settings.EMPTY), securityLifecycleService, new ThreadContext(Settings.EMPTY)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java index 41fe340d05f41..3a67ab9447e32 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.xpack.security.SecurityLifecycleService; import org.elasticsearch.xpack.security.authc.support.CachingUsernamePasswordRealm; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.hamcrest.Matchers; import java.util.Arrays; @@ -75,7 +76,9 @@ public void testResolveRoles() throws Exception { final Client client = mock(Client.class); final SecurityLifecycleService lifecycleService = mock(SecurityLifecycleService.class); - when(lifecycleService.isSecurityIndexAvailable()).thenReturn(true); + SecurityIndexManager securityIndex = mock(SecurityIndexManager.class); + when(lifecycleService.securityIndex()).thenReturn(securityIndex); + when(securityIndex.isAvailable()).thenReturn(true); final NativeRoleMappingStore store = new NativeRoleMappingStore(Settings.EMPTY, client, lifecycleService) { @Override diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/IndexLifecycleManagerIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerIntegTests.java similarity index 97% rename from x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/IndexLifecycleManagerIntegTests.java rename to x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerIntegTests.java index 4934bcd93036f..677be9a94e7ce 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/IndexLifecycleManagerIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerIntegTests.java @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -public class IndexLifecycleManagerIntegTests extends SecurityIntegTestCase { +public class SecurityIndexManagerIntegTests extends SecurityIntegTestCase { public void testConcurrentOperationsTryingToCreateSecurityIndexAndAlias() throws Exception { assertSecurityIndexActive(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/IndexLifecycleManagerTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java similarity index 96% rename from x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/IndexLifecycleManagerTests.java rename to x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java index 9411042e36317..e85c8629f2c80 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/IndexLifecycleManagerTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java @@ -52,17 +52,17 @@ import org.junit.Before; import static org.elasticsearch.cluster.routing.RecoverySource.StoreRecoverySource.EXISTING_STORE_INSTANCE; -import static org.elasticsearch.xpack.security.support.IndexLifecycleManager.TEMPLATE_VERSION_PATTERN; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.TEMPLATE_VERSION_PATTERN; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class IndexLifecycleManagerTests extends ESTestCase { +public class SecurityIndexManagerTests extends ESTestCase { private static final ClusterName CLUSTER_NAME = new ClusterName("index-lifecycle-manager-tests"); private static final ClusterState EMPTY_CLUSTER_STATE = new ClusterState.Builder(CLUSTER_NAME).build(); - public static final String INDEX_NAME = "IndexLifecycleManagerTests"; - private static final String TEMPLATE_NAME = "IndexLifecycleManagerTests-template"; - private IndexLifecycleManager manager; + public static final String INDEX_NAME = "SecurityIndexManagerTests"; + private static final String TEMPLATE_NAME = "SecurityIndexManagerTests-template"; + private SecurityIndexManager manager; private Map, Map>> actions; @Before @@ -86,7 +86,7 @@ void doExecute(Action action, Request request actions.put(action, map); } }; - manager = new IndexLifecycleManager(Settings.EMPTY, client, INDEX_NAME); + manager = new SecurityIndexManager(Settings.EMPTY, client, INDEX_NAME); } public void testIndexWithUpToDateMappingAndTemplate() throws IOException { @@ -221,7 +221,7 @@ public void testIndexOutOfDateListeners() throws Exception { // index doesn't exist and now exists with wrong format ClusterState.Builder clusterStateBuilder = createClusterState(INDEX_NAME, TEMPLATE_NAME, - IndexLifecycleManager.INTERNAL_INDEX_FORMAT - 1); + SecurityIndexManager.INTERNAL_INDEX_FORMAT - 1); markShardsAvailable(clusterStateBuilder); manager.clusterChanged(event(clusterStateBuilder)); assertTrue(listenerCalled.get()); @@ -235,7 +235,7 @@ public void testIndexOutOfDateListeners() throws Exception { listenerCalled.set(false); // index doesn't exist and now exists with correct format - clusterStateBuilder = createClusterState(INDEX_NAME, TEMPLATE_NAME, IndexLifecycleManager.INTERNAL_INDEX_FORMAT); + clusterStateBuilder = createClusterState(INDEX_NAME, TEMPLATE_NAME, SecurityIndexManager.INTERNAL_INDEX_FORMAT); markShardsAvailable(clusterStateBuilder); manager.clusterChanged(event(clusterStateBuilder)); assertFalse(listenerCalled.get()); @@ -255,7 +255,7 @@ private void assertIndexUpToDateButNotAvailable() { } public static ClusterState.Builder createClusterState(String indexName, String templateName) throws IOException { - return createClusterState(indexName, templateName, templateName, IndexLifecycleManager.INTERNAL_INDEX_FORMAT); + return createClusterState(indexName, templateName, templateName, SecurityIndexManager.INTERNAL_INDEX_FORMAT); } public static ClusterState.Builder createClusterState(String indexName, String templateName, int format) throws IOException { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java index c46bac7e6efbc..075ce2772b394 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java @@ -22,11 +22,14 @@ import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.test.SecurityIntegTestCase; import org.elasticsearch.transport.Transport; import org.elasticsearch.xpack.core.TestXPackTransportClient; +import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.common.socket.SocketAccess; +import org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings; import org.elasticsearch.xpack.core.ssl.SSLService; import org.elasticsearch.xpack.security.LocalStateSecurity; @@ -39,7 +42,12 @@ import java.nio.charset.StandardCharsets; import java.security.KeyStore; import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.Locale; +import java.util.Set; import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForStore; import static org.hamcrest.CoreMatchers.is; @@ -65,12 +73,18 @@ protected boolean transportSSLEnabled() { } // no SSL exception as this is the exception is returned when connecting - public void testThatUnconfiguredCiphersAreRejected() { + public void testThatUnconfiguredCiphersAreRejected() throws Exception { + Set supportedCiphers = Sets.newHashSet(SSLContext.getDefault().getSupportedSSLParameters().getCipherSuites()); + Set defaultXPackCiphers = Sets.newHashSet(XPackSettings.DEFAULT_CIPHERS); + final List unconfiguredCiphers = new ArrayList<>(Sets.difference(supportedCiphers, defaultXPackCiphers)); + Collections.shuffle(unconfiguredCiphers, random()); + assumeFalse("the unconfigured ciphers list is empty", unconfiguredCiphers.isEmpty()); + try (TransportClient transportClient = new TestXPackTransportClient(Settings.builder() .put(transportClientSettings()) .put("node.name", "programmatic_transport_client") .put("cluster.name", internalCluster().getClusterName()) - .putList("xpack.ssl.cipher_suites", "TLS_ECDH_anon_WITH_RC4_128_SHA", "SSL_RSA_WITH_3DES_EDE_CBC_SHA") + .putList("xpack.ssl.cipher_suites", unconfiguredCiphers) .build(), LocalStateSecurity.class)) { TransportAddress transportAddress = randomFrom(internalCluster().getInstance(Transport.class).boundAddress().boundAddresses()); diff --git a/x-pack/plugin/security/src/test/resources/IndexLifecycleManagerTests-template.json b/x-pack/plugin/security/src/test/resources/SecurityIndexManagerTests-template.json similarity index 100% rename from x-pack/plugin/security/src/test/resources/IndexLifecycleManagerTests-template.json rename to x-pack/plugin/security/src/test/resources/SecurityIndexManagerTests-template.json diff --git a/x-pack/plugin/upgrade/src/test/java/org/elasticsearch/xpack/upgrade/IndexUpgradeIT.java b/x-pack/plugin/upgrade/src/test/java/org/elasticsearch/xpack/upgrade/IndexUpgradeIT.java index ef5c3acc3d238..9f1fb95ed4835 100644 --- a/x-pack/plugin/upgrade/src/test/java/org/elasticsearch/xpack/upgrade/IndexUpgradeIT.java +++ b/x-pack/plugin/upgrade/src/test/java/org/elasticsearch/xpack/upgrade/IndexUpgradeIT.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.upgrade; +import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.Build; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.action.search.SearchResponse; @@ -30,6 +31,7 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.core.IsEqual.equalTo; +@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/30430") public class IndexUpgradeIT extends IndexUpgradeIntegTestCase { @Before diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index 6c4ac1994ffce..1bd1aa59f7000 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexModule; @@ -216,6 +217,7 @@ public class Watcher extends Plugin implements ActionPlugin, ScriptPlugin { private static final Logger logger = Loggers.getLogger(Watcher.class); private WatcherIndexingListener listener; + private HttpClient httpClient; protected final Settings settings; protected final boolean transportClient; @@ -266,7 +268,7 @@ public Collection createComponents(Client client, ClusterService cluster // TODO: add more auth types, or remove this indirection HttpAuthRegistry httpAuthRegistry = new HttpAuthRegistry(httpAuthFactories); HttpRequestTemplate.Parser httpTemplateParser = new HttpRequestTemplate.Parser(httpAuthRegistry); - final HttpClient httpClient = new HttpClient(settings, httpAuthRegistry, getSslService()); + httpClient = new HttpClient(settings, httpAuthRegistry, getSslService()); // notification EmailService emailService = new EmailService(settings, cryptoService, clusterService.getClusterSettings()); @@ -608,4 +610,9 @@ public List getBootstrapChecks() { public List getContexts() { return Arrays.asList(Watcher.SCRIPT_SEARCH_CONTEXT, Watcher.SCRIPT_EXECUTABLE_CONTEXT, Watcher.SCRIPT_TEMPLATE_CONTEXT); } + + @Override + public void close() throws IOException { + IOUtils.closeWhileHandlingException(httpClient); + } } diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleService.java index eef9e019b7a7e..620d575fc802c 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleService.java @@ -110,8 +110,7 @@ public void clusterChanged(ClusterChangedEvent event) { // if this is not a data node, we need to start it ourselves possibly if (event.state().nodes().getLocalNode().isDataNode() == false && isWatcherStoppedManually == false && this.state.get() == WatcherState.STOPPED) { - watcherService.start(event.state()); - this.state.set(WatcherState.STARTED); + watcherService.start(event.state(), () -> this.state.set(WatcherState.STARTED)); return; } @@ -157,8 +156,8 @@ public void clusterChanged(ClusterChangedEvent event) { if (state.get() == WatcherState.STARTED) { watcherService.reload(event.state(), "new local watcher shard allocation ids"); } else if (state.get() == WatcherState.STOPPED) { - watcherService.start(event.state()); - this.state.set(WatcherState.STARTED); + this.state.set(WatcherState.STARTING); + watcherService.start(event.state(), () -> this.state.set(WatcherState.STARTED)); } } else { clearAllocationIds(); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java index dcfb713a66580..49915674fe9e2 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherService.java @@ -183,23 +183,40 @@ void reload(ClusterState state, String reason) { // by checking the cluster state version before and after loading the watches we can potentially just exit without applying the // changes processedClusterStateVersion.set(state.getVersion()); - pauseExecution(reason); triggerService.pauseExecution(); + int cancelledTaskCount = executionService.clearExecutionsAndQueue(); + logger.info("reloading watcher, reason [{}], cancelled [{}] queued tasks", reason, cancelledTaskCount); executor.execute(wrapWatcherService(() -> reloadInner(state, reason, false), e -> logger.error("error reloading watcher", e))); } - public void start(ClusterState state) { + /** + * start the watcher service, load watches in the background + * + * @param state the current cluster state + * @param postWatchesLoadedCallback the callback to be triggered, when watches where loaded successfully + */ + public void start(ClusterState state, Runnable postWatchesLoadedCallback) { + executionService.unPause(); processedClusterStateVersion.set(state.getVersion()); - executor.execute(wrapWatcherService(() -> reloadInner(state, "starting", true), + executor.execute(wrapWatcherService(() -> { + if (reloadInner(state, "starting", true)) { + postWatchesLoadedCallback.run(); + } + }, e -> logger.error("error starting watcher", e))); } /** - * reload the watches and start scheduling them + * reload watches and start scheduling them + * + * @param state the current cluster state + * @param reason the reason for reloading, will be logged + * @param loadTriggeredWatches should triggered watches be loaded in this run, not needed for reloading, only for starting + * @return true if no other loading of a newer cluster state happened in parallel, false otherwise */ - private synchronized void reloadInner(ClusterState state, String reason, boolean loadTriggeredWatches) { + private synchronized boolean reloadInner(ClusterState state, String reason, boolean loadTriggeredWatches) { // exit early if another thread has come in between if (processedClusterStateVersion.get() != state.getVersion()) { logger.debug("watch service has not been reloaded for state [{}], another reload for state [{}] in progress", @@ -221,9 +238,11 @@ private synchronized void reloadInner(ClusterState state, String reason, boolean executionService.executeTriggeredWatches(triggeredWatches); } logger.debug("watch service has been reloaded, reason [{}]", reason); + return true; } else { logger.debug("watch service has not been reloaded for state [{}], another reload for state [{}] in progress", state.getVersion(), processedClusterStateVersion.get()); + return false; } } diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java index 729696ffa3518..80d12f5fbce4e 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java @@ -46,6 +46,7 @@ import javax.net.ssl.HostnameVerifier; import java.io.ByteArrayOutputStream; +import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.net.URI; @@ -56,9 +57,13 @@ import java.util.List; import java.util.Map; -public class HttpClient extends AbstractComponent { +public class HttpClient extends AbstractComponent implements Closeable { private static final String SETTINGS_SSL_PREFIX = "xpack.http.ssl."; + // picking a reasonable high value here to allow for setups with lots of watch executions or many http inputs/actions + // this is also used as the value per route, if you are connecting to the same endpoint a lot, which is likely, when + // you are querying a remote Elasticsearch cluster + private static final int MAX_CONNECTIONS = 500; private final HttpAuthRegistry httpAuthRegistry; private final CloseableHttpClient client; @@ -84,6 +89,10 @@ public HttpClient(Settings settings, HttpAuthRegistry httpAuthRegistry, SSLServi SSLConnectionSocketFactory factory = new SSLConnectionSocketFactory(sslService.sslSocketFactory(sslSettings), verifier); clientBuilder.setSSLSocketFactory(factory); + clientBuilder.evictExpiredConnections(); + clientBuilder.setMaxConnPerRoute(MAX_CONNECTIONS); + clientBuilder.setMaxConnTotal(MAX_CONNECTIONS); + client = clientBuilder.build(); } @@ -251,6 +260,11 @@ private URI createURI(HttpRequest request) { } } + @Override + public void close() throws IOException { + client.close(); + } + /** * Helper class to have all HTTP methods except HEAD allow for an body, including GET */ diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java index 6901adb0a6937..7b77afb225e4b 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java @@ -121,11 +121,25 @@ public void unPause() { } /** - * Pause the execution of the watcher executor + * Pause the execution of the watcher executor, and empty the state. + * Pausing means, that no new watch executions will be done unless this pausing is explicitely unset. + * This is important when watcher is stopped, so that scheduled watches do not accidentally get executed. + * This should not be used when we need to reload watcher based on some cluster state changes, then just calling + * {@link #clearExecutionsAndQueue()} is the way to go + * * @return the number of tasks that have been removed */ public int pause() { paused.set(true); + return clearExecutionsAndQueue(); + } + + /** + * Empty the currently queued tasks and wait for current executions to finish. + * + * @return the number of tasks that have been removed + */ + public int clearExecutionsAndQueue() { int cancelledTaskCount = executor.queue().drainTo(new ArrayList<>()); this.clearExecutions(); return cancelledTaskCount; diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java index 316cb722f2f1e..700901753d4a1 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherLifeCycleServiceTests.java @@ -180,7 +180,7 @@ public void testManualStartStop() { reset(watcherService); when(watcherService.validate(clusterState)).thenReturn(true); lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, stoppedClusterState)); - verify(watcherService, times(1)).start(eq(clusterState)); + verify(watcherService, times(1)).start(eq(clusterState), anyObject()); // no change, keep going reset(watcherService); @@ -423,7 +423,7 @@ public void testWatcherServiceDoesNotStartIfIndexTemplatesAreMissing() throws Ex when(watcherService.validate(eq(state))).thenReturn(true); lifeCycleService.clusterChanged(new ClusterChangedEvent("any", state, state)); - verify(watcherService, times(0)).start(any(ClusterState.class)); + verify(watcherService, times(0)).start(any(ClusterState.class), anyObject()); } public void testWatcherStopsWhenMasterNodeIsMissing() { diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java index 5f815170215d3..73f9271e3efda 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java @@ -68,6 +68,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -199,7 +200,7 @@ void stopExecutor() { when(client.clearScroll(any(ClearScrollRequest.class))).thenReturn(clearScrollFuture); clearScrollFuture.onResponse(new ClearScrollResponse(true, 1)); - service.start(clusterState); + service.start(clusterState, () -> {}); ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); verify(triggerService).start(captor.capture()); @@ -238,6 +239,27 @@ void stopExecutor() { verify(triggerEngine).pauseExecution(); } + // if we have to reload the watcher service, the execution service should not be paused, as this might + // result in missing executions + public void testReloadingWatcherDoesNotPauseExecutionService() { + ExecutionService executionService = mock(ExecutionService.class); + TriggerService triggerService = mock(TriggerService.class); + WatcherService service = new WatcherService(Settings.EMPTY, triggerService, mock(TriggeredWatchStore.class), + executionService, mock(WatchParser.class), mock(Client.class), executorService) { + @Override + void stopExecutor() { + } + }; + + ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name")); + csBuilder.metaData(MetaData.builder()); + + service.reload(csBuilder.build(), "whatever"); + verify(executionService).clearExecutionsAndQueue(); + verify(executionService, never()).pause(); + verify(triggerService).pauseExecution(); + } + private static DiscoveryNode newNode() { return new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(), new HashSet<>(asList(DiscoveryNode.Role.values())), Version.CURRENT); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java index f57f65f1d6204..09ca57c1708f7 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java @@ -44,7 +44,6 @@ import org.junit.Before; import javax.mail.internet.AddressException; - import java.io.IOException; import java.util.Map; @@ -219,10 +218,9 @@ private WebhookActionFactory webhookFactory(HttpClient client) { public void testThatSelectingProxyWorks() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); - HttpClient httpClient = new HttpClient(Settings.EMPTY, authRegistry, - new SSLService(environment.settings(), environment)); - try (MockWebServer proxyServer = new MockWebServer()) { + try (HttpClient httpClient = new HttpClient(Settings.EMPTY, authRegistry, + new SSLService(environment.settings(), environment)); MockWebServer proxyServer = new MockWebServer()) { proxyServer.start(); proxyServer.enqueue(new MockResponse().setResponseCode(200).setBody("fullProxiedContent")); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java index 2a02c5300bded..10618b36e8ae9 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java @@ -77,6 +77,7 @@ public void init() throws Exception { @After public void shutdown() throws Exception { webServer.close(); + httpClient.close(); } public void testBasics() throws Exception { @@ -184,17 +185,18 @@ public void testHttps() throws Exception { .setSecureSettings(secureSettings) .build(); } - httpClient = new HttpClient(settings, authRegistry, new SSLService(settings, environment)); - secureSettings = new MockSecureSettings(); - // We can't use the client created above for the server since it is only a truststore - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); - Settings settings2 = Settings.builder() - .put("xpack.ssl.keystore.path", getDataPath("/org/elasticsearch/xpack/security/keystore/testnode.jks")) - .setSecureSettings(secureSettings) - .build(); + try (HttpClient client = new HttpClient(settings, authRegistry, new SSLService(settings, environment))) { + secureSettings = new MockSecureSettings(); + // We can't use the client created above for the server since it is only a truststore + secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); + Settings settings2 = Settings.builder() + .put("xpack.ssl.keystore.path", getDataPath("/org/elasticsearch/xpack/security/keystore/testnode.jks")) + .setSecureSettings(secureSettings) + .build(); - TestsSSLService sslService = new TestsSSLService(settings2, environment); - testSslMockWebserver(sslService.sslContext(), false); + TestsSSLService sslService = new TestsSSLService(settings2, environment); + testSslMockWebserver(client, sslService.sslContext(), false); + } } public void testHttpsDisableHostnameVerification() throws Exception { @@ -217,18 +219,19 @@ public void testHttpsDisableHostnameVerification() throws Exception { .setSecureSettings(secureSettings) .build(); } - httpClient = new HttpClient(settings, authRegistry, new SSLService(settings, environment)); - MockSecureSettings secureSettings = new MockSecureSettings(); - // We can't use the client created above for the server since it only defines a truststore - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode-no-subjaltname"); - Settings settings2 = Settings.builder() - .put("xpack.ssl.keystore.path", - getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-no-subjaltname.jks")) - .setSecureSettings(secureSettings) - .build(); + try (HttpClient client = new HttpClient(settings, authRegistry, new SSLService(settings, environment))) { + MockSecureSettings secureSettings = new MockSecureSettings(); + // We can't use the client created above for the server since it only defines a truststore + secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode-no-subjaltname"); + Settings settings2 = Settings.builder() + .put("xpack.ssl.keystore.path", + getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-no-subjaltname.jks")) + .setSecureSettings(secureSettings) + .build(); - TestsSSLService sslService = new TestsSSLService(settings2, environment); - testSslMockWebserver(sslService.sslContext(), false); + TestsSSLService sslService = new TestsSSLService(settings2, environment); + testSslMockWebserver(client, sslService.sslContext(), false); + } } public void testHttpsClientAuth() throws Exception { @@ -241,11 +244,12 @@ public void testHttpsClientAuth() throws Exception { .build(); TestsSSLService sslService = new TestsSSLService(settings, environment); - httpClient = new HttpClient(settings, authRegistry, sslService); - testSslMockWebserver(sslService.sslContext(), true); + try (HttpClient client = new HttpClient(settings, authRegistry, sslService)) { + testSslMockWebserver(client, sslService.sslContext(), true); + } } - private void testSslMockWebserver(SSLContext sslContext, boolean needClientAuth) throws IOException { + private void testSslMockWebserver(HttpClient client, SSLContext sslContext, boolean needClientAuth) throws IOException { try (MockWebServer mockWebServer = new MockWebServer(sslContext, needClientAuth)) { mockWebServer.enqueue(new MockResponse().setResponseCode(200).setBody("body")); mockWebServer.start(); @@ -253,7 +257,7 @@ private void testSslMockWebserver(SSLContext sslContext, boolean needClientAuth) HttpRequest.Builder request = HttpRequest.builder("localhost", mockWebServer.getPort()) .scheme(Scheme.HTTPS) .path("/test"); - HttpResponse response = httpClient.execute(request.build()); + HttpResponse response = client.execute(request.build()); assertThat(response.status(), equalTo(200)); assertThat(response.body().utf8ToString(), equalTo("body")); @@ -288,14 +292,14 @@ public void testHttpResponseWithAnyStatusCodeCanReturnBody() throws Exception { @Network public void testHttpsWithoutTruststore() throws Exception { - HttpClient httpClient = new HttpClient(Settings.EMPTY, authRegistry, new SSLService(Settings.EMPTY, environment)); - - // Known server with a valid cert from a commercial CA - HttpRequest.Builder request = HttpRequest.builder("www.elastic.co", 443).scheme(Scheme.HTTPS); - HttpResponse response = httpClient.execute(request.build()); - assertThat(response.status(), equalTo(200)); - assertThat(response.hasContent(), is(true)); - assertThat(response.body(), notNullValue()); + try (HttpClient client = new HttpClient(Settings.EMPTY, authRegistry, new SSLService(Settings.EMPTY, environment))) { + // Known server with a valid cert from a commercial CA + HttpRequest.Builder request = HttpRequest.builder("www.elastic.co", 443).scheme(Scheme.HTTPS); + HttpResponse response = client.execute(request.build()); + assertThat(response.status(), equalTo(200)); + assertThat(response.hasContent(), is(true)); + assertThat(response.body(), notNullValue()); + } } public void testThatProxyCanBeConfigured() throws Exception { @@ -307,15 +311,16 @@ public void testThatProxyCanBeConfigured() throws Exception { .put(HttpSettings.PROXY_HOST.getKey(), "localhost") .put(HttpSettings.PROXY_PORT.getKey(), proxyServer.getPort()) .build(); - HttpClient httpClient = new HttpClient(settings, authRegistry, new SSLService(settings, environment)); HttpRequest.Builder requestBuilder = HttpRequest.builder("localhost", webServer.getPort()) .method(HttpMethod.GET) .path("/"); - HttpResponse response = httpClient.execute(requestBuilder.build()); - assertThat(response.status(), equalTo(200)); - assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); + try (HttpClient client = new HttpClient(settings, authRegistry, new SSLService(settings, environment))) { + HttpResponse response = client.execute(requestBuilder.build()); + assertThat(response.status(), equalTo(200)); + assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); + } // ensure we hit the proxyServer and not the webserver assertThat(webServer.requests(), hasSize(0)); @@ -386,16 +391,16 @@ public void testProxyCanHaveDifferentSchemeThanRequest() throws Exception { .setSecureSettings(secureSettings) .build(); - HttpClient httpClient = new HttpClient(settings, authRegistry, new SSLService(settings, environment)); - HttpRequest.Builder requestBuilder = HttpRequest.builder("localhost", webServer.getPort()) .method(HttpMethod.GET) .scheme(Scheme.HTTP) .path("/"); - HttpResponse response = httpClient.execute(requestBuilder.build()); - assertThat(response.status(), equalTo(200)); - assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); + try (HttpClient client = new HttpClient(settings, authRegistry, new SSLService(settings, environment))) { + HttpResponse response = client.execute(requestBuilder.build()); + assertThat(response.status(), equalTo(200)); + assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); + } // ensure we hit the proxyServer and not the webserver assertThat(webServer.requests(), hasSize(0)); @@ -413,16 +418,17 @@ public void testThatProxyCanBeOverriddenByRequest() throws Exception { .put(HttpSettings.PROXY_PORT.getKey(), proxyServer.getPort() + 1) .put(HttpSettings.PROXY_HOST.getKey(), "https") .build(); - HttpClient httpClient = new HttpClient(settings, authRegistry, new SSLService(settings, environment)); HttpRequest.Builder requestBuilder = HttpRequest.builder("localhost", webServer.getPort()) .method(HttpMethod.GET) .proxy(new HttpProxy("localhost", proxyServer.getPort(), Scheme.HTTP)) .path("/"); - HttpResponse response = httpClient.execute(requestBuilder.build()); - assertThat(response.status(), equalTo(200)); - assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); + try (HttpClient client = new HttpClient(settings, authRegistry, new SSLService(settings, environment))) { + HttpResponse response = client.execute(requestBuilder.build()); + assertThat(response.status(), equalTo(200)); + assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); + } // ensure we hit the proxyServer and not the webserver assertThat(webServer.requests(), hasSize(0)); @@ -535,12 +541,13 @@ public void testMaxHttpResponseSize() throws Exception { Settings settings = Settings.builder() .put(HttpSettings.MAX_HTTP_RESPONSE_SIZE.getKey(), new ByteSizeValue(randomBytesLength - 1, ByteSizeUnit.BYTES)) .build(); - HttpClient httpClient = new HttpClient(settings, authRegistry, new SSLService(environment.settings(), environment)); HttpRequest.Builder requestBuilder = HttpRequest.builder("localhost", webServer.getPort()).method(HttpMethod.GET).path("/"); - IOException e = expectThrows(IOException.class, () -> httpClient.execute(requestBuilder.build())); - assertThat(e.getMessage(), startsWith("Maximum limit of")); + try (HttpClient client = new HttpClient(settings, authRegistry, new SSLService(environment.settings(), environment))) { + IOException e = expectThrows(IOException.class, () -> client.execute(requestBuilder.build())); + assertThat(e.getMessage(), startsWith("Maximum limit of")); + } } public void testThatGetRedirectIsFollowed() throws Exception { diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java index 2d134681e8b18..fa5a53f4e1da0 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java @@ -40,66 +40,69 @@ public void cleanup() throws Exception { public void testDefaultTimeout() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); - HttpClient httpClient = new HttpClient(Settings.EMPTY, mock(HttpAuthRegistry.class), - new SSLService(environment.settings(), environment)); - HttpRequest request = HttpRequest.builder("localhost", webServer.getPort()) .method(HttpMethod.POST) .path("/") .build(); - long start = System.nanoTime(); - expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); - TimeValue timeout = TimeValue.timeValueNanos(System.nanoTime() - start); - logger.info("http connection timed out after {}", timeout); + try (HttpClient httpClient = new HttpClient(Settings.EMPTY, mock(HttpAuthRegistry.class), + new SSLService(environment.settings(), environment))) { + long start = System.nanoTime(); + + expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); + TimeValue timeout = TimeValue.timeValueNanos(System.nanoTime() - start); + logger.info("http connection timed out after {}", timeout); - // it's supposed to be 10, but we'll give it an error margin of 2 seconds - assertThat(timeout.seconds(), greaterThan(8L)); - assertThat(timeout.seconds(), lessThan(12L)); + // it's supposed to be 10, but we'll give it an error margin of 2 seconds + assertThat(timeout.seconds(), greaterThan(8L)); + assertThat(timeout.seconds(), lessThan(12L)); + } } public void testDefaultTimeoutCustom() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); - HttpClient httpClient = new HttpClient(Settings.builder() - .put("xpack.http.default_read_timeout", "3s").build() - , mock(HttpAuthRegistry.class), new SSLService(environment.settings(), environment)); - HttpRequest request = HttpRequest.builder("localhost", webServer.getPort()) .method(HttpMethod.POST) .path("/") .build(); - long start = System.nanoTime(); - expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); - TimeValue timeout = TimeValue.timeValueNanos(System.nanoTime() - start); - logger.info("http connection timed out after {}", timeout); + try (HttpClient httpClient = new HttpClient(Settings.builder() + .put("xpack.http.default_read_timeout", "3s").build() + , mock(HttpAuthRegistry.class), new SSLService(environment.settings(), environment))) { - // it's supposed to be 3, but we'll give it an error margin of 2 seconds - assertThat(timeout.seconds(), greaterThan(1L)); - assertThat(timeout.seconds(), lessThan(5L)); + long start = System.nanoTime(); + expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); + TimeValue timeout = TimeValue.timeValueNanos(System.nanoTime() - start); + logger.info("http connection timed out after {}", timeout); + + // it's supposed to be 3, but we'll give it an error margin of 2 seconds + assertThat(timeout.seconds(), greaterThan(1L)); + assertThat(timeout.seconds(), lessThan(5L)); + } } public void testTimeoutCustomPerRequest() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); - HttpClient httpClient = new HttpClient(Settings.builder() - .put("xpack.http.default_read_timeout", "10s").build() - , mock(HttpAuthRegistry.class), new SSLService(environment.settings(), environment)); - HttpRequest request = HttpRequest.builder("localhost", webServer.getPort()) .readTimeout(TimeValue.timeValueSeconds(3)) .method(HttpMethod.POST) .path("/") .build(); - long start = System.nanoTime(); - expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); - TimeValue timeout = TimeValue.timeValueNanos(System.nanoTime() - start); - logger.info("http connection timed out after {}", timeout); + try (HttpClient httpClient = new HttpClient(Settings.builder() + .put("xpack.http.default_read_timeout", "10s").build() + , mock(HttpAuthRegistry.class), new SSLService(environment.settings(), environment))) { + + long start = System.nanoTime(); + expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); + TimeValue timeout = TimeValue.timeValueNanos(System.nanoTime() - start); + logger.info("http connection timed out after {}", timeout); - // it's supposed to be 3, but we'll give it an error margin of 2 seconds - assertThat(timeout.seconds(), greaterThan(1L)); - assertThat(timeout.seconds(), lessThan(5L)); + // it's supposed to be 3, but we'll give it an error margin of 2 seconds + assertThat(timeout.seconds(), greaterThan(1L)); + assertThat(timeout.seconds(), lessThan(5L)); + } } } diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/ExecutionVarsIntegrationTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/ExecutionVarsIntegrationTests.java index 85b0280588a6e..2f69cc95a50ef 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/ExecutionVarsIntegrationTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/ExecutionVarsIntegrationTests.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.function.Function; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.loggingAction; import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder; import static org.elasticsearch.xpack.watcher.input.InputBuilders.simpleInput; @@ -36,6 +37,8 @@ public class ExecutionVarsIntegrationTests extends AbstractWatcherIntegrationTestCase { + private String watchId = randomAlphaOfLength(20); + @Override protected List> pluginTypes() { List> types = super.pluginTypes(); @@ -107,7 +110,7 @@ protected Map, Object>> pluginScripts() { public void testVars() throws Exception { WatcherClient watcherClient = watcherClient(); - PutWatchResponse putWatchResponse = watcherClient.preparePutWatch("_id").setSource(watchBuilder() + PutWatchResponse putWatchResponse = watcherClient.preparePutWatch(watchId).setSource(watchBuilder() .trigger(schedule(cron("0/1 * * * * ?"))) .input(simpleInput("value", 5)) .condition(new ScriptCondition( @@ -126,7 +129,7 @@ public void testVars() throws Exception { assertThat(putWatchResponse.isCreated(), is(true)); - timeWarp().trigger("_id"); + timeWarp().trigger(watchId); flush(); refresh(); @@ -135,11 +138,11 @@ public void testVars() throws Exception { // defaults to match all; }); - assertThat(searchResponse.getHits().getTotalHits(), is(1L)); + assertHitCount(searchResponse, 1L); Map source = searchResponse.getHits().getAt(0).getSourceAsMap(); - assertValue(source, "watch_id", is("_id")); + assertValue(source, "watch_id", is(watchId)); assertValue(source, "state", is("executed")); // we don't store the computed vars in history @@ -171,7 +174,7 @@ public void testVars() throws Exception { public void testVarsManual() throws Exception { WatcherClient watcherClient = watcherClient(); - PutWatchResponse putWatchResponse = watcherClient.preparePutWatch("_id").setSource(watchBuilder() + PutWatchResponse putWatchResponse = watcherClient.preparePutWatch(watchId).setSource(watchBuilder() .trigger(schedule(cron("0/1 * * * * ? 2020"))) .input(simpleInput("value", 5)) .condition(new ScriptCondition( @@ -193,13 +196,13 @@ public void testVarsManual() throws Exception { boolean debug = randomBoolean(); ExecuteWatchResponse executeWatchResponse = watcherClient - .prepareExecuteWatch("_id") + .prepareExecuteWatch(watchId) .setDebug(debug) .get(); assertThat(executeWatchResponse.getRecordId(), notNullValue()); XContentSource source = executeWatchResponse.getRecordSource(); - assertValue(source, "watch_id", is("_id")); + assertValue(source, "watch_id", is(watchId)); assertValue(source, "state", is("executed")); if (debug) { diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java index 48a8ba7e2281a..4808700604da2 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java @@ -23,7 +23,7 @@ import org.elasticsearch.xpack.core.monitoring.exporter.MonitoringTemplateUtils; import org.elasticsearch.xpack.core.watcher.client.WatchSourceBuilder; import org.elasticsearch.xpack.core.watcher.support.xcontent.ObjectPath; -import org.elasticsearch.xpack.security.support.IndexLifecycleManager; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.test.rest.XPackRestTestHelper; import org.elasticsearch.xpack.watcher.actions.logging.LoggingAction; import org.elasticsearch.xpack.watcher.common.text.TextTemplate; @@ -138,7 +138,7 @@ public void testSecurityNativeRealm() throws Exception { logger.info("settings map {}", settingsMap); if (settingsMap.containsKey("index")) { int format = Integer.parseInt(String.valueOf(((Map)settingsMap.get("index")).get("format"))); - needsUpgrade = format == IndexLifecycleManager.INTERNAL_INDEX_FORMAT ? false : true; + needsUpgrade = format == SecurityIndexManager.INTERNAL_INDEX_FORMAT ? false : true; } else { needsUpgrade = true; } diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java index c9ad4b3053cbe..6040de8f50cda 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java @@ -8,6 +8,7 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; +import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TimeUnits; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -29,6 +30,7 @@ import static org.hamcrest.Matchers.is; @TimeoutSuite(millis = 5 * TimeUnits.MINUTE) // to account for slow as hell VMs +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/30456") public class UpgradeClusterClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { /**