From 8c1012efbbd3a300afd0dfb9b18250f15ea753f9 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Mon, 2 Sep 2019 11:16:04 +0300 Subject: [PATCH 001/103] SQL: Fix issue with DataType for CASE with NULL (#46173) Previously, if the DataType of all the WHEN conditions of a CASE statement is NULL, then it was set to NULL even if the ELSE clause has a non-NULL data type, e.g.: ``` CASE WHEN a = 1 THEN NULL WHEN a = 5 THEN NULL ELSE 'foo' ``` Fixes: #46032 --- .../predicate/conditional/Case.java | 1 + .../predicate/conditional/CaseTests.java | 40 +++++++++++++++++++ 2 files changed, 41 insertions(+) diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Case.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Case.java index 59ec2c38d00b..7536612a67dd 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Case.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Case.java @@ -58,6 +58,7 @@ public class Case extends ConditionalFunction { for (IfConditional conditional : conditions) { dataType = DataTypeConversion.commonType(dataType, conditional.dataType()); } + dataType = DataTypeConversion.commonType(dataType, elseResult.dataType()); } } return dataType; diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java index 807be397f91b..b4de311c9203 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java @@ -6,19 +6,23 @@ package org.elasticsearch.xpack.sql.expression.predicate.conditional; import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.Equals; import org.elasticsearch.xpack.sql.tree.AbstractNodeTestCase; import org.elasticsearch.xpack.sql.tree.NodeSubclassTests; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.SourceTests; +import org.elasticsearch.xpack.sql.type.DataType; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Objects; import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomIntLiteral; import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomStringLiteral; +import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; import static org.elasticsearch.xpack.sql.tree.SourceTests.randomSource; /** @@ -77,6 +81,42 @@ public class CaseTests extends AbstractNodeTestCase { assertEquals(new Case(c.source(), newChildren), c.replaceChildren(newChildren)); } + public void testDataTypes() { + // CASE WHEN 1 = 1 THEN NULL + // ELSE 'default' + // END + Case c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, Literal.of(EMPTY, 1), Literal.of(EMPTY, 1)), Literal.NULL), + Literal.of(EMPTY, "default"))); + assertEquals(DataType.KEYWORD, c.dataType()); + + // CASE WHEN 1 = 1 THEN 'foo' + // ELSE NULL + // END + c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, Literal.of(EMPTY, 1), Literal.of(EMPTY, 1)), Literal.of(EMPTY, "foo")), + Literal.NULL)); + assertEquals(DataType.KEYWORD, c.dataType()); + + // CASE WHEN 1 = 1 THEN NULL + // ELSE NULL + // END + c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, Literal.of(EMPTY, 1), Literal.of(EMPTY, 1)), Literal.NULL), + Literal.NULL)); + assertEquals(DataType.NULL, c.dataType()); + + // CASE WHEN 1 = 1 THEN NULL + // WHEN 2 = 2 THEN 'foo' + // ELSE NULL + // END + c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, Literal.of(EMPTY, 1), Literal.of(EMPTY, 1)), Literal.NULL), + new IfConditional(EMPTY, new Equals(EMPTY, Literal.of(EMPTY, 2), Literal.of(EMPTY, 2)), Literal.of(EMPTY, "foo")), + Literal.NULL)); + assertEquals(DataType.KEYWORD, c.dataType()); + } + private List mutateChildren(Case c) { boolean removeConditional = randomBoolean(); List expressions = new ArrayList<>(c.children().size()); From dd487a0ab9d42f8caaca48aa85cf9bef6b21314f Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Mon, 2 Sep 2019 10:37:32 +0200 Subject: [PATCH 002/103] Mute 2 tests in S3BlobStoreRepositoryTests (#46221) Muted testSnapshotAndRestore and testMultipleSnapshotAndRollback Relates #46218 and #46219 --- .../s3/S3BlobStoreRepositoryTests.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java index 3e764b69a602..e1af8254ca53 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java @@ -284,4 +284,18 @@ public class S3BlobStoreRepositoryTests extends ESBlobStoreRepositoryIntegTestCa exchange.close(); } } + + // override here to mute only for S3, please remove this overload when un-muting + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46218") + @Override + public void testSnapshotAndRestore() throws Exception { + super.testSnapshotAndRestore(); + } + + // override here to mute only for S3, pleaseremove this overload when un-muting + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46219") + @Override + public void testMultipleSnapshotAndRollback() throws Exception { + super.testMultipleSnapshotAndRollback(); + } } From d980b6120433ed520a8239d753f242c560cb34fb Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 2 Sep 2019 11:26:43 +0200 Subject: [PATCH 003/103] Cleanup BlobStoreRepository Abort and Failure Handling (#46208) Aborts and failures were handled in a somewhat unfortunate way in #42791: Since the tasks for all files are generated before uploading they are all executed when a snapshot is aborted and lead to a massive number of failures added to the original aborted exception. In the case of failures the situation was not very reasonable as well. If one blob fails uploading the snapshot logic would upload all the remaining files as well and then fail (when previously it would just fail all following files). I fixed both of the above issues, by just short-circuiting all remaining tasks for a shard in case of an exception in any one upload. --- .../repositories/blobstore/BlobStoreRepository.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index d88068cf6cf5..47bbb773b0af 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -110,6 +110,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; @@ -1048,17 +1049,27 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp final GroupedActionListener filesListener = new GroupedActionListener<>(allFilesUploadedListener, indexIncrementalFileCount); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); + // Flag to signal that the snapshot has been aborted/failed so we can stop any further blob uploads from starting + final AtomicBoolean alreadyFailed = new AtomicBoolean(); for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { executor.execute(new ActionRunnable<>(filesListener) { @Override protected void doRun() { try { - snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store); + if (alreadyFailed.get() == false) { + snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store); + } filesListener.onResponse(null); } catch (IOException e) { throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); } } + + @Override + public void onFailure(Exception e) { + alreadyFailed.set(true); + super.onFailure(e); + } }); } } catch (Exception e) { From 7a0f2615f2ea73426d5afa585eb4b1b37d678d9e Mon Sep 17 00:00:00 2001 From: markharwood Date: Mon, 2 Sep 2019 11:39:41 +0100 Subject: [PATCH 004/103] Test fix for PinnedQueryBuilderIT (#46187) Fix test issue to stabilise scoring through use of DFS search mode. Randomised index-then-delete docs introduced by the test framework likely caused an imbalance in IDF scores across shards. Also made number of shards used in test a random number for added test coverage. Closes #46174 --- .../xpack/searchbusinessrules/PinnedQueryBuilderIT.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/search-business-rules/src/test/java/org/elasticsearch/xpack/searchbusinessrules/PinnedQueryBuilderIT.java b/x-pack/plugin/search-business-rules/src/test/java/org/elasticsearch/xpack/searchbusinessrules/PinnedQueryBuilderIT.java index 061f6f9968fe..f17ff7e25800 100644 --- a/x-pack/plugin/search-business-rules/src/test/java/org/elasticsearch/xpack/searchbusinessrules/PinnedQueryBuilderIT.java +++ b/x-pack/plugin/search-business-rules/src/test/java/org/elasticsearch/xpack/searchbusinessrules/PinnedQueryBuilderIT.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.LinkedHashSet; import java.util.List; +import static org.elasticsearch.action.search.SearchType.DFS_QUERY_THEN_FETCH; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFirstHit; @@ -56,13 +57,12 @@ public class PinnedQueryBuilderIT extends ESIntegTestCase { return plugins; } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46174") public void testPinnedPromotions() throws Exception { assertAcked(prepareCreate("test") .addMapping("type1", jsonBuilder().startObject().startObject("type1").startObject("properties").startObject("field1") .field("analyzer", "whitespace").field("type", "text").endObject().endObject().endObject().endObject()) - .setSettings(Settings.builder().put(indexSettings()).put("index.number_of_shards", 2))); + .setSettings(Settings.builder().put(indexSettings()).put("index.number_of_shards", randomIntBetween(2, 5)))); int numRelevantDocs = randomIntBetween(1, 100); for (int i = 0; i < numRelevantDocs; i++) { @@ -102,6 +102,7 @@ public class PinnedQueryBuilderIT extends ESIntegTestCase { int from = randomIntBetween(0, numRelevantDocs); int size = randomIntBetween(10, 100); SearchResponse searchResponse = client().prepareSearch().setQuery(pqb).setTrackTotalHits(true).setSize(size).setFrom(from) + .setSearchType(DFS_QUERY_THEN_FETCH) .get(); long numHits = searchResponse.getHits().getTotalHits().value; From 12a4177690e1d27931c24e2a51adcfd564d876fb Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 2 Sep 2019 15:43:51 +0200 Subject: [PATCH 005/103] Wait for all Rec. to Stop on Node Close (#46178) * Wait for all Rec. to Stop on Node Close * This issue is in the `RecoverySourceHandler#acquireStore`. If we submit the store release to the generic threadpool while it is getting shut down we never complete the futue we wait on (in the generic pool as well) and fail to ever release the store potentially. * Fixed by waiting for all recoveries to end on node close so that we aways have a healthy thread pool here * Closes #45956 --- .../recovery/PeerRecoverySourceService.java | 46 ++++++++++++++++++- .../recovery/RecoverySourceHandler.java | 15 +++--- .../java/org/elasticsearch/node/Node.java | 3 ++ .../PeerRecoverySourceServiceTests.java | 1 + 4 files changed, 57 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index ef47b153f535..644a8e2eb5cb 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -24,10 +24,13 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -50,7 +53,7 @@ import java.util.Set; * The source recovery accepts recovery requests from other peer shards and start the recovery process from this * source shard to the target shard. */ -public class PeerRecoverySourceService implements IndexEventListener { +public class PeerRecoverySourceService extends AbstractLifecycleComponent implements IndexEventListener { private static final Logger logger = LogManager.getLogger(PeerRecoverySourceService.class); @@ -74,6 +77,19 @@ public class PeerRecoverySourceService implements IndexEventListener { new StartRecoveryTransportRequestHandler()); } + @Override + protected void doStart() { + } + + @Override + protected void doStop() { + ongoingRecoveries.awaitEmpty(); + } + + @Override + protected void doClose() { + } + @Override public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) { @@ -118,9 +134,14 @@ public class PeerRecoverySourceService implements IndexEventListener { } final class OngoingRecoveries { + private final Map ongoingRecoveries = new HashMap<>(); + @Nullable + private List> emptyListeners; + synchronized RecoverySourceHandler addNewRecovery(StartRecoveryRequest request, IndexShard shard) { + assert lifecycle.started(); final ShardRecoveryContext shardContext = ongoingRecoveries.computeIfAbsent(shard, s -> new ShardRecoveryContext()); RecoverySourceHandler handler = shardContext.addNewRecovery(request, shard); shard.recoveryStats().incCurrentAsSource(); @@ -138,6 +159,13 @@ public class PeerRecoverySourceService implements IndexEventListener { if (shardRecoveryContext.recoveryHandlers.isEmpty()) { ongoingRecoveries.remove(shard); } + if (ongoingRecoveries.isEmpty()) { + if (emptyListeners != null) { + final List> onEmptyListeners = emptyListeners; + emptyListeners = null; + ActionListener.onResponse(onEmptyListeners, null); + } + } } synchronized void cancel(IndexShard shard, String reason) { @@ -157,6 +185,22 @@ public class PeerRecoverySourceService implements IndexEventListener { } } + void awaitEmpty() { + assert lifecycle.stoppedOrClosed(); + final PlainActionFuture future; + synchronized (this) { + if (ongoingRecoveries.isEmpty()) { + return; + } + future = new PlainActionFuture<>(); + if (emptyListeners == null) { + emptyListeners = new ArrayList<>(); + } + emptyListeners.add(future); + } + FutureUtils.get(future); + } + private final class ShardRecoveryContext { final Set recoveryHandlers = new HashSet<>(); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index e5ab1d7890eb..8324dd023b70 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -402,13 +402,14 @@ public class RecoverySourceHandler { store.incRef(); return Releasables.releaseOnce(() -> { final PlainActionFuture future = new PlainActionFuture<>(); - threadPool.generic().execute(new ActionRunnable<>(future) { - @Override - protected void doRun() { - store.decRef(); - listener.onResponse(null); - } - }); + assert threadPool.generic().isShutdown() == false; + // TODO: We shouldn't use the generic thread pool here as we already execute this from the generic pool. + // While practically unlikely at a min pool size of 128 we could technically block the whole pool by waiting on futures + // below and thus make it impossible for the store release to execute which in turn would block the futures forever + threadPool.generic().execute(ActionRunnable.wrap(future, l -> { + store.decRef(); + l.onResponse(null); + })); FutureUtils.get(future); }); } diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index cb7888eddee9..4f892948066d 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -593,6 +593,7 @@ public class Node implements Closeable { .filter(p -> p instanceof LifecycleComponent) .map(p -> (LifecycleComponent) p).collect(Collectors.toList()); resourcesToClose.addAll(pluginLifecycleComponents); + resourcesToClose.add(injector.getInstance(PeerRecoverySourceService.class)); this.pluginLifecycleComponents = Collections.unmodifiableList(pluginLifecycleComponents); client.initialize(injector.getInstance(new Key>() {}), transportService.getTaskManager(), () -> clusterService.localNode().getId(), transportService.getRemoteClusterService()); @@ -689,6 +690,7 @@ public class Node implements Closeable { assert localNodeFactory.getNode() != null; assert transportService.getLocalNode().equals(localNodeFactory.getNode()) : "transportService has a different local node than the factory provided"; + injector.getInstance(PeerRecoverySourceService.class).start(); final MetaData onDiskMetadata; // we load the global state here (the persistent part of the cluster state stored on disk) to // pass it to the bootstrap checks to allow plugins to enforce certain preconditions based on the recovered state. @@ -834,6 +836,7 @@ public class Node implements Closeable { toClose.add(injector.getInstance(IndicesService.class)); // close filter/fielddata caches after indices toClose.add(injector.getInstance(IndicesStore.class)); + toClose.add(injector.getInstance(PeerRecoverySourceService.class)); toClose.add(() -> stopWatch.stop().start("cluster")); toClose.add(injector.getInstance(ClusterService.class)); toClose.add(() -> stopWatch.stop().start("node_connections_service")); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoverySourceServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoverySourceServiceTests.java index 72eb2baeca94..491c3974e5bd 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoverySourceServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoverySourceServiceTests.java @@ -43,6 +43,7 @@ public class PeerRecoverySourceServiceTests extends IndexShardTestCase { StartRecoveryRequest startRecoveryRequest = new StartRecoveryRequest(primary.shardId(), randomAlphaOfLength(10), getFakeDiscoNode("source"), getFakeDiscoNode("target"), Store.MetadataSnapshot.EMPTY, randomBoolean(), randomLong(), SequenceNumbers.UNASSIGNED_SEQ_NO); + peerRecoverySourceService.start(); RecoverySourceHandler handler = peerRecoverySourceService.ongoingRecoveries.addNewRecovery(startRecoveryRequest, primary); DelayRecoveryException delayRecoveryException = expectThrows(DelayRecoveryException.class, () -> peerRecoverySourceService.ongoingRecoveries.addNewRecovery(startRecoveryRequest, primary)); From 69abc644132b874e224615c4207ecbc02d0be5d1 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 2 Sep 2019 16:41:18 +0200 Subject: [PATCH 006/103] Disable request throttling in S3BlobStoreRepositoryTests (#46226) When some high values are randomly picked up - for example the number of indices to snapshot or the number of snapshots to create - the tests in S3BlobStoreRepositoryTests can generate a high number of requests to the internal S3 server. In order to test the retry logic of the S3 client, the internal server is designed to randomly generate random server errors. When many requests are made, it is possible that the S3 client reaches its maximum number of successive retries capacity. Then the S3 client will stop retrying requests until enough retry attempts succeed, but it means that any request could fail before reaching the max retries count and make the test fail too. Closes #46217 Closes #46218 Closes #46219 --- .../s3/S3BlobStoreRepositoryTests.java | 24 ++++--------------- 1 file changed, 5 insertions(+), 19 deletions(-) diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java index e1af8254ca53..ff35e55fd640 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java @@ -121,11 +121,11 @@ public class S3BlobStoreRepositoryTests extends ESBlobStoreRepositoryIntegTestCa final String endpoint = "http://" + InetAddresses.toUriString(address.getAddress()) + ":" + address.getPort(); return Settings.builder() - .put(Settings.builder() - .put(S3ClientSettings.ENDPOINT_SETTING.getConcreteSettingForNamespace("test").getKey(), endpoint) - // Disable chunked encoding as it simplifies a lot the request parsing on the httpServer side - .put(S3ClientSettings.DISABLE_CHUNKED_ENCODING.getConcreteSettingForNamespace("test").getKey(), true) - .build()) + .put(S3ClientSettings.ENDPOINT_SETTING.getConcreteSettingForNamespace("test").getKey(), endpoint) + // Disable chunked encoding as it simplifies a lot the request parsing on the httpServer side + .put(S3ClientSettings.DISABLE_CHUNKED_ENCODING.getConcreteSettingForNamespace("test").getKey(), true) + // Disable request throttling because some random values in tests might generate too many failures for the S3 client + .put(S3ClientSettings.USE_THROTTLE_RETRIES_SETTING.getConcreteSettingForNamespace("test").getKey(), false) .put(super.nodeSettings(nodeOrdinal)) .setSecureSettings(secureSettings) .build(); @@ -284,18 +284,4 @@ public class S3BlobStoreRepositoryTests extends ESBlobStoreRepositoryIntegTestCa exchange.close(); } } - - // override here to mute only for S3, please remove this overload when un-muting - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46218") - @Override - public void testSnapshotAndRestore() throws Exception { - super.testSnapshotAndRestore(); - } - - // override here to mute only for S3, pleaseremove this overload when un-muting - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46219") - @Override - public void testMultipleSnapshotAndRollback() throws Exception { - super.testMultipleSnapshotAndRollback(); - } } From fd8183ee51d7cf08d9def58a2ae027714beb60de Mon Sep 17 00:00:00 2001 From: dengweisysu Date: Tue, 3 Sep 2019 02:56:22 +0800 Subject: [PATCH 007/103] Sync translog without lock when trim unreferenced readers (#46203) With this change, we can avoid blocking writing threads when trimming unreferenced readers; hence improving the translog writing performance in async durability mode. Close #46201 --- .../index/translog/Translog.java | 29 +++++++++++++------ 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 305fcf626891..89ee29689c55 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1670,6 +1670,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * required generation */ public void trimUnreferencedReaders() throws IOException { + List toDeleteReaderList = new ArrayList<>(); try (ReleasableLock ignored = writeLock.acquire()) { if (closed.get()) { // we're shutdown potentially on some tragic event, don't delete anything @@ -1683,22 +1684,14 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC "deletion policy requires a minReferenceGen of [" + minReferencedGen + "] which is higher than the current generation [" + currentFileGeneration() + "]"; - for (Iterator iterator = readers.iterator(); iterator.hasNext(); ) { TranslogReader reader = iterator.next(); if (reader.getGeneration() >= minReferencedGen) { break; } iterator.remove(); + toDeleteReaderList.add(reader); IOUtils.closeWhileHandlingException(reader); - final Path translogPath = reader.path(); - logger.trace("delete translog file [{}], not referenced and not current anymore", translogPath); - // The checkpoint is used when opening the translog to know which files should be recovered from. - // We now update the checkpoint to ignore the file we are going to remove. - // Note that there is a provision in recoverFromFiles to allow for the case where we synced the checkpoint - // but crashed before we could delete the file. - current.sync(); - deleteReaderFiles(reader); } assert readers.isEmpty() == false || current.generation == minReferencedGen : "all readers were cleaned but the minReferenceGen [" + minReferencedGen + "] is not the current writer's gen [" + @@ -1707,6 +1700,24 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC closeOnTragicEvent(ex); throw ex; } + // Do sync outside the writeLock to avoid blocking all writing thread. + if (toDeleteReaderList.isEmpty() == false) { + try { + // The checkpoint is used when opening the translog to know which files should be recovered from. + // We now update the checkpoint to ignore the file we are going to remove. + // Note that there is a provision in recoverFromFiles to allow for the case where we synced the checkpoint + // but crashed before we could delete the file. + sync(); + for (TranslogReader reader : toDeleteReaderList) { + final Path translogPath = reader.path(); + logger.trace("delete translog file [{}], not referenced and not current anymore", translogPath); + deleteReaderFiles(reader); + } + } catch (final Exception ex) { + closeOnTragicEvent(ex); + throw ex; + } + } } /** From 907b4c54ed2c5ef263bd76e60a2574cd9acda772 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Mon, 2 Sep 2019 14:08:18 -0700 Subject: [PATCH 008/103] Add debug assertions for userhome not existing (#46206) The elasticsearch user should not have a homedir, yet we have seen this particular test fail rather frequently with a failed check that the userhome does not exist. This commit adds some additional assertions on the presumptive userhome to narrow down where it might be created. relates #45903 --- qa/os/bats/default/certgen.bash | 3 +++ 1 file changed, 3 insertions(+) diff --git a/qa/os/bats/default/certgen.bash b/qa/os/bats/default/certgen.bash index 90292561d17a..8d6f085806d6 100644 --- a/qa/os/bats/default/certgen.bash +++ b/qa/os/bats/default/certgen.bash @@ -94,9 +94,11 @@ install_node_using_archive() { export ESHOME="$DEFAULT_ARCHIVE_ESHOME" export_elasticsearch_paths + assert_file_not_exist "/home/elasticsearch" install_archive set_debug_logging verify_archive_installation + assert_file_not_exist "/home/elasticsearch" export ESPLUGIN_COMMAND_USER=$DEFAULT_ARCHIVE_USER generate_trial_license @@ -122,6 +124,7 @@ install_node_using_package() { export ESHOME="$DEFAULT_PACKAGE_ESHOME" export_elasticsearch_paths + assert_file_not_exist "/home/elasticsearch" install_package set_debug_logging verify_package_installation From d67a3c98c52a1c0948296583953334996f861f78 Mon Sep 17 00:00:00 2001 From: Anup Date: Tue, 3 Sep 2019 04:59:21 +0530 Subject: [PATCH 009/103] Remove duplicate line in SearchAfterBuilder (#45994) --- .../org/elasticsearch/search/searchafter/SearchAfterBuilder.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/searchafter/SearchAfterBuilder.java b/server/src/main/java/org/elasticsearch/search/searchafter/SearchAfterBuilder.java index 7b2cedea64ab..bdd44c3438dd 100644 --- a/server/src/main/java/org/elasticsearch/search/searchafter/SearchAfterBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/searchafter/SearchAfterBuilder.java @@ -91,7 +91,6 @@ public class SearchAfterBuilder implements ToXContentObject, Writeable { if (values[i] instanceof Double) continue; if (values[i] instanceof Float) continue; if (values[i] instanceof Boolean) continue; - if (values[i] instanceof Boolean) continue; throw new IllegalArgumentException("Can't handle " + SEARCH_AFTER + " field value of type [" + values[i].getClass() + "]"); } sortValues = new Object[values.length]; From daffcf1a2ad443c7182d469ca45dbc2d46ceb368 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Tue, 3 Sep 2019 08:21:10 +0200 Subject: [PATCH 010/103] reset queryGeometry in ShapeQueryTests (#45974) --- .../xpack/spatial/search/ShapeQueryTests.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/ShapeQueryTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/ShapeQueryTests.java index 7a9476aa5221..937790b56abe 100644 --- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/ShapeQueryTests.java +++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/ShapeQueryTests.java @@ -62,6 +62,8 @@ public class ShapeQueryTests extends ESSingleNodeTestCase { // index random shapes numDocs = randomIntBetween(25, 50); + // reset query geometry to make sure we pick one from the indexed shapes + queryGeometry = null; Geometry geometry; for (int i = 0; i < numDocs; ++i) { geometry = ShapeTestUtils.randomGeometry(false); @@ -77,6 +79,10 @@ public class ShapeQueryTests extends ESSingleNodeTestCase { client().prepareIndex(IGNORE_MALFORMED_INDEX, FIELD_TYPE).setRefreshPolicy(IMMEDIATE).setSource(geoJson).get(); } catch (Exception e) { // sometimes GeoTestUtil will create invalid geometry; catch and continue: + if (queryGeometry == geometry) { + // reset query geometry as it didn't get indexed + queryGeometry = null; + } --i; continue; } @@ -227,7 +233,6 @@ public class ShapeQueryTests extends ESSingleNodeTestCase { assertHitCount(result, numDocs); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/45628") public void testFieldAlias() { SearchResponse response = client().prepareSearch(INDEX) .setQuery(new ShapeQueryBuilder("alias", queryGeometry).relation(ShapeRelation.INTERSECTS)) From 4d67dae69b4f1970cdf5008e2f208d7f6f27444e Mon Sep 17 00:00:00 2001 From: David Roberts Date: Tue, 3 Sep 2019 12:41:44 +0100 Subject: [PATCH 011/103] [ML-DataFrame] Fix off-by-one error in checkpoint operations_behind (#46235) Fixes a problem where operations_behind would be one less than expected per shard in a new index matched by the data frame transform source pattern. For example, if a data frame transform had a source of foo* and a new index foo-new was created with 2 shards and 7 documents indexed in it then operations_behind would be 5 prior to this change. The problem was that an empty index has a global checkpoint number of -1 and the sequence number of the first document that is indexed into an index is 0, not 1. This doesn't matter for indices included in both the last and next checkpoints, as the off-by-one errors cancelled, but for a new index it affected the observed result. --- .../transforms/DataFrameTransformCheckpoint.java | 16 +++++++++------- .../DataFrameTransformCheckpointTests.java | 8 ++++---- .../checkpoint/DefaultCheckpointProvider.java | 7 +++---- ...ataFrameTransformsCheckpointServiceTests.java | 2 +- 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpoint.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpoint.java index 81a06eb4524d..69877c4cbe70 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpoint.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpoint.java @@ -279,28 +279,30 @@ public class DataFrameTransformCheckpoint implements Writeable, ToXContentObject throw new IllegalArgumentException("old checkpoint is newer than new checkpoint"); } - // get the sum of of shard checkpoints + // get the sum of of shard operations (that are fully replicated), which is 1 higher than the global checkpoint for each shard // note: we require shard checkpoints to strictly increase and never decrease - long oldCheckPointSum = 0; - long newCheckPointSum = 0; + long oldCheckPointOperationsSum = 0; + long newCheckPointOperationsSum = 0; for (Entry entry : oldCheckpoint.indicesCheckpoints.entrySet()) { // ignore entries that aren't part of newCheckpoint, e.g. deleted indices if (newCheckpoint.indicesCheckpoints.containsKey(entry.getKey())) { - oldCheckPointSum += Arrays.stream(entry.getValue()).sum(); + // Add 1 per shard as sequence numbers start at 0, i.e. sequence number 0 means there has been 1 operation + oldCheckPointOperationsSum += Arrays.stream(entry.getValue()).sum() + entry.getValue().length; } } for (long[] v : newCheckpoint.indicesCheckpoints.values()) { - newCheckPointSum += Arrays.stream(v).sum(); + // Add 1 per shard as sequence numbers start at 0, i.e. sequence number 0 means there has been 1 operation + newCheckPointOperationsSum += Arrays.stream(v).sum() + v.length; } // this should not be possible - if (newCheckPointSum < oldCheckPointSum) { + if (newCheckPointOperationsSum < oldCheckPointOperationsSum) { return -1L; } - return newCheckPointSum - oldCheckPointSum; + return newCheckPointOperationsSum - oldCheckPointOperationsSum; } private static Map readCheckpoints(Map readMap) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpointTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpointTests.java index 67cc4b91584c..298b018ce453 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpointTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformCheckpointTests.java @@ -106,7 +106,7 @@ public class DataFrameTransformCheckpointTests extends AbstractSerializingDataFr List checkpoints2 = new ArrayList<>(); for (int j = 0; j < shards; ++j) { - long shardCheckpoint = randomLongBetween(0, 1_000_000); + long shardCheckpoint = randomLongBetween(-1, 1_000_000); checkpoints1.add(shardCheckpoint); checkpoints2.add(shardCheckpoint + 10); } @@ -152,11 +152,11 @@ public class DataFrameTransformCheckpointTests extends AbstractSerializingDataFr assertEquals(-1L, DataFrameTransformCheckpoint.getBehind(checkpointOlderButNewerShardsCheckpoint, checkpointOld)); // test cases where indices sets do not match - // remove something from old, so newer has 1 index more than old + // remove something from old, so newer has 1 index more than old: should be equivalent to old index existing but empty checkpointsByIndexOld.remove(checkpointsByIndexOld.firstKey()); long behind = DataFrameTransformCheckpoint.getBehind(checkpointOld, checkpointTransientNew); - assertTrue("Expected behind (" + behind + ") > sum of shard checkpoints (" + indices * shards * 10L + ")", - behind > indices * shards * 10L); + assertTrue("Expected behind (" + behind + ") => sum of shard checkpoint differences (" + indices * shards * 10L + ")", + behind >= indices * shards * 10L); // remove same key: old and new should have equal indices again checkpointsByIndexNew.remove(checkpointsByIndexNew.firstKey()); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/DefaultCheckpointProvider.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/DefaultCheckpointProvider.java index 23b1bdde12b4..5464304d5b8e 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/DefaultCheckpointProvider.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/DefaultCheckpointProvider.java @@ -183,8 +183,8 @@ public class DefaultCheckpointProvider implements CheckpointProvider { String indexName = shard.getShardRouting().getIndexName(); if (userIndices.contains(indexName)) { - // SeqNoStats could be `null`, assume the global checkpoint to be 0 in this case - long globalCheckpoint = shard.getSeqNoStats() == null ? 0 : shard.getSeqNoStats().getGlobalCheckpoint(); + // SeqNoStats could be `null`, assume the global checkpoint to be -1 in this case + long globalCheckpoint = shard.getSeqNoStats() == null ? -1L : shard.getSeqNoStats().getGlobalCheckpoint(); if (checkpointsByIndex.containsKey(indexName)) { // we have already seen this index, just check/add shards TreeMap checkpoints = checkpointsByIndex.get(indexName); @@ -215,8 +215,7 @@ public class DefaultCheckpointProvider implements CheckpointProvider { userIndicesClone.removeAll(checkpointsByIndex.keySet()); if (userIndicesClone.isEmpty() == false) { - logger.debug("Original set of user indices contained more indexes [{}]", - userIndicesClone); + logger.debug("Original set of user indices contained more indexes [{}]", userIndicesClone); } } diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformsCheckpointServiceTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformsCheckpointServiceTests.java index 1947e7b1d064..8a9a5bd50af9 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformsCheckpointServiceTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformsCheckpointServiceTests.java @@ -184,7 +184,7 @@ public class DataFrameTransformsCheckpointServiceTests extends ESTestCase { // add broken seqNoStats if requested if (missingSeqNoStats && index.getName().equals(missingSeqNoStatsIndex)) { - checkpoints.add(0L); + checkpoints.add(-1L); } else { validSeqNoStats = new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint); checkpoints.add(globalCheckpoint); From 5ec9d7be4dc4e4e126cba4804fdeca2aa9789dcc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Letterer?= <48132449+111andre111@users.noreply.github.com> Date: Tue, 3 Sep 2019 14:56:36 +0200 Subject: [PATCH 012/103] Fixed synchronizing REST API inflight breaker names with internal variable (#40878) The internal configuration settings were like that: network.breaker.inflight_requests But the exposed REST API had the value names with underscore like that: network.breaker.in_flight_requests This was now corrected to without underscores like that: network.breaker.inflight_requests --- docs/reference/migration/migrate_8_0.asciidoc | 2 ++ docs/reference/migration/migrate_8_0/breaker.asciidoc | 11 +++++++++++ .../elasticsearch/common/breaker/CircuitBreaker.java | 2 +- .../breaker/HierarchyCircuitBreakerServiceTests.java | 4 ++-- 4 files changed, 16 insertions(+), 3 deletions(-) create mode 100644 docs/reference/migration/migrate_8_0/breaker.asciidoc diff --git a/docs/reference/migration/migrate_8_0.asciidoc b/docs/reference/migration/migrate_8_0.asciidoc index a346e35841de..05b2c6a9123c 100644 --- a/docs/reference/migration/migrate_8_0.asciidoc +++ b/docs/reference/migration/migrate_8_0.asciidoc @@ -13,6 +13,7 @@ coming[8.0.0] * <> * <> +* <> * <> * <> * <> @@ -53,6 +54,7 @@ Elasticsearch 7.x in order to be readable by Elasticsearch 8.x. include::migrate_8_0/analysis.asciidoc[] include::migrate_8_0/allocation.asciidoc[] +include::migrate_8_0/breaker.asciidoc[] include::migrate_8_0/discovery.asciidoc[] include::migrate_8_0/mappings.asciidoc[] include::migrate_8_0/packaging.asciidoc[] diff --git a/docs/reference/migration/migrate_8_0/breaker.asciidoc b/docs/reference/migration/migrate_8_0/breaker.asciidoc new file mode 100644 index 000000000000..59ab2b4ae8d9 --- /dev/null +++ b/docs/reference/migration/migrate_8_0/breaker.asciidoc @@ -0,0 +1,11 @@ +[float] +[[breaking_80_breaker_changes]] +=== Packaging changes + +//tag::notable-breaking-changes[] +[float] +==== In Flight Request Circuit Breaker + +The name of the in flight requests circuit breaker in log output and diagnostic APIs (such as the node stats API) changes from `in_flight_requests` to `inflight_requests` to align it with the name of the corresponding settings. + +//end::notable-breaking-changes[] diff --git a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java index 6cc818e03aca..2908856b84f1 100644 --- a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java +++ b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java @@ -53,7 +53,7 @@ public interface CircuitBreaker { * The in-flight request breaker tracks bytes allocated for reading and * writing requests on the network layer. */ - String IN_FLIGHT_REQUESTS = "in_flight_requests"; + String IN_FLIGHT_REQUESTS = "inflight_requests"; /** * The accounting breaker tracks things held in memory that is independent * of the request lifecycle. This includes memory used by Lucene for diff --git a/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java b/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java index 9918b0c6e618..b72d8b980093 100644 --- a/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerServiceTests.java @@ -201,7 +201,7 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase { assertThat(exception.getMessage(), containsString("[parent] Data too large, data for [should break] would be")); assertThat(exception.getMessage(), containsString("which is larger than the limit of [209715200/200mb]")); assertThat(exception.getMessage(), - containsString("usages [request=157286400/150mb, fielddata=54001664/51.5mb, in_flight_requests=0/0b, accounting=0/0b]")); + containsString("usages [request=157286400/150mb, fielddata=54001664/51.5mb, accounting=0/0b, inflight_requests=0/0b]")); assertThat(exception.getDurability(), equalTo(CircuitBreaker.Durability.TRANSIENT)); } } @@ -249,7 +249,7 @@ public class HierarchyCircuitBreakerServiceTests extends ESTestCase { final long requestCircuitBreakerUsed = (requestBreaker.getUsed() + reservationInBytes) * 2; assertThat(exception.getMessage(), containsString("usages [request=" + requestCircuitBreakerUsed + "/" + new ByteSizeValue(requestCircuitBreakerUsed) + - ", fielddata=0/0b, in_flight_requests=0/0b, accounting=0/0b]")); + ", fielddata=0/0b, accounting=0/0b, inflight_requests=0/0b]")); assertThat(exception.getDurability(), equalTo(CircuitBreaker.Durability.TRANSIENT)); assertEquals(0, requestBreaker.getTrippedCount()); assertEquals(1, service.stats().getStats(CircuitBreaker.PARENT).getTrippedCount()); From 1e211ef7164f1dd16a09032ac6fa5bf7e8b1ae0b Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 3 Sep 2019 09:10:46 -0400 Subject: [PATCH 013/103] [DOCS] Add delete index alias API docs (#46080) --- docs/reference/indices.asciidoc | 3 ++ docs/reference/indices/delete-alias.asciidoc | 49 ++++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 docs/reference/indices/delete-alias.asciidoc diff --git a/docs/reference/indices.asciidoc b/docs/reference/indices.asciidoc index 403b9c7903a7..8fbb63a94d64 100644 --- a/docs/reference/indices.asciidoc +++ b/docs/reference/indices.asciidoc @@ -35,6 +35,7 @@ index settings, aliases, mappings, and index templates. [[alias-management]] === Alias management: * <> +* <> * <> * <> * <> @@ -103,6 +104,8 @@ include::indices/types-exists.asciidoc[] include::indices/add-alias.asciidoc[] +include::indices/delete-alias.asciidoc[] + include::indices/get-alias.asciidoc[] include::indices/alias-exists.asciidoc[] diff --git a/docs/reference/indices/delete-alias.asciidoc b/docs/reference/indices/delete-alias.asciidoc new file mode 100644 index 000000000000..e7ef772cbbd6 --- /dev/null +++ b/docs/reference/indices/delete-alias.asciidoc @@ -0,0 +1,49 @@ +[[indices-delete-alias]] +=== Delete index alias API +++++ +Delete index alias +++++ + +Deletes an existing index alias. + +include::alias-exists.asciidoc[tag=index-alias-def] + +[source,js] +---- +DELETE /twitter/_alias/alias1 +---- +// CONSOLE +// TEST[setup:twitter] +// TEST[s/^/PUT twitter\/_alias\/alias1\n/] + +[[delete-alias-api-request]] +==== {api-request-title} + +`DELETE //_alias/` + +`DELETE //_aliases/` + + +[[delete-alias-api-path-params]] +==== {api-path-parms-title} + +``:: +(Required, string) +Comma-separated list or wildcard expression of index names +used to limit the request. ++ +To include all indices in the cluster, +use a value of `_all` or `*`. + +``:: +(Required, string) +include::{docdir}/rest-api/common-parms.asciidoc[tag=index-alias] ++ +To delete all aliases, +use a value of `_all` or `*`. + + +[[delete-alias-api-query-params]] +==== {api-query-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=timeoutparms] From 38c323eb3c989c1431c0ce984507cb4e53c170a7 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 3 Sep 2019 08:51:04 -0500 Subject: [PATCH 014/103] [ML][Transforms] fixing stop on changes check bug (#46162) * [ML][Transforms] fixing stop on changes check bug * Adding new method finishAndCheckState to cover race conditions in early terminations * changing stopping conditions in `onStart` * allow indexer to finish when exiting early --- .../core/indexing/AsyncTwoPhaseIndexer.java | 4 +++- .../DataFrameGetAndGetStatsIT.java | 2 -- .../transforms/DataFrameTransformTask.java | 24 ++++++++++++++++++- 3 files changed, 26 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java index ac5bbb144940..52aa9304ce26 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java @@ -158,7 +158,9 @@ public abstract class AsyncTwoPhaseIndexer doSaveState(finishAndSetState(), position.get(), () -> {}), + onFinishFailure -> doSaveState(finishAndSetState(), position.get(), () -> {}))); } }, this::finishWithFailure)); diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameGetAndGetStatsIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameGetAndGetStatsIT.java index 76bb90d300ad..3a7809125c7d 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameGetAndGetStatsIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameGetAndGetStatsIT.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.dataframe.integration; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.client.Request; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.xpack.core.dataframe.DataFrameField; @@ -27,7 +26,6 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.oneOf; -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/45610") public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase { private static final String TEST_USER_NAME = "df_user"; diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java index 7a4162ad6e5e..0515640f2baa 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java @@ -640,6 +640,8 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S private final DataFrameTransformTask transformTask; private final AtomicInteger failureCount; private volatile boolean auditBulkFailures = true; + // Indicates that the source has changed for the current run + private volatile boolean hasSourceChanged = true; // Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index private volatile String lastAuditedExceptionMessage = null; private final AtomicBoolean oldStatsCleanedUp = new AtomicBoolean(false); @@ -760,18 +762,26 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S if (transformTask.currentCheckpoint.get() > 0 && initialRun()) { sourceHasChanged(ActionListener.wrap( hasChanged -> { + hasSourceChanged = hasChanged; if (hasChanged) { transformTask.changesLastDetectedAt = Instant.now(); logger.debug("[{}] source has changed, triggering new indexer run.", transformId); changedSourceListener.onResponse(null); } else { + logger.trace("[{}] source has not changed, finish indexer early.", transformId); // No changes, stop executing listener.onResponse(false); } }, - listener::onFailure + failure -> { + // If we failed determining if the source changed, it's safer to assume there were changes. + // We should allow the failure path to complete as normal + hasSourceChanged = true; + listener.onFailure(failure); + } )); } else { + hasSourceChanged = true; changedSourceListener.onResponse(null); } } @@ -869,6 +879,13 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S next.run(); return; } + // This means that the indexer was triggered to discover changes, found none, and exited early. + // If the state is `STOPPED` this means that DataFrameTransformTask#stop was called while we were checking for changes. + // Allow the stop call path to continue + if (hasSourceChanged == false && indexerState.equals(IndexerState.STOPPED) == false) { + next.run(); + return; + } // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING` // OR we called `doSaveState` manually as the indexer was not actively running. // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state @@ -959,6 +976,11 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S @Override protected void onFinish(ActionListener listener) { try { + // This indicates an early exit since no changes were found. + // So, don't treat this like a checkpoint being completed, as no work was done. + if (hasSourceChanged == false) { + listener.onResponse(null); + } // TODO: needs cleanup super is called with a listener, but listener.onResponse is called below // super.onFinish() fortunately ignores the listener super.onFinish(listener); From f4e9d90ca49bd66a20416325f246cad5b051f2de Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Tue, 3 Sep 2019 17:29:28 +0200 Subject: [PATCH 015/103] Fix testSyncFailsIfOperationIsInFlight (#46269) testSyncFailsIfOperationIsInFlight could fail due to the index request spawing a GCP sync (new since 7.4). Test now waits for it to finish before testing that flushed sync fails. --- .../indices/flush/SyncedFlushSingleNodeTests.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java b/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java index 9d7f3d5e253c..0a97b4978f39 100644 --- a/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.threadpool.ThreadPool; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -110,12 +109,18 @@ public class SyncedFlushSingleNodeTests extends ESSingleNodeTestCase { assertTrue(response.success()); } - public void testSyncFailsIfOperationIsInFlight() throws InterruptedException, ExecutionException { + public void testSyncFailsIfOperationIsInFlight() throws Exception { createIndex("test"); client().prepareIndex("test", "test", "1").setSource("{}", XContentType.JSON).get(); IndexService test = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test")); IndexShard shard = test.getShardOrNull(0); + // wait for the GCP sync spawned from the index request above to complete to avoid that request disturbing the check below + assertBusy(() -> { + assertEquals(0, shard.getLastSyncedGlobalCheckpoint()); + assertEquals(0, shard.getActiveOperationsCount()); + }); + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); final ShardId shardId = shard.shardId(); PlainActionFuture fut = new PlainActionFuture<>(); From 23cacaf0e64a4e6bb4ca51642b3fd9afa2c8455e Mon Sep 17 00:00:00 2001 From: Dimitris Athanasiou Date: Tue, 3 Sep 2019 18:35:25 +0300 Subject: [PATCH 016/103] [ML] Unmute testStopOutlierDetectionWithEnoughDocumentsToScroll (#46271) The test seems to have been failing due to a race condition between stopping the task and refreshing the destination index. In particular, we were going forward with refreshing the destination index even though the task stopped in the meantime. This was fixed in request. Closes #43960 --- .../xpack/ml/integration/RunDataFrameAnalyticsIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/RunDataFrameAnalyticsIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/RunDataFrameAnalyticsIT.java index 6920949bb9a1..88cc74a48b02 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/RunDataFrameAnalyticsIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/RunDataFrameAnalyticsIT.java @@ -236,7 +236,6 @@ public class RunDataFrameAnalyticsIT extends MlNativeDataFrameAnalyticsIntegTest assertThat(searchStoredProgress(id).getHits().getTotalHits().value, equalTo(1L)); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/43960") public void testStopOutlierDetectionWithEnoughDocumentsToScroll() { String sourceIndex = "test-stop-outlier-detection-with-enough-docs-to-scroll"; From 91f7a0e3cd8f4946a4ba9b9cb7990c2c9b7c6c07 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 3 Sep 2019 11:06:20 -0500 Subject: [PATCH 017/103] [ML][Transforms] protecting doSaveState with optimistic concurrency (#46156) * [ML][Transforms] protecting doSaveState with optimistic concurrency * task code cleanup --- ...ansportUpdateDataFrameTransformAction.java | 7 +- .../DataFrameTransformsConfigManager.java | 60 +++++----- .../persistence/SeqNoPrimaryTermAndIndex.java | 73 +++++++++++++ ...FrameTransformPersistentTasksExecutor.java | 16 ++- .../transforms/DataFrameTransformTask.java | 103 +++++++++--------- .../DataFrameSingleNodeTestCase.java | 4 +- ...DataFrameTransformsConfigManagerTests.java | 46 ++++++-- .../SeqNoPrimaryTermAndIndexTests.java | 56 ++++++++++ 8 files changed, 263 insertions(+), 102 deletions(-) create mode 100644 x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndex.java create mode 100644 x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndexTests.java diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportUpdateDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportUpdateDataFrameTransformAction.java index ae6b0090a6b9..548ecf09dc91 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportUpdateDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportUpdateDataFrameTransformAction.java @@ -49,6 +49,7 @@ import org.elasticsearch.xpack.core.security.support.Exceptions; import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; import org.elasticsearch.xpack.dataframe.persistence.DataframeIndex; +import org.elasticsearch.xpack.dataframe.persistence.SeqNoPrimaryTermAndIndex; import org.elasticsearch.xpack.dataframe.transforms.SourceDestValidator; import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; @@ -138,7 +139,7 @@ public class TransportUpdateDataFrameTransformAction extends TransportMasterNode private void handlePrivsResponse(String username, Request request, DataFrameTransformConfig config, - DataFrameTransformsConfigManager.SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, ClusterState clusterState, HasPrivilegesResponse privilegesResponse, ActionListener listener) { @@ -161,7 +162,7 @@ public class TransportUpdateDataFrameTransformAction extends TransportMasterNode private void validateAndUpdateDataFrame(Request request, ClusterState clusterState, DataFrameTransformConfig config, - DataFrameTransformsConfigManager.SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, ActionListener listener) { try { SourceDestValidator.validate(config, clusterState, indexNameExpressionResolver, request.isDeferValidation()); @@ -186,7 +187,7 @@ public class TransportUpdateDataFrameTransformAction extends TransportMasterNode } private void updateDataFrame(Request request, DataFrameTransformConfig config, - DataFrameTransformsConfigManager.SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, ClusterState clusterState, ActionListener listener) { diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManager.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManager.java index 3d5c8b28aaa1..b58b96a8d193 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManager.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManager.java @@ -227,7 +227,8 @@ public class DataFrameTransformsConfigManager { .id(DataFrameTransformConfig.documentId(transformConfig.getId())) .source(source); if (seqNoPrimaryTermAndIndex != null) { - indexRequest.setIfSeqNo(seqNoPrimaryTermAndIndex.seqNo).setIfPrimaryTerm(seqNoPrimaryTermAndIndex.primaryTerm); + indexRequest.setIfSeqNo(seqNoPrimaryTermAndIndex.getSeqNo()) + .setIfPrimaryTerm(seqNoPrimaryTermAndIndex.getPrimaryTerm()); } executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, IndexAction.INSTANCE, indexRequest, ActionListener.wrap(r -> { listener.onResponse(true); @@ -433,21 +434,31 @@ public class DataFrameTransformsConfigManager { })); } - public void putOrUpdateTransformStoredDoc(DataFrameTransformStoredDoc stats, ActionListener listener) { + public void putOrUpdateTransformStoredDoc(DataFrameTransformStoredDoc stats, + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex, + ActionListener listener) { try (XContentBuilder builder = XContentFactory.jsonBuilder()) { XContentBuilder source = stats.toXContent(builder, new ToXContent.MapParams(TO_XCONTENT_PARAMS)); IndexRequest indexRequest = new IndexRequest(DataFrameInternalIndex.LATEST_INDEX_NAME) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .id(DataFrameTransformStoredDoc.documentId(stats.getId())) - .opType(DocWriteRequest.OpType.INDEX) .source(source); - + if (seqNoPrimaryTermAndIndex != null && + seqNoPrimaryTermAndIndex.getIndex().equals(DataFrameInternalIndex.LATEST_INDEX_NAME)) { + indexRequest.opType(DocWriteRequest.OpType.INDEX) + .setIfSeqNo(seqNoPrimaryTermAndIndex.getSeqNo()) + .setIfPrimaryTerm(seqNoPrimaryTermAndIndex.getPrimaryTerm()); + } else { + // If the index is NOT the latest or we are null, that means we have not created this doc before + // so, it should be a create option without the seqNo and primaryTerm set + indexRequest.opType(DocWriteRequest.OpType.CREATE); + } executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, IndexAction.INSTANCE, indexRequest, ActionListener.wrap( - r -> listener.onResponse(true), + r -> listener.onResponse(SeqNoPrimaryTermAndIndex.fromIndexResponse(r)), e -> listener.onFailure(new RuntimeException( - DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_FAILED_TO_PERSIST_STATS, stats.getId()), - e)) + DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_FAILED_TO_PERSIST_STATS, stats.getId()), + e)) )); } catch (IOException e) { // not expected to happen but for the sake of completeness @@ -457,13 +468,15 @@ public class DataFrameTransformsConfigManager { } } - public void getTransformStoredDoc(String transformId, ActionListener resultListener) { + public void getTransformStoredDoc(String transformId, + ActionListener> resultListener) { QueryBuilder queryBuilder = QueryBuilders.termQuery("_id", DataFrameTransformStoredDoc.documentId(transformId)); SearchRequest searchRequest = client.prepareSearch(DataFrameInternalIndex.INDEX_NAME_PATTERN) .setQuery(queryBuilder) // use sort to get the last .addSort("_index", SortOrder.DESC) .setSize(1) + .seqNoAndPrimaryTerm(true) .request(); executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap( @@ -473,11 +486,14 @@ public class DataFrameTransformsConfigManager { DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_UNKNOWN_TRANSFORM_STATS, transformId))); return; } - BytesReference source = searchResponse.getHits().getHits()[0].getSourceRef(); + SearchHit searchHit = searchResponse.getHits().getHits()[0]; + BytesReference source = searchHit.getSourceRef(); try (InputStream stream = source.streamInput(); XContentParser parser = XContentFactory.xContent(XContentType.JSON) .createParser(xContentRegistry, LoggingDeprecationHandler.INSTANCE, stream)) { - resultListener.onResponse(DataFrameTransformStoredDoc.fromXContent(parser)); + resultListener.onResponse( + Tuple.tuple(DataFrameTransformStoredDoc.fromXContent(parser), + SeqNoPrimaryTermAndIndex.fromSearchHit(searchHit))); } catch (Exception e) { logger.error(DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_PARSE_TRANSFORM_STATISTICS_CONFIGURATION, transformId), e); @@ -595,28 +611,4 @@ public class DataFrameTransformsConfigManager { } return new Tuple<>(status, reason); } - - public static class SeqNoPrimaryTermAndIndex { - private final long seqNo; - private final long primaryTerm; - private final String index; - - public SeqNoPrimaryTermAndIndex(long seqNo, long primaryTerm, String index) { - this.seqNo = seqNo; - this.primaryTerm = primaryTerm; - this.index = index; - } - - public long getSeqNo() { - return seqNo; - } - - public long getPrimaryTerm() { - return primaryTerm; - } - - public String getIndex() { - return index; - } - } } diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndex.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndex.java new file mode 100644 index 000000000000..68d88c18fe6d --- /dev/null +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndex.java @@ -0,0 +1,73 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.dataframe.persistence; + +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.search.SearchHit; + +import java.util.Objects; + +/** + * Simple class to keep track of information needed for optimistic concurrency + */ +public class SeqNoPrimaryTermAndIndex { + private final long seqNo; + private final long primaryTerm; + private final String index; + + public static SeqNoPrimaryTermAndIndex fromSearchHit(SearchHit hit) { + return new SeqNoPrimaryTermAndIndex(hit.getSeqNo(), hit.getPrimaryTerm(), hit.getIndex()); + } + + public static SeqNoPrimaryTermAndIndex fromIndexResponse(IndexResponse response) { + return new SeqNoPrimaryTermAndIndex(response.getSeqNo(), response.getPrimaryTerm(), response.getIndex()); + } + + SeqNoPrimaryTermAndIndex(long seqNo, long primaryTerm, String index) { + this.seqNo = seqNo; + this.primaryTerm = primaryTerm; + this.index = index; + } + + public long getSeqNo() { + return seqNo; + } + + public long getPrimaryTerm() { + return primaryTerm; + } + + public String getIndex() { + return index; + } + + @Override + public int hashCode() { + return Objects.hash(seqNo, primaryTerm, index); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + SeqNoPrimaryTermAndIndex other = (SeqNoPrimaryTermAndIndex) obj; + return Objects.equals(seqNo, other.seqNo) + && Objects.equals(primaryTerm, other.primaryTerm) + && Objects.equals(index, other.index); + } + + @Override + public String toString() { + return "{seqNo=" + seqNo + ",primaryTerm=" + primaryTerm + ",index=" + index + "}"; + } +} diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java index 593c3c6e8a56..8374edbbf217 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java @@ -20,6 +20,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTaskState; @@ -42,6 +43,7 @@ import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpoin import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; +import org.elasticsearch.xpack.dataframe.persistence.SeqNoPrimaryTermAndIndex; import org.elasticsearch.xpack.dataframe.transforms.pivot.SchemaUtil; import java.util.ArrayList; @@ -189,8 +191,12 @@ public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksEx // <3> Set the previous stats (if they exist), initialize the indexer, start the task (If it is STOPPED) // Since we don't create the task until `_start` is called, if we see that the task state is stopped, attempt to start // Schedule execution regardless - ActionListener transformStatsActionListener = ActionListener.wrap( - stateAndStats -> { + ActionListener> transformStatsActionListener = ActionListener.wrap( + stateAndStatsAndSeqNoPrimaryTermAndIndex -> { + DataFrameTransformStoredDoc stateAndStats = stateAndStatsAndSeqNoPrimaryTermAndIndex.v1(); + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = stateAndStatsAndSeqNoPrimaryTermAndIndex.v2(); + // Since we have not set the value for this yet, it SHOULD be null + buildTask.updateSeqNoPrimaryTermAndIndex(null, seqNoPrimaryTermAndIndex); logger.trace("[{}] initializing state and stats: [{}]", transformId, stateAndStats.toString()); indexerBuilder.setInitialStats(stateAndStats.getTransformStats()) .setInitialPosition(stateAndStats.getTransformState().getPosition()) @@ -217,10 +223,10 @@ public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksEx String msg = DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_LOAD_TRANSFORM_STATE, transformId); logger.error(msg, error); markAsFailed(buildTask, msg); + } else { + logger.trace("[{}] No stats found (new transform), starting the task", transformId); + startTask(buildTask, indexerBuilder, null, startTaskListener); } - - logger.trace("[{}] No stats found(new transform), starting the task", transformId); - startTask(buildTask, indexerBuilder, null, startTaskListener); } ); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java index 0515640f2baa..cdd3888e330b 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; @@ -54,6 +55,7 @@ import org.elasticsearch.xpack.dataframe.checkpoint.CheckpointProvider; import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService; import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; +import org.elasticsearch.xpack.dataframe.persistence.SeqNoPrimaryTermAndIndex; import org.elasticsearch.xpack.dataframe.transforms.pivot.AggregationResultUtils; import java.time.Instant; @@ -98,6 +100,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S private final AtomicReference taskState; private final AtomicReference stateReason; + private final AtomicReference seqNoPrimaryTermAndIndex = new AtomicReference<>(null); // the checkpoint of this data frame, storing the checkpoint until data indexing from source to dest is _complete_ // Note: Each indexer run creates a new future checkpoint which becomes the current checkpoint only after the indexer run finished private final AtomicLong currentCheckpoint; @@ -216,31 +219,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S )); } - public DataFrameTransformCheckpoint getLastCheckpoint() { - return getIndexer().getLastCheckpoint(); - } - - public DataFrameTransformCheckpoint getNextCheckpoint() { - return getIndexer().getNextCheckpoint(); - } - - /** - * Get the in-progress checkpoint - * - * @return checkpoint in progress or 0 if task/indexer is not active - */ - public long getInProgressCheckpoint() { - if (getIndexer() == null) { - return 0; - } else { - return indexer.get().getState().equals(IndexerState.INDEXING) ? currentCheckpoint.get() + 1L : 0; - } - } - - public synchronized void setTaskStateStopped() { - taskState.set(DataFrameTransformTaskState.STOPPED); - } - /** * Start the background indexer and set the task's state to started * @param startingCheckpoint Set the current checkpoint to this value. If null the @@ -270,6 +248,15 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S msg)); return; } + // If we are already in a `STARTED` state, we should not attempt to call `.start` on the indexer again. + if (taskState.get() == DataFrameTransformTaskState.STARTED) { + listener.onFailure(new ElasticsearchStatusException( + "Cannot start transform [{}] as it is already STARTED.", + RestStatus.CONFLICT, + getTransformId() + )); + return; + } final IndexerState newState = getIndexer().start(); if (Arrays.stream(RUNNING_STATES).noneMatch(newState::equals)) { listener.onFailure(new ElasticsearchException("Cannot start task for data frame transform [{}], because state was [{}]", @@ -325,7 +312,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S return; } - if (getIndexer().getState() == IndexerState.STOPPED) { + if (getIndexer().getState() == IndexerState.STOPPED || getIndexer().getState() == IndexerState.STOPPING) { return; } @@ -339,10 +326,11 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S IndexerState state = getIndexer().stop(); stateReason.set(null); - // We just don't want it to be failed if it is failed - // Either we are running, and the STATE is already started or failed - // doSaveState should transfer the state to STOPPED when it needs to. - taskState.set(DataFrameTransformTaskState.STARTED); + // No reason to keep it in the potentially failed state. + // Since we have called `stop` against the indexer, we have no more fear of triggering again. + // But, since `doSaveState` is asynchronous, it is best to set the state as STARTED so that another `start` call cannot be + // executed while we are wrapping up. + taskState.compareAndSet(DataFrameTransformTaskState.FAILED, DataFrameTransformTaskState.STARTED); if (state == IndexerState.STOPPED) { getIndexer().onStop(); getIndexer().doSaveState(state, getIndexer().getPosition(), () -> {}); @@ -361,8 +349,10 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S return; } - if (taskState.get() == DataFrameTransformTaskState.FAILED) { - logger.debug("[{}] schedule was triggered for transform but task is failed. Ignoring trigger.", getTransformId()); + if (taskState.get() == DataFrameTransformTaskState.FAILED || taskState.get() == DataFrameTransformTaskState.STOPPED) { + logger.debug("[{}] schedule was triggered for transform but task is [{}]. Ignoring trigger.", + getTransformId(), + taskState.get()); return; } @@ -379,7 +369,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S // if it runs for the 1st time we just do it, if not we check for changes if (currentCheckpoint.get() == 0) { - logger.debug("Trigger initial run."); + logger.debug("[{}] trigger initial run.", getTransformId()); getIndexer().maybeTriggerAsyncJob(System.currentTimeMillis()); } else if (getIndexer().isContinuous()) { getIndexer().maybeTriggerAsyncJob(System.currentTimeMillis()); @@ -395,17 +385,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S markAsCompleted(); } - public DataFrameTransformProgress getProgress() { - if (indexer.get() == null) { - return null; - } - DataFrameTransformProgress indexerProgress = indexer.get().getProgress(); - if (indexerProgress == null) { - return null; - } - return new DataFrameTransformProgress(indexerProgress); - } - void persistStateToClusterState(DataFrameTransformState state, ActionListener> listener) { updatePersistentTaskState(state, ActionListener.wrap( @@ -520,6 +499,19 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S indexer.set(indexerBuilder.build(this)); } + void updateSeqNoPrimaryTermAndIndex(SeqNoPrimaryTermAndIndex expectedValue, SeqNoPrimaryTermAndIndex newValue) { + boolean updated = seqNoPrimaryTermAndIndex.compareAndSet(expectedValue, newValue); + // This should never happen. We ONLY ever update this value if at initialization or we just finished updating the document + // famous last words... + assert updated : + "[" + getTransformId() + "] unexpected change to seqNoPrimaryTermAndIndex."; + } + + @Nullable + SeqNoPrimaryTermAndIndex getSeqNoPrimaryTermAndIndex() { + return seqNoPrimaryTermAndIndex.get(); + } + static class ClientDataFrameIndexerBuilder { private Client client; private DataFrameTransformsConfigManager transformsConfigManager; @@ -879,6 +871,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S next.run(); return; } + // This means that the indexer was triggered to discover changes, found none, and exited early. // If the state is `STOPPED` this means that DataFrameTransformTask#stop was called while we were checking for changes. // Allow the stop call path to continue @@ -886,12 +879,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S next.run(); return; } - // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING` - // OR we called `doSaveState` manually as the indexer was not actively running. - // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state - if (indexerState.equals(IndexerState.STOPPED)) { - transformTask.setTaskStateStopped(); - } DataFrameTransformTaskState taskState = transformTask.taskState.get(); @@ -899,13 +886,21 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S && transformTask.currentCheckpoint.get() == 1 && this.isContinuous() == false) { // set both to stopped so they are persisted as such - taskState = DataFrameTransformTaskState.STOPPED; indexerState = IndexerState.STOPPED; auditor.info(transformConfig.getId(), "Data frame finished indexing all data, initiating stop"); logger.info("[{}] data frame transform finished indexing all data, initiating stop.", transformConfig.getId()); } + // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING` + // OR we called `doSaveState` manually as the indexer was not actively running. + // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state + if (indexerState.equals(IndexerState.STOPPED)) { + // We don't want adjust the stored taskState because as soon as it is `STOPPED` a user could call + // .start again. + taskState = DataFrameTransformTaskState.STOPPED; + } + final DataFrameTransformState state = new DataFrameTransformState( taskState, indexerState, @@ -915,13 +910,18 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S getProgress()); logger.debug("[{}] updating persistent state of transform to [{}].", transformConfig.getId(), state.toString()); + // This could be `null` but the putOrUpdateTransformStoredDoc handles that case just fine + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = transformTask.getSeqNoPrimaryTermAndIndex(); + // Persist the current state and stats in the internal index. The interval of this method being // called is controlled by AsyncTwoPhaseIndexer#onBulkResponse which calls doSaveState every so // often when doing bulk indexing calls or at the end of one indexing run. transformsConfigManager.putOrUpdateTransformStoredDoc( new DataFrameTransformStoredDoc(transformId, state, getStats()), + seqNoPrimaryTermAndIndex, ActionListener.wrap( r -> { + transformTask.updateSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex, r); // for auto stop shutdown the task if (state.getTaskState().equals(DataFrameTransformTaskState.STOPPED)) { transformTask.shutdown(); @@ -989,6 +989,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S nextCheckpoint = null; // Reset our failure count as we have finished and may start again with a new checkpoint failureCount.set(0); + transformTask.stateReason.set(null); // With bucket_selector we could have read all the buckets and completed the transform // but not "see" all the buckets since they were filtered out. Consequently, progress would diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/DataFrameSingleNodeTestCase.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/DataFrameSingleNodeTestCase.java index 34c16ebc9e73..38820d315f75 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/DataFrameSingleNodeTestCase.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/DataFrameSingleNodeTestCase.java @@ -23,6 +23,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import static org.hamcrest.Matchers.equalTo; + public abstract class DataFrameSingleNodeTestCase extends ESSingleNodeTestCase { @Before @@ -56,7 +58,7 @@ public abstract class DataFrameSingleNodeTestCase extends ESSingleNodeTestCase { if (expected == null) { fail("expected an exception but got a response"); } else { - assertEquals(r, expected); + assertThat(r, equalTo(expected)); } if (onAnswer != null) { onAnswer.accept(r); diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManagerTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManagerTests.java index e403d102adff..56c8d9ce07f6 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManagerTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameTransformsConfigManagerTests.java @@ -261,23 +261,50 @@ public class DataFrameTransformsConfigManagerTests extends DataFrameSingleNodeTe String transformId = "transform_test_stored_doc_create_read_update"; DataFrameTransformStoredDoc storedDocs = DataFrameTransformStoredDocTests.randomDataFrameTransformStoredDoc(transformId); + SeqNoPrimaryTermAndIndex firstIndex = new SeqNoPrimaryTermAndIndex(0, 1, DataFrameInternalIndex.LATEST_INDEX_NAME); - assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(storedDocs, listener), Boolean.TRUE, null, null); - assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(transformId, listener), storedDocs, null, null); + assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(storedDocs, null, listener), + firstIndex, + null, + null); + assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(transformId, listener), + Tuple.tuple(storedDocs, firstIndex), + null, + null); + SeqNoPrimaryTermAndIndex secondIndex = new SeqNoPrimaryTermAndIndex(1, 1, DataFrameInternalIndex.LATEST_INDEX_NAME); DataFrameTransformStoredDoc updated = DataFrameTransformStoredDocTests.randomDataFrameTransformStoredDoc(transformId); - assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(updated, listener), Boolean.TRUE, null, null); - assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(transformId, listener), updated, null, null); + assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(updated, firstIndex, listener), + secondIndex, + null, + null); + assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(transformId, listener), + Tuple.tuple(updated, secondIndex), + null, + null); + + assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(updated, firstIndex, listener), + (SeqNoPrimaryTermAndIndex)null, + r -> fail("did not fail with version conflict."), + e -> assertThat( + e.getMessage(), + equalTo("Failed to persist data frame statistics for transform [transform_test_stored_doc_create_read_update]")) + ); } public void testGetStoredDocMultiple() throws InterruptedException { int numStats = randomIntBetween(10, 15); List expectedDocs = new ArrayList<>(); for (int i=0; i transformsConfigManager.putOrUpdateTransformStoredDoc(stat, listener), Boolean.TRUE, null, null); + assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(stat, null, listener), + initialSeqNo, + null, + null); } // remove one of the put docs so we don't retrieve all @@ -338,8 +365,11 @@ public class DataFrameTransformsConfigManagerTests extends DataFrameSingleNodeTe client().index(request).actionGet(); } - assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(dataFrameTransformStoredDoc, listener), - true, + // Put when referencing the old index should create the doc in the new index, even if we have seqNo|primaryTerm info + assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(dataFrameTransformStoredDoc, + new SeqNoPrimaryTermAndIndex(3, 1, oldIndex), + listener), + new SeqNoPrimaryTermAndIndex(0, 1, DataFrameInternalIndex.LATEST_INDEX_NAME), null, null); diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndexTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndexTests.java new file mode 100644 index 000000000000..747c7baf3903 --- /dev/null +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/persistence/SeqNoPrimaryTermAndIndexTests.java @@ -0,0 +1,56 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.dataframe.persistence; + +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class SeqNoPrimaryTermAndIndexTests extends ESTestCase { + + public void testEquals() { + for (int i = 0; i < 30; i++) { + long seqNo = randomLongBetween(-2, 10_000); + long primaryTerm = randomLongBetween(-2, 10_000); + String index = randomAlphaOfLength(10); + SeqNoPrimaryTermAndIndex first = new SeqNoPrimaryTermAndIndex(seqNo, primaryTerm, index); + SeqNoPrimaryTermAndIndex second = new SeqNoPrimaryTermAndIndex(seqNo, primaryTerm, index); + assertThat(first, equalTo(second)); + } + } + + public void testFromSearchHit() { + SearchHit searchHit = new SearchHit(1); + long seqNo = randomLongBetween(-2, 10_000); + long primaryTerm = randomLongBetween(-2, 10_000); + String index = randomAlphaOfLength(10); + searchHit.setSeqNo(seqNo); + searchHit.setPrimaryTerm(primaryTerm); + searchHit.shard(new SearchShardTarget("anynode", new ShardId(index, randomAlphaOfLength(10), 1), null, null)); + assertThat(SeqNoPrimaryTermAndIndex.fromSearchHit(searchHit), equalTo(new SeqNoPrimaryTermAndIndex(seqNo, primaryTerm, index))); + } + + public void testFromIndexResponse() { + long seqNo = randomLongBetween(-2, 10_000); + long primaryTerm = randomLongBetween(-2, 10_000); + String index = randomAlphaOfLength(10); + IndexResponse indexResponse = new IndexResponse(new ShardId(index, randomAlphaOfLength(10), 1), + "_doc", + "asdf", + seqNo, + primaryTerm, + 1, + randomBoolean()); + + assertThat(SeqNoPrimaryTermAndIndex.fromIndexResponse(indexResponse), + equalTo(new SeqNoPrimaryTermAndIndex(seqNo, primaryTerm, index))); + } +} From a91f09a2200be7b95e21ecd68a4568d370601761 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 3 Sep 2019 12:09:08 -0400 Subject: [PATCH 018/103] Suppress warning from background sync on relocated primary (#46247) If a primary as being relocated, then the global checkpoint and retention lease background sync can emit unnecessary warning logs. This side effect was introduced in #42241. Relates #40800 Relates #42241 --- .../java/org/elasticsearch/index/IndexService.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index be4098cb1a71..61f8856faa72 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -67,6 +67,7 @@ import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.index.shard.SearchOperationListener; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardNotFoundException; +import org.elasticsearch.index.shard.ShardNotInPrimaryModeException; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; @@ -835,14 +836,11 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust case STARTED: try { shard.runUnderPrimaryPermit( - () -> { - if (shard.isRelocatedPrimary() == false) { - sync.accept(shard); - } - }, + () -> sync.accept(shard), e -> { if (e instanceof AlreadyClosedException == false - && e instanceof IndexShardClosedException == false) { + && e instanceof IndexShardClosedException == false + && e instanceof ShardNotInPrimaryModeException == false) { logger.warn( new ParameterizedMessage( "{} failed to execute {} sync", shard.shardId(), source), e); From 758f7999b732226f356e8dbd8187bbffd6bfe917 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 3 Sep 2019 12:10:34 -0400 Subject: [PATCH 019/103] Add CumulativeCard pipeline agg to pipeline index (#46279) The Cumulative Cardinality docs weren't linked from the pipeline index page --- docs/reference/aggregations/pipeline.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/reference/aggregations/pipeline.asciidoc b/docs/reference/aggregations/pipeline.asciidoc index 90938bee5d56..f31aaa08b9f3 100644 --- a/docs/reference/aggregations/pipeline.asciidoc +++ b/docs/reference/aggregations/pipeline.asciidoc @@ -281,6 +281,7 @@ include::pipeline/percentiles-bucket-aggregation.asciidoc[] include::pipeline/movavg-aggregation.asciidoc[] include::pipeline/movfn-aggregation.asciidoc[] include::pipeline/cumulative-sum-aggregation.asciidoc[] +include::pipeline/cumulative-cardinality-aggregation.asciidoc[] include::pipeline/bucket-script-aggregation.asciidoc[] include::pipeline/bucket-selector-aggregation.asciidoc[] include::pipeline/bucket-sort-aggregation.asciidoc[] From fcdfb80f85ff117395b9dce0deabd8cb62f748c1 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Tue, 3 Sep 2019 11:02:20 -0700 Subject: [PATCH 020/103] Add more assertions and cleanup to setup passwords tests (#46289) This commit is a followup to #46206 to continue debugging failures in an elasticsearch homedir being created. A couple more assertions are added as well as a final cleanup at the end of the previous test to the one that fails. --- qa/os/bats/default/setup_passwords.bash | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/qa/os/bats/default/setup_passwords.bash b/qa/os/bats/default/setup_passwords.bash index d892b379df4f..e64030cd6bb7 100644 --- a/qa/os/bats/default/setup_passwords.bash +++ b/qa/os/bats/default/setup_passwords.bash @@ -48,6 +48,7 @@ else fi @test "[$GROUP] test auto generated passwords" { + assert_file_not_exist "/home/elasticsearch" run_elasticsearch_service 0 wait_for_xpack @@ -84,4 +85,10 @@ SETUP_AUTO set -H stop_elasticsearch_service + assert_file_not_exist "/home/elasticsearch" +} + +@test "[$GROUP] remove Elasticsearch" { + # NOTE: this must be the last test, so that running oss tests does not already have the default distro still installed + clean_before_test } From 6fb2f9e2e8bc72cb16f509cf5e7721043ec5142b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 3 Sep 2019 14:28:21 -0400 Subject: [PATCH 021/103] Multi-get requests should wait for search active (#46283) When a shard has fallen search idle, and a non-realtime multi-get request is executed, today such requests do not wait for the shard to become search active and therefore such requests do not wait for a refresh to see the latest changes to the index. This also prevents such requests from triggering the shard as non-search idle, influencing the behavior of scheduled refreshes. This commit addresses this by attaching a listener to the shard search active state for multi-get requests. In this way, when the next scheduled refresh is executed, the multi-get request will then proceed. --- .../get/TransportShardMultiGetAction.java | 21 ++++++++ .../index/shard/IndexShardIT.java | 49 +++++++++++++------ 2 files changed, 54 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java index e7921ca489ce..4ae18010016f 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.get; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportActions; @@ -38,6 +39,8 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; + public class TransportShardMultiGetAction extends TransportSingleShardAction { private static final String ACTION_NAME = MultiGetAction.NAME + "[shard]"; @@ -75,6 +78,24 @@ public class TransportShardMultiGetAction extends TransportSingleShardAction listener) throws IOException { + IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); + IndexShard indexShard = indexService.getShard(shardId.id()); + if (request.realtime()) { // we are not tied to a refresh cycle here anyway + super.asyncShardOperation(request, shardId, listener); + } else { + indexShard.awaitShardSearchActive(b -> { + try { + super.asyncShardOperation(request, shardId, listener); + } catch (Exception ex) { + listener.onFailure(ex); + } + }); + } + } + @Override protected MultiGetShardResponse shardOperation(MultiGetShardRequest request, ShardId shardId) { IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 1d7969dbfde9..d513c80fcf25 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; @@ -108,6 +109,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.IntToLongFunction; import java.util.function.Predicate; import java.util.stream.Stream; @@ -687,7 +689,23 @@ public class IndexShardIT extends ESSingleNodeTestCase { return shardRouting; } - public void testAutomaticRefresh() throws InterruptedException { + public void testAutomaticRefreshSearch() throws InterruptedException { + runTestAutomaticRefresh(numDocs -> client().prepareSearch("test").get().getHits().getTotalHits().value); + } + + public void testAutomaticRefreshMultiGet() throws InterruptedException { + runTestAutomaticRefresh( + numDocs -> { + final MultiGetRequest request = new MultiGetRequest(); + request.realtime(false); + for (int i = 0; i < numDocs; i++) { + request.add("test", "" + i); + } + return Arrays.stream(client().multiGet(request).actionGet().getResponses()).filter(r -> r.getResponse().isExists()).count(); + }); + } + + private void runTestAutomaticRefresh(final IntToLongFunction count) throws InterruptedException { TimeValue randomTimeValue = randomFrom(random(), null, TimeValue.ZERO, TimeValue.timeValueMillis(randomIntBetween(0, 1000))); Settings.Builder builder = Settings.builder(); if (randomTimeValue != null) { @@ -720,31 +738,31 @@ public class IndexShardIT extends ESSingleNodeTestCase { ensureNoPendingScheduledRefresh(indexService.getThreadPool()); } } + CountDownLatch started = new CountDownLatch(1); Thread t = new Thread(() -> { - SearchResponse searchResponse; started.countDown(); do { - searchResponse = client().prepareSearch().get(); - } while (searchResponse.getHits().getTotalHits().value != totalNumDocs.get()); + + } while (count.applyAsLong(totalNumDocs.get()) != totalNumDocs.get()); }); t.start(); started.await(); - assertHitCount(client().prepareSearch().get(), 1); + assertThat(count.applyAsLong(totalNumDocs.get()), equalTo(1L)); for (int i = 1; i < numDocs; i++) { client().prepareIndex("test", "test", "" + i).setSource("{\"foo\" : \"bar\"}", XContentType.JSON) .execute(new ActionListener() { - @Override - public void onResponse(IndexResponse indexResponse) { - indexingDone.countDown(); - } + @Override + public void onResponse(IndexResponse indexResponse) { + indexingDone.countDown(); + } - @Override - public void onFailure(Exception e) { - indexingDone.countDown(); - throw new AssertionError(e); - } - }); + @Override + public void onFailure(Exception e) { + indexingDone.countDown(); + throw new AssertionError(e); + } + }); } indexingDone.await(); t.join(); @@ -756,7 +774,6 @@ public class IndexShardIT extends ESSingleNodeTestCase { IndexService indexService = createIndex("test", builder.build()); assertFalse(indexService.getIndexSettings().isExplicitRefresh()); ensureGreen(); - assertNoSearchHits(client().prepareSearch().get()); client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); IndexShard shard = indexService.getShard(0); assertFalse(shard.scheduledRefresh()); From 2d460e04d9714ff73df61672c41a7a4abc0bede7 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 3 Sep 2019 13:30:13 -0500 Subject: [PATCH 022/103] [ML][Transforms] fixing listener being called twice (#46284) --- .../xpack/dataframe/transforms/DataFrameTransformTask.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java index cdd3888e330b..bc39cbab0425 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java @@ -980,6 +980,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S // So, don't treat this like a checkpoint being completed, as no work was done. if (hasSourceChanged == false) { listener.onResponse(null); + return; } // TODO: needs cleanup super is called with a listener, but listener.onResponse is called below // super.onFinish() fortunately ignores the listener From 1054f351d395c568989dc38bf954523aa21c261f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 3 Sep 2019 14:51:16 -0400 Subject: [PATCH 023/103] Mute testRecoveryFromFailureOnTrimming Tracked at #46267 --- .../java/org/elasticsearch/index/translog/TranslogTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index daa6fdfe625e..a1330a36912a 100644 --- a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2285,6 +2285,7 @@ public class TranslogTests extends ESTestCase { * Tests the situation where the node crashes after a translog gen was committed to lucene, but before the translog had the chance * to clean up its files. */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46267") public void testRecoveryFromFailureOnTrimming() throws IOException { Path tempDir = createTempDir(); final FailSwitch fail = new FailSwitch(); From 25f84be99908439dc2cadb44df1d0674052a6a66 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 3 Sep 2019 16:19:25 -0600 Subject: [PATCH 024/103] Move MockRespository into test framework (#46298) This moves the `MockRespository` class into `test/framework/src/main` so it can be used across all modules and plugins in tests. --- .../elasticsearch/snapshots/mockstore/BlobContainerWrapper.java | 0 .../org/elasticsearch/snapshots/mockstore/BlobStoreWrapper.java | 0 .../org/elasticsearch/snapshots/mockstore/MockRepository.java | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/snapshots/mockstore/BlobContainerWrapper.java (100%) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/snapshots/mockstore/BlobStoreWrapper.java (100%) rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/snapshots/mockstore/MockRepository.java (100%) diff --git a/server/src/test/java/org/elasticsearch/snapshots/mockstore/BlobContainerWrapper.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/BlobContainerWrapper.java similarity index 100% rename from server/src/test/java/org/elasticsearch/snapshots/mockstore/BlobContainerWrapper.java rename to test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/BlobContainerWrapper.java diff --git a/server/src/test/java/org/elasticsearch/snapshots/mockstore/BlobStoreWrapper.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/BlobStoreWrapper.java similarity index 100% rename from server/src/test/java/org/elasticsearch/snapshots/mockstore/BlobStoreWrapper.java rename to test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/BlobStoreWrapper.java diff --git a/server/src/test/java/org/elasticsearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java similarity index 100% rename from server/src/test/java/org/elasticsearch/snapshots/mockstore/MockRepository.java rename to test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java From 126f87b36b1c96f92fe42849df2bd1a3d9c2cb2c Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Tue, 3 Sep 2019 16:30:12 -0700 Subject: [PATCH 025/103] First round of optimizations for vector functions. (#46294) This PR merges the `vectors-optimize-brute-force` feature branch, which makes the following changes to how vector functions are computed: * Precompute the L2 norm of each vector at indexing time. (#45390) * Switch to ByteBuffer for vector encoding. (#45936) * Decode vectors and while computing the vector function. (#46103) * Use an array instead of a List for the query vector. (#46155) * Precompute the normalized query vector when using cosine similarity. (#46190) Co-authored-by: Mayya Sharipova --- .../mapping/types/dense-vector.asciidoc | 2 +- .../mapping/types/sparse-vector.asciidoc | 2 +- .../search/function/ScriptScoreFunction.java | 8 +- .../ScriptScoreFunctionBuilder.java | 3 +- .../org/elasticsearch/script/ScoreScript.java | 22 ++ .../vectors/20_dense_vector_special_cases.yml | 2 +- .../mapper/DenseVectorFieldMapper.java | 27 +- .../mapper/SparseVectorFieldMapper.java | 2 +- .../vectors/mapper/VectorEncoderDecoder.java | 108 +++--- .../xpack/vectors/query/ScoreScriptUtils.java | 341 ++++++++++-------- .../xpack/vectors/query/whitelist.txt | 18 +- .../mapper/DenseVectorFieldMapperTests.java | 67 +++- .../mapper/SparseVectorFieldMapperTests.java | 81 ++++- .../mapper/VectorEncoderDecoderTests.java | 75 ++-- .../vectors/query/ScoreScriptUtilsTests.java | 101 ++++-- 15 files changed, 560 insertions(+), 299 deletions(-) diff --git a/docs/reference/mapping/types/dense-vector.asciidoc b/docs/reference/mapping/types/dense-vector.asciidoc index 9462fe544af9..a1799fae7188 100644 --- a/docs/reference/mapping/types/dense-vector.asciidoc +++ b/docs/reference/mapping/types/dense-vector.asciidoc @@ -54,4 +54,4 @@ PUT my_index/_doc/2 Internally, each document's dense vector is encoded as a binary doc value. Its size in bytes is equal to -`4 * dims`, where `dims`—the number of the vector's dimensions. \ No newline at end of file +`4 * dims + 4`, where `dims`—the number of the vector's dimensions. \ No newline at end of file diff --git a/docs/reference/mapping/types/sparse-vector.asciidoc b/docs/reference/mapping/types/sparse-vector.asciidoc index 7b437031513b..af3b6a510377 100644 --- a/docs/reference/mapping/types/sparse-vector.asciidoc +++ b/docs/reference/mapping/types/sparse-vector.asciidoc @@ -56,5 +56,5 @@ PUT my_index/_doc/2 Internally, each document's sparse vector is encoded as a binary doc value. Its size in bytes is equal to -`6 * NUMBER_OF_DIMENSIONS`, where `NUMBER_OF_DIMENSIONS` - +`6 * NUMBER_OF_DIMENSIONS + 4`, where `NUMBER_OF_DIMENSIONS` - number of the vector's dimensions. \ No newline at end of file diff --git a/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java b/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java index 960df44a6251..65dacd51e13c 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.Scorable; import org.elasticsearch.script.ExplainableScoreScript; import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.Script; +import org.elasticsearch.Version; import java.io.IOException; import java.util.Objects; @@ -52,7 +53,7 @@ public class ScriptScoreFunction extends ScoreFunction { private final int shardId; private final String indexName; - + private final Version indexVersion; public ScriptScoreFunction(Script sScript, ScoreScript.LeafFactory script) { super(CombineFunction.REPLACE); @@ -60,14 +61,16 @@ public class ScriptScoreFunction extends ScoreFunction { this.script = script; this.indexName = null; this.shardId = -1; + this.indexVersion = null; } - public ScriptScoreFunction(Script sScript, ScoreScript.LeafFactory script, String indexName, int shardId) { + public ScriptScoreFunction(Script sScript, ScoreScript.LeafFactory script, String indexName, int shardId, Version indexVersion) { super(CombineFunction.REPLACE); this.sScript = sScript; this.script = script; this.indexName = indexName; this.shardId = shardId; + this.indexVersion = indexVersion; } @Override @@ -77,6 +80,7 @@ public class ScriptScoreFunction extends ScoreFunction { leafScript.setScorer(scorer); leafScript._setIndexName(indexName); leafScript._setShard(shardId); + leafScript._setIndexVersion(indexVersion); return new LeafScoreFunction() { @Override public double score(int docId, float subQueryScore) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java index accfd2f65699..8fc2d4ff6b1a 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java @@ -94,7 +94,8 @@ public class ScriptScoreFunctionBuilder extends ScoreFunctionBuilder params, SearchLookup lookup, LeafReaderContext leafContext) { // null check needed b/c of expression engine subclass @@ -155,6 +157,19 @@ public abstract class ScoreScript { } } + /** + * Starting a name with underscore, so that the user cannot access this function directly through a script + * It is only used within predefined painless functions. + * @return index version or throws an exception if the index version is not set up for this script instance + */ + public Version _getIndexVersion() { + if (indexVersion != null) { + return indexVersion; + } else { + throw new IllegalArgumentException("index version can not be looked up!"); + } + } + /** * Starting a name with underscore, so that the user cannot access this function directly through a script */ @@ -169,6 +184,13 @@ public abstract class ScoreScript { this.indexName = indexName; } + /** + * Starting a name with underscore, so that the user cannot access this function directly through a script + */ + public void _setIndexVersion(Version indexVersion) { + this.indexVersion = indexVersion; + } + /** A factory to construct {@link ScoreScript} instances. */ public interface LeafFactory { diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/20_dense_vector_special_cases.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/20_dense_vector_special_cases.yml index 070844094c41..98a68cab9ca0 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/20_dense_vector_special_cases.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/vectors/20_dense_vector_special_cases.yml @@ -46,7 +46,7 @@ setup: index: test-index id: 2 body: - my_dense_vector: [10.9, 10.9, 10.9] + my_dense_vector: [10.5, 10.9, 10.4] - do: indices.refresh: {} diff --git a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapper.java b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapper.java index a7773e3e3c52..b1518d3ecd58 100644 --- a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapper.java +++ b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapper.java @@ -13,6 +13,7 @@ import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser.Token; @@ -25,10 +26,11 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.query.QueryShardContext; -import org.elasticsearch.xpack.vectors.query.VectorDVIndexFieldData; import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.xpack.vectors.query.VectorDVIndexFieldData; import java.io.IOException; +import java.nio.ByteBuffer; import java.time.ZoneId; import java.util.List; import java.util.Map; @@ -180,8 +182,11 @@ public class DenseVectorFieldMapper extends FieldMapper implements ArrayValueMap // encode array of floats as array of integers and store into buf // this code is here and not int the VectorEncoderDecoder so not to create extra arrays - byte[] buf = new byte[dims * INT_BYTES]; - int offset = 0; + byte[] bytes = indexCreatedVersion.onOrAfter(Version.V_7_5_0) ? new byte[dims * INT_BYTES + INT_BYTES] : new byte[dims * INT_BYTES]; + + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + double dotProduct = 0f; + int dim = 0; for (Token token = context.parser().nextToken(); token != Token.END_ARRAY; token = context.parser().nextToken()) { if (dim++ >= dims) { @@ -190,18 +195,22 @@ public class DenseVectorFieldMapper extends FieldMapper implements ArrayValueMap } ensureExpectedToken(Token.VALUE_NUMBER, token, context.parser()::getTokenLocation); float value = context.parser().floatValue(true); - int intValue = Float.floatToIntBits(value); - buf[offset++] = (byte) (intValue >> 24); - buf[offset++] = (byte) (intValue >> 16); - buf[offset++] = (byte) (intValue >> 8); - buf[offset++] = (byte) intValue; + + byteBuffer.putFloat(value); + dotProduct += value * value; } if (dim != dims) { throw new IllegalArgumentException("Field [" + name() + "] of type [" + typeName() + "] of doc [" + context.sourceToParse().id() + "] has number of dimensions [" + dim + "] less than defined in the mapping [" + dims +"]"); } - BinaryDocValuesField field = new BinaryDocValuesField(fieldType().name(), new BytesRef(buf, 0, offset)); + + if (indexCreatedVersion.onOrAfter(Version.V_7_5_0)) { + // encode vector magnitude at the end + float vectorMagnitude = (float) Math.sqrt(dotProduct); + byteBuffer.putFloat(vectorMagnitude); + } + BinaryDocValuesField field = new BinaryDocValuesField(fieldType().name(), new BytesRef(bytes)); if (context.doc().getByKey(fieldType().name()) != null) { throw new IllegalArgumentException("Field [" + name() + "] of type [" + typeName() + "] doesn't not support indexing multiple values for the same field in the same document"); diff --git a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapper.java b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapper.java index 3c551a4ee525..38ea21922f44 100644 --- a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapper.java +++ b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapper.java @@ -181,7 +181,7 @@ public class SparseVectorFieldMapper extends FieldMapper { } } - BytesRef br = VectorEncoderDecoder.encodeSparseVector(dims, values, dimCount); + BytesRef br = VectorEncoderDecoder.encodeSparseVector(indexCreatedVersion, dims, values, dimCount); BinaryDocValuesField field = new BinaryDocValuesField(fieldType().name(), br); context.doc().addWithKey(fieldType().name(), field); } diff --git a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoder.java b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoder.java index 67078b370ea9..2d591aaccd48 100644 --- a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoder.java +++ b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoder.java @@ -9,6 +9,9 @@ package org.elasticsearch.xpack.vectors.mapper; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InPlaceMergeSorter; +import org.elasticsearch.Version; + +import java.nio.ByteBuffer; // static utility functions for encoding and decoding dense_vector and sparse_vector fields public final class VectorEncoderDecoder { @@ -19,81 +22,84 @@ public final class VectorEncoderDecoder { /** * Encodes a sparse array represented by values, dims and dimCount into a bytes array - BytesRef - * BytesRef: int[] floats encoded as integers values, 2 bytes for each dimension - * @param values - values of the sparse array + * BytesRef: int[] floats encoded as integers values, 2 bytes for each dimension, length of vector + * @param indexVersion - index version * @param dims - dims of the sparse array + * @param values - values of the sparse array * @param dimCount - number of the dimensions, necessary as values and dims are dynamically created arrays, * and may be over-allocated * @return BytesRef */ - public static BytesRef encodeSparseVector(int[] dims, float[] values, int dimCount) { + public static BytesRef encodeSparseVector(Version indexVersion, int[] dims, float[] values, int dimCount) { // 1. Sort dims and values sortSparseDimsValues(dims, values, dimCount); - byte[] buf = new byte[dimCount * (INT_BYTES + SHORT_BYTES)]; // 2. Encode dimensions // as each dimension is a positive value that doesn't exceed 65535, 2 bytes is enough for encoding it - int offset = 0; + byte[] bytes = indexVersion.onOrAfter(Version.V_7_5_0) ? new byte[dimCount * (INT_BYTES + SHORT_BYTES) + INT_BYTES] : + new byte[dimCount * (INT_BYTES + SHORT_BYTES)]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + for (int dim = 0; dim < dimCount; dim++) { - buf[offset] = (byte) (dims[dim] >> 8); - buf[offset+1] = (byte) dims[dim]; - offset += SHORT_BYTES; + int dimValue = dims[dim]; + byteBuffer.put((byte) (dimValue >> 8)); + byteBuffer.put((byte) dimValue); } // 3. Encode values + double dotProduct = 0.0f; for (int dim = 0; dim < dimCount; dim++) { - int intValue = Float.floatToIntBits(values[dim]); - buf[offset] = (byte) (intValue >> 24); - buf[offset+1] = (byte) (intValue >> 16); - buf[offset+2] = (byte) (intValue >> 8); - buf[offset+3] = (byte) intValue; - offset += INT_BYTES; + float value = values[dim]; + byteBuffer.putFloat(value); + dotProduct += value * value; } - return new BytesRef(buf); + // 4. Encode vector magnitude at the end + if (indexVersion.onOrAfter(Version.V_7_5_0)) { + float vectorMagnitude = (float) Math.sqrt(dotProduct); + byteBuffer.putFloat(vectorMagnitude); + } + + return new BytesRef(bytes); } /** * Decodes the first part of BytesRef into sparse vector dimensions + * @param indexVersion - index version * @param vectorBR - sparse vector encoded in BytesRef */ - public static int[] decodeSparseVectorDims(BytesRef vectorBR) { - if (vectorBR == null) { - throw new IllegalArgumentException("A document doesn't have a value for a vector field!"); - } - int dimCount = vectorBR.length / (INT_BYTES + SHORT_BYTES); + public static int[] decodeSparseVectorDims(Version indexVersion, BytesRef vectorBR) { + int dimCount = indexVersion.onOrAfter(Version.V_7_5_0) + ? (vectorBR.length - INT_BYTES) / (INT_BYTES + SHORT_BYTES) + : vectorBR.length / (INT_BYTES + SHORT_BYTES); + ByteBuffer byteBuffer = ByteBuffer.wrap(vectorBR.bytes, vectorBR.offset, dimCount * SHORT_BYTES); + int[] dims = new int[dimCount]; - int offset = vectorBR.offset; for (int dim = 0; dim < dimCount; dim++) { - dims[dim] = ((vectorBR.bytes[offset] & 0xFF) << 8) | (vectorBR.bytes[offset+1] & 0xFF); - offset += SHORT_BYTES; + dims[dim] = ((byteBuffer.get() & 0xFF) << 8) | (byteBuffer.get() & 0xFF); } return dims; } /** * Decodes the second part of the BytesRef into sparse vector values + * @param indexVersion - index version * @param vectorBR - sparse vector encoded in BytesRef */ - public static float[] decodeSparseVector(BytesRef vectorBR) { - if (vectorBR == null) { - throw new IllegalArgumentException("A document doesn't have a value for a vector field!"); - } - int dimCount = vectorBR.length / (INT_BYTES + SHORT_BYTES); - int offset = vectorBR.offset + SHORT_BYTES * dimCount; //calculate the offset from where values are encoded + public static float[] decodeSparseVector(Version indexVersion, BytesRef vectorBR) { + int dimCount = indexVersion.onOrAfter(Version.V_7_5_0) + ? (vectorBR.length - INT_BYTES) / (INT_BYTES + SHORT_BYTES) + : vectorBR.length / (INT_BYTES + SHORT_BYTES); + int offset = vectorBR.offset + SHORT_BYTES * dimCount; float[] vector = new float[dimCount]; + + ByteBuffer byteBuffer = ByteBuffer.wrap(vectorBR.bytes, offset, dimCount * INT_BYTES); for (int dim = 0; dim < dimCount; dim++) { - int intValue = ((vectorBR.bytes[offset] & 0xFF) << 24) | - ((vectorBR.bytes[offset+1] & 0xFF) << 16) | - ((vectorBR.bytes[offset+2] & 0xFF) << 8) | - (vectorBR.bytes[offset+3] & 0xFF); - vector[dim] = Float.intBitsToFloat(intValue); - offset = offset + INT_BYTES; + vector[dim] = byteBuffer.getFloat(); } return vector; } - /** * Sorts dimensions in the ascending order and * sorts values in the same order as their corresponding dimensions @@ -150,24 +156,20 @@ public final class VectorEncoderDecoder { }.sort(0, n); } + public static int denseVectorLength(Version indexVersion, BytesRef vectorBR) { + return indexVersion.onOrAfter(Version.V_7_5_0) + ? (vectorBR.length - INT_BYTES) / INT_BYTES + : vectorBR.length / INT_BYTES; + } + /** - * Decodes a BytesRef into an array of floats - * @param vectorBR - dense vector encoded in BytesRef + * Decodes the last 4 bytes of the encoded vector, which contains the vector magnitude. + * NOTE: this function can only be called on vectors from an index version greater than or + * equal to 7.5.0, since vectors created prior to that do not store the magnitude. */ - public static float[] decodeDenseVector(BytesRef vectorBR) { - if (vectorBR == null) { - throw new IllegalArgumentException("A document doesn't have a value for a vector field!"); - } - int dimCount = vectorBR.length / INT_BYTES; - float[] vector = new float[dimCount]; - int offset = vectorBR.offset; - for (int dim = 0; dim < dimCount; dim++) { - int intValue = ((vectorBR.bytes[offset++] & 0xFF) << 24) | - ((vectorBR.bytes[offset++] & 0xFF) << 16) | - ((vectorBR.bytes[offset++] & 0xFF) << 8) | - (vectorBR.bytes[offset++] & 0xFF); - vector[dim] = Float.intBitsToFloat(intValue); - } - return vector; + public static float decodeVectorMagnitude(Version indexVersion, BytesRef vectorBR) { + assert indexVersion.onOrAfter(Version.V_7_5_0); + ByteBuffer byteBuffer = ByteBuffer.wrap(vectorBR.bytes, vectorBR.offset, vectorBR.length); + return byteBuffer.getFloat(vectorBR.offset + vectorBR.length - 4); } } diff --git a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtils.java b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtils.java index 9c54f267ca14..f286ab732855 100644 --- a/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtils.java +++ b/x-pack/plugin/vectors/src/main/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtils.java @@ -8,9 +8,11 @@ package org.elasticsearch.xpack.vectors.query; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.xpack.vectors.mapper.VectorEncoderDecoder; -import java.util.Iterator; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -19,132 +21,166 @@ import static org.elasticsearch.xpack.vectors.mapper.VectorEncoderDecoder.sortSp public class ScoreScriptUtils { //**************FUNCTIONS FOR DENSE VECTORS + // Functions are implemented as classes to accept a hidden parameter scoreScript that contains some index settings. + // Also, constructors for some functions accept queryVector to calculate and cache queryVectorMagnitude only once + // per script execution for all documents. - /** - * Calculate l1 norm - Manhattan distance - * between a query's dense vector and documents' dense vectors - * - * @param queryVector the query vector parsed as {@code List} from json - * @param dvs VectorScriptDocValues representing encoded documents' vectors - */ - public static double l1norm(List queryVector, VectorScriptDocValues.DenseVectorScriptDocValues dvs){ - BytesRef value = dvs.getEncodedValue(); - float[] docVector = VectorEncoderDecoder.decodeDenseVector(value); - if (queryVector.size() != docVector.length) { - throw new IllegalArgumentException("Can't calculate l1norm! The number of dimensions of the query vector [" + - queryVector.size() + "] is different from the documents' vectors [" + docVector.length + "]."); + public static class DenseVectorFunction { + final ScoreScript scoreScript; + final float[] queryVector; + + public DenseVectorFunction(ScoreScript scoreScript, List queryVector) { + this(scoreScript, queryVector, false); } - Iterator queryVectorIter = queryVector.iterator(); - double l1norm = 0; - for (int dim = 0; dim < docVector.length; dim++){ - l1norm += Math.abs(queryVectorIter.next().floatValue() - docVector[dim]); + + /** + * Constructs a dense vector function. + * + * @param scoreScript The script in which this function was referenced. + * @param queryVector The query vector. + * @param normalizeQuery Whether the provided query should be normalized to unit length. + */ + public DenseVectorFunction(ScoreScript scoreScript, + List queryVector, + boolean normalizeQuery) { + this.scoreScript = scoreScript; + + this.queryVector = new float[queryVector.size()]; + double queryMagnitude = 0.0; + for (int i = 0; i < queryVector.size(); i++) { + float value = queryVector.get(i).floatValue(); + this.queryVector[i] = value; + queryMagnitude += value * value; + } + queryMagnitude = Math.sqrt(queryMagnitude); + + if (normalizeQuery) { + for (int dim = 0; dim < this.queryVector.length; dim++) { + this.queryVector[dim] /= queryMagnitude; + } + } + } + + public void validateDocVector(BytesRef vector) { + if (vector == null) { + throw new IllegalArgumentException("A document doesn't have a value for a vector field!"); + } + + int vectorLength = VectorEncoderDecoder.denseVectorLength(scoreScript._getIndexVersion(), vector); + if (queryVector.length != vectorLength) { + throw new IllegalArgumentException("The query vector has a different number of dimensions [" + + queryVector.length + "] than the document vectors [" + vectorLength + "]."); + } } - return l1norm; } - /** - * Calculate l2 norm - Euclidean distance - * between a query's dense vector and documents' dense vectors - * - * @param queryVector the query vector parsed as {@code List} from json - * @param dvs VectorScriptDocValues representing encoded documents' vectors - */ - public static double l2norm(List queryVector, VectorScriptDocValues.DenseVectorScriptDocValues dvs){ - BytesRef value = dvs.getEncodedValue(); - float[] docVector = VectorEncoderDecoder.decodeDenseVector(value); - if (queryVector.size() != docVector.length) { - throw new IllegalArgumentException("Can't calculate l2norm! The number of dimensions of the query vector [" + - queryVector.size() + "] is different from the documents' vectors [" + docVector.length + "]."); + // Calculate l1 norm (Manhattan distance) between a query's dense vector and documents' dense vectors + public static final class L1Norm extends DenseVectorFunction { + + public L1Norm(ScoreScript scoreScript, List queryVector) { + super(scoreScript, queryVector); } - Iterator queryVectorIter = queryVector.iterator(); - double l2norm = 0; - for (int dim = 0; dim < docVector.length; dim++){ - double diff = queryVectorIter.next().floatValue() - docVector[dim]; - l2norm += diff * diff; + + public double l1norm(VectorScriptDocValues.DenseVectorScriptDocValues dvs) { + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); + ByteBuffer byteBuffer = ByteBuffer.wrap(vector.bytes, vector.offset, vector.length); + double l1norm = 0; + + for (float queryValue : queryVector) { + l1norm += Math.abs(queryValue - byteBuffer.getFloat()); + } + return l1norm; } - return Math.sqrt(l2norm); } + // Calculate l2 norm (Euclidean distance) between a query's dense vector and documents' dense vectors + public static final class L2Norm extends DenseVectorFunction { - /** - * Calculate a dot product between a query's dense vector and documents' dense vectors - * - * @param queryVector the query vector parsed as {@code List} from json - * @param dvs VectorScriptDocValues representing encoded documents' vectors - */ - public static double dotProduct(List queryVector, VectorScriptDocValues.DenseVectorScriptDocValues dvs){ - BytesRef value = dvs.getEncodedValue(); - float[] docVector = VectorEncoderDecoder.decodeDenseVector(value); - if (queryVector.size() != docVector.length) { - throw new IllegalArgumentException("Can't calculate dotProduct! The number of dimensions of the query vector [" + - queryVector.size() + "] is different from the documents' vectors [" + docVector.length + "]."); + public L2Norm(ScoreScript scoreScript, List queryVector) { + super(scoreScript, queryVector); + } + + public double l2norm(VectorScriptDocValues.DenseVectorScriptDocValues dvs) { + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); + ByteBuffer byteBuffer = ByteBuffer.wrap(vector.bytes, vector.offset, vector.length); + + double l2norm = 0; + for (float queryValue : queryVector) { + double diff = queryValue - byteBuffer.getFloat(); + l2norm += diff * diff; + } + return Math.sqrt(l2norm); } - return intDotProduct(queryVector, docVector); } - /** - * Calculate cosine similarity between a query's dense vector and documents' dense vectors - * - * CosineSimilarity is implemented as a class to use - * painless script caching to calculate queryVectorMagnitude - * only once per script execution for all documents. - * A user will call `cosineSimilarity(params.queryVector, doc['my_vector'])` - */ - public static final class CosineSimilarity { - final double queryVectorMagnitude; - final List queryVector; + // Calculate a dot product between a query's dense vector and documents' dense vectors + public static final class DotProduct extends DenseVectorFunction { - // calculate queryVectorMagnitude once per query execution - public CosineSimilarity(List queryVector) { - this.queryVector = queryVector; + public DotProduct(ScoreScript scoreScript, List queryVector) { + super(scoreScript, queryVector); + } + + public double dotProduct(VectorScriptDocValues.DenseVectorScriptDocValues dvs){ + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); + ByteBuffer byteBuffer = ByteBuffer.wrap(vector.bytes, vector.offset, vector.length); double dotProduct = 0; - for (Number value : queryVector) { - float floatValue = value.floatValue(); - dotProduct += floatValue * floatValue; + for (float queryValue : queryVector) { + dotProduct += queryValue * byteBuffer.getFloat(); } - this.queryVectorMagnitude = Math.sqrt(dotProduct); + return dotProduct; + } + } + + // Calculate cosine similarity between a query's dense vector and documents' dense vectors + public static final class CosineSimilarity extends DenseVectorFunction { + + public CosineSimilarity(ScoreScript scoreScript, List queryVector) { + super(scoreScript, queryVector, true); } public double cosineSimilarity(VectorScriptDocValues.DenseVectorScriptDocValues dvs) { - BytesRef value = dvs.getEncodedValue(); - float[] docVector = VectorEncoderDecoder.decodeDenseVector(value); - if (queryVector.size() != docVector.length) { - throw new IllegalArgumentException("Can't calculate cosineSimilarity! The number of dimensions of the query vector [" + - queryVector.size() + "] is different from the documents' vectors [" + docVector.length + "]."); - } + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); - // calculate docVector magnitude - double dotProduct = 0f; - for (int dim = 0; dim < docVector.length; dim++) { - dotProduct += (double) docVector[dim] * docVector[dim]; - } - final double docVectorMagnitude = Math.sqrt(dotProduct); + ByteBuffer byteBuffer = ByteBuffer.wrap(vector.bytes, vector.offset, vector.length); - double docQueryDotProduct = intDotProduct(queryVector, docVector); - return docQueryDotProduct / (docVectorMagnitude * queryVectorMagnitude); + double dotProduct = 0.0; + double vectorMagnitude = 0.0f; + if (scoreScript._getIndexVersion().onOrAfter(Version.V_7_5_0)) { + for (float queryValue : queryVector) { + dotProduct += queryValue * byteBuffer.getFloat(); + } + vectorMagnitude = VectorEncoderDecoder.decodeVectorMagnitude(scoreScript._getIndexVersion(), vector); + } else { + for (float queryValue : queryVector) { + float docValue = byteBuffer.getFloat(); + dotProduct += queryValue * docValue; + vectorMagnitude += docValue * docValue; + } + vectorMagnitude = (float) Math.sqrt(vectorMagnitude); + } + return dotProduct / vectorMagnitude; } } - private static double intDotProduct(List v1, float[] v2){ - double v1v2DotProduct = 0; - Iterator v1Iter = v1.iterator(); - for (int dim = 0; dim < v2.length; dim++) { - v1v2DotProduct += v1Iter.next().floatValue() * v2[dim]; - } - return v1v2DotProduct; - } - - //**************FUNCTIONS FOR SPARSE VECTORS + // Functions are implemented as classes to accept a hidden parameter scoreScript that contains some index settings. + // Also, constructors for some functions accept queryVector to calculate and cache queryVectorMagnitude only once + // per script execution for all documents. - public static class VectorSparseFunctions { + public static class SparseVectorFunction { + final ScoreScript scoreScript; final float[] queryValues; final int[] queryDims; // prepare queryVector once per script execution // queryVector represents a map of dimensions to values - public VectorSparseFunctions(Map queryVector) { + public SparseVectorFunction(ScoreScript scoreScript, Map queryVector) { + this.scoreScript = scoreScript; //break vector into two arrays dims and values int n = queryVector.size(); queryValues = new float[n]; @@ -162,26 +198,26 @@ public class ScoreScriptUtils { // Sort dimensions in the ascending order and sort values in the same order as their corresponding dimensions sortSparseDimsFloatValues(queryDims, queryValues, n); } + + public void validateDocVector(BytesRef vector) { + if (vector == null) { + throw new IllegalArgumentException("A document doesn't have a value for a vector field!"); + } + } } - /** - * Calculate l1 norm - Manhattan distance - * between a query's sparse vector and documents' sparse vectors - * - * L1NormSparse is implemented as a class to use - * painless script caching to prepare queryVector - * only once per script execution for all documents. - * A user will call `l1normSparse(params.queryVector, doc['my_vector'])` - */ - public static final class L1NormSparse extends VectorSparseFunctions { - public L1NormSparse(Map queryVector) { - super(queryVector); + // Calculate l1 norm (Manhattan distance) between a query's sparse vector and documents' sparse vectors + public static final class L1NormSparse extends SparseVectorFunction { + public L1NormSparse(ScoreScript scoreScript,Map queryVector) { + super(scoreScript, queryVector); } public double l1normSparse(VectorScriptDocValues.SparseVectorScriptDocValues dvs) { - BytesRef value = dvs.getEncodedValue(); - int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(value); - float[] docValues = VectorEncoderDecoder.decodeSparseVector(value); + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); + + int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(scoreScript._getIndexVersion(), vector); + float[] docValues = VectorEncoderDecoder.decodeSparseVector(scoreScript._getIndexVersion(), vector); int queryIndex = 0; int docIndex = 0; double l1norm = 0; @@ -210,24 +246,18 @@ public class ScoreScriptUtils { } } - /** - * Calculate l2 norm - Euclidean distance - * between a query's sparse vector and documents' sparse vectors - * - * L2NormSparse is implemented as a class to use - * painless script caching to prepare queryVector - * only once per script execution for all documents. - * A user will call `l2normSparse(params.queryVector, doc['my_vector'])` - */ - public static final class L2NormSparse extends VectorSparseFunctions { - public L2NormSparse(Map queryVector) { - super(queryVector); + // Calculate l2 norm (Euclidean distance) between a query's sparse vector and documents' sparse vectors + public static final class L2NormSparse extends SparseVectorFunction { + public L2NormSparse(ScoreScript scoreScript, Map queryVector) { + super(scoreScript, queryVector); } public double l2normSparse(VectorScriptDocValues.SparseVectorScriptDocValues dvs) { - BytesRef value = dvs.getEncodedValue(); - int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(value); - float[] docValues = VectorEncoderDecoder.decodeSparseVector(value); + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); + + int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(scoreScript._getIndexVersion(), vector); + float[] docValues = VectorEncoderDecoder.decodeSparseVector(scoreScript._getIndexVersion(), vector); int queryIndex = 0; int docIndex = 0; double l2norm = 0; @@ -259,40 +289,28 @@ public class ScoreScriptUtils { } } - /** - * Calculate a dot product between a query's sparse vector and documents' sparse vectors - * - * DotProductSparse is implemented as a class to use - * painless script caching to prepare queryVector - * only once per script execution for all documents. - * A user will call `dotProductSparse(params.queryVector, doc['my_vector'])` - */ - public static final class DotProductSparse extends VectorSparseFunctions { - public DotProductSparse(Map queryVector) { - super(queryVector); + // Calculate a dot product between a query's sparse vector and documents' sparse vectors + public static final class DotProductSparse extends SparseVectorFunction { + public DotProductSparse(ScoreScript scoreScript, Map queryVector) { + super(scoreScript, queryVector); } public double dotProductSparse(VectorScriptDocValues.SparseVectorScriptDocValues dvs) { - BytesRef value = dvs.getEncodedValue(); - int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(value); - float[] docValues = VectorEncoderDecoder.decodeSparseVector(value); + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); + + int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(scoreScript._getIndexVersion(), vector); + float[] docValues = VectorEncoderDecoder.decodeSparseVector(scoreScript._getIndexVersion(), vector); return intDotProductSparse(queryValues, queryDims, docValues, docDims); } } - /** - * Calculate cosine similarity between a query's sparse vector and documents' sparse vectors - * - * CosineSimilaritySparse is implemented as a class to use - * painless script caching to prepare queryVector and calculate queryVectorMagnitude - * only once per script execution for all documents. - * A user will call `cosineSimilaritySparse(params.queryVector, doc['my_vector'])` - */ - public static final class CosineSimilaritySparse extends VectorSparseFunctions { + // Calculate cosine similarity between a query's sparse vector and documents' sparse vectors + public static final class CosineSimilaritySparse extends SparseVectorFunction { final double queryVectorMagnitude; - public CosineSimilaritySparse(Map queryVector) { - super(queryVector); + public CosineSimilaritySparse(ScoreScript scoreScript, Map queryVector) { + super(scoreScript, queryVector); double dotProduct = 0; for (int i = 0; i< queryDims.length; i++) { dotProduct += queryValues[i] * queryValues[i]; @@ -301,18 +319,23 @@ public class ScoreScriptUtils { } public double cosineSimilaritySparse(VectorScriptDocValues.SparseVectorScriptDocValues dvs) { - BytesRef value = dvs.getEncodedValue(); - int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(value); - float[] docValues = VectorEncoderDecoder.decodeSparseVector(value); + BytesRef vector = dvs.getEncodedValue(); + validateDocVector(vector); - // calculate docVector magnitude - double dotProduct = 0; - for (float docValue : docValues) { - dotProduct += (double) docValue * docValue; - } - final double docVectorMagnitude = Math.sqrt(dotProduct); + int[] docDims = VectorEncoderDecoder.decodeSparseVectorDims(scoreScript._getIndexVersion(), vector); + float[] docValues = VectorEncoderDecoder.decodeSparseVector(scoreScript._getIndexVersion(), vector); double docQueryDotProduct = intDotProductSparse(queryValues, queryDims, docValues, docDims); + double docVectorMagnitude = 0.0f; + if (scoreScript._getIndexVersion().onOrAfter(Version.V_7_5_0)) { + docVectorMagnitude = VectorEncoderDecoder.decodeVectorMagnitude(scoreScript._getIndexVersion(), vector); + } else { + for (float docValue : docValues) { + docVectorMagnitude += docValue * docValue; + } + docVectorMagnitude = (float) Math.sqrt(docVectorMagnitude); + } + return docQueryDotProduct / (docVectorMagnitude * queryVectorMagnitude); } } diff --git a/x-pack/plugin/vectors/src/main/resources/org/elasticsearch/xpack/vectors/query/whitelist.txt b/x-pack/plugin/vectors/src/main/resources/org/elasticsearch/xpack/vectors/query/whitelist.txt index 252d4356f9ca..42d6e6d0b0f7 100644 --- a/x-pack/plugin/vectors/src/main/resources/org/elasticsearch/xpack/vectors/query/whitelist.txt +++ b/x-pack/plugin/vectors/src/main/resources/org/elasticsearch/xpack/vectors/query/whitelist.txt @@ -9,14 +9,16 @@ class org.elasticsearch.xpack.vectors.query.VectorScriptDocValues$DenseVectorScr } class org.elasticsearch.xpack.vectors.query.VectorScriptDocValues$SparseVectorScriptDocValues { } +class org.elasticsearch.script.ScoreScript @no_import { +} static_import { - double l1norm(List, VectorScriptDocValues.DenseVectorScriptDocValues) from_class org.elasticsearch.xpack.vectors.query.ScoreScriptUtils - double l2norm(List, VectorScriptDocValues.DenseVectorScriptDocValues) from_class org.elasticsearch.xpack.vectors.query.ScoreScriptUtils - double cosineSimilarity(List, VectorScriptDocValues.DenseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$CosineSimilarity - double dotProduct(List, VectorScriptDocValues.DenseVectorScriptDocValues) from_class org.elasticsearch.xpack.vectors.query.ScoreScriptUtils - double l1normSparse(Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$L1NormSparse - double l2normSparse(Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$L2NormSparse - double dotProductSparse(Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$DotProductSparse - double cosineSimilaritySparse(Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$CosineSimilaritySparse + double l1norm(org.elasticsearch.script.ScoreScript, List, VectorScriptDocValues.DenseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$L1Norm + double l2norm(org.elasticsearch.script.ScoreScript, List, VectorScriptDocValues.DenseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$L2Norm + double cosineSimilarity(org.elasticsearch.script.ScoreScript, List, VectorScriptDocValues.DenseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$CosineSimilarity + double dotProduct(org.elasticsearch.script.ScoreScript, List, VectorScriptDocValues.DenseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$DotProduct + double l1normSparse(org.elasticsearch.script.ScoreScript, Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$L1NormSparse + double l2normSparse(org.elasticsearch.script.ScoreScript, Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$L2NormSparse + double dotProductSparse(org.elasticsearch.script.ScoreScript, Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$DotProductSparse + double cosineSimilaritySparse(org.elasticsearch.script.ScoreScript, Map, VectorScriptDocValues.SparseVectorScriptDocValues) bound_to org.elasticsearch.xpack.vectors.query.ScoreScriptUtils$CosineSimilaritySparse } \ No newline at end of file diff --git a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapperTests.java b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapperTests.java index d1b37c73a246..52ef487935b6 100644 --- a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapperTests.java +++ b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/DenseVectorFieldMapperTests.java @@ -10,9 +10,12 @@ package org.elasticsearch.xpack.vectors.mapper; import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; @@ -27,6 +30,7 @@ import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.vectors.Vectors; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collection; import static org.hamcrest.Matchers.containsString; @@ -38,6 +42,12 @@ public class DenseVectorFieldMapperTests extends ESSingleNodeTestCase { return pluginList(Vectors.class, XPackPlugin.class); } + // this allows to set indexVersion as it is a private setting + @Override + protected boolean forbidPrivateIndexSettings() { + return false; + } + public void testMappingExceedDimsLimit() throws IOException { IndexService indexService = createIndex("test-index"); DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); @@ -55,6 +65,7 @@ public class DenseVectorFieldMapperTests extends ESSingleNodeTestCase { } public void testDefaults() throws Exception { + Version indexVersion = Version.CURRENT; IndexService indexService = createIndex("test-index"); DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); String mapping = Strings.toString(XContentFactory.jsonBuilder() @@ -69,6 +80,11 @@ public class DenseVectorFieldMapperTests extends ESSingleNodeTestCase { DocumentMapper mapper = parser.parse("_doc", new CompressedXContent(mapping)); float[] validVector = {-12.1f, 100.7f, -4}; + double dotProduct = 0.0f; + for (float value: validVector) { + dotProduct += value * value; + } + float expectedMagnitude = (float) Math.sqrt(dotProduct); ParsedDocument doc1 = mapper.parse(new SourceToParse("test-index", "_doc", "1", BytesReference .bytes(XContentFactory.jsonBuilder() .startObject() @@ -80,7 +96,9 @@ public class DenseVectorFieldMapperTests extends ESSingleNodeTestCase { assertThat(fields[0], instanceOf(BinaryDocValuesField.class)); // assert that after decoding the indexed value is equal to expected BytesRef vectorBR = fields[0].binaryValue(); - float[] decodedValues = VectorEncoderDecoder.decodeDenseVector(vectorBR); + float[] decodedValues = decodeDenseVector(indexVersion, vectorBR); + float decodedMagnitude = VectorEncoderDecoder.decodeVectorMagnitude(indexVersion, vectorBR); + assertEquals(expectedMagnitude, decodedMagnitude, 0.001f); assertArrayEquals( "Decoded dense vector values is not equal to the indexed one.", validVector, @@ -89,6 +107,53 @@ public class DenseVectorFieldMapperTests extends ESSingleNodeTestCase { ); } + public void testAddDocumentsToIndexBefore_V_7_5_0() throws Exception { + Version indexVersion = Version.V_7_4_0; + IndexService indexService = createIndex("test-index7_4", + Settings.builder().put(IndexMetaData.SETTING_INDEX_VERSION_CREATED.getKey(), indexVersion).build()); + DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); + String mapping = Strings.toString(XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("my-dense-vector").field("type", "dense_vector").field("dims", 3) + .endObject() + .endObject() + .endObject() + .endObject()); + DocumentMapper mapper = parser.parse("_doc", new CompressedXContent(mapping)); + float[] validVector = {-12.1f, 100.7f, -4}; + ParsedDocument doc1 = mapper.parse(new SourceToParse("test-index7_4", "_doc", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .startArray("my-dense-vector").value(validVector[0]).value(validVector[1]).value(validVector[2]).endArray() + .endObject()), + XContentType.JSON)); + IndexableField[] fields = doc1.rootDoc().getFields("my-dense-vector"); + assertEquals(1, fields.length); + assertThat(fields[0], instanceOf(BinaryDocValuesField.class)); + // assert that after decoding the indexed value is equal to expected + BytesRef vectorBR = fields[0].binaryValue(); + float[] decodedValues = decodeDenseVector(indexVersion, vectorBR); + assertArrayEquals( + "Decoded dense vector values is not equal to the indexed one.", + validVector, + decodedValues, + 0.001f + ); + } + + private static float[] decodeDenseVector(Version indexVersion, BytesRef encodedVector) { + int dimCount = VectorEncoderDecoder.denseVectorLength(indexVersion, encodedVector); + float[] vector = new float[dimCount]; + + ByteBuffer byteBuffer = ByteBuffer.wrap(encodedVector.bytes, encodedVector.offset, encodedVector.length); + for (int dim = 0; dim < dimCount; dim++) { + vector[dim] = byteBuffer.getFloat(); + } + return vector; + } + public void testDocumentsWithIncorrectDims() throws Exception { IndexService indexService = createIndex("test-index"); int dims = 3; diff --git a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapperTests.java b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapperTests.java index e1e110a750b9..915908ade428 100644 --- a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapperTests.java +++ b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/SparseVectorFieldMapperTests.java @@ -10,9 +10,12 @@ package org.elasticsearch.xpack.vectors.mapper; import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; @@ -43,7 +46,7 @@ public class SparseVectorFieldMapperTests extends ESSingleNodeTestCase { @Before public void setUpMapper() throws Exception { - IndexService indexService = createIndex("test-index"); + IndexService indexService = createIndex("test-index"); DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); String mapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() @@ -62,7 +65,14 @@ public class SparseVectorFieldMapperTests extends ESSingleNodeTestCase { return pluginList(Vectors.class, XPackPlugin.class); } + // this allows to set indexVersion as it is a private setting + @Override + protected boolean forbidPrivateIndexSettings() { + return false; + } + public void testDefaults() throws Exception { + Version indexVersion = Version.CURRENT; int[] indexedDims = {65535, 50, 2}; float[] indexedValues = {0.5f, 1800f, -34567.11f}; ParsedDocument doc1 = mapper.parse(new SourceToParse("test-index", "_doc", "1", BytesReference @@ -82,16 +92,76 @@ public class SparseVectorFieldMapperTests extends ESSingleNodeTestCase { // assert that after decoding the indexed values are equal to expected int[] expectedDims = {2, 50, 65535}; //the same as indexed but sorted float[] expectedValues = {-34567.11f, 1800f, 0.5f}; //the same as indexed but sorted by their dimensions + double dotProduct = 0.0f; + for (float value: expectedValues) { + dotProduct += value * value; + } + float expectedMagnitude = (float) Math.sqrt(dotProduct); - // assert that after decoding the indexed dims and values are equal to expected - BytesRef vectorBR = ((BinaryDocValuesField) fields[0]).binaryValue(); - int[] decodedDims = VectorEncoderDecoder.decodeSparseVectorDims(vectorBR); + // assert that after decoded magnitude, dims and values are equal to expected + BytesRef vectorBR = fields[0].binaryValue(); + int[] decodedDims = VectorEncoderDecoder.decodeSparseVectorDims(indexVersion, vectorBR); assertArrayEquals( "Decoded sparse vector dimensions are not equal to the indexed ones.", expectedDims, decodedDims ); - float[] decodedValues = VectorEncoderDecoder.decodeSparseVector(vectorBR); + float[] decodedValues = VectorEncoderDecoder.decodeSparseVector(indexVersion, vectorBR); + assertArrayEquals( + "Decoded sparse vector values are not equal to the indexed ones.", + expectedValues, + decodedValues, + 0.001f + ); + float decodedMagnitude = VectorEncoderDecoder.decodeVectorMagnitude(indexVersion, vectorBR); + assertEquals(expectedMagnitude, decodedMagnitude, 0.001f); + } + + public void testAddDocumentsToIndexBefore_V_7_5_0() throws Exception { + Version indexVersion = Version.V_7_4_0; + IndexService indexService = createIndex("test-index7_4", + Settings.builder().put(IndexMetaData.SETTING_INDEX_VERSION_CREATED.getKey(), indexVersion).build()); + DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); + String mapping = Strings.toString(XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("my-sparse-vector").field("type", "sparse_vector") + .endObject() + .endObject() + .endObject() + .endObject()); + mapper = parser.parse("_doc", new CompressedXContent(mapping)); + + int[] indexedDims = {65535, 50, 2}; + float[] indexedValues = {0.5f, 1800f, -34567.11f}; + ParsedDocument doc1 = mapper.parse(new SourceToParse("test-index7_4", "_doc", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .startObject("my-sparse-vector") + .field(Integer.toString(indexedDims[0]), indexedValues[0]) + .field(Integer.toString(indexedDims[1]), indexedValues[1]) + .field(Integer.toString(indexedDims[2]), indexedValues[2]) + .endObject() + .endObject()), + XContentType.JSON)); + IndexableField[] fields = doc1.rootDoc().getFields("my-sparse-vector"); + assertEquals(1, fields.length); + assertThat(fields[0], Matchers.instanceOf(BinaryDocValuesField.class)); + + // assert that after decoding the indexed values are equal to expected + int[] expectedDims = {2, 50, 65535}; //the same as indexed but sorted + float[] expectedValues = {-34567.11f, 1800f, 0.5f}; //the same as indexed but sorted by their dimensions + + // assert that after decoded magnitude, dims and values are equal to expected + BytesRef vectorBR = fields[0].binaryValue(); + int[] decodedDims = VectorEncoderDecoder.decodeSparseVectorDims(indexVersion, vectorBR); + assertArrayEquals( + "Decoded sparse vector dimensions are not equal to the indexed ones.", + expectedDims, + decodedDims + ); + float[] decodedValues = VectorEncoderDecoder.decodeSparseVector(indexVersion, vectorBR); assertArrayEquals( "Decoded sparse vector values are not equal to the indexed ones.", expectedValues, @@ -185,4 +255,5 @@ public class SparseVectorFieldMapperTests extends ESSingleNodeTestCase { new SourceToParse("test-index", "_doc", "1", invalidDoc, XContentType.JSON))); assertThat(e.getDetailedMessage(), containsString("has exceeded the maximum allowed number of dimensions")); } + } diff --git a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoderTests.java b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoderTests.java index 939d999b0d9a..c81bdfe147eb 100644 --- a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoderTests.java +++ b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/mapper/VectorEncoderDecoderTests.java @@ -7,33 +7,59 @@ package org.elasticsearch.xpack.vectors.mapper; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; import org.elasticsearch.test.ESTestCase; +import java.nio.ByteBuffer; import java.util.HashSet; import java.util.Set; import java.util.Arrays; public class VectorEncoderDecoderTests extends ESTestCase { - public void testDenseVectorEncodingDecoding() { - int dimCount = randomIntBetween(0, DenseVectorFieldMapper.MAX_DIMS_COUNT); + public void testSparseVectorEncodingDecoding() { + Version indexVersion = Version.CURRENT; + int dimCount = randomIntBetween(0, 100); float[] expectedValues = new float[dimCount]; + int[] expectedDims = randomUniqueDims(dimCount); + double dotProduct = 0.0f; for (int i = 0; i < dimCount; i++) { expectedValues[i] = randomFloat(); + dotProduct += expectedValues[i] * expectedValues[i]; } + float expectedMagnitude = (float) Math.sqrt(dotProduct); + + // test that sorting in the encoding works as expected + int[] sortedDims = Arrays.copyOf(expectedDims, dimCount); + Arrays.sort(sortedDims); + VectorEncoderDecoder.sortSparseDimsValues(expectedDims, expectedValues, dimCount); + assertArrayEquals( + "Sparse vector dims are not properly sorted!", + sortedDims, + expectedDims + ); // test that values that went through encoding and decoding are equal to their original - BytesRef encodedDenseVector = mockEncodeDenseVector(expectedValues); - float[] decodedValues = VectorEncoderDecoder.decodeDenseVector(encodedDenseVector); + BytesRef encodedSparseVector = VectorEncoderDecoder.encodeSparseVector(indexVersion, expectedDims, expectedValues, dimCount); + int[] decodedDims = VectorEncoderDecoder.decodeSparseVectorDims(indexVersion, encodedSparseVector); + float[] decodedValues = VectorEncoderDecoder.decodeSparseVector(indexVersion, encodedSparseVector); + float decodedMagnitude = VectorEncoderDecoder.decodeVectorMagnitude(indexVersion, encodedSparseVector); + assertEquals(expectedMagnitude, decodedMagnitude, 0.0f); assertArrayEquals( - "Decoded dense vector values are not equal to their original.", + "Decoded sparse vector dims are not equal to their original!", + expectedDims, + decodedDims + ); + assertArrayEquals( + "Decoded sparse vector values are not equal to their original.", expectedValues, decodedValues, 0.001f ); } - public void testSparseVectorEncodingDecoding() { + public void testSparseVectorEncodingDecodingBefore_V_7_5_0() { + Version indexVersion = Version.V_7_4_0; int dimCount = randomIntBetween(0, 100); float[] expectedValues = new float[dimCount]; int[] expectedDims = randomUniqueDims(dimCount); @@ -52,9 +78,9 @@ public class VectorEncoderDecoderTests extends ESTestCase { ); // test that values that went through encoding and decoding are equal to their original - BytesRef encodedSparseVector = VectorEncoderDecoder.encodeSparseVector(expectedDims, expectedValues, dimCount); - int[] decodedDims = VectorEncoderDecoder.decodeSparseVectorDims(encodedSparseVector); - float[] decodedValues = VectorEncoderDecoder.decodeSparseVector(encodedSparseVector); + BytesRef encodedSparseVector = VectorEncoderDecoder.encodeSparseVector(indexVersion, expectedDims, expectedValues, dimCount); + int[] decodedDims = VectorEncoderDecoder.decodeSparseVectorDims(indexVersion, encodedSparseVector); + float[] decodedValues = VectorEncoderDecoder.decodeSparseVector(indexVersion, encodedSparseVector); assertArrayEquals( "Decoded sparse vector dims are not equal to their original!", expectedDims, @@ -69,23 +95,28 @@ public class VectorEncoderDecoderTests extends ESTestCase { } // imitates the code in DenseVectorFieldMapper::parse - public static BytesRef mockEncodeDenseVector(float[] values) { - final short INT_BYTES = VectorEncoderDecoder.INT_BYTES; - byte[] buf = new byte[INT_BYTES * values.length]; - int offset = 0; - int intValue; - for (float value: values) { - intValue = Float.floatToIntBits(value); - buf[offset++] = (byte) (intValue >> 24); - buf[offset++] = (byte) (intValue >> 16); - buf[offset++] = (byte) (intValue >> 8); - buf[offset++] = (byte) intValue; + public static BytesRef mockEncodeDenseVector(float[] values, Version indexVersion) { + byte[] bytes = indexVersion.onOrAfter(Version.V_7_5_0) + ? new byte[VectorEncoderDecoder.INT_BYTES * values.length + VectorEncoderDecoder.INT_BYTES] + : new byte[VectorEncoderDecoder.INT_BYTES * values.length]; + double dotProduct = 0f; + + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + for (float value : values) { + byteBuffer.putFloat(value); + dotProduct += value * value; } - return new BytesRef(buf, 0, offset); + + if (indexVersion.onOrAfter(Version.V_7_5_0)) { + // encode vector magnitude at the end + float vectorMagnitude = (float) Math.sqrt(dotProduct); + byteBuffer.putFloat(vectorMagnitude); + } + return new BytesRef(bytes); } // generate unique random dims - private int[] randomUniqueDims(int dimCount) { + private static int[] randomUniqueDims(int dimCount) { int[] values = new int[dimCount]; Set usedValues = new HashSet<>(); int value; diff --git a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtilsTests.java b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtilsTests.java index 87f8f83c06bd..343db845e6e3 100644 --- a/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtilsTests.java +++ b/x-pack/plugin/vectors/src/test/java/org/elasticsearch/xpack/vectors/query/ScoreScriptUtilsTests.java @@ -7,12 +7,17 @@ package org.elasticsearch.xpack.vectors.query; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.xpack.vectors.mapper.VectorEncoderDecoder; +import org.elasticsearch.Version; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.vectors.mapper.VectorEncoderDecoder; import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.CosineSimilarity; -import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.DotProductSparse; import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.CosineSimilaritySparse; +import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.DotProduct; +import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.DotProductSparse; +import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.L1Norm; import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.L1NormSparse; +import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.L2Norm; import org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.L2NormSparse; import java.util.Arrays; @@ -21,65 +26,85 @@ import java.util.List; import java.util.Map; import static org.elasticsearch.xpack.vectors.mapper.VectorEncoderDecoderTests.mockEncodeDenseVector; -import static org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.dotProduct; -import static org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.l1norm; -import static org.elasticsearch.xpack.vectors.query.ScoreScriptUtils.l2norm; - import static org.hamcrest.Matchers.containsString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; - public class ScoreScriptUtilsTests extends ESTestCase { + public void testDenseVectorFunctions() { + testDenseVectorFunctions(Version.V_7_4_0); + testDenseVectorFunctions(Version.CURRENT); + } + + private void testDenseVectorFunctions(Version indexVersion) { float[] docVector = {230.0f, 300.33f, -34.8988f, 15.555f, -200.0f}; - BytesRef encodedDocVector = mockEncodeDenseVector(docVector); + BytesRef encodedDocVector = mockEncodeDenseVector(docVector, indexVersion); VectorScriptDocValues.DenseVectorScriptDocValues dvs = mock(VectorScriptDocValues.DenseVectorScriptDocValues.class); when(dvs.getEncodedValue()).thenReturn(encodedDocVector); + + ScoreScript scoreScript = mock(ScoreScript.class); + when(scoreScript._getIndexVersion()).thenReturn(indexVersion); + List queryVector = Arrays.asList(0.5f, 111.3f, -13.0f, 14.8f, -156.0f); // test dotProduct - double result = dotProduct(queryVector, dvs); + DotProduct dotProduct = new DotProduct(scoreScript, queryVector); + double result = dotProduct.dotProduct(dvs); assertEquals("dotProduct result is not equal to the expected value!", 65425.624, result, 0.001); // test cosineSimilarity - CosineSimilarity cosineSimilarity = new CosineSimilarity(queryVector); + CosineSimilarity cosineSimilarity = new CosineSimilarity(scoreScript, queryVector); double result2 = cosineSimilarity.cosineSimilarity(dvs); assertEquals("cosineSimilarity result is not equal to the expected value!", 0.790, result2, 0.001); // test l1Norm - double result3 = l1norm(queryVector, dvs); + L1Norm l1norm = new L1Norm(scoreScript, queryVector); + double result3 = l1norm.l1norm(dvs); assertEquals("l1norm result is not equal to the expected value!", 485.184, result3, 0.001); // test l2norm - double result4 = l2norm(queryVector, dvs); + L2Norm l2norm = new L2Norm(scoreScript, queryVector); + double result4 = l2norm.l2norm(dvs); assertEquals("l2norm result is not equal to the expected value!", 301.361, result4, 0.001); // test dotProduct fails when queryVector has wrong number of dims List invalidQueryVector = Arrays.asList(0.5, 111.3); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> dotProduct(invalidQueryVector, dvs)); - assertThat(e.getMessage(), containsString("dimensions of the query vector [2] is different from the documents' vectors [5]")); + DotProduct dotProduct2 = new DotProduct(scoreScript, invalidQueryVector); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> dotProduct2.dotProduct(dvs)); + assertThat(e.getMessage(), containsString("query vector has a different number of dimensions [2] than the document vectors [5]")); // test cosineSimilarity fails when queryVector has wrong number of dims - CosineSimilarity cosineSimilarity2 = new CosineSimilarity(invalidQueryVector); + CosineSimilarity cosineSimilarity2 = new CosineSimilarity(scoreScript, invalidQueryVector); e = expectThrows(IllegalArgumentException.class, () -> cosineSimilarity2.cosineSimilarity(dvs)); - assertThat(e.getMessage(), containsString("dimensions of the query vector [2] is different from the documents' vectors [5]")); + assertThat(e.getMessage(), containsString("query vector has a different number of dimensions [2] than the document vectors [5]")); // test l1norm fails when queryVector has wrong number of dims - e = expectThrows(IllegalArgumentException.class, () -> l1norm(invalidQueryVector, dvs)); - assertThat(e.getMessage(), containsString("dimensions of the query vector [2] is different from the documents' vectors [5]")); + L1Norm l1norm2 = new L1Norm(scoreScript, invalidQueryVector); + e = expectThrows(IllegalArgumentException.class, () -> l1norm2.l1norm(dvs)); + assertThat(e.getMessage(), containsString("query vector has a different number of dimensions [2] than the document vectors [5]")); // test l2norm fails when queryVector has wrong number of dims - e = expectThrows(IllegalArgumentException.class, () -> l2norm(invalidQueryVector, dvs)); - assertThat(e.getMessage(), containsString("dimensions of the query vector [2] is different from the documents' vectors [5]")); + L2Norm l2norm2 = new L2Norm(scoreScript, invalidQueryVector); + e = expectThrows(IllegalArgumentException.class, () -> l2norm2.l2norm(dvs)); + assertThat(e.getMessage(), containsString("query vector has a different number of dimensions [2] than the document vectors [5]")); } public void testSparseVectorFunctions() { + testSparseVectorFunctions(Version.V_7_4_0); + testSparseVectorFunctions(Version.CURRENT); + } + + private void testSparseVectorFunctions(Version indexVersion) { int[] docVectorDims = {2, 10, 50, 113, 4545}; float[] docVectorValues = {230.0f, 300.33f, -34.8988f, 15.555f, -200.0f}; - BytesRef encodedDocVector = VectorEncoderDecoder.encodeSparseVector(docVectorDims, docVectorValues, docVectorDims.length); + BytesRef encodedDocVector = VectorEncoderDecoder.encodeSparseVector( + indexVersion, docVectorDims, docVectorValues, docVectorDims.length); VectorScriptDocValues.SparseVectorScriptDocValues dvs = mock(VectorScriptDocValues.SparseVectorScriptDocValues.class); when(dvs.getEncodedValue()).thenReturn(encodedDocVector); + ScoreScript scoreScript = mock(ScoreScript.class); + when(scoreScript._getIndexVersion()).thenReturn(indexVersion); + Map queryVector = new HashMap() {{ put("2", 0.5); put("10", 111.3); @@ -89,22 +114,22 @@ public class ScoreScriptUtilsTests extends ESTestCase { }}; // test dotProduct - DotProductSparse docProductSparse = new DotProductSparse(queryVector); + DotProductSparse docProductSparse = new DotProductSparse(scoreScript, queryVector); double result = docProductSparse.dotProductSparse(dvs); assertEquals("dotProductSparse result is not equal to the expected value!", 65425.624, result, 0.001); // test cosineSimilarity - CosineSimilaritySparse cosineSimilaritySparse = new CosineSimilaritySparse(queryVector); + CosineSimilaritySparse cosineSimilaritySparse = new CosineSimilaritySparse(scoreScript, queryVector); double result2 = cosineSimilaritySparse.cosineSimilaritySparse(dvs); assertEquals("cosineSimilaritySparse result is not equal to the expected value!", 0.790, result2, 0.001); // test l1norm - L1NormSparse l1Norm = new L1NormSparse(queryVector); + L1NormSparse l1Norm = new L1NormSparse(scoreScript, queryVector); double result3 = l1Norm.l1normSparse(dvs); assertEquals("l1normSparse result is not equal to the expected value!", 485.184, result3, 0.001); // test l2norm - L2NormSparse l2Norm = new L2NormSparse(queryVector); + L2NormSparse l2Norm = new L2NormSparse(scoreScript, queryVector); double result4 = l2Norm.l2normSparse(dvs); assertEquals("l2normSparse result is not equal to the expected value!", 301.361, result4, 0.001); } @@ -113,9 +138,12 @@ public class ScoreScriptUtilsTests extends ESTestCase { // Document vector's biggest dimension > query vector's biggest dimension int[] docVectorDims = {2, 10, 50, 113, 4545, 4546}; float[] docVectorValues = {230.0f, 300.33f, -34.8988f, 15.555f, -200.0f, 11.5f}; - BytesRef encodedDocVector = VectorEncoderDecoder.encodeSparseVector(docVectorDims, docVectorValues, docVectorDims.length); + BytesRef encodedDocVector = VectorEncoderDecoder.encodeSparseVector( + Version.CURRENT, docVectorDims, docVectorValues, docVectorDims.length); VectorScriptDocValues.SparseVectorScriptDocValues dvs = mock(VectorScriptDocValues.SparseVectorScriptDocValues.class); when(dvs.getEncodedValue()).thenReturn(encodedDocVector); + ScoreScript scoreScript = mock(ScoreScript.class); + when(scoreScript._getIndexVersion()).thenReturn(Version.CURRENT); Map queryVector = new HashMap() {{ put("2", 0.5); put("10", 111.3); @@ -126,22 +154,22 @@ public class ScoreScriptUtilsTests extends ESTestCase { }}; // test dotProduct - DotProductSparse docProductSparse = new DotProductSparse(queryVector); + DotProductSparse docProductSparse = new DotProductSparse(scoreScript, queryVector); double result = docProductSparse.dotProductSparse(dvs); assertEquals("dotProductSparse result is not equal to the expected value!", 65425.624, result, 0.001); // test cosineSimilarity - CosineSimilaritySparse cosineSimilaritySparse = new CosineSimilaritySparse(queryVector); + CosineSimilaritySparse cosineSimilaritySparse = new CosineSimilaritySparse(scoreScript, queryVector); double result2 = cosineSimilaritySparse.cosineSimilaritySparse(dvs); assertEquals("cosineSimilaritySparse result is not equal to the expected value!", 0.786, result2, 0.001); // test l1norm - L1NormSparse l1Norm = new L1NormSparse(queryVector); + L1NormSparse l1Norm = new L1NormSparse(scoreScript, queryVector); double result3 = l1Norm.l1normSparse(dvs); assertEquals("l1normSparse result is not equal to the expected value!", 517.184, result3, 0.001); // test l2norm - L2NormSparse l2Norm = new L2NormSparse(queryVector); + L2NormSparse l2Norm = new L2NormSparse(scoreScript, queryVector); double result4 = l2Norm.l2normSparse(dvs); assertEquals("l2normSparse result is not equal to the expected value!", 302.277, result4, 0.001); } @@ -150,9 +178,12 @@ public class ScoreScriptUtilsTests extends ESTestCase { // Document vector's biggest dimension < query vector's biggest dimension int[] docVectorDims = {2, 10, 50, 113, 4545, 4546}; float[] docVectorValues = {230.0f, 300.33f, -34.8988f, 15.555f, -200.0f, 11.5f}; - BytesRef encodedDocVector = VectorEncoderDecoder.encodeSparseVector(docVectorDims, docVectorValues, docVectorDims.length); + BytesRef encodedDocVector = VectorEncoderDecoder.encodeSparseVector( + Version.CURRENT, docVectorDims, docVectorValues, docVectorDims.length); VectorScriptDocValues.SparseVectorScriptDocValues dvs = mock(VectorScriptDocValues.SparseVectorScriptDocValues.class); when(dvs.getEncodedValue()).thenReturn(encodedDocVector); + ScoreScript scoreScript = mock(ScoreScript.class); + when(scoreScript._getIndexVersion()).thenReturn(Version.CURRENT); Map queryVector = new HashMap() {{ put("2", 0.5); put("10", 111.3); @@ -163,22 +194,22 @@ public class ScoreScriptUtilsTests extends ESTestCase { }}; // test dotProduct - DotProductSparse docProductSparse = new DotProductSparse(queryVector); + DotProductSparse docProductSparse = new DotProductSparse(scoreScript, queryVector); double result = docProductSparse.dotProductSparse(dvs); assertEquals("dotProductSparse result is not equal to the expected value!", 65425.624, result, 0.001); // test cosineSimilarity - CosineSimilaritySparse cosineSimilaritySparse = new CosineSimilaritySparse(queryVector); + CosineSimilaritySparse cosineSimilaritySparse = new CosineSimilaritySparse(scoreScript, queryVector); double result2 = cosineSimilaritySparse.cosineSimilaritySparse(dvs); assertEquals("cosineSimilaritySparse result is not equal to the expected value!", 0.786, result2, 0.001); // test l1norm - L1NormSparse l1Norm = new L1NormSparse(queryVector); + L1NormSparse l1Norm = new L1NormSparse(scoreScript, queryVector); double result3 = l1Norm.l1normSparse(dvs); assertEquals("l1normSparse result is not equal to the expected value!", 517.184, result3, 0.001); // test l2norm - L2NormSparse l2Norm = new L2NormSparse(queryVector); + L2NormSparse l2Norm = new L2NormSparse(scoreScript, queryVector); double result4 = l2Norm.l2normSparse(dvs); assertEquals("l2normSparse result is not equal to the expected value!", 302.277, result4, 0.001); } From 488d84354d392b22712eb9530cd11eeb50baa93f Mon Sep 17 00:00:00 2001 From: Yogesh Gaikwad <902768+bizybot@users.noreply.github.com> Date: Wed, 4 Sep 2019 12:36:04 +1000 Subject: [PATCH 026/103] Initialize document subset bit set cache used for DLS (#46211) This commit initializes DocumentSubsetBitsetCache even if DLS is disabled. Previously it would throw null pointer when querying usage stats if we explicitly disabled DLS as there would be no instance of DocumentSubsetBitsetCache to query. It is okay to initialize DocumentSubsetBitsetCache which will be empty as the license enforcement would prevent usage of DLS feature and it will not fail when accessing usage stats. Closes #45147 --- .../xpack/security/Security.java | 5 +- .../authz/store/CompositeRolesStore.java | 22 +++-- .../authz/store/CompositeRolesStoreTests.java | 90 +++++++++++++++---- 3 files changed, 82 insertions(+), 35 deletions(-) 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 408288a3f8de..533897bac446 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 @@ -361,10 +361,6 @@ public class Security extends Plugin implements ActionPlugin, IngestPlugin, Netw securityContext.set(new SecurityContext(settings, threadPool.getThreadContext())); components.add(securityContext.get()); - if (XPackSettings.DLS_FLS_ENABLED.get(settings)) { - dlsBitsetCache.set(new DocumentSubsetBitsetCache(settings)); - } - // audit trail service construction final List auditTrails = XPackSettings.AUDIT_ENABLED.get(settings) ? Collections.singletonList(new LoggingAuditTrail(settings, clusterService, threadPool)) @@ -408,6 +404,7 @@ public class Security extends Plugin implements ActionPlugin, IngestPlugin, Netw final NativePrivilegeStore privilegeStore = new NativePrivilegeStore(settings, client, securityIndex.get()); components.add(privilegeStore); + dlsBitsetCache.set(new DocumentSubsetBitsetCache(settings)); final FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(settings); final FileRolesStore fileRolesStore = new FileRolesStore(settings, env, resourceWatcherService, getLicenseState()); final NativeRolesStore nativeRolesStore = new NativeRolesStore(settings, client, getLicenseState(), securityIndex.get()); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStore.java index 5db9fab2f4da..667b53c1c332 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStore.java @@ -106,17 +106,17 @@ public class CompositeRolesStore { ReservedRolesStore reservedRolesStore, NativePrivilegeStore privilegeStore, List, ActionListener>> rolesProviders, ThreadContext threadContext, XPackLicenseState licenseState, FieldPermissionsCache fieldPermissionsCache, - ApiKeyService apiKeyService, @Nullable DocumentSubsetBitsetCache dlsBitsetCache, + ApiKeyService apiKeyService, DocumentSubsetBitsetCache dlsBitsetCache, Consumer> effectiveRoleDescriptorsConsumer) { - this.fileRolesStore = fileRolesStore; - this.dlsBitsetCache = dlsBitsetCache; + this.fileRolesStore = Objects.requireNonNull(fileRolesStore); + this.dlsBitsetCache = Objects.requireNonNull(dlsBitsetCache); fileRolesStore.addListener(this::invalidate); - this.nativeRolesStore = nativeRolesStore; - this.privilegeStore = privilegeStore; - this.licenseState = licenseState; - this.fieldPermissionsCache = fieldPermissionsCache; - this.apiKeyService = apiKeyService; - this.effectiveRoleDescriptorsConsumer = effectiveRoleDescriptorsConsumer; + this.nativeRolesStore = Objects.requireNonNull(nativeRolesStore); + this.privilegeStore = Objects.requireNonNull(privilegeStore); + this.licenseState = Objects.requireNonNull(licenseState); + this.fieldPermissionsCache = Objects.requireNonNull(fieldPermissionsCache); + this.apiKeyService = Objects.requireNonNull(apiKeyService); + this.effectiveRoleDescriptorsConsumer = Objects.requireNonNull(effectiveRoleDescriptorsConsumer); CacheBuilder builder = CacheBuilder.builder(); final int cacheSize = CACHE_SIZE_SETTING.get(settings); if (cacheSize >= 0) { @@ -415,9 +415,7 @@ public class CompositeRolesStore { try (ReleasableLock ignored = roleCacheHelper.acquireUpdateLock()) { roleCache.invalidateAll(); } - if (dlsBitsetCache != null) { - dlsBitsetCache.clear("role store invalidation"); - } + dlsBitsetCache.clear("role store invalidation"); } public void invalidate(String role) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java index 58d19ed2c97e..7da88f0231b8 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.xpack.core.security.authc.Authentication.Authentication import org.elasticsearch.xpack.core.security.authc.Authentication.RealmRef; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor.IndicesPrivileges; +import org.elasticsearch.xpack.core.security.authz.accesscontrol.DocumentSubsetBitsetCache; import org.elasticsearch.xpack.core.security.authz.accesscontrol.IndicesAccessControl; import org.elasticsearch.xpack.core.security.authz.permission.ClusterPermission; import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissionsCache; @@ -151,9 +152,10 @@ public class CompositeRolesStoreTests extends ESTestCase { when(fileRolesStore.roleDescriptors(Collections.singleton("fls_dls"))).thenReturn(Collections.singleton(flsDlsRole)); when(fileRolesStore.roleDescriptors(Collections.singleton("no_fls_dls"))).thenReturn(Collections.singleton(noFlsDlsRole)); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); CompositeRolesStore compositeRolesStore = new CompositeRolesStore(Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), - new ThreadContext(Settings.EMPTY), licenseState, cache, mock(ApiKeyService.class), null, + new ThreadContext(Settings.EMPTY), licenseState, cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); PlainActionFuture roleFuture = new PlainActionFuture<>(); @@ -226,9 +228,10 @@ public class CompositeRolesStoreTests extends ESTestCase { when(fileRolesStore.roleDescriptors(Collections.singleton("fls_dls"))).thenReturn(Collections.singleton(flsDlsRole)); when(fileRolesStore.roleDescriptors(Collections.singleton("no_fls_dls"))).thenReturn(Collections.singleton(noFlsDlsRole)); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); CompositeRolesStore compositeRolesStore = new CompositeRolesStore(Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), - new ThreadContext(Settings.EMPTY), licenseState, cache, mock(ApiKeyService.class), null, + new ThreadContext(Settings.EMPTY), licenseState, cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); PlainActionFuture roleFuture = new PlainActionFuture<>(); @@ -277,11 +280,12 @@ public class CompositeRolesStoreTests extends ESTestCase { }).when(nativePrivilegeStore).getPrivileges(isA(Set.class), isA(Set.class), any(ActionListener.class)); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, nativePrivilegeStore, Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, - rds -> effectiveRoleDescriptors.set(rds)); + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), + documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor final String roleName = randomAlphaOfLengthBetween(1, 10); @@ -339,9 +343,11 @@ public class CompositeRolesStoreTests extends ESTestCase { .put("xpack.security.authz.store.roles.negative_lookup_cache.max_size", 0) .build(); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(settings, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(settings), - new XPackLicenseState(settings), cache, mock(ApiKeyService.class), null, rds -> effectiveRoleDescriptors.set(rds)); + new XPackLicenseState(settings), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, + rds -> effectiveRoleDescriptors.set(rds)); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor final String roleName = randomAlphaOfLengthBetween(1, 10); @@ -375,10 +381,11 @@ public class CompositeRolesStoreTests extends ESTestCase { final ReservedRolesStore reservedRolesStore = spy(new ReservedRolesStore()); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor @@ -460,11 +467,13 @@ public class CompositeRolesStoreTests extends ESTestCase { })); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Arrays.asList(inMemoryProvider1, inMemoryProvider2), new ThreadContext(SECURITY_ENABLED_SETTINGS), new XPackLicenseState(SECURITY_ENABLED_SETTINGS), - cache, mock(ApiKeyService.class), null, rds -> effectiveRoleDescriptors.set(rds)); + cache, mock(ApiKeyService.class), documentSubsetBitsetCache, + rds -> effectiveRoleDescriptors.set(rds)); final Set roleNames = Sets.newHashSet("roleA", "roleB", "unknown"); PlainActionFuture future = new PlainActionFuture<>(); @@ -687,11 +696,12 @@ public class CompositeRolesStoreTests extends ESTestCase { (roles, listener) -> listener.onFailure(new Exception("fake failure")); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Arrays.asList(inMemoryProvider1, failingProvider), new ThreadContext(SECURITY_ENABLED_SETTINGS), new XPackLicenseState(SECURITY_ENABLED_SETTINGS), - cache, mock(ApiKeyService.class), null, rds -> effectiveRoleDescriptors.set(rds)); + cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); final Set roleNames = Sets.newHashSet("roleA", "roleB", "unknown"); PlainActionFuture future = new PlainActionFuture<>(); @@ -734,10 +744,11 @@ public class CompositeRolesStoreTests extends ESTestCase { // these licenses don't allow custom role providers xPackLicenseState.update(randomFrom(OperationMode.BASIC, OperationMode.GOLD, OperationMode.STANDARD), true, null); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); CompositeRolesStore compositeRolesStore = new CompositeRolesStore( Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Arrays.asList(inMemoryProvider), new ThreadContext(Settings.EMPTY), xPackLicenseState, cache, - mock(ApiKeyService.class), null, rds -> effectiveRoleDescriptors.set(rds)); + mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); Set roleNames = Sets.newHashSet("roleA"); PlainActionFuture future = new PlainActionFuture<>(); @@ -752,7 +763,7 @@ public class CompositeRolesStoreTests extends ESTestCase { compositeRolesStore = new CompositeRolesStore( Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Arrays.asList(inMemoryProvider), new ThreadContext(Settings.EMPTY), xPackLicenseState, cache, - mock(ApiKeyService.class), null, rds -> effectiveRoleDescriptors.set(rds)); + mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); // these licenses allow custom role providers xPackLicenseState.update(randomFrom(OperationMode.PLATINUM, OperationMode.TRIAL), true, null); roleNames = Sets.newHashSet("roleA"); @@ -769,7 +780,7 @@ public class CompositeRolesStoreTests extends ESTestCase { compositeRolesStore = new CompositeRolesStore( Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Arrays.asList(inMemoryProvider), new ThreadContext(Settings.EMPTY), xPackLicenseState, cache, - mock(ApiKeyService.class), null, rds -> effectiveRoleDescriptors.set(rds)); + mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); xPackLicenseState.update(randomFrom(OperationMode.PLATINUM, OperationMode.TRIAL), false, null); roleNames = Sets.newHashSet("roleA"); future = new PlainActionFuture<>(); @@ -797,10 +808,12 @@ public class CompositeRolesStoreTests extends ESTestCase { doCallRealMethod().when(reservedRolesStore).accept(any(Set.class), any(ActionListener.class)); NativeRolesStore nativeRolesStore = mock(NativeRolesStore.class); doCallRealMethod().when(nativeRolesStore).accept(any(Set.class), any(ActionListener.class)); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); CompositeRolesStore compositeRolesStore = new CompositeRolesStore( Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(Settings.EMPTY), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, rds -> {}) { + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, + rds -> {}) { @Override public void invalidateAll() { numInvalidation.incrementAndGet(); @@ -849,10 +862,12 @@ public class CompositeRolesStoreTests extends ESTestCase { doCallRealMethod().when(reservedRolesStore).accept(any(Set.class), any(ActionListener.class)); NativeRolesStore nativeRolesStore = mock(NativeRolesStore.class); doCallRealMethod().when(nativeRolesStore).accept(any(Set.class), any(ActionListener.class)); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, rds -> {}) { + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), + documentSubsetBitsetCache, rds -> {}) { @Override public void invalidateAll() { numInvalidation.incrementAndGet(); @@ -879,10 +894,12 @@ public class CompositeRolesStoreTests extends ESTestCase { }).when(nativeRolesStore).getRoleDescriptors(isA(Set.class), any(ActionListener.class)); final ReservedRolesStore reservedRolesStore = spy(new ReservedRolesStore()); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, rds -> {}); + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, + rds -> {}); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor PlainActionFuture rolesFuture = new PlainActionFuture<>(); @@ -918,10 +935,11 @@ public class CompositeRolesStoreTests extends ESTestCase { }).when(nativeRolesStore).getRoleDescriptors(isA(Set.class), any(ActionListener.class)); final ReservedRolesStore reservedRolesStore = spy(new ReservedRolesStore()); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(settings, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(settings), - new XPackLicenseState(settings), cache, mock(ApiKeyService.class), null, rds -> {}); + new XPackLicenseState(settings), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> {}); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor PlainActionFuture rolesFuture = new PlainActionFuture<>(); @@ -945,11 +963,12 @@ public class CompositeRolesStoreTests extends ESTestCase { }).when(nativeRolesStore).getRoleDescriptors(isA(Set.class), any(ActionListener.class)); final ReservedRolesStore reservedRolesStore = spy(new ReservedRolesStore()); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor @@ -975,11 +994,12 @@ public class CompositeRolesStoreTests extends ESTestCase { }).when(nativeRolesStore).getRoleDescriptors(isA(Set.class), any(ActionListener.class)); final ReservedRolesStore reservedRolesStore = spy(new ReservedRolesStore()); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), null, + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); verify(fileRolesStore).addListener(any(Consumer.class)); // adds a listener in ctor IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, @@ -1010,11 +1030,12 @@ public class CompositeRolesStoreTests extends ESTestCase { return Void.TYPE; }).when(nativePrivStore).getPrivileges(any(Collection.class), any(Collection.class), any(ActionListener.class)); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, nativePrivStore, Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, apiKeyService, null, + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, apiKeyService, documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); AuditUtil.getOrGenerateRequestId(threadContext); final Authentication authentication = new Authentication(new User("test api key user", "superuser"), @@ -1055,11 +1076,12 @@ public class CompositeRolesStoreTests extends ESTestCase { return Void.TYPE; }).when(nativePrivStore).getPrivileges(any(Collection.class), any(Collection.class), any(ActionListener.class)); + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final AtomicReference> effectiveRoleDescriptors = new AtomicReference>(); final CompositeRolesStore compositeRolesStore = new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, nativePrivStore, Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), - new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, apiKeyService, null, + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, apiKeyService, documentSubsetBitsetCache, rds -> effectiveRoleDescriptors.set(rds)); AuditUtil.getOrGenerateRequestId(threadContext); final Authentication authentication = new Authentication(new User("test api key user", "api_key"), @@ -1081,6 +1103,36 @@ public class CompositeRolesStoreTests extends ESTestCase { verify(apiKeyService).getRoleForApiKey(eq(authentication), any(ActionListener.class)); } + public void testUsageStats() { + final FileRolesStore fileRolesStore = mock(FileRolesStore.class); + final Map fileRolesStoreUsageStats = Map.of("size", "1", "fls", Boolean.FALSE, "dls", Boolean.TRUE); + when(fileRolesStore.usageStats()).thenReturn(fileRolesStoreUsageStats); + + final NativeRolesStore nativeRolesStore = mock(NativeRolesStore.class); + final Map nativeRolesStoreUsageStats = Map.of(); + doAnswer((invocationOnMock) -> { + ActionListener> usageStats = (ActionListener>) invocationOnMock.getArguments()[0]; + usageStats.onResponse(nativeRolesStoreUsageStats); + return Void.TYPE; + }).when(nativeRolesStore).usageStats(any(ActionListener.class)); + final ReservedRolesStore reservedRolesStore = spy(new ReservedRolesStore()); + + final DocumentSubsetBitsetCache documentSubsetBitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); + + final CompositeRolesStore compositeRolesStore = + new CompositeRolesStore(SECURITY_ENABLED_SETTINGS, fileRolesStore, nativeRolesStore, reservedRolesStore, + mock(NativePrivilegeStore.class), Collections.emptyList(), new ThreadContext(SECURITY_ENABLED_SETTINGS), + new XPackLicenseState(SECURITY_ENABLED_SETTINGS), cache, mock(ApiKeyService.class), documentSubsetBitsetCache, rds -> { + }); + + PlainActionFuture> usageStatsListener = new PlainActionFuture<>(); + compositeRolesStore.usageStats(usageStatsListener); + Map usageStats = usageStatsListener.actionGet(); + assertThat(usageStats.get("file"), is(fileRolesStoreUsageStats)); + assertThat(usageStats.get("native"), is(nativeRolesStoreUsageStats)); + assertThat(usageStats.get("dls"), is(Map.of("bit_set_cache", documentSubsetBitsetCache.usageStats()))); + } + private static class InMemoryRolesProvider implements BiConsumer, ActionListener> { private final Function, RoleRetrievalResult> roleDescriptorsFunc; From 8d5bd3bcc04767838f851c8cbbeee56cba503879 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Wed, 4 Sep 2019 08:29:10 +0200 Subject: [PATCH 027/103] [ML-DataFrame] unmute tests for debuging purposes (#46121) unmute testGetCheckpointStats closes #45238 --- .../checkpoint/DataFrameTransformCheckpointServiceNodeTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java index 6e5ee8c8bc86..fe6cc7c7525a 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java @@ -178,7 +178,6 @@ public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingl DataFrameTransformCheckpoint.EMPTY, null, null); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/45238") public void testGetCheckpointStats() throws InterruptedException { String transformId = randomAlphaOfLengthBetween(3, 10); long timestamp = 1000; From 9a885a3ac47bc8f52c07770d1d8d670ce0af1e59 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 4 Sep 2019 10:32:01 +0300 Subject: [PATCH 028/103] SQL: Fix issue with IIF function when condition folds (#46290) Previously, when the condition (1st argument) of the IIF function could be evaluated (folded) to false, the `IfConditional` was eliminated which caused `IndexOutOfBoundsException` to be thrown when `info()` and `resolveType()` methods where called. Fixes: #46268 --- .../src/main/resources/conditionals.csv-spec | 28 +++++++++++++++++++ .../expression/predicate/conditional/Iif.java | 9 ++++-- .../predicate/conditional/CaseTests.java | 9 ++++++ .../predicate/conditional/IifTests.java | 11 ++++++++ .../xpack/sql/optimizer/OptimizerTests.java | 8 +++++- 5 files changed, 62 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/sql/qa/src/main/resources/conditionals.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/conditionals.csv-spec index a5d7411caf6e..3b7a8a8de372 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/conditionals.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/conditionals.csv-spec @@ -52,6 +52,21 @@ ORDER BY emp_no LIMIT 10; 10014 | null ; +caseWithConditionsFolded +schema::emp_no:i|CASE_1:byte|CASE_2:i +SELECT emp_no, CASE WHEN NULL = 1 THEN emp_no WHEN 10 < 5 THEN emp_no ELSE languages END AS "CASE_1", +CASE WHEN NULL = 1 THEN languages WHEN 10 > 5 THEN emp_no ELSE languages END AS "CASE_2" +FROM test_emp ORDER BY 1 LIMIT 5; + + emp_no | CASE_1 | CASE_2 +--------+--------+------- +10001 | 2 | 10001 +10002 | 5 | 10002 +10003 | 4 | 10003 +10004 | 5 | 10004 +10005 | 1 | 10005 +; + caseWhere SELECT last_name FROM test_emp WHERE CASE WHEN LENGTH(last_name) < 7 THEN 'ShortName' ELSE 'LongName' END = 'LongName' ORDER BY emp_no LIMIT 10; @@ -235,6 +250,19 @@ ORDER BY emp_no LIMIT 10; 10014 | null ; +iifWithConditionFolded +schema::emp_no:i|IIF_1:i|IIF_2:byte|IIF_3:i +SELECT emp_no, IIF(NULL, emp_no) AS IIF_1, IIF(NULL, emp_no, languages) AS IIF_2, IIF(10 > 5, emp_no, languages) AS IIF_3 FROM test_emp ORDER BY 1 LIMIT 5; + + emp_no | IIF_1 | IIF_2 | IIF_3 +--------+-------+-------+------ +10001 | null | 2 | 10001 +10002 | null | 5 | 10002 +10003 | null | 4 | 10003 +10004 | null | 5 | 10004 +10005 | null | 1 | 10005 +; + iifWhere SELECT last_name FROM test_emp WHERE IIF(LENGTH(last_name) < 7, 'ShortName') IS NOT NULL ORDER BY emp_no LIMIT 10; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Iif.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Iif.java index 49faeb1233a7..ac6a7d342939 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Iif.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Iif.java @@ -18,6 +18,7 @@ import java.util.List; import static org.elasticsearch.common.logging.LoggerMessageFormat.format; import static org.elasticsearch.xpack.sql.expression.TypeResolutions.isBoolean; +import static org.elasticsearch.xpack.sql.util.CollectionUtils.combine; public class Iif extends Case { @@ -25,13 +26,13 @@ public class Iif extends Case { super(source, Arrays.asList(new IfConditional(source, condition, thenResult), elseResult != null ? elseResult : Literal.NULL)); } - private Iif(Source source, List expressions) { + Iif(Source source, List expressions) { super(source, expressions); } @Override protected NodeInfo info() { - return NodeInfo.create(this, Iif::new, conditions().get(0).condition(), conditions().get(0).result(), elseResult()); + return NodeInfo.create(this, Iif::new, combine(conditions(), elseResult())); } @Override @@ -41,6 +42,10 @@ public class Iif extends Case { @Override protected TypeResolution resolveType() { + if (conditions().isEmpty()) { + return TypeResolution.TYPE_RESOLVED; + } + TypeResolution conditionTypeResolution = isBoolean(conditions().get(0).condition(), sourceText(), Expressions.ParamOrdinal.FIRST); if (conditionTypeResolution.unresolved()) { return conditionTypeResolution; diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java index b4de311c9203..00004598f5c9 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java @@ -17,9 +17,11 @@ import org.elasticsearch.xpack.sql.type.DataType; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Objects; +import static org.elasticsearch.xpack.sql.expression.Expression.TypeResolution; import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomIntLiteral; import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomStringLiteral; import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; @@ -117,6 +119,13 @@ public class CaseTests extends AbstractNodeTestCase { assertEquals(DataType.KEYWORD, c.dataType()); } + public void testAllConditionsFolded() { + Case c = new Case(EMPTY, Collections.singletonList(Literal.of(EMPTY, "foo"))); + assertEquals(DataType.KEYWORD, c.dataType()); + assertEquals(TypeResolution.TYPE_RESOLVED, c.typeResolved()); + assertNotNull(c.info()); + } + private List mutateChildren(Case c) { boolean removeConditional = randomBoolean(); List expressions = new ArrayList<>(c.children().size()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IifTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IifTests.java index 054bf2c879f7..a07663b188d2 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IifTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IifTests.java @@ -6,20 +6,24 @@ package org.elasticsearch.xpack.sql.expression.predicate.conditional; import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.Equals; import org.elasticsearch.xpack.sql.tree.AbstractNodeTestCase; import org.elasticsearch.xpack.sql.tree.NodeSubclassTests; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.SourceTests; +import org.elasticsearch.xpack.sql.type.DataType; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Objects; import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomIntLiteral; import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomStringLiteral; +import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; import static org.elasticsearch.xpack.sql.tree.SourceTests.randomSource; /** @@ -74,6 +78,13 @@ public class IifTests extends AbstractNodeTestCase { newChildren.get(2)))); } + public void testConditionFolded() { + Iif iif = new Iif(EMPTY, Collections.singletonList(Literal.of(EMPTY, "foo"))); + assertEquals(DataType.KEYWORD, iif.dataType()); + assertEquals(Expression.TypeResolution.TYPE_RESOLVED, iif.typeResolved()); + assertNotNull(iif.info()); + } + private List mutateChildren(Iif iif) { List expressions = new ArrayList<>(3); Equals eq = (Equals) iif.conditions().get(0).condition(); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java index 93f6515f7106..1607c4db5244 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java @@ -49,9 +49,9 @@ import org.elasticsearch.xpack.sql.expression.predicate.conditional.Case; import org.elasticsearch.xpack.sql.expression.predicate.conditional.Coalesce; import org.elasticsearch.xpack.sql.expression.predicate.conditional.ConditionalFunction; import org.elasticsearch.xpack.sql.expression.predicate.conditional.Greatest; -import org.elasticsearch.xpack.sql.expression.predicate.conditional.Iif; import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfConditional; import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfNull; +import org.elasticsearch.xpack.sql.expression.predicate.conditional.Iif; import org.elasticsearch.xpack.sql.expression.predicate.conditional.Least; import org.elasticsearch.xpack.sql.expression.predicate.conditional.NullIf; import org.elasticsearch.xpack.sql.expression.predicate.logical.And; @@ -112,6 +112,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.singletonList; +import static org.elasticsearch.xpack.sql.expression.Expression.TypeResolution; import static org.elasticsearch.xpack.sql.expression.Literal.FALSE; import static org.elasticsearch.xpack.sql.expression.Literal.NULL; import static org.elasticsearch.xpack.sql.expression.Literal.TRUE; @@ -630,6 +631,7 @@ public class OptimizerTests extends ESTestCase { assertThat(c.conditions().get(0).condition().toString(), startsWith("Equals[a{f}#")); assertThat(c.conditions().get(1).condition().toString(), startsWith("GreaterThan[a{f}#")); assertFalse(c.foldable()); + assertEquals(TypeResolution.TYPE_RESOLVED, c.typeResolved()); } public void testSimplifyCaseConditionsFoldWhenTrue() { @@ -661,6 +663,7 @@ public class OptimizerTests extends ESTestCase { assertThat(c.conditions().get(0).condition().toString(), startsWith("Equals[a{f}#")); assertThat(c.conditions().get(1).condition().toString(), startsWith("Equals[=1,=1]#")); assertFalse(c.foldable()); + assertEquals(TypeResolution.TYPE_RESOLVED, c.typeResolved()); } public void testSimplifyCaseConditionsFoldCompletely() { @@ -685,6 +688,7 @@ public class OptimizerTests extends ESTestCase { assertThat(c.conditions().get(0).condition().toString(), startsWith("Equals[=1,=1]#")); assertTrue(c.foldable()); assertEquals("foo2", c.fold()); + assertEquals(TypeResolution.TYPE_RESOLVED, c.typeResolved()); } public void testSimplifyIif_ConditionTrue() { @@ -696,6 +700,7 @@ public class OptimizerTests extends ESTestCase { assertEquals(1, iif.conditions().size()); assertTrue(iif.foldable()); assertEquals("foo", iif.fold()); + assertEquals(TypeResolution.TYPE_RESOLVED, iif.typeResolved()); } public void testSimplifyIif_ConditionFalse() { @@ -707,6 +712,7 @@ public class OptimizerTests extends ESTestCase { assertEquals(0, iif.conditions().size()); assertTrue(iif.foldable()); assertEquals("bar", iif.fold()); + assertEquals(TypeResolution.TYPE_RESOLVED, iif.typeResolved()); } // From f6466f484035343190adb8563f54077e5b72f97a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Istv=C3=A1n=20Zolt=C3=A1n=20Szab=C3=B3?= Date: Wed, 4 Sep 2019 10:14:30 +0200 Subject: [PATCH 029/103] [DOCS] Reformats multi search API (#46256) * [DOCS] Reformats multi search API. Co-Authored-By: James Rodewig --- docs/reference/search/multi-search.asciidoc | 121 ++++++++++++++------ 1 file changed, 86 insertions(+), 35 deletions(-) diff --git a/docs/reference/search/multi-search.asciidoc b/docs/reference/search/multi-search.asciidoc index 42dbd0f5c2a1..51e0f680a3be 100644 --- a/docs/reference/search/multi-search.asciidoc +++ b/docs/reference/search/multi-search.asciidoc @@ -1,13 +1,22 @@ [[search-multi-search]] === Multi Search API -The multi search API allows to execute several search requests within -the same API. The endpoint for it is `_msearch`. +Executes several searches with a single API request. -The format of the request is similar to the bulk API format and makes -use of the newline delimited JSON (NDJSON) format. The structure is as -follows (the structure is specifically optimized to reduce parsing if -a specific search ends up redirected to another node): +[[search-multi-search-api-request]] +==== {api-request-title} + +`GET //_msearch` + + +[[search-multi-search-api-desc]] +==== {api-description-title} + +The multi search AP executes several searches from a single API request. +The format of the request is similar to the bulk API format and makes use +of the newline delimited JSON (NDJSON) format. The structure is as follows (the +structure is specifically optimized to reduce parsing if a specific search ends +up redirected to another node): [source,js] -------------------------------------------------- @@ -22,9 +31,67 @@ body\n may be preceded by a carriage return `\r`. When sending requests to this endpoint the `Content-Type` header should be set to `application/x-ndjson`. -The header part includes which index / indices to search on, the `search_type`, `preference`, -and `routing`. The body includes the typical search body request (including the `query`, -`aggregations`, `from`, `size`, and so on). Here is an example: +[[search-multi-search-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index] + + +[[search-multi-search-api-request-body]] +==== {api-request-body-title} + +`aggregations`:: + (Optional, object) Specifies aggregations. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=from] + +`max_concurrent_searches`:: + (Optional, integer) Specifies the maximum number of concurrent searches the + multi search API will execute. This default is based on the number of data + nodes and the default search thread pool size. + +`max_concurrent_shard_requests`:: + (Optional, integer) Specifies the maximum number of concurrent shard requests + that each sub search request will execute per node. This parameter should be + used to protect a single request from overloading a cluster (for example a + default request will hit all indices in a cluster which could cause shard + request rejections if the number of shards per node is high). Defaults to `5`. + In certain scenarios parallelism isn't achieved through concurrent request + such that this protection will result in poor performance. For instance in an + environment where only a very low number of concurrent search requests are + expected it might help to increase this value to a higher number. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=preference] + +`query`:: + (Optional, <>) Defines the search definition using the + <>. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=routing] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=search_type] + +`size`:: + (Optional, integer) The number of hits to return. Defaults to `10`. + + +[[search-multi-search-api-response-body]] +==== {api-response-body-title} + +`responses`:: + (array) Includes the search response and status code for each search request + matching its order in the original multi search request. If there was a + complete failure for a specific search request, an object with `error` message + and corresponding status code will be returned in place of the actual search + response. + + +[[search-multi-search-api-example]] +==== {api-examples-title} + +The header part includes which index / indices to search on, the `search_type`, +`preference`, and `routing`. The body includes the typical search body request +(including the `query`, `aggregations`, `from`, `size`, and so on). [source,js] -------------------------------------------------- @@ -51,11 +118,6 @@ $ curl -H "Content-Type: application/x-ndjson" -XGET localhost:9200/_msearch --d Note, the above includes an example of an empty header (can also be just without any content) which is supported as well. -The response returns a `responses` array, which includes the search -response and status code for each search request matching its order in -the original multi search request. If there was a complete failure for that -specific search request, an object with `error` message and corresponding -status code will be returned in place of the actual search response. The endpoint allows to also search against an index/indices in the URI itself, in which case it will be used as the default unless explicitly defined otherwise @@ -81,33 +143,19 @@ against the `twitter2` index. The `search_type` can be set in a similar manner to globally apply to all search requests. -The msearch's `max_concurrent_searches` request parameter can be used to control -the maximum number of concurrent searches the multi search api will execute. -This default is based on the number of data nodes and the default search thread pool size. -The request parameter `max_concurrent_shard_requests` can be used to control -the maximum number of concurrent shard requests that each sub search request -will execute per node. This parameter should be used to protect a single -request from overloading a cluster (e.g., a default request will hit all -indices in a cluster which could cause shard request rejections if the number -of shards per node is high). This default value is `5`.In certain scenarios -parallelism isn't achieved through concurrent request such that this protection -will result in poor performance. For instance in an environment where only a -very low number of concurrent search requests are expected it might help to -increase this value to a higher number. - -[float] [[msearch-security]] ==== Security See <> -[float] + [[template-msearch]] ==== Template support Much like described in <> for the _search resource, _msearch -also provides support for templates. Submit them like follows: +also provides support for templates. Submit them like follows for inline +templates: [source,js] ----------------------------------------------- @@ -120,7 +168,6 @@ GET _msearch/template // CONSOLE // TEST[setup:twitter] -for inline templates. You can also create search templates: @@ -143,6 +190,7 @@ POST /_scripts/my_template_1 // CONSOLE // TEST[setup:twitter] + [source,js] ------------------------------------------ POST /_scripts/my_template_2 @@ -162,7 +210,7 @@ POST /_scripts/my_template_2 // CONSOLE // TEST[continued] -and later use them in a _msearch: +You can use search templates in a _msearch: [source,js] ----------------------------------------------- @@ -175,7 +223,10 @@ GET _msearch/template // CONSOLE // TEST[continued] -[float] + [[multi-search-partial-responses]] ==== Partial responses -To ensure fast responses, the multi search API will respond with partial results if one or more shards fail. See <> for more information. \ No newline at end of file + +To ensure fast responses, the multi search API will respond with partial results +if one or more shards fail. See <> for more +information. From c99224a32d26db985053b7b36e2049036e438f97 Mon Sep 17 00:00:00 2001 From: Andrey Ershov Date: Wed, 4 Sep 2019 10:48:32 +0200 Subject: [PATCH 030/103] Remove stack trace logging in Security(Transport|Http)ExceptionHandler (#45966) As per #45852 comment we no longer need to log stack-traces in SecurityTransportExceptionHandler and SecurityHttpExceptionHandler even if trace logging is enabled. --- .../SecurityTransportExceptionHandler.java | 13 ++---------- .../SecurityHttpExceptionHandler.java | 21 +++---------------- 2 files changed, 5 insertions(+), 29 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/SecurityTransportExceptionHandler.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/SecurityTransportExceptionHandler.java index ecc55fb47523..44789e81ce96 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/SecurityTransportExceptionHandler.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/SecurityTransportExceptionHandler.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.core.security.transport; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.network.CloseableChannel; import org.elasticsearch.transport.TcpChannel; @@ -33,18 +32,10 @@ public final class SecurityTransportExceptionHandler implements BiConsumer Date: Wed, 4 Sep 2019 10:52:17 +0200 Subject: [PATCH 031/103] [DOCS] Reformats request body search API (#46254) * [DOCS] Reformats request body search API. Co-Authored-By: James Rodewig --- docs/reference/search/request-body.asciidoc | 170 +++++++++++--------- 1 file changed, 92 insertions(+), 78 deletions(-) diff --git a/docs/reference/search/request-body.asciidoc b/docs/reference/search/request-body.asciidoc index cd9ee8592c49..b84d695e9f10 100644 --- a/docs/reference/search/request-body.asciidoc +++ b/docs/reference/search/request-body.asciidoc @@ -1,9 +1,7 @@ [[search-request-body]] === Request Body Search -The search request can be executed with a search DSL, which includes the -<>, within its body. Here is an -example: +Specifies search criteria as request body parameters. [source,js] -------------------------------------------------- @@ -17,7 +15,94 @@ GET /twitter/_search // CONSOLE // TEST[setup:twitter] -And here is a sample response: + +[[search-request-body-api-request]] +==== {api-request-title} + + +`GET //_search +{ + "query": {} +}` + + +[[search-request-body-api-desc]] +==== {api-description-title} + +The search request can be executed with a search DSL, which includes the +<>, within its body. + + +[[search-request-body-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index] + + +[[search-request-body-api-request-body]] +==== {api-request-body-title} + +`allow_partial_search_results`:: + (Optional, boolean) Set to `false` to fail the request if only partial results + are available. Defaults to `true`, which returns partial results in the event + of timeouts or partial failures You can override the default behavior for all + requests by setting `search.default_allow_partial_results` to `false` in the + cluster settings. + +`batched_reduce_size`:: + (Optional, integer) The number of shard results that should be reduced at once + on the coordinating node. This value should be used as a protection mechanism + to reduce the memory overhead per search request if the potential number of + shards in the request can be large. + +[[ccs-minimize-roundtrips]] +`ccs_minimize_roundtrips`:: + (Optional, boolean) If `true`, the network round-trips between the + coordinating node and the remote clusters ewill be minimized when executing + {ccs} requests. See <> for more. Defaults to `true`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=from] + +`request_cache`:: + (Optional, boolean) If `true`, the caching of search results is enabled for + requests where `size` is `0`. See <>. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=search_type] + +`size`:: + (Optional, integer) The number of hits to return. Defaults to `10`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=terminate_after] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=timeout] + + +Out of the above, the `search_type`, `request_cache` and the +`allow_partial_search_results` settings must be passed as query-string +parameters. The rest of the search request should be passed within the body +itself. The body content can also be passed as a REST parameter named `source`. + +Both HTTP GET and HTTP POST can be used to execute search with body. Since not +all clients support GET with body, POST is allowed as well. + + +[[search-request-body-api-example]] +==== {api-examples-title} + +[source,js] +-------------------------------------------------- +GET /twitter/_search +{ + "query" : { + "term" : { "user" : "kimchy" } + } +} +-------------------------------------------------- +// CONSOLE +// TEST[setup:twitter] + + +The API returns the following response: [source,js] -------------------------------------------------- @@ -55,81 +140,7 @@ And here is a sample response: -------------------------------------------------- // TESTRESPONSE[s/"took": 1/"took": $body.took/] -[float] -==== Parameters -[horizontal] -`timeout`:: - - A search timeout, bounding the search request to be executed within the - specified time value and bail with the hits accumulated up to that point - when expired. Search requests are canceled after the timeout is reached using - the <> mechanism. - Defaults to no timeout. See <>. - -`from`:: - - To retrieve hits from a certain offset. Defaults to `0`. - -`size`:: - - The number of hits to return. Defaults to `10`. If you do not care about - getting some hits back but only about the number of matches and/or - aggregations, setting the value to `0` will help performance. - -`search_type`:: - - The type of the search operation to perform. Can be - `dfs_query_then_fetch` or `query_then_fetch`. - Defaults to `query_then_fetch`. - See <> for more. - -`request_cache`:: - - Set to `true` or `false` to enable or disable the caching - of search results for requests where `size` is 0, ie - aggregations and suggestions (no top hits returned). - See <>. - -`allow_partial_search_results`:: - - Set to `false` to return an overall failure if the request would produce partial - results. Defaults to true, which will allow partial results in the case of timeouts - or partial failures. This default can be controlled using the cluster-level setting - `search.default_allow_partial_results`. - -`terminate_after`:: - - The maximum number of documents to collect for each shard, - upon reaching which the query execution will terminate early. If set, the - response will have a boolean field `terminated_early` to indicate whether - the query execution has actually terminated_early. Defaults to no - terminate_after. - -`batched_reduce_size`:: - - The number of shard results that should be reduced at once on the - coordinating node. This value should be used as a protection mechanism to - reduce the memory overhead per search request if the potential number of - shards in the request can be large. - -[[ccs-minimize-roundtrips]] -`ccs_minimize_roundtrips`:: - - Defaults to `true`. Set to `false` to disable minimizing network round-trips - between the coordinating node and the remote clusters when executing - {ccs} requests. See <> for more. - - -Out of the above, the `search_type`, `request_cache` and the `allow_partial_search_results` -settings must be passed as query-string parameters. The rest of the search request should -be passed within the body itself. The body content can also be passed as a REST -parameter named `source`. - -Both HTTP GET and HTTP POST can be used to execute search with body. Since not -all clients support GET with body, POST is allowed as well. - -[float] ==== Fast check for any matching docs NOTE: `terminate_after` is always applied **after** the `post_filter` and stops @@ -151,6 +162,7 @@ GET /_search?q=message:number&size=0&terminate_after=1 // CONSOLE // TEST[setup:twitter] + The response will not contain any hits as the `size` was set to `0`. The `hits.total` will be either equal to `0`, indicating that there were no matching documents, or greater than `0` meaning that there were at least @@ -182,6 +194,7 @@ be set to `true` in the response. -------------------------------------------------- // TESTRESPONSE[s/"took": 3/"took": $body.took/] + The `took` time in the response contains the milliseconds that this request took for processing, beginning quickly after the node received the query, up until all search related work is done and before the above JSON is returned @@ -189,6 +202,7 @@ to the client. This means it includes the time spent waiting in thread pools, executing a distributed search across the whole cluster and gathering all the results. + include::request/docvalue-fields.asciidoc[] include::request/explain.asciidoc[] From 1d99f8afb2bf4f6cfd0d325eee2650e1f86f5a0c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Sep 2019 10:56:20 +0200 Subject: [PATCH 032/103] Reenable+Fix testMasterShutdownDuringFailedSnapshot (#46303) Reenable this test since it was fixed by #45689 in production code (specifically, the fact that we write the `snap-` blobs without overwrite checks now). Only required adding the assumed blocking on index file writes to test code to properly work again. * Closes #25281 --- .../snapshots/DedicatedClusterSnapshotRestoreIT.java | 1 - .../org/elasticsearch/snapshots/mockstore/MockRepository.java | 3 +++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index d9bc6dd02209..e07107f790b4 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -931,7 +931,6 @@ public class DedicatedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTest } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/25281") public void testMasterShutdownDuringFailedSnapshot() throws Exception { logger.info("--> starting two master nodes and two data nodes"); internalCluster().startMasterOnlyNodes(2); diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java index fa33f8aef867..9a157e11c722 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java @@ -369,6 +369,9 @@ public class MockRepository extends FsRepository { public void writeBlobAtomic(final String blobName, final InputStream inputStream, final long blobSize, final boolean failIfAlreadyExists) throws IOException { final Random random = RandomizedContext.current().getRandom(); + if (blobName.startsWith("index-") && blockOnWriteIndexFile) { + blockExecutionAndFail(blobName); + } if ((delegate() instanceof FsBlobContainer) && (random.nextBoolean())) { // Simulate a failure between the write and move operation in FsBlobContainer final String tempBlobName = FsBlobContainer.tempBlobName(blobName); From 091fa889c51f89eaca01d292721a0b42ad34a8af Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Wed, 4 Sep 2019 12:10:23 +0300 Subject: [PATCH 033/103] DOCS Link to kib reference from es reference on PKI authn (#46260) --- .../en/rest-api/security/delegate-pki-authentication.asciidoc | 2 +- .../en/security/authentication/configuring-pki-realm.asciidoc | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc b/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc index 92d82f1c273e..b8d20b23953b 100644 --- a/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc +++ b/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc @@ -5,7 +5,7 @@ Delegate PKI authentication ++++ -Implements the exchange of an {@code X509Certificate} chain into an {es} access +Implements the exchange of an _X509Certificate_ chain into an {es} access token. [[security-api-delegate-pki-authentication-request]] diff --git a/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc index a3fc1a6c0b0a..0964d1795b00 100644 --- a/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc @@ -246,7 +246,9 @@ NOTE: PKI authentication delegation requires that the `xpack.security.authc.token.enabled` setting be `true` and that SSL/TLS be configured (without SSL/TLS client authentication). -NOTE: {kib} also needs to be configured to allow PKI certificate authentication. +NOTE: {kib} also needs to be +{kibana-ref}/kibana-authentication.html#pki-authentication[configured to allow +PKI certificate authentication]. A PKI realm with `delegation.enabled` still works unchanged for clients connecting directly to {es}. Directly authenticated users, and users that are PKI From fc2f4f1d2d162b99e849204459c32b4fb106229d Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 4 Sep 2019 12:18:23 +0300 Subject: [PATCH 034/103] Quote the task name in reproduction line printer (#46266) Some tasks have `#` for instance that doesn't play well with some shells ( e.x. zsh ) --- .../test/junit/listeners/ReproduceInfoPrinter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java b/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java index 24e38bbec58d..b94c8551562e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java +++ b/test/framework/src/main/java/org/elasticsearch/test/junit/listeners/ReproduceInfoPrinter.java @@ -79,7 +79,7 @@ public class ReproduceInfoPrinter extends RunListener { String task = System.getProperty("tests.task"); // append Gradle test runner test filter string - b.append(task); + b.append("'" + task + "'"); b.append(" --tests \""); b.append(failure.getDescription().getClassName()); final String methodName = failure.getDescription().getMethodName(); From ded27911ddc9b2f127afeb9f729efa6e0ca2e3e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Istv=C3=A1n=20Zolt=C3=A1n=20Szab=C3=B3?= Date: Wed, 4 Sep 2019 11:34:30 +0200 Subject: [PATCH 035/103] [DOCS] Reformats search shards API (#46240) * [DOCS] Reformats search shards API Co-Authored-By: James Rodewig --- docs/reference/search/search-shards.asciidoc | 93 ++++++++++++-------- 1 file changed, 58 insertions(+), 35 deletions(-) diff --git a/docs/reference/search/search-shards.asciidoc b/docs/reference/search/search-shards.asciidoc index 9912a791af13..288f8b7545ec 100644 --- a/docs/reference/search/search-shards.asciidoc +++ b/docs/reference/search/search-shards.asciidoc @@ -1,17 +1,7 @@ [[search-shards]] === Search Shards API -The search shards api returns the indices and shards that a search request would -be executed against. This can give useful feedback for working out issues or -planning optimizations with routing and shard preferences. When filtered aliases -are used, the filter is returned as part of the `indices` section [5.1.0] Added in 5.1.0. - -The `index` may be a single value, or comma-separated. - -[float] -==== Usage - -Full example: +Returns the indices and shards that a search request would be executed against. [source,js] -------------------------------------------------- @@ -20,7 +10,59 @@ GET /twitter/_search_shards // CONSOLE // TEST[s/^/PUT twitter\n{"settings":{"index.number_of_shards":5}}\n/] -This will yield the following result: + +[[search-shards-api-request]] +==== {api-request-title} + +`GET //_search_shards` + + +[[search-shards-api-desc]] +==== {api-description-title} + +The search shards api returns the indices and shards that a search request would +be executed against. This can give useful feedback for working out issues or +planning optimizations with routing and shard preferences. When filtered aliases +are used, the filter is returned as part of the `indices` section. + + +[[search-shards-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index] + + +[[search-shards-api-query-params]] +==== {api-query-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=allow-no-indices] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] ++ +-- +Defaults to `open`. +-- + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=local] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=preference] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=routing] + + +[[search-shards-api-example]] +==== {api-examples-title} + +[source,js] +-------------------------------------------------- +GET /twitter/_search_shards +-------------------------------------------------- +// CONSOLE +// TEST[s/^/PUT twitter\n{"settings":{"index.number_of_shards":5}}\n/] + +The API returns the following result: [source,js] -------------------------------------------------- @@ -96,7 +138,7 @@ This will yield the following result: // TESTRESPONSE[s/bU_KLGJISbW0RejwnwDPKw/$body.shards.3.0.allocation_id.id/] // TESTRESPONSE[s/DMs7_giNSwmdqVukF7UydA/$body.shards.4.0.allocation_id.id/] -And specifying the same request, this time with a routing value: +Specifying the same request, this time with a routing value: [source,js] -------------------------------------------------- @@ -105,7 +147,7 @@ GET /twitter/_search_shards?routing=foo,bar // CONSOLE // TEST[s/^/PUT twitter\n{"settings":{"index.number_of_shards":5}}\n/] -This will yield the following result: +The API returns the following result: [source,js] -------------------------------------------------- @@ -145,24 +187,5 @@ This will yield the following result: // TESTRESPONSE[s/0TvkCyF7TAmM1wHP4a42-A/$body.shards.1.0.allocation_id.id/] // TESTRESPONSE[s/fMju3hd1QHWmWrIgFnI4Ww/$body.shards.0.0.allocation_id.id/] -This time the search will only be executed against two of the shards, because -routing values have been specified. - -[float] -==== All parameters: - -[horizontal] -`routing`:: - A comma-separated list of routing values to take into account when - determining which shards a request would be executed against. - -`preference`:: - Controls a `preference` of which shard replicas to execute the search - request on. By default, the operation is randomized between the shard - replicas. See the link:search-request-body.html#request-body-search-preference[preference] - documentation for a list of all acceptable values. - -`local`:: - A boolean value whether to read the cluster state locally in order to - determine where shards are allocated instead of using the Master node's - cluster state. \ No newline at end of file +The search will only be executed against two of the shards, because outing +values have been specified. From 79cd3e1e410570d09db921339dd294fd0d98e5ae Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Wed, 4 Sep 2019 14:07:26 +0200 Subject: [PATCH 036/103] Fix SearchService.createContext exception handling (#46258) An exception from the DefaultSearchContext constructor could leak a searcher, causing future issues like shard lock obtained exceptions. The underlying cause of the exception in the constructor has been fixed, but as a safety precaution we also fix the exception handling in createContext. Closes #45378 --- .../elasticsearch/search/SearchService.java | 12 ++++++--- .../search/SearchServiceTests.java | 25 +++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 83e15eb5e73f..9311ae287d9a 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -626,11 +626,12 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv indexShard.shardId(), request.getClusterAlias(), OriginalIndices.NONE); Engine.Searcher searcher = indexShard.acquireSearcher(source); - final DefaultSearchContext searchContext = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, - searcher, clusterService, indexService, indexShard, bigArrays, threadPool::relativeTimeInMillis, timeout, - fetchPhase, clusterService.state().nodes().getMinNodeVersion()); boolean success = false; + DefaultSearchContext searchContext = null; try { + searchContext = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, + searcher, clusterService, indexService, indexShard, bigArrays, threadPool::relativeTimeInMillis, timeout, + fetchPhase, clusterService.state().nodes().getMinNodeVersion()); // we clone the query shard context here just for rewriting otherwise we // might end up with incorrect state since we are using now() or script services // during rewrite and normalized / evaluate templates etc. @@ -641,6 +642,11 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv } finally { if (success == false) { IOUtils.closeWhileHandlingException(searchContext); + if (searchContext == null) { + // we handle the case where the DefaultSearchContext constructor throws an exception since we would otherwise + // leak a searcher and this can have severe implications (unable to obtain shard lock exceptions). + IOUtils.closeWhileHandlingException(searcher); + } } } return searchContext; diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index f8ef11abe9bf..dc3f91f4502a 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchTask; +import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; @@ -680,4 +681,28 @@ public class SearchServiceTests extends ESSingleNodeTestCase { assertSame(searchShardTarget, searchContext.fetchResult().getSearchShardTarget()); } } + + /** + * While we have no NPE in DefaultContext constructor anymore, we still want to guard against it (or other failures) in the future to + * avoid leaking searchers. + */ + public void testCreateSearchContextFailure() throws IOException { + final String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); + final IndexService indexService = createIndex(index); + final SearchService service = getInstanceFromNode(SearchService.class); + final ShardId shardId = new ShardId(indexService.index(), 0); + + NullPointerException e = expectThrows(NullPointerException.class, + () -> service.createContext( + new ShardSearchLocalRequest(shardId, 0, null) { + @Override + public SearchType searchType() { + // induce an artificial NPE + throw new NullPointerException("expected"); + } + } + )); + assertEquals("expected", e.getMessage()); + assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount()); + } } From a238248706e7daa8950297520964b20496a4b410 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 4 Sep 2019 15:39:37 +0300 Subject: [PATCH 037/103] Bwc testclusters all (#46265) Convert all bwc projects to testclusters --- .../testclusters/ElasticsearchNode.java | 28 +- .../gradle/VersionProperties.java | 4 + qa/full-cluster-restart/build.gradle | 11 +- qa/mixed-cluster/build.gradle | 85 +++-- qa/rolling-upgrade/build.gradle | 149 ++++---- .../elasticsearch/upgrades/RecoveryIT.java | 69 +--- qa/verify-version-constants/build.gradle | 46 +-- .../test/cat.allocation/10_basic.yml | 16 +- .../test/rest/ESRestTestCase.java | 22 +- x-pack/qa/full-cluster-restart/build.gradle | 262 +++++-------- x-pack/qa/rolling-upgrade-basic/build.gradle | 161 ++++---- .../build.gradle | 351 ++++++------------ .../AbstractMultiClusterUpgradeTestCase.java | 1 + .../upgrades/CcrRollingUpgradeIT.java | 6 +- x-pack/qa/rolling-upgrade/build.gradle | 315 +++++----------- .../elasticsearch/upgrades/IndexingIT.java | 22 +- .../upgrades/MlMappingsUpgradeIT.java | 1 + 17 files changed, 608 insertions(+), 941 deletions(-) diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java index 5cca8dc0e191..c0d4c4baafc3 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java @@ -464,13 +464,6 @@ public class ElasticsearchNode implements TestClusterConfiguration { public void restart() { LOGGER.info("Restarting {}", this); stop(false); - try { - Files.delete(httpPortsFile); - Files.delete(transportPortFile); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - start(); } @@ -479,11 +472,11 @@ public class ElasticsearchNode implements TestClusterConfiguration { if (currentDistro + 1 >= distributions.size()) { throw new TestClustersException("Ran out of versions to go to for " + this); } - LOGGER.info("Switch version from {} to {} for {}", - getVersion(), distributions.get(currentDistro + 1).getVersion(), this - ); + logToProcessStdout("Switch version from " + getVersion() + " to " + distributions.get(currentDistro + 1).getVersion()); + stop(false); currentDistro += 1; - restart(); + setting("node.attr.upgraded", "true"); + start(); } private boolean isSettingMissingOrTrue(String name) { @@ -716,6 +709,17 @@ public class ElasticsearchNode implements TestClusterConfiguration { @Override public synchronized void stop(boolean tailLogs) { + logToProcessStdout("Stopping node"); + try { + if (Files.exists(httpPortsFile)) { + Files.delete(httpPortsFile); + } + if (Files.exists(transportPortFile)) { + Files.delete(transportPortFile); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } if (esProcess == null && tailLogs) { // This is a special case. If start() throws an exception the plugin will still call stop // Another exception here would eat the orriginal. @@ -867,6 +871,8 @@ public class ElasticsearchNode implements TestClusterConfiguration { private void createWorkingDir(Path distroExtractDir) throws IOException { syncWithLinks(distroExtractDir, distroDir); + // Start configuration from scratch in case of a restart + project.delete(configFile.getParent()); Files.createDirectories(configFile.getParent()); Files.createDirectories(confPathRepo); Files.createDirectories(confPathData); diff --git a/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/VersionProperties.java b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/VersionProperties.java index cdb0f01cf75a..bbcb1e5845e5 100644 --- a/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/VersionProperties.java +++ b/buildSrc/src/minimumRuntime/java/org/elasticsearch/gradle/VersionProperties.java @@ -15,6 +15,10 @@ public class VersionProperties { return elasticsearch; } + public static Version getElasticsearchVersion() { + return Version.fromString(elasticsearch); + } + public static String getLucene() { return lucene; } diff --git a/qa/full-cluster-restart/build.gradle b/qa/full-cluster-restart/build.gradle index 1dc5ddd1acee..524d1e1516c4 100644 --- a/qa/full-cluster-restart/build.gradle +++ b/qa/full-cluster-restart/build.gradle @@ -21,12 +21,10 @@ import org.elasticsearch.gradle.Version import org.elasticsearch.gradle.testclusters.RestTestRunnerTask -apply plugin: 'elasticsearch.standalone-test' apply plugin: 'elasticsearch.testclusters' +apply plugin: 'elasticsearch.standalone-test' -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } @@ -81,9 +79,6 @@ for (Version bwcVersion : bwcVersions.indexCompatible) { } } -test.enabled = false // no unit tests for rolling upgrades, only the rest integration test - -// basic integ tests includes testing bwc against the most recent version task bwcTestSnapshots { if (project.bwc_tests_enabled) { for (final def version : bwcVersions.unreleasedIndexCompatible) { @@ -106,3 +101,5 @@ task testJar(type: Jar) { artifacts { testArtifacts testJar } + +test.enabled = false \ No newline at end of file diff --git a/qa/mixed-cluster/build.gradle b/qa/mixed-cluster/build.gradle index 91480c4e3d24..5f3690c293d8 100644 --- a/qa/mixed-cluster/build.gradle +++ b/qa/mixed-cluster/build.gradle @@ -17,58 +17,89 @@ * under the License. */ -import org.elasticsearch.gradle.test.RestIntegTestTask import org.elasticsearch.gradle.Version +import org.elasticsearch.gradle.VersionProperties +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } -for (Version version : bwcVersions.wireCompatible) { - String baseName = "v${version}" +configurations { + restSpec +} - Task mixedClusterTest = tasks.create(name: "${baseName}#mixedClusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) - includePackaged = true +dependencies { + restSpec project(':rest-api-spec') +} + +processTestResources { + from ({ zipTree(configurations.restSpec.singleFile) }) + dependsOn configurations.restSpec +} + +for (Version bwcVersion : bwcVersions.wireCompatible) { + if (bwcVersion == VersionProperties.getElasticsearchVersion()) { + // Not really a mixed cluster + continue ; } + String baseName = "v${bwcVersion}" + /* This project runs the core REST tests against a 4 node cluster where two of the nodes has a different minor. */ - Object extension = extensions.findByName("${baseName}#mixedClusterTestCluster") - configure(extension) { - numNodes = 4 - numBwcNodes = 2 - bwcVersion = version + testClusters { + "${baseName}" { + versions = [ bwcVersion.toString(), project.version ] + numberOfNodes = 4 + + setting 'path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + javaHome = project.file(project.ext.runtimeJavaHome) + } } - Task versionBwcTest = tasks.create(name: "${baseName}#bwcTest") { - dependsOn = [mixedClusterTest] + tasks.register("${baseName}#mixedClusterTest", RestTestRunnerTask) { + useCluster testClusters."${baseName}" + mustRunAfter(precommit) + doFirst { + project.delete("${buildDir}/cluster/shared/repo/${baseName}") + // Getting the endpoints causes a wait for the cluster + println "Test cluster endpoints are: ${-> testClusters."${baseName}".allHttpSocketURI.join(",") }" + println "Upgrading one node to create a mixed cluster" + testClusters."${baseName}".nextNodeToNextVersion() + // Getting the endpoints causes a wait for the cluster + println "Upgrade complete, endpoints are: ${-> testClusters."${baseName}".allHttpSocketURI.join(",") }" + println "Upgrading another node to create a mixed cluster" + testClusters."${baseName}".nextNodeToNextVersion() + + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") + } + systemProperty 'tests.path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + onlyIf { project.bwc_tests_enabled } } - if (project.bwc_tests_enabled) { - bwcTest.dependsOn(versionBwcTest) + tasks.register("${baseName}#bwcTest") { + dependsOn "${baseName}#mixedClusterTest" } - tasks.getByName("${baseName}#mixedClusterTestRunner").configure { - /* To support taking index snapshots, we have to set path.repo setting */ - systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") - } + tasks.bwcTest.dependsOn "${baseName}#bwcTest" } -test.enabled = false // no unit tests for rolling upgrades, only the rest integration test - -// basic integ tests includes testing bwc against the most recent version task bwcTestSnapshots { if (project.bwc_tests_enabled) { - for (final def version : bwcVersions.unreleasedWireCompatible) { - dependsOn "v${version}#bwcTest" + for (Version bwcVersion : bwcVersions.unreleasedWireCompatible) { + if (bwcVersion != VersionProperties.getElasticsearchVersion()) { + dependsOn "v${bwcVersion}#bwcTest" + } } } } check.dependsOn(bwcTestSnapshots) + +test.enabled = false diff --git a/qa/rolling-upgrade/build.gradle b/qa/rolling-upgrade/build.gradle index fae318bd9390..9d352f8522e5 100644 --- a/qa/rolling-upgrade/build.gradle +++ b/qa/rolling-upgrade/build.gradle @@ -17,9 +17,10 @@ * under the License. */ -import org.elasticsearch.gradle.test.RestIntegTestTask import org.elasticsearch.gradle.Version +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' // This is a top level task which we will add dependencies to below. @@ -29,112 +30,102 @@ task bwcTest { group = 'verification' } -for (Version version : bwcVersions.wireCompatible) { +configurations { + restSpec +} + +dependencies { + restSpec project(':rest-api-spec') +} + +processTestResources { + from ({ zipTree(configurations.restSpec.singleFile) }) { + include 'rest-api-spec/api/**' + } + dependsOn configurations.restSpec +} + +for (Version bwcVersion : bwcVersions.wireCompatible) { /* * The goal here is to: *
    *
  • start three nodes on the old version *
  • run tests with systemProperty 'tests.rest.suite', 'old_cluster' - *
  • shut down one node - *
  • start a node with the new version + *
  • upgrade one node *
  • run tests with systemProperty 'tests.rest.suite', 'mixed_cluster' - *
  • shut down one node on the old version - *
  • start a node with the new version + *
  • upgrade one more node *
  • run tests with systemProperty 'tests.rest.suite', 'mixed_cluster' again - *
  • shut down the last node with the old version - *
  • start a node with the new version + *
  • updgrade the last node *
  • run tests with systemProperty 'tests.rest.suite', 'upgraded_cluster' - *
  • shut down the entire cluster *
- * - * Be careful: gradle dry run spits out tasks in the wrong order but, - * strangely, running the tasks works properly. */ - String baseName = "v${version}" + String baseName = "v${bwcVersion}" - Task oldClusterTest = tasks.create(name: "${baseName}#oldClusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) - } + testClusters { + "${baseName}" { + versions = [ bwcVersion.toString(), project.version ] + numberOfNodes = 3 - configure(extensions.findByName("${baseName}#oldClusterTestCluster")) { - bwcVersion = version - numBwcNodes = 3 - numNodes = 3 - clusterName = 'rolling-upgrade' - setting 'repositories.url.allowed_urls', 'http://snapshot.test*' - if (version.onOrAfter('5.3.0')) { - setting 'http.content_type.required', 'true' - } - } - - Task oldClusterTestRunner = tasks.getByName("${baseName}#oldClusterTestRunner") - oldClusterTestRunner.configure { - systemProperty 'tests.rest.suite', 'old_cluster' - } - - Closure configureUpgradeCluster = {String name, Task lastRunner, int stopNode, Closure getOtherUnicastHostAddresses -> - configure(extensions.findByName("${baseName}#${name}")) { - dependsOn lastRunner, "${baseName}#oldClusterTestCluster#node${stopNode}.stop" - clusterName = 'rolling-upgrade' - otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - /* Override the data directory so the new node always gets the node we - * just stopped's data directory. */ - dataDir = { nodeNumber -> oldClusterTest.nodes[stopNode].dataDir } setting 'repositories.url.allowed_urls', 'http://snapshot.test*' - setting 'node.name', "upgraded-node-${stopNode}" + setting 'path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + setting 'http.content_type.required', 'true' + javaHome = project.file(project.ext.runtimeJavaHome) } } - Task oneThirdUpgradedTest = tasks.create(name: "${baseName}#oneThirdUpgradedTest", type: RestIntegTestTask) + tasks.register("${baseName}#oldClusterTest", RestTestRunnerTask) { + dependsOn processTestResources + useCluster testClusters."${baseName}" + mustRunAfter(precommit) + doFirst { + project.delete("${buildDir}/cluster/shared/repo/${baseName}") + } + systemProperty 'tests.rest.suite', 'old_cluster' + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") + } - configureUpgradeCluster("oneThirdUpgradedTestCluster", oldClusterTestRunner, 0, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oldClusterTest.nodes.get(1).transportUri(), oldClusterTest.nodes.get(2).transportUri()] }) - - Task oneThirdUpgradedTestRunner = tasks.getByName("${baseName}#oneThirdUpgradedTestRunner") - oneThirdUpgradedTestRunner.configure { + tasks.register("${baseName}#oneThirdUpgradedTest", RestTestRunnerTask) { + dependsOn "${baseName}#oldClusterTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } systemProperty 'tests.rest.suite', 'mixed_cluster' systemProperty 'tests.first_round', 'true' - finalizedBy "${baseName}#oldClusterTestCluster#node1.stop" + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") } - Task twoThirdsUpgradedTest = tasks.create(name: "${baseName}#twoThirdsUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster("twoThirdsUpgradedTestCluster", oneThirdUpgradedTestRunner, 1, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oldClusterTest.nodes.get(2).transportUri(), oneThirdUpgradedTest.nodes.get(0).transportUri()] }) - - Task twoThirdsUpgradedTestRunner = tasks.getByName("${baseName}#twoThirdsUpgradedTestRunner") - twoThirdsUpgradedTestRunner.configure { + tasks.register("${baseName}#twoThirdsUpgradedTest", RestTestRunnerTask) { + dependsOn "${baseName}#oneThirdUpgradedTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } systemProperty 'tests.rest.suite', 'mixed_cluster' systemProperty 'tests.first_round', 'false' - finalizedBy "${baseName}#oldClusterTestCluster#node2.stop" + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") } - Task upgradedClusterTest = tasks.create(name: "${baseName}#upgradedClusterTest", type: RestIntegTestTask) - - configureUpgradeCluster("upgradedClusterTestCluster", twoThirdsUpgradedTestRunner, 2, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oneThirdUpgradedTest.nodes.get(0).transportUri(), twoThirdsUpgradedTest.nodes.get(0).transportUri()] }) - - Task upgradedClusterTestRunner = tasks.getByName("${baseName}#upgradedClusterTestRunner") - upgradedClusterTestRunner.configure { + tasks.register("${baseName}#upgradedClusterTest", RestTestRunnerTask) { + dependsOn "${baseName}#twoThirdsUpgradedTest" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } + useCluster testClusters."${baseName}" systemProperty 'tests.rest.suite', 'upgraded_cluster' - /* - * Force stopping all the upgraded nodes after the test runner - * so they are alive during the test. - */ - finalizedBy "${baseName}#oneThirdUpgradedTestCluster#stop" - finalizedBy "${baseName}#twoThirdsUpgradedTestCluster#stop" - } - - Task versionBwcTest = tasks.create(name: "${baseName}#bwcTest") { - enabled = project.bwc_tests_enabled - dependsOn = [upgradedClusterTest] + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") } if (project.bwc_tests_enabled) { - bwcTest.dependsOn(versionBwcTest) + bwcTest.dependsOn( + tasks.register("${baseName}#bwcTest") { + dependsOn tasks.named("${baseName}#upgradedClusterTest") + } + ) } } @@ -150,3 +141,5 @@ task bwcTestSnapshots { } check.dependsOn(bwcTestSnapshots) + +test.enabled = false diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java index c0d192d38ec0..df38fab7787d 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java @@ -65,6 +65,8 @@ import static org.hamcrest.Matchers.nullValue; */ public class RecoveryIT extends AbstractRollingTestCase { + private static String CLUSTER_NAME = System.getProperty("tests.clustername"); + public void testHistoryUUIDIsGenerated() throws Exception { final String index = "index_history_uuid"; if (CLUSTER_TYPE == ClusterType.OLD) { @@ -299,54 +301,6 @@ public class RecoveryIT extends AbstractRollingTestCase { } } - /** - * This test ensures that peer recovery won't get stuck in a situation where the recovery target and recovery source - * have an identical sync id but different local checkpoint in the commit in particular the target does not have - * sequence numbers yet. This is possible if the primary is on 6.x while the replica was on 5.x and some write - * operations with sequence numbers have taken place. If this is not the case, then peer recovery should utilize - * syncId and skip copying files. - */ - public void testRecoverSyncedFlushIndex() throws Exception { - final String index = "recover_synced_flush_index"; - if (CLUSTER_TYPE == ClusterType.OLD) { - Settings.Builder settings = Settings.builder() - .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) - .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 2); - if (randomBoolean()) { - settings.put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING.getKey(), "256b"); - } - createIndex(index, settings.build()); - ensureGreen(index); - indexDocs(index, 0, 40); - syncedFlush(index); - } else if (CLUSTER_TYPE == ClusterType.MIXED) { - ensureGreen(index); - if (firstMixedRound) { - assertPeerRecoveredFiles("peer recovery with syncId should not copy files", index, "upgraded-node-0", equalTo(0)); - assertDocCountOnAllCopies(index, 40); - indexDocs(index, 40, 50); - syncedFlush(index); - } else { - assertPeerRecoveredFiles("peer recovery with syncId should not copy files", index, "upgraded-node-1", equalTo(0)); - assertDocCountOnAllCopies(index, 90); - indexDocs(index, 90, 60); - syncedFlush(index); - // exclude node-2 from allocation-filter so we can trim translog on the primary before node-2 starts recover - if (randomBoolean()) { - updateIndexSettings(index, Settings.builder().put("index.routing.allocation.include._name", "upgraded-*")); - } - } - } else { - final int docsAfterUpgraded = randomIntBetween(0, 100); - indexDocs(index, 150, docsAfterUpgraded); - ensureGreen(index); - assertPeerRecoveredFiles("peer recovery with syncId should not copy files", index, "upgraded-node-2", equalTo(0)); - assertDocCountOnAllCopies(index, 150 + docsAfterUpgraded); - } - } - public void testRecoveryWithSoftDeletes() throws Exception { final String index = "recover_with_soft_deletes"; if (CLUSTER_TYPE == ClusterType.OLD) { @@ -530,12 +484,15 @@ public class RecoveryIT extends AbstractRollingTestCase { .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1) .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none") .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "120s") - .put("index.routing.allocation.include._name", "node-0") + .put("index.routing.allocation.include._name", CLUSTER_NAME + "-0") .build()); indexDocs(indexName, 0, randomInt(10)); // allocate replica to node-2 updateIndexSettings(indexName, - Settings.builder().put("index.routing.allocation.include._name", "node-0,node-2,upgraded-node-*")); + Settings.builder().put( + "index.routing.allocation.include._name", + CLUSTER_NAME + "-0," + CLUSTER_NAME + "-2," + CLUSTER_NAME + "-*") + ); ensureGreen(indexName); closeIndex(indexName); } @@ -546,8 +503,16 @@ public class RecoveryIT extends AbstractRollingTestCase { // so we expect the index to be closed and replicated ensureGreen(indexName); assertClosedIndex(indexName, true); - if (CLUSTER_TYPE != ClusterType.OLD && minimumNodeVersion().onOrAfter(Version.V_7_2_0)) { - assertNoFileBasedRecovery(indexName, s-> s.startsWith("upgraded")); + if (minimumNodeVersion().onOrAfter(Version.V_7_2_0)) { + switch (CLUSTER_TYPE) { + case OLD: break; + case MIXED: + assertNoFileBasedRecovery(indexName, s -> s.startsWith(CLUSTER_NAME + "-0")); + break; + case UPGRADED: + assertNoFileBasedRecovery(indexName, s -> s.startsWith(CLUSTER_NAME)); + break; + } } } else { assertClosedIndex(indexName, false); diff --git a/qa/verify-version-constants/build.gradle b/qa/verify-version-constants/build.gradle index 42c2195dd13f..7e472e4f01d7 100644 --- a/qa/verify-version-constants/build.gradle +++ b/qa/verify-version-constants/build.gradle @@ -19,45 +19,45 @@ import org.elasticsearch.gradle.Version import org.elasticsearch.gradle.VersionProperties -import org.elasticsearch.gradle.test.RestIntegTestTask +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } -for (Version version : bwcVersions.indexCompatible) { - String baseName = "v${version}" - Task oldClusterTest = tasks.create(name: "${baseName}#oldClusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) - } - configure(extensions.findByName("${baseName}#oldClusterTestCluster")) { - bwcVersion = version - numBwcNodes = 1 - numNodes = 1 - clusterName = 'verify-version-constants' - if (version.onOrAfter('5.3.0')) { +for (Version bwcVersion : bwcVersions.indexCompatible) { + String baseName = "v${bwcVersion}" + + testClusters { + "${baseName}" { + version = bwcVersion.toString() setting 'http.content_type.required', 'true' + javaHome = project.file(project.ext.runtimeJavaHome) } } - Task versionBwcTest = tasks.create(name: "${baseName}#bwcTest") { - dependsOn = [oldClusterTest] + tasks.register("${baseName}#integTest", RestTestRunnerTask) { + useCluster testClusters."${baseName}" + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") } - bwcTest.dependsOn(versionBwcTest) -} + tasks.register("${baseName}#bwcTest") { + dependsOn "${baseName}#integTest" + } -test.enabled = false + bwcTest.dependsOn("${baseName}#bwcTest") +} task bwcTestSnapshots { if (project.bwc_tests_enabled) { - final def version = bwcVersions.unreleasedIndexCompatible.first() - dependsOn "v${version}#bwcTest" + for (version in bwcVersions.unreleasedIndexCompatible) { + dependsOn "v${version}#bwcTest" + } } } @@ -84,3 +84,5 @@ task verifyDocsLuceneVersion { } check.dependsOn bwcTestSnapshots, verifyDocsLuceneVersion + +test.enabled = false \ No newline at end of file diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml index bdb328fad79c..73e35972aeca 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml @@ -34,7 +34,7 @@ (\d+ \s+)? #no value from client nodes [-\w.]+ \s+ \d+(\.\d+){3} \s+ - [-\w]+ + [-\w.]+ \n )+ $/ @@ -61,7 +61,7 @@ (\d+ \s+) #always should return value since we filter out non data nodes by default [-\w.]+ \s+ \d+(\.\d+){3} \s+ - [-\w]+ + [-\w.]+ \n )+ ( @@ -90,7 +90,7 @@ (\d+ \s+)? #no value from client nodes [-\w.]+ \s+ \d+(\.\d+){3} \s+ - [-\w]+ + [-\w.]+ \n ) $/ @@ -124,7 +124,7 @@ (\d+ \s+)? #no value from client nodes [-\w.]+ \s+ \d+(\.\d+){3} \s+ - [-\w]+ + [-\w.]+ \n )+ ( @@ -164,7 +164,7 @@ (\d+ \s+) #always should return value since we filter out non data nodes by default [-\w.]+ \s+ \d+(\.\d+){3} \s+ - [-\w]+ + [-\w.]+ \n )+ $/ @@ -181,7 +181,7 @@ $body: | /^ ( \d* \s+ - [-\w]+ + [-\w.]+ \n )+ $/ @@ -199,7 +199,7 @@ \n ( \s+\d* \s+ - [-\w]+ + [-\w.]+ \n )+ $/ @@ -224,7 +224,7 @@ (\d+ \s+) #always should return value since we filter out non data nodes by default [-\w.]+ \s+ \d+(\.\d+){3} \s+ - [-\w]+ + [-\w.]+ \n )+ $/ diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 0f7ad85b4656..90a0a656142a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -873,17 +873,27 @@ public abstract class ESRestTestCase extends ESTestCase { * @param index index to test for **/ protected static void ensureGreen(String index) throws IOException { - Request request = new Request("GET", "/_cluster/health/" + index); - request.addParameter("wait_for_status", "green"); - request.addParameter("wait_for_no_relocating_shards", "true"); - request.addParameter("timeout", "70s"); - request.addParameter("level", "shards"); + ensureHealth(index, (request) -> { + request.addParameter("wait_for_status", "green"); + request.addParameter("wait_for_no_relocating_shards", "true"); + request.addParameter("timeout", "70s"); + request.addParameter("level", "shards"); + }); + } + + protected static void ensureHealth(Consumer requestConsumer) throws IOException { + ensureHealth("", requestConsumer); + } + + protected static void ensureHealth(String index, Consumer requestConsumer) throws IOException { + Request request = new Request("GET", "/_cluster/health" + (index.isBlank() ? "" : "/" + index)); + requestConsumer.accept(request); try { client().performRequest(request); } catch (ResponseException e) { if (e.getResponse().getStatusLine().getStatusCode() == HttpStatus.SC_REQUEST_TIMEOUT) { try { - final Response clusterStateResponse = client().performRequest(new Request("GET", "/_cluster/state")); + final Response clusterStateResponse = client().performRequest(new Request("GET", "/_cluster/state?pretty")); fail("timed out waiting for green state for index [" + index + "] " + "cluster state [" + EntityUtils.toString(clusterStateResponse.getEntity()) + "]"); } catch (Exception inner) { diff --git a/x-pack/qa/full-cluster-restart/build.gradle b/x-pack/qa/full-cluster-restart/build.gradle index b962e912cf43..4df2edce169f 100644 --- a/x-pack/qa/full-cluster-restart/build.gradle +++ b/x-pack/qa/full-cluster-restart/build.gradle @@ -1,14 +1,10 @@ -import org.elasticsearch.gradle.test.NodeInfo -import org.elasticsearch.gradle.test.RestIntegTestTask import org.elasticsearch.gradle.Version +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask -import java.nio.charset.StandardCharsets -// Apply the java plugin to this project so the sources can be edited in an IDE +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' -test.enabled = false - dependencies { // TODO: Remove core dependency and change tests to not use builders that are part of xpack-core. // Currently needed for ml tests are using the building for datafeed and job config) @@ -18,49 +14,6 @@ dependencies { testCompile project(':x-pack:qa') } -Closure waitWithAuth = { NodeInfo node, AntBuilder ant -> - File tmpFile = new File(node.cwd, 'wait.success') - - // wait up to twenty seconds - final long stopTime = System.currentTimeMillis() + 20000L; - Exception lastException = null; - - while (System.currentTimeMillis() < stopTime) { - lastException = null; - // we use custom wait logic here as the elastic user is not available immediately and ant.get will fail when a 401 is returned - HttpURLConnection httpURLConnection = null; - try { - httpURLConnection = (HttpURLConnection) new URL("http://${node.httpUri()}/_cluster/health?wait_for_nodes=${node.config.numNodes}&wait_for_status=yellow").openConnection(); - httpURLConnection.setRequestProperty("Authorization", "Basic " + - Base64.getEncoder().encodeToString("test_user:x-pack-test-password".getBytes(StandardCharsets.UTF_8))); - httpURLConnection.setRequestMethod("GET"); - httpURLConnection.setConnectTimeout(1000); - httpURLConnection.setReadTimeout(30000); // read needs to wait for nodes! - httpURLConnection.connect(); - if (httpURLConnection.getResponseCode() == 200) { - tmpFile.withWriter StandardCharsets.UTF_8.name(), { - it.write(httpURLConnection.getInputStream().getText(StandardCharsets.UTF_8.name())) - } - break; - } - } catch (Exception e) { - logger.debug("failed to call cluster health", e) - lastException = e - } finally { - if (httpURLConnection != null) { - httpURLConnection.disconnect(); - } - } - - // did not start, so wait a bit before trying again - Thread.sleep(500L); - } - if (tmpFile.exists() == false && lastException != null) { - logger.error("final attempt of calling cluster health failed", lastException) - } - return tmpFile.exists() -} - licenseHeaders { approvedLicenses << 'Apache' } @@ -71,124 +24,114 @@ forbiddenPatterns { String outputDir = "${buildDir}/generated-resources/${project.name}" -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } -task copyTestNodeKeyMaterial(type: Copy) { - from project(':x-pack:plugin:core').files('src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem', - 'src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt') +tasks.register("copyTestNodeKeyMaterial", Copy) { + from project(':x-pack:plugin:core') + .files( + 'src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem', + 'src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt' + ) into outputDir } -for (Version version : bwcVersions.indexCompatible) { - String baseName = "v${version}" +configurations { + restSpec +} - Task oldClusterTest = tasks.create(name: "${baseName}#oldClusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) +dependencies { + restSpec project(':rest-api-spec') +} + +processTestResources { + dependsOn configurations.restSpec + from ({ zipTree(configurations.restSpec.singleFile) }) { + include 'rest-api-spec/api/**' } - - Object extension = extensions.findByName("${baseName}#oldClusterTestCluster") - configure(extensions.findByName("${baseName}#oldClusterTestCluster")) { - dependsOn copyTestNodeKeyMaterial - if (version.before('6.3.0')) { - String depVersion = version; - if (project.bwcVersions.unreleased.contains(version)) { - depVersion += "-SNAPSHOT" - } - mavenPlugin 'x-pack', "org.elasticsearch.plugin:x-pack:${depVersion}" - - } - bwcVersion = version - numBwcNodes = 2 - numNodes = 2 - clusterName = 'full-cluster-restart' - String usersCli = version.before('6.3.0') ? 'bin/x-pack/users' : 'bin/elasticsearch-users' - setupCommand 'setupTestUser', usersCli, 'useradd', 'test_user', '-p', 'x-pack-test-password', '-r', 'superuser' - waitCondition = waitWithAuth - - // some tests rely on the translog not being flushed - setting 'indices.memory.shard_inactive_time', '20m' - - setting 'xpack.security.enabled', 'true' - setting 'xpack.security.transport.ssl.enabled', 'true' - - setting 'xpack.security.transport.ssl.key', 'testnode.pem' - setting 'xpack.security.transport.ssl.certificate', 'testnode.crt' - keystoreSetting 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' - - setting 'xpack.license.self_generated.type', 'trial' - dependsOn copyTestNodeKeyMaterial - extraConfigFile 'testnode.pem', new File(outputDir + '/testnode.pem') - extraConfigFile 'testnode.crt', new File(outputDir + '/testnode.crt') - - keystoreFile 'xpack.watcher.encryption_key', "${project.projectDir}/src/test/resources/system_key" - setting 'xpack.watcher.encrypt_sensitive_data', 'true' - } - - Task oldClusterTestRunner = tasks.getByName("${baseName}#oldClusterTestRunner") - oldClusterTestRunner.configure { - systemProperty 'tests.is_old_cluster', 'true' - systemProperty 'tests.old_cluster_version', version.toString().minus("-SNAPSHOT") - systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") - exclude 'org/elasticsearch/upgrades/FullClusterRestartIT.class' - exclude 'org/elasticsearch/upgrades/FullClusterRestartSettingsUpgradeIT.class' - exclude 'org/elasticsearch/upgrades/QueryBuilderBWCIT.class' - } - - Task upgradedClusterTest = tasks.create(name: "${baseName}#upgradedClusterTest", type: RestIntegTestTask) - - configure(extensions.findByName("${baseName}#upgradedClusterTestCluster")) { - dependsOn oldClusterTestRunner, - "${baseName}#oldClusterTestCluster#node0.stop", - "${baseName}#oldClusterTestCluster#node1.stop" - numNodes = 2 - clusterName = 'full-cluster-restart' - dataDir = { nodeNum -> oldClusterTest.nodes[nodeNum].dataDir } - cleanShared = false // We want to keep snapshots made by the old cluster! - setupCommand 'setupTestUser', 'bin/elasticsearch-users', 'useradd', 'test_user', '-p', 'x-pack-test-password', '-r', 'superuser' - waitCondition = waitWithAuth - - // some tests rely on the translog not being flushed - setting 'indices.memory.shard_inactive_time', '20m' - setting 'xpack.security.enabled', 'true' - - setting 'xpack.security.transport.ssl.key', 'testnode.pem' - setting 'xpack.security.transport.ssl.certificate', 'testnode.crt' - keystoreSetting 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' - - setting 'xpack.license.self_generated.type', 'trial' - dependsOn copyTestNodeKeyMaterial - extraConfigFile 'testnode.pem', new File(outputDir + '/testnode.pem') - extraConfigFile 'testnode.crt', new File(outputDir + '/testnode.crt') - - setting 'xpack.watcher.encrypt_sensitive_data', 'true' - keystoreFile 'xpack.watcher.encryption_key', "${project.projectDir}/src/test/resources/system_key" - } - - Task upgradedClusterTestRunner = tasks.getByName("${baseName}#upgradedClusterTestRunner") - upgradedClusterTestRunner.configure { - systemProperty 'tests.is_old_cluster', 'false' - systemProperty 'tests.old_cluster_version', version.toString().minus("-SNAPSHOT") - systemProperty 'tests.path.repo', new File(buildDir, "cluster/shared/repo") - exclude 'org/elasticsearch/upgrades/FullClusterRestartIT.class' - exclude 'org/elasticsearch/upgrades/FullClusterRestartSettingsUpgradeIT.class' - exclude 'org/elasticsearch/upgrades/QueryBuilderBWCIT.class' - } - - Task versionBwcTest = tasks.create(name: "${baseName}#bwcTest") { - dependsOn = [upgradedClusterTest] - } - - if (project.bwc_tests_enabled) { - bwcTest.dependsOn(versionBwcTest) + from (project(xpackModule('core')).sourceSets.test.resources) { + include 'rest-api-spec/api/**' + } +} + +for (Version bwcVersion : bwcVersions.indexCompatible) { + String baseName = "v${bwcVersion}" + + testClusters { + "${baseName}" { + testDistribution = "DEFAULT" + versions = [bwcVersion.toString(), project.version] + numberOfNodes = 2 + setting 'path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + javaHome = project.file(project.ext.runtimeJavaHome) + user username: "test_user", password: "x-pack-test-password" + + setting 'path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + // some tests rely on the translog not being flushed + setting 'indices.memory.shard_inactive_time', '20m' + setting 'xpack.security.enabled', 'true' + setting 'xpack.security.transport.ssl.enabled', 'true' + setting 'xpack.license.self_generated.type', 'trial' + + extraConfigFile 'testnode.pem', file("${outputDir}/testnode.pem") + extraConfigFile 'testnode.crt', file("${outputDir}/testnode.crt") + + keystore 'xpack.watcher.encryption_key', file("${project.projectDir}/src/test/resources/system_key") + setting 'xpack.watcher.encrypt_sensitive_data', 'true' + + setting 'xpack.security.transport.ssl.key', 'testnode.pem' + setting 'xpack.security.transport.ssl.certificate', 'testnode.crt' + keystore 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' + } + } + + tasks.register("${baseName}#oldClusterTest", RestTestRunnerTask) { + mustRunAfter(precommit) + useCluster testClusters."${baseName}" + dependsOn copyTestNodeKeyMaterial + doFirst { + project.delete("${buildDir}/cluster/shared/repo/${baseName}") + } + systemProperty 'tests.is_old_cluster', 'true' + systemProperty 'tests.old_cluster_version', version.toString().minus("-SNAPSHOT") + exclude 'org/elasticsearch/upgrades/FullClusterRestartIT.class' + exclude 'org/elasticsearch/upgrades/FullClusterRestartSettingsUpgradeIT.class' + exclude 'org/elasticsearch/upgrades/QueryBuilderBWCIT.class' + } + + + tasks.register("${baseName}#upgradedClusterTest", RestTestRunnerTask) { + mustRunAfter(precommit) + useCluster testClusters."${baseName}" + dependsOn "${baseName}#oldClusterTest" + doFirst { + testClusters."${baseName}".goToNextVersion() + } + systemProperty 'tests.is_old_cluster', 'false' + systemProperty 'tests.old_cluster_version', version.toString().minus("-SNAPSHOT") + exclude 'org/elasticsearch/upgrades/FullClusterRestartIT.class' + exclude 'org/elasticsearch/upgrades/FullClusterRestartSettingsUpgradeIT.class' + exclude 'org/elasticsearch/upgrades/QueryBuilderBWCIT.class' + } + + tasks.matching { it.name.startsWith(baseName) && it.name.endsWith("ClusterTest") }.configureEach { + it.systemProperty 'tests.old_cluster_version', bwcVersion.toString().minus("-SNAPSHOT") + it.systemProperty 'tests.path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + it.nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + it.nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") + } + + tasks.register("${baseName}#bwcTest") { + dependsOn "${baseName}#upgradedClusterTest" + } + + if (project.bwc_tests_enabled) { + bwcTest.dependsOn("${baseName}#bwcTest") } } -// basic integ tests includes testing bwc against the most recent version task bwcTestSnapshots { if (project.bwc_tests_enabled) { for (final def version : bwcVersions.unreleasedIndexCompatible) { @@ -199,11 +142,4 @@ task bwcTestSnapshots { check.dependsOn(bwcTestSnapshots) -// copy x-pack plugin info so it is on the classpath and security manager has the right permissions -task copyXPackRestSpec(type: Copy) { - dependsOn(project.configurations.restSpec, 'processTestResources') - from project(xpackModule('core')).sourceSets.test.resources - include 'rest-api-spec/api/**' - into project.sourceSets.test.output.resourcesDir -} -project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackRestSpec) +test.enabled = false diff --git a/x-pack/qa/rolling-upgrade-basic/build.gradle b/x-pack/qa/rolling-upgrade-basic/build.gradle index 64a2eda823e0..4b376e748134 100644 --- a/x-pack/qa/rolling-upgrade-basic/build.gradle +++ b/x-pack/qa/rolling-upgrade-basic/build.gradle @@ -1,126 +1,113 @@ import org.elasticsearch.gradle.Version -import org.elasticsearch.gradle.test.RestIntegTestTask +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' dependencies { testCompile project(':x-pack:qa') } -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } -String outputDir = "${buildDir}/generated-resources/${project.name}" +configurations { + restSpec +} -for (Version version : bwcVersions.wireCompatible) { - String baseName = "v${version}" +dependencies { + restSpec project(':rest-api-spec') +} - Task oldClusterTest = tasks.create(name: "${baseName}#oldClusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) +processTestResources { + dependsOn configurations.restSpec + from ({ zipTree(configurations.restSpec.singleFile) }) { + include 'rest-api-spec/api/**' } - - configure(extensions.findByName("${baseName}#oldClusterTestCluster")) { - if (version.before('6.3.0')) { - String depVersion = version; - if (project.bwcVersions.unreleased.contains(version)) { - depVersion += "-SNAPSHOT" - } - mavenPlugin 'x-pack', "org.elasticsearch.plugin:x-pack:${depVersion}" - - } - bwcVersion = version - numBwcNodes = 3 - numNodes = 3 - clusterName = 'rolling-upgrade-basic' - setting 'xpack.security.enabled', 'false' - setting 'xpack.monitoring.enabled', 'false' - setting 'xpack.ml.enabled', 'false' - setting 'xpack.watcher.enabled', 'false' - setting 'xpack.license.self_generated.type', 'basic' + from (project(xpackProject('plugin').path).sourceSets.test.resources) { + include 'rest-api-spec/api/**' } +} - Task oldClusterTestRunner = tasks.getByName("${baseName}#oldClusterTestRunner") - oldClusterTestRunner.configure { - systemProperty 'tests.rest.suite', 'old_cluster' - } - Closure configureUpgradeCluster = {String name, Task lastRunner, int stopNode, Closure getOtherUnicastHostAddresses -> - configure(extensions.findByName("${baseName}#${name}")) { - dependsOn lastRunner, "${baseName}#oldClusterTestCluster#node${stopNode}.stop" - clusterName = 'rolling-upgrade-basic' - otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - /* Override the data directory so the new node always gets the node we - * just stopped's data directory. */ - dataDir = { nodeNumber -> oldClusterTest.nodes[stopNode].dataDir } +for (Version bwcVersion : bwcVersions.wireCompatible) { + String baseName = "v${bwcVersion}" + + testClusters { + "${baseName}" { + testDistribution = "DEFAULT" + versions = [bwcVersion.toString(), project.version] + numberOfNodes = 3 + setting 'repositories.url.allowed_urls', 'http://snapshot.test*' setting 'xpack.security.enabled', 'false' setting 'xpack.monitoring.enabled', 'false' setting 'xpack.ml.enabled', 'false' setting 'xpack.watcher.enabled', 'false' setting 'xpack.license.self_generated.type', 'basic' - setting 'node.name', "upgraded-node-${stopNode}" + javaHome = project.file(project.ext.runtimeJavaHome) } } - Task oneThirdUpgradedTest = tasks.create(name: "${baseName}#oneThirdUpgradedTest", type: RestIntegTestTask) + tasks.register("${baseName}#oldClusterTest", RestTestRunnerTask) { + useCluster testClusters."${baseName}" + mustRunAfter(precommit) + systemProperty 'tests.rest.suite', 'old_cluster' + systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") + } - configureUpgradeCluster("oneThirdUpgradedTestCluster", oldClusterTestRunner, 0, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oldClusterTest.nodes.get(1).transportUri(), oldClusterTest.nodes.get(2).transportUri()] }) - Task oneThirdUpgradedTestRunner = tasks.getByName("${baseName}#oneThirdUpgradedTestRunner") - oneThirdUpgradedTestRunner.configure { + tasks.register("${baseName}#oneThirdUpgradedTest", RestTestRunnerTask) { + dependsOn "${baseName}#oldClusterTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",")}") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName()}") systemProperty 'tests.rest.suite', 'mixed_cluster' systemProperty 'tests.first_round', 'true' - finalizedBy "${baseName}#oldClusterTestCluster#node1.stop" + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') } - Task twoThirdsUpgradedTest = tasks.create(name: "${baseName}#twoThirdsUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster("twoThirdsUpgradedTestCluster", oneThirdUpgradedTestRunner, 1, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oldClusterTest.nodes.get(2).transportUri(), oneThirdUpgradedTest.nodes.get(0).transportUri()] }) - - Task twoThirdsUpgradedTestRunner = tasks.getByName("${baseName}#twoThirdsUpgradedTestRunner") - twoThirdsUpgradedTestRunner.configure { + tasks.register("${baseName}#twoThirdsUpgradedTest", RestTestRunnerTask) { + dependsOn "${baseName}#oneThirdUpgradedTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") systemProperty 'tests.rest.suite', 'mixed_cluster' systemProperty 'tests.first_round', 'false' - finalizedBy "${baseName}#oldClusterTestCluster#node2.stop" + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') } - Task upgradedClusterTest = tasks.create(name: "${baseName}#upgradedClusterTest", type: RestIntegTestTask) - - configureUpgradeCluster("upgradedClusterTestCluster", twoThirdsUpgradedTestRunner, 2, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oneThirdUpgradedTest.nodes.get(0).transportUri(), twoThirdsUpgradedTest.nodes.get(0).transportUri()] }) - - Task upgradedClusterTestRunner = tasks.getByName("${baseName}#upgradedClusterTestRunner") - upgradedClusterTestRunner.configure { + tasks.register("${baseName}#upgradedClusterTest", RestTestRunnerTask) { + dependsOn "${baseName}#twoThirdsUpgradedTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") systemProperty 'tests.rest.suite', 'upgraded_cluster' - /* - * Force stopping all the upgraded nodes after the test runner - * so they are alive during the test. - */ - finalizedBy "${baseName}#oneThirdUpgradedTestCluster#stop" - finalizedBy "${baseName}#twoThirdsUpgradedTestCluster#stop" + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') } - Task versionBwcTest = tasks.create(name: "${baseName}#bwcTest") { - dependsOn = [upgradedClusterTest] + tasks.register("${baseName}#bwcTest") { + dependsOn "${baseName}#upgradedClusterTest" } if (project.bwc_tests_enabled) { - bwcTest.dependsOn(versionBwcTest) + bwcTest.dependsOn("${baseName}#bwcTest") } } -test.enabled = false // no unit tests for rolling upgrades, only the rest integration test - -// basic integ tests includes testing bwc against the most recent version task bwcTestSnapshots { if (project.bwc_tests_enabled) { for (final def version : bwcVersions.unreleasedWireCompatible) { @@ -132,22 +119,4 @@ check.dependsOn(bwcTestSnapshots) compileTestJava.options.compilerArgs << "-Xlint:-cast,-rawtypes,-unchecked" -// copy x-pack plugin info so it is on the classpath and security manager has the right permissions -task copyXPackRestSpec(type: Copy) { - dependsOn(project.configurations.restSpec, 'processTestResources') - from project(xpackProject('plugin').path).sourceSets.test.resources - include 'rest-api-spec/api/**' - into project.sourceSets.test.output.resourcesDir -} -project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackRestSpec) - -repositories { - maven { - name "elastic" - url "https://artifacts.elastic.co/maven" - } - maven { - name "elastic-snapshots" - url "https://snapshots.elastic.co/maven" - } -} +test.enabled = false \ No newline at end of file diff --git a/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle b/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle index ade76e361e6c..4467cf5d0a5e 100644 --- a/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle +++ b/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle @@ -1,266 +1,135 @@ import org.elasticsearch.gradle.Version -import org.elasticsearch.gradle.test.RestIntegTestTask +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' dependencies { testCompile project(':x-pack:qa') } -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } -for (Version version : bwcVersions.wireCompatible) { - String taskPrefix = "v${version}" +configurations { + restSpec +} - // ============================================================================================ - // Create leader cluster - // ============================================================================================ +dependencies { + restSpec project(':rest-api-spec') +} - RestIntegTestTask leaderClusterTest = tasks.create(name: "${taskPrefix}#leader#clusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) +processTestResources { + dependsOn configurations.restSpec + from ({ zipTree(configurations.restSpec.singleFile) }) { + include 'rest-api-spec/api/**' } - - configure(extensions.findByName("${taskPrefix}#leader#clusterTestCluster")) { - bwcVersion = version - numBwcNodes = 3 - numNodes = 3 - clusterName = 'leader' - setting 'xpack.security.enabled', 'false' - setting 'xpack.monitoring.enabled', 'false' - setting 'xpack.ml.enabled', 'false' - setting 'xpack.watcher.enabled', 'false' - setting 'xpack.license.self_generated.type', 'trial' - } - - Task leaderClusterTestRunner = tasks.getByName("${taskPrefix}#leader#clusterTestRunner") - leaderClusterTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'none' - systemProperty 'tests.rest.cluster_name', 'leader' - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderClusterTest.nodes.get(0).transportUri()}" - } - - // ============================================================================================ - // Create follower cluster - // ============================================================================================ - - RestIntegTestTask followerClusterTest = tasks.create(name: "${taskPrefix}#follower#clusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) - } - - configure(extensions.findByName("${taskPrefix}#follower#clusterTestCluster")) { - dependsOn leaderClusterTestRunner - bwcVersion = version - numBwcNodes = 3 - numNodes = 3 - clusterName = 'follower' - setting 'xpack.security.enabled', 'false' - setting 'xpack.monitoring.enabled', 'false' - setting 'xpack.ml.enabled', 'false' - setting 'xpack.watcher.enabled', 'false' - setting 'xpack.license.self_generated.type', 'trial' - } - - Task followerClusterTestRunner = tasks.getByName("${taskPrefix}#follower#clusterTestRunner") - followerClusterTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'none' - systemProperty 'tests.rest.cluster_name', 'follower' - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderClusterTest.nodes.get(0).transportUri()}" - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerClusterTest.nodes.get(0).transportUri()}" - } - - // ============================================================================================ - // Upgrade follower cluster - // ============================================================================================ - - Closure configureUpgradeCluster = {String prefix, String cluster, String name, Task lastRunner, int stopNode, - RestIntegTestTask clusterTest, Closure getOtherUnicastHostAddresses -> - configure(extensions.findByName("${prefix}#${cluster}#${name}")) { - dependsOn lastRunner, "${prefix}#${cluster}#clusterTestCluster#node${stopNode}.stop" - clusterName = cluster - otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - /* Override the data directory so the new node always gets the node we - * just stopped's data directory. */ - dataDir = { nodeNumber -> clusterTest.nodes[stopNode].dataDir } - setting 'repositories.url.allowed_urls', 'http://snapshot.test*' - setting 'xpack.security.enabled', 'false' - setting 'xpack.monitoring.enabled', 'false' - setting 'xpack.ml.enabled', 'false' - setting 'xpack.watcher.enabled', 'false' - setting 'xpack.license.self_generated.type', 'trial' - setting 'node.name', "upgraded-node-${cluster}-${stopNode}" - setting 'node.attr.upgraded', 'true' - } - } - - Task followerOneThirdUpgradedTest = tasks.create(name: "${taskPrefix}#follower#oneThirdUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster(taskPrefix, 'follower', 'oneThirdUpgradedTestCluster', followerClusterTestRunner, 0, followerClusterTest, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [followerClusterTest.nodes.get(1).transportUri(), followerClusterTest.nodes.get(2).transportUri()] }) - - Task followerOneThirdUpgradedTestRunner = tasks.getByName("${taskPrefix}#follower#oneThirdUpgradedTestRunner") - followerOneThirdUpgradedTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'one_third' - systemProperty 'tests.rest.cluster_name', 'follower' - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerClusterTest.nodes.get(1).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerClusterTest.nodes.get(1).transportUri()}" - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderClusterTest.nodes.get(0).transportUri()}" - - finalizedBy "${taskPrefix}#follower#clusterTestCluster#node1.stop" - } - - Task followerTwoThirdsUpgradedTest = tasks.create(name: "${taskPrefix}#follower#twoThirdsUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster(taskPrefix, 'follower', 'twoThirdsUpgradedTestCluster', followerOneThirdUpgradedTestRunner, 1, followerClusterTest, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [followerClusterTest.nodes.get(2).transportUri(), followerOneThirdUpgradedTest.nodes.get(0).transportUri()] }) - - Task followerTwoThirdsUpgradedTestRunner = tasks.getByName("${taskPrefix}#follower#twoThirdsUpgradedTestRunner") - followerTwoThirdsUpgradedTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'two_third' - systemProperty 'tests.rest.cluster_name', 'follower' - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerClusterTest.nodes.get(2).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerClusterTest.nodes.get(2).transportUri()}" - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderClusterTest.nodes.get(0).transportUri()}" - - finalizedBy "${taskPrefix}#follower#clusterTestCluster#node2.stop" - } - - Task followerUpgradedClusterTest = tasks.create(name: "${taskPrefix}#follower#upgradedClusterTest", type: RestIntegTestTask) - - configureUpgradeCluster(taskPrefix, 'follower', 'upgradedClusterTestCluster', followerTwoThirdsUpgradedTestRunner, 2, followerClusterTest, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [followerOneThirdUpgradedTest.nodes.get(0).transportUri(), followerTwoThirdsUpgradedTest.nodes.get(0).transportUri()] }) - - Task followerUpgradedClusterTestRunner = tasks.getByName("${taskPrefix}#follower#upgradedClusterTestRunner") - followerUpgradedClusterTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'all' - systemProperty 'tests.rest.cluster_name', 'follower' - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerOneThirdUpgradedTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerOneThirdUpgradedTest.nodes.get(0).transportUri()}" - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderClusterTest.nodes.get(0).transportUri()}" - - // This is needed, otherwise leader node 0 will stop after the leaderClusterTestRunner task has run. - // Here it is ok to stop, because in the next task, the leader node 0 gets upgraded. - finalizedBy "v${version}#leader#clusterTestCluster#node0.stop" - } - - // ============================================================================================ - // Upgrade leader cluster - // ============================================================================================ - - Task leaderOneThirdUpgradedTest = tasks.create(name: "${taskPrefix}#leader#oneThirdUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster(taskPrefix, 'leader', 'oneThirdUpgradedTestCluster', followerUpgradedClusterTestRunner, 0, leaderClusterTest, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [leaderClusterTest.nodes.get(1).transportUri(), leaderClusterTest.nodes.get(2).transportUri()] }) - - Task leaderOneThirdUpgradedTestRunner = tasks.getByName("${taskPrefix}#leader#oneThirdUpgradedTestRunner") - leaderOneThirdUpgradedTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'one_third' - systemProperty 'tests.rest.cluster_name', 'leader' - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerUpgradedClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerUpgradedClusterTest.nodes.get(0).transportUri()}" - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderClusterTest.nodes.get(2).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderClusterTest.nodes.get(2).transportUri()}" - - finalizedBy "${taskPrefix}#leader#clusterTestCluster#node1.stop" - } - - Task leaderTwoThirdsUpgradedTest = tasks.create(name: "${taskPrefix}#leader#twoThirdsUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster(taskPrefix, 'leader', 'twoThirdsUpgradedTestCluster', leaderOneThirdUpgradedTestRunner, 1, leaderClusterTest, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [leaderClusterTest.nodes.get(2).transportUri(), leaderOneThirdUpgradedTest.nodes.get(0).transportUri()] }) - - Task leaderTwoThirdsUpgradedTestRunner = tasks.getByName("${taskPrefix}#leader#twoThirdsUpgradedTestRunner") - leaderTwoThirdsUpgradedTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'two_third' - systemProperty 'tests.rest.cluster_name', 'leader' - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerUpgradedClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerUpgradedClusterTest.nodes.get(0).transportUri()}" - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderOneThirdUpgradedTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderOneThirdUpgradedTest.nodes.get(0).transportUri()}" - - finalizedBy "${taskPrefix}#leader#clusterTestCluster#node2.stop" - } - - Task leaderUpgradedClusterTest = tasks.create(name: "${taskPrefix}#leader#upgradedClusterTest", type: RestIntegTestTask) - - configureUpgradeCluster(taskPrefix, 'leader', "upgradedClusterTestCluster", leaderTwoThirdsUpgradedTestRunner, 2, leaderClusterTest, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [leaderOneThirdUpgradedTest.nodes.get(0).transportUri(), leaderTwoThirdsUpgradedTest.nodes.get(0).transportUri()] }) - - Task leaderUpgradedClusterTestRunner = tasks.getByName("${taskPrefix}#leader#upgradedClusterTestRunner") - leaderUpgradedClusterTestRunner.configure { - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - systemProperty 'tests.rest.upgrade_state', 'all' - systemProperty 'tests.rest.cluster_name', 'leader' - - nonInputProperties.systemProperty 'tests.follower_host', "${-> followerUpgradedClusterTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.follower_remote_cluster_seed', "${-> followerUpgradedClusterTest.nodes.get(0).transportUri()}" - - nonInputProperties.systemProperty 'tests.leader_host', "${-> leaderTwoThirdsUpgradedTest.nodes.get(0).httpUri()}" - nonInputProperties.systemProperty 'tests.leader_remote_cluster_seed', "${-> leaderTwoThirdsUpgradedTest.nodes.get(0).transportUri()}" - - /* - * Force stopping all the upgraded nodes after the test runner - * so they are alive during the test. - */ - finalizedBy "${taskPrefix}#follower#oneThirdUpgradedTestCluster#stop" - finalizedBy "${taskPrefix}#follower#twoThirdsUpgradedTestCluster#stop" - finalizedBy "${taskPrefix}#follower#upgradedClusterTestCluster#stop" - finalizedBy "${taskPrefix}#leader#oneThirdUpgradedTestCluster#stop" - finalizedBy "${taskPrefix}#leader#twoThirdsUpgradedTestCluster#stop" - } - - if (project.bwc_tests_enabled) { - Task versionBwcTest = tasks.create(name: "${taskPrefix}#bwcTest") { - dependsOn = [leaderUpgradedClusterTest] - } - bwcTest.dependsOn(versionBwcTest) + from (project(xpackProject('plugin').path).sourceSets.test.resources) { + include 'rest-api-spec/api/**' } } -test.enabled = false // no unit tests for rolling upgrades, only the rest integration test +for (Version bwcVersion : bwcVersions.wireCompatible) { + String baseName = "v${bwcVersion}" -// basic integ tests includes testing bwc against the most recent version -task integTest { + testClusters { + "${baseName}-leader" { + numberOfNodes = 3 + } + "${baseName}-follower" { + numberOfNodes = 3 + } + } + testClusters.matching{ it.name.startsWith(baseName)}.all { + testDistribution = "DEFAULT" + versions = [bwcVersion.toString(), project.version] + + setting 'repositories.url.allowed_urls', 'http://snapshot.test*' + setting 'xpack.security.enabled', 'false' + setting 'xpack.monitoring.enabled', 'false' + setting 'xpack.ml.enabled', 'false' + setting 'xpack.watcher.enabled', 'false' + setting 'xpack.license.self_generated.type', 'trial' + + javaHome = project.file(project.ext.runtimeJavaHome) + } + + tasks.withType(RestTestRunnerTask).matching{it.name.startsWith(baseName)}.configureEach { + useCluster testClusters."${baseName}-leader" + useCluster testClusters."${baseName}-follower" + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') + + doFirst { + if (name.endsWith("#clusterTest") == false ) { + println "Upgrade node $it" + testClusters."${baseName}-${kindExt}".nextNodeToNextVersion() + } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}-${kindExt}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}-${kindExt}".getName() }") + nonInputProperties.systemProperty('tests.leader_host', "${-> testClusters."${baseName}-leader".allHttpSocketURI.last()}") + nonInputProperties.systemProperty('tests.leader_remote_cluster_seed', "${-> testClusters."${baseName}-leader".allTransportPortURI.last()}") + nonInputProperties.systemProperty('tests.follower_host', "${-> testClusters."${baseName}-follower".allHttpSocketURI.last()}") + nonInputProperties.systemProperty('tests.follower_remote_cluster_seed', "${-> testClusters."${baseName}-follower".allTransportPortURI.last()}") + } + } + + for (kind in ["follower", "leader"]) { + // Attention!! Groovy trap: do not pass `kind` to a closure + + tasks.register("${baseName}#${kind}#clusterTest", RestTestRunnerTask) { + systemProperty 'tests.rest.upgrade_state', 'none' + systemProperty 'tests.rest.cluster_name', kind + ext.kindExt = kind + } + + tasks.register("${baseName}#${kind}#oneThirdUpgradedTest", RestTestRunnerTask) { + systemProperty 'tests.rest.upgrade_state', 'one_third' + systemProperty 'tests.rest.cluster_name', kind + dependsOn "${baseName}#leader#clusterTest", "${baseName}#follower#clusterTest" + ext.kindExt = kind + } + + tasks.register("${baseName}#${kind}#twoThirdsUpgradedTest", RestTestRunnerTask) { + systemProperty 'tests.rest.upgrade_state', 'two_third' + systemProperty 'tests.rest.cluster_name', kind + dependsOn "${baseName}#${kind}#oneThirdUpgradedTest" + ext.kindExt = kind + } + + tasks.register("${baseName}#${kind}#upgradedClusterTest", RestTestRunnerTask) { + systemProperty 'tests.rest.upgrade_state', 'all' + systemProperty 'tests.rest.cluster_name', kind + dependsOn "${baseName}#${kind}#twoThirdsUpgradedTest" + ext.kindExt = kind + } + } + + tasks.named("${baseName}#follower#clusterTest") { + dependsOn "${baseName}#leader#clusterTest" + } + + tasks.named("${baseName}#leader#oneThirdUpgradedTest") { + dependsOn "${baseName}#follower#upgradedClusterTest" + } + + tasks.register("${baseName}#bwcTest") { + dependsOn "${baseName}#leader#upgradedClusterTest" + } + + if (project.bwc_tests_enabled) { + bwcTest.dependsOn("${baseName}#bwcTest") + } +} + +task bwcTestSnapshots { if (project.bwc_tests_enabled) { for (final def version : bwcVersions.unreleasedWireCompatible) { dependsOn "v${version}#bwcTest" } } } -check.dependsOn(integTest) \ No newline at end of file +check.dependsOn(bwcTestSnapshots) + +test.enabled = false \ No newline at end of file diff --git a/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/AbstractMultiClusterUpgradeTestCase.java b/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/AbstractMultiClusterUpgradeTestCase.java index 3221e899eac3..419fcd529dda 100644 --- a/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/AbstractMultiClusterUpgradeTestCase.java +++ b/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/AbstractMultiClusterUpgradeTestCase.java @@ -106,6 +106,7 @@ public abstract class AbstractMultiClusterUpgradeTestCase extends ESRestTestCase } else { throw new AssertionError("unknown cluster name: " + clusterName); } + logger.info("Leader host: {}, follower host: {}", leaderHost, followerHost); configureLeaderRemoteClusters(); configureFollowerRemoteClusters(); diff --git a/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java b/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java index 2850aa80cd56..e9e4b671f564 100644 --- a/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java @@ -133,6 +133,7 @@ public class CcrRollingUpgradeIT extends AbstractMultiClusterUpgradeTestCase { createLeaderIndex(leaderClient(), leaderIndex1); index(leaderClient(), leaderIndex1, 64); assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex1; assertThat(getNumberOfSuccessfulFollowedIndices(), equalTo(1)); assertTotalHitCount(followerIndex, 64, followerClient()); @@ -320,7 +321,8 @@ public class CcrRollingUpgradeIT extends AbstractMultiClusterUpgradeTestCase { private static void putAutoFollowPattern(RestClient client, String name, String remoteCluster, String pattern) throws IOException { Request request = new Request("PUT", "/_ccr/auto_follow/" + name); - request.setJsonEntity("{\"leader_index_patterns\": [\"" + pattern + "\"], \"remote_cluster\": \"" + remoteCluster + "\"," + + request.setJsonEntity("{\"leader_index_patterns\": [\"" + pattern + "\"], \"remote_cluster\": \"" + + remoteCluster + "\"," + "\"follow_index_pattern\": \"copy-{{leader_index}}\", \"read_poll_timeout\": \"10ms\"}"); assertOK(client.performRequest(request)); } @@ -332,7 +334,7 @@ public class CcrRollingUpgradeIT extends AbstractMultiClusterUpgradeTestCase { private int getNumberOfSuccessfulFollowedIndices() throws IOException { Request statsRequest = new Request("GET", "/_ccr/stats"); - Map response = toMap(client().performRequest(statsRequest)); + Map response = toMap(followerClient().performRequest(statsRequest)); Integer actualSuccessfulFollowedIndices = ObjectPath.eval("auto_follow_stats.number_of_successful_follow_indices", response); if (actualSuccessfulFollowedIndices != null) { return actualSuccessfulFollowedIndices; diff --git a/x-pack/qa/rolling-upgrade/build.gradle b/x-pack/qa/rolling-upgrade/build.gradle index 329837320994..d1066a7957ac 100644 --- a/x-pack/qa/rolling-upgrade/build.gradle +++ b/x-pack/qa/rolling-upgrade/build.gradle @@ -1,71 +1,14 @@ -import org.elasticsearch.gradle.test.NodeInfo -import org.elasticsearch.gradle.test.RestIntegTestTask import org.elasticsearch.gradle.Version +import org.elasticsearch.gradle.testclusters.RestTestRunnerTask -import java.nio.charset.StandardCharsets - -// Apply the java plugin to this project so the sources can be edited in an IDE +apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.standalone-test' -test.enabled = false - dependencies { testCompile project(':x-pack:qa') testCompile project(':client:rest-high-level') } -Closure waitWithAuth = { NodeInfo node, AntBuilder ant -> - File tmpFile = new File(node.cwd, 'wait.success') - - // wait up to two minutes - final long stopTime = System.currentTimeMillis() + (2 * 60000L); - Exception lastException = null; - int lastResponseCode = 0 - - while (System.currentTimeMillis() < stopTime) { - - lastException = null; - // we use custom wait logic here as the elastic user is not available immediately and ant.get will fail when a 401 is returned - HttpURLConnection httpURLConnection = null; - try { - // TODO this sucks having to hardcode number of nodes, but node.config.numNodes isn't necessarily accurate for rolling - httpURLConnection = (HttpURLConnection) new URL("http://${node.httpUri()}/_cluster/health?wait_for_nodes=3&wait_for_status=yellow").openConnection(); - httpURLConnection.setRequestProperty("Authorization", "Basic " + - Base64.getEncoder().encodeToString("test_user:x-pack-test-password".getBytes(StandardCharsets.UTF_8))); - httpURLConnection.setRequestMethod("GET"); - httpURLConnection.setConnectTimeout(1000); - httpURLConnection.setReadTimeout(30000); // read needs to wait for nodes! - httpURLConnection.connect(); - lastResponseCode = httpURLConnection.getResponseCode() - if (lastResponseCode == 200) { - tmpFile.withWriter StandardCharsets.UTF_8.name(), { - it.write(httpURLConnection.getInputStream().getText(StandardCharsets.UTF_8.name())) - } - break; - } - } catch (Exception e) { - logger.debug("failed to call cluster health", e) - lastException = e - } finally { - if (httpURLConnection != null) { - httpURLConnection.disconnect(); - } - } - - // did not start, so wait a bit before trying again - Thread.sleep(500L); - } - if (tmpFile.exists() == false) { - final String message = "final attempt of calling cluster health failed [lastResponseCode=${lastResponseCode}]" - if (lastException != null) { - logger.error(message, lastException) - } else { - logger.error(message + " [no exception]") - } - } - return tmpFile.exists() -} - compileTestJava.options.compilerArgs << "-Xlint:-cast,-rawtypes,-unchecked" forbiddenPatterns { @@ -74,104 +17,47 @@ forbiddenPatterns { String outputDir = "${buildDir}/generated-resources/${project.name}" -// This is a top level task which we will add dependencies to below. -// It is a single task that can be used to backcompat tests against all versions. -task bwcTest { +tasks.register("bwcTest") { description = 'Runs backwards compatibility tests.' group = 'verification' } +configurations { + restSpec +} + +dependencies { + restSpec project(':rest-api-spec') +} + +processTestResources { + dependsOn configurations.restSpec + from ({ zipTree(configurations.restSpec.singleFile) }) { + include 'rest-api-spec/api/**' + } + from (project(xpackProject('plugin').path).sourceSets.test.resources) { + include 'rest-api-spec/api/**' + } +} + task copyTestNodeKeyMaterial(type: Copy) { from project(':x-pack:plugin:core').files('src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem', 'src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt') into outputDir } -for (Version version : bwcVersions.wireCompatible) { - String baseName = "v${version}" - +for (Version bwcVersion : bwcVersions.wireCompatible) { + String baseName = "v${bwcVersion}" - Task oldClusterTest = tasks.create(name: "${baseName}#oldClusterTest", type: RestIntegTestTask) { - mustRunAfter(precommit) - } + testClusters { + "${baseName}" { + testDistribution = "DEFAULT" + versions = [ bwcVersion.toString(), project.version ] + numberOfNodes = 3 - configure(extensions.findByName("${baseName}#oldClusterTestCluster")) { - dependsOn copyTestNodeKeyMaterial - if (version.before('6.3.0')) { - String depVersion = version; - if (project.bwcVersions.unreleased.contains(version)) { - depVersion += "-SNAPSHOT" - } - mavenPlugin 'x-pack', "org.elasticsearch.plugin:x-pack:${depVersion}" - } - String usersCli = version.before('6.3.0') ? 'bin/x-pack/users' : 'bin/elasticsearch-users' - setupCommand 'setupTestUser', usersCli, 'useradd', 'test_user', '-p', 'x-pack-test-password', '-r', 'superuser' - bwcVersion = version - numBwcNodes = 3 - numNodes = 3 - clusterName = 'rolling-upgrade' - waitCondition = waitWithAuth - setting 'xpack.monitoring.exporters._http.type', 'http' - setting 'xpack.monitoring.exporters._http.enabled', 'false' - setting 'xpack.monitoring.exporters._http.auth.username', 'test_user' - setting 'xpack.monitoring.exporters._http.auth.password', 'x-pack-test-password' - setting 'xpack.license.self_generated.type', 'trial' - setting 'xpack.security.enabled', 'true' - setting 'xpack.security.transport.ssl.enabled', 'true' - setting 'xpack.security.authc.token.enabled', 'true' - setting 'xpack.security.authc.token.timeout', '60m' - setting 'xpack.security.audit.enabled', 'true' - setting 'xpack.security.transport.ssl.key', 'testnode.pem' - setting 'xpack.security.transport.ssl.certificate', 'testnode.crt' - keystoreSetting 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' - - dependsOn copyTestNodeKeyMaterial - extraConfigFile 'testnode.pem', new File(outputDir + '/testnode.pem') - extraConfigFile 'testnode.crt', new File(outputDir + '/testnode.crt') - if (version.onOrAfter('7.0.0')) { - setting 'xpack.security.authc.realms.file.file1.order', '0' - setting 'xpack.security.authc.realms.native.native1.order', '1' - } else { - setting 'xpack.security.authc.realms.file1.type', 'file' - setting 'xpack.security.authc.realms.file1.order', '0' - setting 'xpack.security.authc.realms.native1.type', 'native' - setting 'xpack.security.authc.realms.native1.order', '1' - } - - keystoreFile 'xpack.watcher.encryption_key', "${project.projectDir}/src/test/resources/system_key" - setting 'xpack.watcher.encrypt_sensitive_data', 'true' - - if (version.onOrAfter('6.6.0')) { - setting 'ccr.auto_follow.wait_for_metadata_timeout', '1s' - } - - // Old versions of the code contain an invalid assertion that trips - // during tests. Versions 5.6.9 and 6.2.4 have been fixed by removing - // the assertion, but this is impossible for released versions. - // However, released versions run without assertions, so end users won't - // be suffering the effects. This argument effectively removes the - // incorrect assertion from the older versions used in the BWC tests. - if (version.before('5.6.9') || (version.onOrAfter('6.0.0') && version.before('6.2.4'))) { - jvmArgs '-da:org.elasticsearch.xpack.monitoring.exporter.http.HttpExportBulk' - } - } - - Task oldClusterTestRunner = tasks.getByName("${baseName}#oldClusterTestRunner") - oldClusterTestRunner.configure { - systemProperty 'tests.rest.suite', 'old_cluster' - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - } - - Closure configureUpgradeCluster = {String name, Task lastRunner, int stopNode, Closure getOtherUnicastHostAddresses -> - configure(extensions.findByName("${baseName}#${name}")) { - dependsOn lastRunner, "${baseName}#oldClusterTestCluster#node${stopNode}.stop" - setupCommand 'setupTestUser', 'bin/elasticsearch-users', 'useradd', 'test_user', '-p', 'x-pack-test-password', '-r', 'superuser' - clusterName = 'rolling-upgrade' - otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - /* Override the data directory so the new node always gets the node we - * just stopped's data directory. */ - dataDir = { nodeNumber -> oldClusterTest.nodes[stopNode].dataDir } - waitCondition = waitWithAuth + setting 'repositories.url.allowed_urls', 'http://snapshot.test*' + setting 'path.repo', "${buildDir}/cluster/shared/repo/${baseName}" + setting 'http.content_type.required', 'true' setting 'xpack.monitoring.exporters._http.type', 'http' setting 'xpack.monitoring.exporters._http.enabled', 'false' setting 'xpack.monitoring.exporters._http.auth.username', 'test_user' @@ -179,19 +65,14 @@ for (Version version : bwcVersions.wireCompatible) { setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.security.transport.ssl.enabled', 'true' + setting 'xpack.security.authc.token.enabled', 'true' setting 'xpack.security.authc.token.timeout', '60m' + setting 'xpack.security.audit.enabled', 'true' setting 'xpack.security.transport.ssl.key', 'testnode.pem' setting 'xpack.security.transport.ssl.certificate', 'testnode.crt' - keystoreSetting 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' - - setting 'node.attr.upgraded', 'true' - setting 'xpack.security.authc.token.enabled', 'true' - setting 'xpack.security.audit.enabled', 'true' - setting 'node.name', "upgraded-node-${stopNode}" - dependsOn copyTestNodeKeyMaterial - extraConfigFile 'testnode.pem', new File(outputDir + '/testnode.pem') - extraConfigFile 'testnode.crt', new File(outputDir + '/testnode.crt') - if (version.onOrAfter('7.0.0')) { + keystore 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' + + if (bwcVersion.onOrAfter('7.0.0')) { setting 'xpack.security.authc.realms.file.file1.order', '0' setting 'xpack.security.authc.realms.native.native1.order', '1' } else { @@ -200,25 +81,53 @@ for (Version version : bwcVersions.wireCompatible) { setting 'xpack.security.authc.realms.native1.type', 'native' setting 'xpack.security.authc.realms.native1.order', '1' } - setting 'xpack.watcher.encrypt_sensitive_data', 'true' - keystoreFile 'xpack.watcher.encryption_key', "${project.projectDir}/src/test/resources/system_key" - if (version.before('6.0.0')) { - keystoreSetting 'xpack.security.authc.token.passphrase', 'token passphrase' + if (bwcVersion.onOrAfter('6.6.0')) { + setting 'ccr.auto_follow.wait_for_metadata_timeout', '1s' } + + user username: "test_user", password: "x-pack-test-password" + + extraConfigFile 'testnode.pem', file("$outputDir/testnode.pem") + extraConfigFile 'testnode.crt', file("$outputDir/testnode.crt") + + keystore 'xpack.watcher.encryption_key', file("${project.projectDir}/src/test/resources/system_key") + setting 'xpack.watcher.encrypt_sensitive_data', 'true' + + // Old versions of the code contain an invalid assertion that trips + // during tests. Versions 5.6.9 and 6.2.4 have been fixed by removing + // the assertion, but this is impossible for released versions. + // However, released versions run without assertions, so end users won't + // be suffering the effects. This argument effectively removes the + // incorrect assertion from the older versions used in the BWC tests. + if (bwcVersion.before('5.6.9') || (bwcVersion.onOrAfter('6.0.0') && bwcVersion.before('6.2.4'))) { + jvmArgs '-da:org.elasticsearch.xpack.monitoring.exporter.http.HttpExportBulk' + } + + javaHome = project.file(project.ext.runtimeJavaHome) } } - Task oneThirdUpgradedTest = tasks.create(name: "${baseName}#oneThirdUpgradedTest", type: RestIntegTestTask) + tasks.register("${baseName}#oldClusterTest", RestTestRunnerTask) { + useCluster testClusters."${baseName}" + mustRunAfter(precommit) + dependsOn copyTestNodeKeyMaterial + systemProperty 'tests.rest.suite', 'old_cluster' + systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") + } - configureUpgradeCluster("oneThirdUpgradedTestCluster", oldClusterTestRunner, 0, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oldClusterTest.nodes.get(1).transportUri(), oldClusterTest.nodes.get(2).transportUri()] }) - - Task oneThirdUpgradedTestRunner = tasks.getByName("${baseName}#oneThirdUpgradedTestRunner") - oneThirdUpgradedTestRunner.configure { + tasks.register("${baseName}#oneThirdUpgradedTest", RestTestRunnerTask) { + dependsOn "${baseName}#oldClusterTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") systemProperty 'tests.rest.suite', 'mixed_cluster' systemProperty 'tests.first_round', 'true' - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') // We only need to run these tests once so we may as well do it when we're two thirds upgraded systemProperty 'tests.rest.blacklist', [ 'mixed_cluster/10_basic/Start scroll in mixed cluster on upgraded node that we will continue after upgrade', @@ -229,63 +138,42 @@ for (Version version : bwcVersions.wireCompatible) { 'mixed_cluster/80_data_frame_jobs_crud/Test put batch data frame transforms on mixed cluster', 'mixed_cluster/80_data_frame_jobs_crud/Test put continuous data frame transform on mixed cluster' ].join(',') - finalizedBy "${baseName}#oldClusterTestCluster#node1.stop" } - Task twoThirdsUpgradedTest = tasks.create(name: "${baseName}#twoThirdsUpgradedTest", type: RestIntegTestTask) - - configureUpgradeCluster("twoThirdsUpgradedTestCluster", oneThirdUpgradedTestRunner, 1, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oldClusterTest.nodes.get(2).transportUri(), oneThirdUpgradedTest.nodes.get(0).transportUri()] }) - - Task twoThirdsUpgradedTestRunner = tasks.getByName("${baseName}#twoThirdsUpgradedTestRunner") - twoThirdsUpgradedTestRunner.configure { + tasks.register("${baseName}#twoThirdsUpgradedTest", RestTestRunnerTask) { + dependsOn "${baseName}#oneThirdUpgradedTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() + } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") systemProperty 'tests.rest.suite', 'mixed_cluster' systemProperty 'tests.first_round', 'false' - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - finalizedBy "${baseName}#oldClusterTestCluster#node2.stop" + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') } - Task upgradedClusterTest = tasks.create(name: "${baseName}#upgradedClusterTest", type: RestIntegTestTask) - - configureUpgradeCluster("upgradedClusterTestCluster", twoThirdsUpgradedTestRunner, 2, - // Use all running nodes as seed nodes so there is no race between pinging and the tests - { [oneThirdUpgradedTest.nodes.get(0).transportUri(), twoThirdsUpgradedTest.nodes.get(0).transportUri()] }) - - Task upgradedClusterTestRunner = tasks.getByName("${baseName}#upgradedClusterTestRunner") - upgradedClusterTestRunner.configure { - systemProperty 'tests.rest.suite', 'upgraded_cluster' - systemProperty 'tests.upgrade_from_version', version.toString().replace('-SNAPSHOT', '') - /* - * Force stopping all the upgraded nodes after the test runner - * so they are alive during the test. - */ - finalizedBy "${baseName}#oneThirdUpgradedTestCluster#stop" - finalizedBy "${baseName}#twoThirdsUpgradedTestCluster#stop" - - // migration tests should only run when the original/old cluster nodes where versions < 5.2.0. - // this stinks but we do the check here since our rest tests do not support conditionals - // otherwise we could check the index created version - String versionStr = project.extensions.findByName("${baseName}#oldClusterTestCluster").properties.get('bwcVersion') - String[] versionParts = versionStr.split('\\.') - if (versionParts[0].equals("5")) { - Integer minor = Integer.parseInt(versionParts[1]) - if (minor >= 2) { - systemProperty 'tests.rest.blacklist', '/20_security/Verify default password migration results in upgraded cluster' - } + tasks.register("${baseName}#upgradedClusterTest", RestTestRunnerTask) { + dependsOn "${baseName}#twoThirdsUpgradedTest" + useCluster testClusters."${baseName}" + doFirst { + testClusters."${baseName}".nextNodeToNextVersion() } + nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",") }") + nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName() }") + systemProperty 'tests.rest.suite', 'upgraded_cluster' + systemProperty 'tests.upgrade_from_version', bwcVersion.toString().replace('-SNAPSHOT', '') } - Task versionBwcTest = tasks.create(name: "${baseName}#bwcTest") { - dependsOn = [upgradedClusterTest] + tasks.register("${baseName}#bwcTest") { + dependsOn "${baseName}#upgradedClusterTest" } if (project.bwc_tests_enabled) { - bwcTest.dependsOn(versionBwcTest) + bwcTest.dependsOn("${baseName}#bwcTest") } } -// basic integ tests includes testing bwc against the most recent version task bwcTestSnapshots { if (project.bwc_tests_enabled) { for (final def version : bwcVersions.unreleasedWireCompatible) { @@ -295,11 +183,4 @@ task bwcTestSnapshots { } check.dependsOn(bwcTestSnapshots) -// copy x-pack plugin info so it is on the classpath and security manager has the right permissions -task copyXPackRestSpec(type: Copy) { - dependsOn(project.configurations.restSpec, 'processTestResources') - from project(xpackProject('plugin').path).sourceSets.test.resources - include 'rest-api-spec/api/**' - into project.sourceSets.test.output.resourcesDir -} -project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackRestSpec) +test.enabled = false \ No newline at end of file diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java index b44ae0360d88..a392fea9beb3 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java @@ -29,19 +29,19 @@ public class IndexingIT extends AbstractUpgradeTestCase { case OLD: break; case MIXED: - Request waitForYellow = new Request("GET", "/_cluster/health"); - waitForYellow.addParameter("wait_for_nodes", "3"); - waitForYellow.addParameter("wait_for_status", "yellow"); - client().performRequest(waitForYellow); + ensureHealth((request -> { + request.addParameter("timeout", "70s"); + request.addParameter("wait_for_nodes", "3"); + request.addParameter("wait_for_status", "yellow"); + })); break; case UPGRADED: - Request waitForGreen = new Request("GET", "/_cluster/health/test_index,index_with_replicas,empty_index"); - waitForGreen.addParameter("wait_for_nodes", "3"); - waitForGreen.addParameter("wait_for_status", "green"); - // wait for long enough that we give delayed unassigned shards to stop being delayed - waitForGreen.addParameter("timeout", "70s"); - waitForGreen.addParameter("level", "shards"); - client().performRequest(waitForGreen); + ensureHealth("test_index,index_with_replicas,empty_index", (request -> { + request.addParameter("wait_for_nodes", "3"); + request.addParameter("wait_for_status", "green"); + request.addParameter("timeout", "70s"); + request.addParameter("level", "shards"); + })); break; default: throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]"); diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java index 13ed2dafc5f3..8099812c8ea1 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java @@ -38,6 +38,7 @@ public class MlMappingsUpgradeIT extends AbstractUpgradeTestCase { * The purpose of this test is to ensure that when a job is open through a rolling upgrade we upgrade the results * index mappings when it is assigned to an upgraded node even if no other ML endpoint is called after the upgrade */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46262") public void testMappingsUpgrade() throws Exception { switch (CLUSTER_TYPE) { From 2a6ec9892942f14a3d0867c2ff83c7ba6c1be882 Mon Sep 17 00:00:00 2001 From: markharwood Date: Wed, 4 Sep 2019 14:05:46 +0100 Subject: [PATCH 038/103] Adjacency_matrix aggregation optimisation. (#46257) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Avoid pre-allocating ((N * N) - N) / 2 “BitsIntersector” objects given N filters. Most adjacency matrices will be sparse and we typically don’t need to allocate all of these objects - can save a lot of allocations when the number of filters is high. Closes #46212 --- .../adjacency/AdjacencyMatrixAggregator.java | 53 +++++++------------ 1 file changed, 18 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java index c1bf1d648d61..aa36e3a26fbc 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/adjacency/AdjacencyMatrixAggregator.java @@ -52,10 +52,6 @@ import static org.elasticsearch.index.query.AbstractQueryBuilder.parseInnerQuery /** * Aggregation for adjacency matrices. * - * TODO the aggregation produces a sparse response but in the - * computation it uses a non-sparse structure (an array of Bits - * objects). This could be changed to a sparse structure in future. - * */ public class AdjacencyMatrixAggregator extends BucketsAggregator { @@ -143,51 +139,38 @@ public class AdjacencyMatrixAggregator extends BucketsAggregator { this.totalNumKeys = keys.length + totalNumIntersections; } - private static class BitsIntersector implements Bits { - Bits a; - Bits b; - - BitsIntersector(Bits a, Bits b) { - super(); - this.a = a; - this.b = b; - } - - @Override - public boolean get(int index) { - return a.get(index) && b.get(index); - } - - @Override - public int length() { - return Math.min(a.length(), b.length()); - } - - } - @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException { // no need to provide deleted docs to the filter - final Bits[] bits = new Bits[filters.length + totalNumIntersections]; + final Bits[] bits = new Bits[filters.length]; for (int i = 0; i < filters.length; ++i) { bits[i] = Lucene.asSequentialAccessBits(ctx.reader().maxDoc(), filters[i].scorerSupplier(ctx)); } - // Add extra Bits for intersections - int pos = filters.length; - for (int i = 0; i < filters.length; i++) { - for (int j = i + 1; j < filters.length; j++) { - bits[pos++] = new BitsIntersector(bits[i], bits[j]); - } - } - assert pos == bits.length; return new LeafBucketCollectorBase(sub, null) { @Override public void collect(int doc, long bucket) throws IOException { + // Check each of the provided filters for (int i = 0; i < bits.length; i++) { if (bits[i].get(doc)) { collectBucket(sub, doc, bucketOrd(bucket, i)); } } + // Check all the possible intersections of the provided filters + int pos = filters.length; + for (int i = 0; i < filters.length; i++) { + if (bits[i].get(doc)) { + for (int j = i + 1; j < filters.length; j++) { + if (bits[j].get(doc)) { + collectBucket(sub, doc, bucketOrd(bucket, pos)); + } + pos++; + } + } else { + // Skip checks on all the other filters given one half of the pairing failed + pos += (filters.length - (i + 1)); + } + } + assert pos == bits.length + totalNumIntersections; } }; } From 4a0713aa0b62909dc3878314c7590b685c863f91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Istv=C3=A1n=20Zolt=C3=A1n=20Szab=C3=B3?= Date: Wed, 4 Sep 2019 15:12:49 +0200 Subject: [PATCH 039/103] [DOCS] Reformats search template and multi search template APIs (#46236) * [DOCS] Reformats search template and multi search template APIs. Co-Authored-By: James Rodewig --- .../reference/search/search-template.asciidoc | 191 +++++++++++++----- 1 file changed, 135 insertions(+), 56 deletions(-) diff --git a/docs/reference/search/search-template.asciidoc b/docs/reference/search/search-template.asciidoc index 950477aa7d32..fb6fe4d41a25 100644 --- a/docs/reference/search/search-template.asciidoc +++ b/docs/reference/search/search-template.asciidoc @@ -1,8 +1,7 @@ [[search-template]] === Search Template -The `/_search/template` endpoint allows to use the mustache language to pre render search requests, -before they are executed and fill existing templates with template parameters. +Allows you to use the mustache language to pre render search requests. [source,js] ------------------------------------------ @@ -22,19 +21,87 @@ GET _search/template // CONSOLE // TEST[setup:twitter] +[[search-template-api-request]] +==== {api-request-title} + +`GET _search/template` + + +[[search-template-api-desc]] +==== {api-description-title} + +The `/_search/template` endpoint allows you to use the mustache language to pre- +render search requests, before they are executed and fill existing templates +with template parameters. + For more information on how Mustache templating and what kind of templating you can do with it check out the http://mustache.github.io/mustache.5.html[online documentation of the mustache project]. -NOTE: The mustache language is implemented in Elasticsearch as a sandboxed -scripting language, hence it obeys settings that may be used to enable or -disable scripts per type and context as described in the -<> +NOTE: The mustache language is implemented in {es} as a sandboxed scripting +language, hence it obeys settings that may be used to enable or disable scripts +per type and context as described in the +<>. + + +[[search-template-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index] + + +[[search-template-api-query-params]] +==== {api-query-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=allow-no-indices] + +`ccs_minimize_roundtrips`:: + (Optional, boolean) If `true`, network round-trips are minimized for + cross-cluster search requests. Defaults to `true`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + +`explain`:: + (Optional, boolean) If `true`, the response includes additional details about + score computation as part of a hit. Defaults to `false`. + +`ignore_throttled`:: + (Optional, boolean) If `true`, specified concrete, expanded or aliased indices + are not included in the response when throttled. Defaults to `false`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=preference] + +`profile`:: + (Optional, boolean) If `true`, the query execution is profiled. Defaults + to `false`. + +`rest_total_hits_as_int`:: + (Optional, boolean) If `true`, `hits.total` are rendered as an integer in + the response. Defaults to `false`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=routing] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=scroll] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=search_type] + +`typed_keys`:: + (Optional, boolean) If `true`, aggregation and suggester names are + prefixed by their respective types in the response. Defaults to `false`. + + +[[search-template-api-request-body]] +==== {api-request-body-title} + +The API request body must contain the search definition template and its parameters. + + +[[search-template-api-example]] +==== {api-response-codes-title} -[float] -==== Examples -[float] [[pre-registered-templates]] ===== Store a search template @@ -61,8 +128,8 @@ POST _scripts/ ////////////////////////// -We want to be sure that the template has been created, -because we'll use it later. +The API returns the following result if the template has been successfully +created: [source,js] -------------------------------------------------- @@ -74,7 +141,8 @@ because we'll use it later. ////////////////////////// -This template can be retrieved by + +The template can be retrieved by calling [source,js] ------------------------------------------ @@ -83,7 +151,7 @@ GET _scripts/ // CONSOLE // TEST[continued] -which is rendered as: +The API returns the following result: [source,js] ------------------------------------------ @@ -101,7 +169,8 @@ which is rendered as: ------------------------------------------ // TESTRESPONSE -This template can be deleted by + +This template can be deleted by calling [source,js] ------------------------------------------ @@ -110,26 +179,12 @@ DELETE _scripts/ // CONSOLE // TEST[continued] -////////////////////////// -We want to be sure that the template has been created, -because we'll use it later. -[source,js] --------------------------------------------------- -{ - "acknowledged" : true -} --------------------------------------------------- -// TESTRESPONSE - -////////////////////////// - -[float] [[use-registered-templates]] -===== Use a stored search template +===== Using a stored search template -To use a stored template at search time use: +To use a stored template at search time send the following request: [source,js] ------------------------------------------ @@ -145,11 +200,12 @@ GET _search/template // TEST[catch:missing] <1> Name of the stored template script. -[float] -[[_validating_templates]] -==== Validate a search template -A template can be rendered in a response with given parameters using +[[_validating_templates]] +==== Validating a search template + +A template can be rendered in a response with given parameters by using the +following request: [source,js] ------------------------------------------ @@ -165,7 +221,8 @@ GET _render/template ------------------------------------------ // CONSOLE -This call will return the rendered template: + +The API returns the rendered template: [source,js] ------------------------------------------ @@ -185,7 +242,8 @@ This call will return the rendered template: // TESTRESPONSE <1> `status` array has been populated with values from the `params` object. -Stored templates can also be rendered using + +Stored templates can also be rendered by calling the following request: [source,js] ------------------------------------------ @@ -198,10 +256,10 @@ GET _render/template/ ------------------------------------------ // NOTCONSOLE -[float] -===== Explain +[[search-template-explain-parameter]] +===== Using the explain parameter -You can use `explain` parameter when running a template: +You can use the `explain` parameter when running a template: [source,js] ------------------------------------------ @@ -217,10 +275,11 @@ GET _search/template // CONSOLE // TEST[catch:missing] -[float] + +[[search-template-profile-parameter]] ===== Profiling -You can use `profile` parameter when running a template: +You can use the `profile` parameter when running a template: [source,js] ------------------------------------------ @@ -236,7 +295,8 @@ GET _search/template // CONSOLE // TEST[catch:missing] -[float] + +[[search-template-query-string-single]] ===== Filling in a query string with a single value [source,js] @@ -258,7 +318,7 @@ GET _search/template // CONSOLE // TEST[setup:twitter] -[float] +[[search-template-converting-to-json]] ===== Converting parameters to JSON The `{{#toJson}}parameter{{/toJson}}` function can be used to convert parameters @@ -337,7 +397,7 @@ which is rendered as: ------------------------------------------ // NOTCONSOLE -[float] +[[search-template-concatenate-array]] ===== Concatenating array of values The `{{#join}}array{{/join}}` function can be used to concatenate the @@ -422,7 +482,7 @@ which is rendered as: ------------------------------------------ // NOTCONSOLE -[float] +[[search-template-default-values]] ===== Default values A default value is written as `{{var}}{{^var}}default{{/var}}` for instance: @@ -476,7 +536,7 @@ for `end`: ------------------------------------------ // NOTCONSOLE -[float] +[[search-template-conditional-clauses]] ===== Conditional clauses Conditional clauses cannot be expressed using the JSON form of the template. @@ -485,6 +545,7 @@ we wanted to run a `match` query on the `line` field, and optionally wanted to filter by line numbers, where `start` and `end` are optional. The `params` would look like: + [source,js] ------------------------------------------ { @@ -500,6 +561,7 @@ The `params` would look like: // NOTCONSOLE <1> All three of these elements are optional. + We could write the query as: [source,js] @@ -556,11 +618,12 @@ via the REST API, should be written as a string: ================================== -[float] +[[search-template-encode-urls]] ===== Encoding URLs The `{{#url}}value{{/url}}` function can be used to encode a string value -in a HTML encoding form as defined in by the http://www.w3.org/TR/html4/[HTML specification]. +in a HTML encoding form as defined in by the +http://www.w3.org/TR/html4/[HTML specification]. As an example, it is useful to encode a URL: @@ -583,6 +646,7 @@ GET _render/template ------------------------------------------ // CONSOLE + The previous query will be rendered as: [source,js] @@ -602,8 +666,19 @@ The previous query will be rendered as: [[multi-search-template]] === Multi Search Template -The multi search template API allows to execute several search template -requests within the same API using the `_msearch/template` endpoint. +Allows to execute several search template requests. + +[[multi-search-template-api-request]] +==== {api-request-title} + +`GET _msearch/template` + + +[[multi-search-template-api-desc]] +==== {api-description-title} + +Allows to execute several search template requests within the same API using the +`_msearch/template` endpoint. The format of the request is similar to the <> format: @@ -617,11 +692,15 @@ body\n -------------------------------------------------- // NOTCONSOLE -The header part supports the same `index`, `search_type`, -`preference`, and `routing` options as the usual Multi Search API. +The header part supports the same `index`, `search_type`, `preference`, and +`routing` options as the Multi Search API. -The body includes a search template body request and supports inline, -stored and file templates. Here is an example: +The body includes a search template body request and supports inline, stored and +file templates. + + +[[multi-search-template-api-example]] +==== {api-examples-title} [source,js] -------------------------------------------------- @@ -643,5 +722,5 @@ $ curl -H "Content-Type: application/x-ndjson" -XGET localhost:9200/_msearch/tem The response returns a `responses` array, which includes the search template response for each search template request matching its order in the original multi search template request. If there was a complete failure for that specific -search template request, an object with `error` message will be returned in place -of the actual search response. +search template request, an object with `error` message will be returned in +place of the actual search response. From 97bd8de1d25e3ef66e471c3300653b68d650d4f9 Mon Sep 17 00:00:00 2001 From: Przemyslaw Gomulka Date: Wed, 4 Sep 2019 15:48:37 +0200 Subject: [PATCH 040/103] Improve documentation for X-Opaque-ID (#46167) this field can be present in search slow logs and deprecation logs. The docs describes how to enable this functionality and what expect in logs. closes #44851 --- docs/reference/index-modules/slowlog.asciidoc | 35 +++++++++++++++++++ docs/reference/setup/logging-config.asciidoc | 18 ++++++++++ 2 files changed, 53 insertions(+) diff --git a/docs/reference/index-modules/slowlog.asciidoc b/docs/reference/index-modules/slowlog.asciidoc index a96c8fe995b7..18dbab995eb7 100644 --- a/docs/reference/index-modules/slowlog.asciidoc +++ b/docs/reference/index-modules/slowlog.asciidoc @@ -83,6 +83,41 @@ logger.index_search_slowlog_rolling.appenderRef.index_search_slowlog_rolling.ref logger.index_search_slowlog_rolling.additivity = false -------------------------------------------------- +[float] +==== Identifying search slow log origin + +It is often useful to identify what triggered a slow running query. If a call was initiated with an `X-Opaque-ID` header, then the user ID +is included in Search Slow logs as an additional **id** field (scroll to the right). +[source,txt] +--------------------------- +[2030-08-30T11:59:37,786][WARN ][i.s.s.query ] [node-0] [index6][0] took[78.4micros], took_millis[0], total_hits[0 hits], stats[], search_type[QUERY_THEN_FETCH], total_shards[1], source[{"query":{"match_all":{"boost":1.0}}}], id[MY_USER_ID], +--------------------------- +// NOTCONSOLE +The user ID is also included in JSON logs. +[source,js] +--------------------------- +{ + "type": "index_search_slowlog", + "timestamp": "2030-08-30T11:59:37,786+02:00", + "level": "WARN", + "component": "i.s.s.query", + "cluster.name": "distribution_run", + "node.name": "node-0", + "message": "[index6][0]", + "took": "78.4micros", + "took_millis": "0", + "total_hits": "0 hits", + "stats": "[]", + "search_type": "QUERY_THEN_FETCH", + "total_shards": "1", + "source": "{\"query\":{\"match_all\":{\"boost\":1.0}}}", + "id": "MY_USER_ID", + "cluster.uuid": "Aq-c-PAeQiK3tfBYtig9Bw", + "node.id": "D7fUYfnfTLa2D7y-xw6tZg" +} +--------------------------- +// NOTCONSOLE + [float] [[index-slow-log]] === Index Slow log diff --git a/docs/reference/setup/logging-config.asciidoc b/docs/reference/setup/logging-config.asciidoc index dcea83a7f5d6..ce747c76621e 100644 --- a/docs/reference/setup/logging-config.asciidoc +++ b/docs/reference/setup/logging-config.asciidoc @@ -214,6 +214,24 @@ files (four rolled logs, and the active log). You can disable it in the `config/log4j2.properties` file by setting the deprecation log level to `error`. +You can identify what is triggering deprecated functionality if `X-Opaque-Id` was used as an HTTP header. +The user ID is included in the `X-Opaque-ID` field in deprecation JSON logs. +[source,js] +--------------------------- +{ + "type": "deprecation", + "timestamp": "2019-08-30T12:07:07,126+02:00", + "level": "WARN", + "component": "o.e.d.r.a.a.i.RestCreateIndexAction", + "cluster.name": "distribution_run", + "node.name": "node-0", + "message": "[types removal] Using include_type_name in create index requests is deprecated. The parameter will be removed in the next major version.", + "x-opaque-id": "MY_USER_ID", + "cluster.uuid": "Aq-c-PAeQiK3tfBYtig9Bw", + "node.id": "D7fUYfnfTLa2D7y-xw6tZg" +} +--------------------------- +// NOTCONSOLE [float] [[json-logging]] From c43ad1eebd0a97c4116fc2c152ef156980ba46c5 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Wed, 4 Sep 2019 10:58:24 -0400 Subject: [PATCH 041/103] [DOCS] Add "get index template" API docs (#46296) --- docs/reference/indices.asciidoc | 3 + .../indices/get-index-template.asciidoc | 89 +++++++++++++++++++ docs/reference/indices/templates.asciidoc | 30 +------ 3 files changed, 95 insertions(+), 27 deletions(-) create mode 100644 docs/reference/indices/get-index-template.asciidoc diff --git a/docs/reference/indices.asciidoc b/docs/reference/indices.asciidoc index 8fbb63a94d64..9eb0bf7c4766 100644 --- a/docs/reference/indices.asciidoc +++ b/docs/reference/indices.asciidoc @@ -51,6 +51,7 @@ index settings, aliases, mappings, and index templates. [[index-templates]] === Index templates: * <> +* <> * <> * <> @@ -120,6 +121,8 @@ include::indices/analyze.asciidoc[] include::indices/delete-index-template.asciidoc[] +include::indices/get-index-template.asciidoc[] + include::indices/template-exists.asciidoc[] include::indices/templates.asciidoc[] diff --git a/docs/reference/indices/get-index-template.asciidoc b/docs/reference/indices/get-index-template.asciidoc new file mode 100644 index 000000000000..703c134abc9f --- /dev/null +++ b/docs/reference/indices/get-index-template.asciidoc @@ -0,0 +1,89 @@ +[[indices-get-template]] +=== Get index template API +++++ +Get template index +++++ + +Returns information about one or more index templates. + +//// +[source,js] +-------------------------------------------------- +PUT _template/template_1 +{ + "index_patterns" : ["te*"], + "settings": { + "number_of_shards": 1 + } +} +-------------------------------------------------- +// CONSOLE +// TESTSETUP +//// + +[source,js] +-------------------------------------------------- +GET /_template/template_1 +-------------------------------------------------- +// CONSOLE + + +[[get-template-api-request]] +==== {api-request-title} + +`GET /_template/` + + +[[get-template-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index-template] ++ +To return all index templates, omit this parameter +or use a value of `_all` or `*`. + + +[[get-template-api-query-params]] +==== {api-query-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=flat-settings] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=include-type-name] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=local] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=master-timeout] + + +[[get-template-api-example]] +==== {api-examples-title} + + +[[get-template-api-multiple-ex]] +===== Get multiple index templates + +[source,js] +-------------------------------------------------- +GET /_template/template_1,template_2 +-------------------------------------------------- +// CONSOLE + + +[[get-template-api-wildcard-ex]] +===== Get index templates using a wildcard expression + +[source,js] +-------------------------------------------------- +GET /_template/temp* +-------------------------------------------------- +// CONSOLE + + +[[get-template-api-all-ex]] +===== Get all index templates + +[source,js] +-------------------------------------------------- +GET /_template +-------------------------------------------------- +// CONSOLE diff --git a/docs/reference/indices/templates.asciidoc b/docs/reference/indices/templates.asciidoc index 6d6591cc4ff3..ddd310e10c27 100644 --- a/docs/reference/indices/templates.asciidoc +++ b/docs/reference/indices/templates.asciidoc @@ -90,35 +90,11 @@ DELETE /_template/template_1 -------------------------------------------------- // CONSOLE -[float] -[[getting]] +[float] +[[getting]] ==== Getting templates -Index templates are identified by a name (in the above case -`template_1`) and can be retrieved using the following: - -[source,js] --------------------------------------------------- -GET /_template/template_1 --------------------------------------------------- -// CONSOLE - -You can also match several templates by using wildcards like: - -[source,js] --------------------------------------------------- -GET /_template/temp* -GET /_template/template_1,template_2 --------------------------------------------------- -// CONSOLE - -To get list of all index templates you can run: - -[source,js] --------------------------------------------------- -GET /_template --------------------------------------------------- -// CONSOLE +See <>. [float] [[multiple-templates]] From b38f46440386de969404268bc532a369c9ebba39 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 4 Sep 2019 11:01:01 -0400 Subject: [PATCH 042/103] Do not send recovery requests with CancellableThreads (#46287) Previously, we send recovery requests using CancellableThreads because we send requests and wait for responses in a blocking manner. With async recovery, we no longer need to do so. Moreover, if we fail to submit a request, then we can release the Store using an interruptible thread which can risk invalidating the node lock. This PR is the first step to avoid forking when releasing the Store. Relates #45409 Relates #46178 --- .../recovery/RecoverySourceHandler.java | 70 ++++++++++--------- 1 file changed, 36 insertions(+), 34 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 8324dd023b70..8428b5a47221 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -517,9 +517,9 @@ public class RecoverySourceHandler { final StepListener sendFilesStep = new StepListener<>(); final StepListener createRetentionLeaseStep = new StepListener<>(); final StepListener cleanFilesStep = new StepListener<>(); - cancellableThreads.execute(() -> - recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, - phase1ExistingFileSizes, translogOps.getAsInt(), sendFileInfoStep)); + cancellableThreads.checkForCancel(); + recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, + phase1ExistingFileSizes, translogOps.getAsInt(), sendFileInfoStep); sendFileInfoStep.whenComplete(r -> sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFilesStep), listener::onFailure); @@ -634,8 +634,8 @@ public class RecoverySourceHandler { // Send a request preparing the new shard's translog to receive operations. This ensures the shard engine is started and disables // garbage collection (not the JVM's GC!) of tombstone deletes. logger.trace("recovery [phase1]: prepare remote engine for translog"); - cancellableThreads.execute(() -> - recoveryTarget.prepareForTranslogOperations(totalTranslogOps, wrappedListener)); + cancellableThreads.checkForCancel(); + recoveryTarget.prepareForTranslogOperations(totalTranslogOps, wrappedListener); } /** @@ -741,30 +741,29 @@ public class RecoverySourceHandler { final List operations = nextBatch.get(); // send the leftover operations or if no operations were sent, request the target to respond with its local checkpoint if (operations.isEmpty() == false || firstBatch) { - cancellableThreads.execute(() -> { - recoveryTarget.indexTranslogOperations( - operations, - totalTranslogOps, - maxSeenAutoIdTimestamp, - maxSeqNoOfUpdatesOrDeletes, - retentionLeases, - mappingVersionOnPrimary, - ActionListener.wrap( - newCheckpoint -> { - sendBatch( - nextBatch, - false, - SequenceNumbers.max(targetLocalCheckpoint, newCheckpoint), - totalTranslogOps, - maxSeenAutoIdTimestamp, - maxSeqNoOfUpdatesOrDeletes, - retentionLeases, - mappingVersionOnPrimary, - listener); - }, - listener::onFailure - )); - }); + cancellableThreads.checkForCancel(); + recoveryTarget.indexTranslogOperations( + operations, + totalTranslogOps, + maxSeenAutoIdTimestamp, + maxSeqNoOfUpdatesOrDeletes, + retentionLeases, + mappingVersionOnPrimary, + ActionListener.wrap( + newCheckpoint -> { + sendBatch( + nextBatch, + false, + SequenceNumbers.max(targetLocalCheckpoint, newCheckpoint), + totalTranslogOps, + maxSeenAutoIdTimestamp, + maxSeqNoOfUpdatesOrDeletes, + retentionLeases, + mappingVersionOnPrimary, + listener); + }, + listener::onFailure + )); } else { listener.onResponse(targetLocalCheckpoint); } @@ -787,7 +786,8 @@ public class RecoverySourceHandler { shardId + " marking " + request.targetAllocationId() + " as in sync", shard, cancellableThreads, logger); final long globalCheckpoint = shard.getLastKnownGlobalCheckpoint(); // this global checkpoint is persisted in finalizeRecovery final StepListener finalizeListener = new StepListener<>(); - cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint, trimAboveSeqNo, finalizeListener)); + cancellableThreads.checkForCancel(); + recoveryTarget.finalizeRecovery(globalCheckpoint, trimAboveSeqNo, finalizeListener); finalizeListener.whenComplete(r -> { runUnderPrimaryPermit(() -> shard.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), shardId + " updating " + request.targetAllocationId() + "'s global checkpoint", shard, cancellableThreads, logger); @@ -894,8 +894,9 @@ public class RecoverySourceHandler { @Override protected void sendChunkRequest(FileChunk request, ActionListener listener) { - cancellableThreads.execute(() -> recoveryTarget.writeFileChunk( - request.md, request.position, request.content, request.lastChunk, translogOps.getAsInt(), listener)); + cancellableThreads.checkForCancel(); + recoveryTarget.writeFileChunk( + request.md, request.position, request.content, request.lastChunk, translogOps.getAsInt(), listener); } @Override @@ -922,13 +923,14 @@ public class RecoverySourceHandler { // Once the files have been renamed, any other files that are not // related to this recovery (out of date segments, for example) // are deleted - cancellableThreads.execute(() -> recoveryTarget.cleanFiles(translogOps.getAsInt(), globalCheckpoint, sourceMetadata, + cancellableThreads.checkForCancel(); + recoveryTarget.cleanFiles(translogOps.getAsInt(), globalCheckpoint, sourceMetadata, ActionListener.delegateResponse(listener, (l, e) -> ActionListener.completeWith(l, () -> { StoreFileMetaData[] mds = StreamSupport.stream(sourceMetadata.spliterator(), false).toArray(StoreFileMetaData[]::new); ArrayUtil.timSort(mds, Comparator.comparingLong(StoreFileMetaData::length)); // check small files first handleErrorOnSendFiles(store, e, mds); throw e; - })))); + }))); } private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetaData[] mds) throws Exception { From 948cdab15ae3a3716c3d06be9c7b1662949f48d5 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 4 Sep 2019 11:23:29 -0400 Subject: [PATCH 043/103] Build: Enable testing without magic comments (#46180) Previously we only turned on tests if we saw either `// CONSOLE` or `// TEST`. These magic comments are difficult for the docs build to deal with so it has moved away from using them where possible. We should catch up. This adds another trigger to enable testing: marking a snippet with the `console` language. It looks like this: ``` [source,console] ---- GET / ---- ``` This saves a line which is nice, I guess. But it is more important to me that this is consistent with the way the docs build works now. Similarly this enables response testing when you mark a snippet with the language `console-result`. That looks like: ``` [source,console-result] ---- { "result": "0.1" } ---- ``` `// TESTRESPONSE` is still available for situations like `// TEST`: when the response isn't *in* the console-result language (like `_cat`) or when you want to perform substitutions on the generated test. Should unblock #46159. --- .../doc/RestTestsFromSnippetsTask.groovy | 5 +-- docs/README.asciidoc | 35 +++++++++++++------ .../painless-execute-script.asciidoc | 18 ++++------ docs/reference/docs/index_.asciidoc | 3 +- 4 files changed, 34 insertions(+), 27 deletions(-) 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 d89ed12b8167..54db2cd71628 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy @@ -202,11 +202,12 @@ public class RestTestsFromSnippetsTask extends SnippetsTask { previousTest = snippet return } - if (snippet.testResponse) { + if (snippet.testResponse || snippet.language == 'console-result') { response(snippet) return } - if (snippet.test || snippet.console) { + if (snippet.test || snippet.console || + snippet.language == 'console') { test(snippet) previousTest = snippet return diff --git a/docs/README.asciidoc b/docs/README.asciidoc index 9cd2c0b4e7bd..6ec7083fba85 100644 --- a/docs/README.asciidoc +++ b/docs/README.asciidoc @@ -3,10 +3,14 @@ Elasticsearch documentation build process. See: https://github.com/elastic/docs -Snippets marked with `// CONSOLE` are automatically annotated with "VIEW IN -CONSOLE" and "COPY AS CURL" in the documentation and are automatically tested -by the command `gradle :docs:check`. To test just the docs from a single page, -use e.g. `./gradlew :docs:integTestRunner --tests "*rollover*"`. +Snippets marked with `[source,console]` are automatically annotated with +"VIEW IN CONSOLE" and "COPY AS CURL" in the documentation and are automatically +tested by the command `./gradlew -pdocs check`. To test just the docs from a +single page, use e.g. `./gradlew -ddocs integTestRunner --tests "*rollover*"`. + +NOTE: Previously we use `// CONSOLE` instead of `[source,console]`. This worked +well for a long time so you'll see it all over early branches but we're phasing +it out because it requires some unpleasant hackery on the docs build side. NOTE: If you have an elasticsearch-extra folder alongside your elasticsearch folder, you must temporarily rename it when you are testing 6.3 or later branches. @@ -45,10 +49,21 @@ for its modifiers: header. If the response doesn't include a `Warning` header with the exact text then the test fails. If the response includes `Warning` headers that aren't expected then the test fails. -* `// TESTRESPONSE`: Matches this snippet against the body of the response of - the last test. If the response is JSON then order is ignored. If you add - `// TEST[continued]` to the snippet after `// TESTRESPONSE` it will continue - in the same test, allowing you to interleave requests with responses to check. +* `[source,console-result]`: Matches this snippet against the body of the + response of the last test. If the response is JSON then order is ignored. If + you add `// TEST[continued]` to the snippet after `[source,console-result]` + it will continue in the same test, allowing you to interleave requests with + responses to check. +* `// TESTRESPONSE`: Explicitly marks a snippet as a test response even without + `[source,console-result]`. Similarly to `// TEST` this is mostly used for + its modifiers. + * You can't use `[source,console-result]` immediately after `// TESTSETUP`. + Instead, consider using `// TEST[continued]` or rearrange your snippets. + + NOTE: Previously we only used `// TESTRESPONSE` instead of + `[source,console-result]` so you'll see that a lot in older branches but we + prefer `[source,console-result]` now. + * `// TESTRESPONSE[s/foo/bar/]`: Substitutions. See `// TEST[s/foo/bar]` for how it works. These are much more common than `// TEST[s/foo/bar]` because they are useful for eliding portions of the response that are not pertinent @@ -62,8 +77,6 @@ for its modifiers: "figures out" the path. This is especially useful for making sweeping assertions like "I made up all the numbers in this example, don't compare them" which looks like `// TESTRESPONSE[s/\d+/$body.$_path/]`. - * You can't use `// TESTRESPONSE` immediately after `// TESTSETUP`. Instead, - consider using `// TEST[continued]` or rearrange your snippets. * `// TESTRESPONSE[non_json]`: Add substitutions for testing responses in a format other than JSON. Use this after all other substitutions so it doesn't make other substitutions difficult. @@ -98,7 +111,7 @@ 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 +is not supported by `[source,console]` the usual way to incorporate it is with a `// TEST[s//]` marker like this: ``` diff --git a/docs/painless/painless-guide/painless-execute-script.asciidoc b/docs/painless/painless-guide/painless-execute-script.asciidoc index fc5a6bf71d14..02c15cb75dd1 100644 --- a/docs/painless/painless-guide/painless-execute-script.asciidoc +++ b/docs/painless/painless-guide/painless-execute-script.asciidoc @@ -30,7 +30,7 @@ If no context is specified then this context is used by default. Request: -[source,js] +[source,console] ---------------------------------------------------------------- POST /_scripts/painless/_execute { @@ -43,17 +43,15 @@ POST /_scripts/painless/_execute } } ---------------------------------------------------------------- -// CONSOLE Response: -[source,js] +[source,console-result] -------------------------------------------------- { "result": "0.1" } -------------------------------------------------- -// TESTRESPONSE ===== Filter context @@ -69,7 +67,7 @@ index:: The name of an index containing a mapping that is compatible with the do *Example* -[source,js] +[source,console] ---------------------------------------------------------------- PUT /my-index { @@ -99,17 +97,15 @@ POST /_scripts/painless/_execute } } ---------------------------------------------------------------- -// CONSOLE Response: -[source,js] +[source,console-result] -------------------------------------------------- { "result": true } -------------------------------------------------- -// TESTRESPONSE ===== Score context @@ -125,7 +121,7 @@ query:: If `_score` is used in the script then a query can specified that will b *Example* -[source,js] +[source,console] ---------------------------------------------------------------- PUT /my-index { @@ -159,14 +155,12 @@ POST /_scripts/painless/_execute } } ---------------------------------------------------------------- -// CONSOLE Response: -[source,js] +[source,console-result] -------------------------------------------------- { "result": 0.8 } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index ab1da88da6a0..3e70cf45dbc5 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -141,7 +141,7 @@ comma-separated list of patterns you want to allow, or prefix each pattern with `+` or `-` to indicate whether it should be allowed or blocked. When a list is specified, the default behaviour is to disallow. -[source,js] +[source,console] -------------------------------------------------- PUT _cluster/settings { @@ -164,7 +164,6 @@ PUT _cluster/settings } } -------------------------------------------------- -// CONSOLE <1> Allow auto-creation of indices called `twitter` or `index10`, block the creation of indices that match the pattern `index1*`, and allow creation of From 4472773e0dc9a1874068ad6209197fa00d2c91e6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 4 Sep 2019 16:37:00 +0100 Subject: [PATCH 044/103] Docs for translog, history retention and flushing (#46245) This commit updates the docs about translog retention and flushing to reflect recent changes in how peer recoveries work. It also adds some docs to describe how history is retained for replay using soft deletes and shard history retention leases. Relates #45473 --- docs/reference/index-modules.asciidoc | 6 + .../index-modules/history-retention.asciidoc | 72 ++++++++++ .../reference/index-modules/translog.asciidoc | 108 +++++++------- docs/reference/indices/flush.asciidoc | 135 ++++++++++-------- 4 files changed, 216 insertions(+), 105 deletions(-) create mode 100644 docs/reference/index-modules/history-retention.asciidoc diff --git a/docs/reference/index-modules.asciidoc b/docs/reference/index-modules.asciidoc index b3729f8477df..93da06d9b967 100644 --- a/docs/reference/index-modules.asciidoc +++ b/docs/reference/index-modules.asciidoc @@ -280,6 +280,10 @@ Other index settings are available in index modules: Control over the transaction log and background flush operations. +<>:: + + Control over the retention of a history of operations in the index. + [float] [[x-pack-index-settings]] === [xpack]#{xpack} index settings# @@ -305,4 +309,6 @@ include::index-modules/store.asciidoc[] include::index-modules/translog.asciidoc[] +include::index-modules/history-retention.asciidoc[] + include::index-modules/index-sorting.asciidoc[] diff --git a/docs/reference/index-modules/history-retention.asciidoc b/docs/reference/index-modules/history-retention.asciidoc new file mode 100644 index 000000000000..94e17e49251e --- /dev/null +++ b/docs/reference/index-modules/history-retention.asciidoc @@ -0,0 +1,72 @@ +[[index-modules-history-retention]] +== History retention + +{es} sometimes needs to replay some of the operations that were performed on a +shard. For instance, if a replica is briefly offline then it may be much more +efficient to replay the few operations it missed while it was offline than to +rebuild it from scratch. Similarly, {ccr} works by performing operations on the +leader cluster and then replaying those operations on the follower cluster. + +At the Lucene level there are really only two write operations that {es} +performs on an index: a new document may be indexed, or an existing document may +be deleted. Updates are implemented by atomically deleting the old document and +then indexing the new document. A document indexed into Lucene already contains +all the information needed to replay that indexing operation, but this is not +true of document deletions. To solve this, {es} uses a feature called _soft +deletes_ to preserve recent deletions in the Lucene index so that they can be +replayed. + +{es} only preserves certain recently-deleted documents in the index because a +soft-deleted document still takes up some space. Eventually {es} will fully +discard these soft-deleted documents to free up that space so that the index +does not grow larger and larger over time. Fortunately {es} does not need to be +able to replay every operation that has ever been performed on a shard, because +it is always possible to make a full copy of a shard on a remote node. However, +copying the whole shard may take much longer than replaying a few missing +operations, so {es} tries to retain all of the operations it expects to need to +replay in future. + +{es} keeps track of the operations it expects to need to replay in future using +a mechanism called _shard history retention leases_. Each shard copy that might +need operations to be replayed must first create a shard history retention lease +for itself. For example, this shard copy might be a replica of a shard or it +might be a shard of a follower index when using {ccr}. Each retention lease +keeps track of the sequence number of the first operation that the corresponding +shard copy has not received. As the shard copy receives new operations, it +increases the sequence number contained in its retention lease to indicate that +it will not need to replay those operations in future. {es} discards +soft-deleted operations once they are not being held by any retention lease. + +If a shard copy fails then it stops updating its shard history retention lease, +which means that {es} will preserve all new operations so they can be replayed +when the failed shard copy recovers. However, retention leases only last for a +limited amount of time. If the shard copy does not recover quickly enough then +its retention lease may expire. This protects {es} from retaining history +forever if a shard copy fails permanently, because once a retention lease has +expired {es} can start to discard history again. If a shard copy recovers after +its retention lease has expired then {es} will fall back to copying the whole +index since it can no longer simply replay the missing history. The expiry time +of a retention lease defaults to `12h` which should be long enough for most +reasonable recovery scenarios. + +Soft deletes are enabled by default on indices created in recent versions, but +they can be explicitly enabled or disabled at index creation time. If soft +deletes are disabled then peer recoveries can still sometimes take place by +copying just the missing operations from the translog +<>. {ccr-cap} will not function if soft deletes are disabled. + +[float] +=== History retention settings + +`index.soft_deletes.enabled`:: + + Whether or not soft deletes are enabled on the index. Soft deletes can only be + configured at index creation and only on indices created on or after 6.5.0. + The default value is `true`. + +`index.soft_deletes.retention_lease.period`:: + + The maximum length of time to retain a shard history retention lease before + it expires and the history that it retains can be discarded. The default + value is `12h`. diff --git a/docs/reference/index-modules/translog.asciidoc b/docs/reference/index-modules/translog.asciidoc index 414ac59f0ba2..48947b348a47 100644 --- a/docs/reference/index-modules/translog.asciidoc +++ b/docs/reference/index-modules/translog.asciidoc @@ -7,55 +7,57 @@ delete operation. Changes that happen after one commit and before another will be removed from the index by Lucene in the event of process exit or hardware failure. -Because Lucene commits are too expensive to perform on every individual change, -each shard copy also has a _transaction log_ known as its _translog_ associated -with it. All index and delete operations are written to the translog after +Lucene commits are too expensive to perform on every individual change, so each +shard copy also writes operations into its _transaction log_ known as the +_translog_. All index and delete operations are written to the translog after being processed by the internal Lucene index but before they are acknowledged. -In the event of a crash, recent transactions that have been acknowledged but -not yet included in the last Lucene commit can instead be recovered from the -translog when the shard recovers. +In the event of a crash, recent operations that have been acknowledged but not +yet included in the last Lucene commit are instead recovered from the translog +when the shard recovers. -An Elasticsearch flush is the process of performing a Lucene commit and -starting a new translog. Flushes are performed automatically in the background -in order to make sure the translog doesn't grow too large, which would make -replaying its operations take a considerable amount of time during recovery. -The ability to perform a flush manually is also exposed through an API, -although this is rarely needed. +An {es} <> is the process of performing a Lucene commit and +starting a new translog generation. Flushes are performed automatically in the +background in order to make sure the translog does not grow too large, which +would make replaying its operations take a considerable amount of time during +recovery. The ability to perform a flush manually is also exposed through an +API, although this is rarely needed. [float] === Translog settings The data in the translog is only persisted to disk when the translog is -++fsync++ed and committed. In the event of a hardware failure or an operating +++fsync++ed and committed. In the event of a hardware failure or an operating system crash or a JVM crash or a shard failure, any data written since the previous translog commit will be lost. -By default, `index.translog.durability` is set to `request` meaning that Elasticsearch will only report success of an index, delete, -update, or bulk request to the client after the translog has been successfully -++fsync++ed and committed on the primary and on every allocated replica. If -`index.translog.durability` is set to `async` then Elasticsearch ++fsync++s -and commits the translog every `index.translog.sync_interval` (defaults to 5 seconds). +By default, `index.translog.durability` is set to `request` meaning that +Elasticsearch will only report success of an index, delete, update, or bulk +request to the client after the translog has been successfully ++fsync++ed and +committed on the primary and on every allocated replica. If +`index.translog.durability` is set to `async` then Elasticsearch ++fsync++s and +commits the translog only every `index.translog.sync_interval` which means that +any operations that were performed just before a crash may be lost when the node +recovers. The following <> per-index settings control the behaviour of the translog: `index.translog.sync_interval`:: -How often the translog is ++fsync++ed to disk and committed, regardless of -write operations. Defaults to `5s`. Values less than `100ms` are not allowed. + How often the translog is ++fsync++ed to disk and committed, regardless of + write operations. Defaults to `5s`. Values less than `100ms` are not allowed. `index.translog.durability`:: + -- Whether or not to `fsync` and commit the translog after every index, delete, -update, or bulk request. This setting accepts the following parameters: +update, or bulk request. This setting accepts the following parameters: `request`:: - (default) `fsync` and commit after every request. In the event - of hardware failure, all acknowledged writes will already have been - committed to disk. + (default) `fsync` and commit after every request. In the event of hardware + failure, all acknowledged writes will already have been committed to disk. `async`:: @@ -66,33 +68,43 @@ update, or bulk request. This setting accepts the following parameters: `index.translog.flush_threshold_size`:: -The translog stores all operations that are not yet safely persisted in Lucene -(i.e., are not part of a Lucene commit point). Although these operations are -available for reads, they will need to be reindexed if the shard was to -shutdown and has to be recovered. This settings controls the maximum total size -of these operations, to prevent recoveries from taking too long. Once the -maximum size has been reached a flush will happen, generating a new Lucene -commit point. Defaults to `512mb`. + The translog stores all operations that are not yet safely persisted in Lucene + (i.e., are not part of a Lucene commit point). Although these operations are + available for reads, they will need to be replayed if the shard was stopped + and had to be recovered. This setting controls the maximum total size of these + operations, to prevent recoveries from taking too long. Once the maximum size + has been reached a flush will happen, generating a new Lucene commit point. + Defaults to `512mb`. + +[float] +[[index-modules-translog-retention]] +==== Translog retention + +If an index is not using <> to +retain historical operations then {es} recovers each replica shard by replaying +operations from the primary's translog. This means it is important for the +primary to preserve extra operations in its translog in case it needs to +rebuild a replica. Moreover it is important for each replica to preserve extra +operations in its translog in case it is promoted to primary and then needs to +rebuild its own replicas in turn. The following settings control how much +translog is retained for peer recoveries. `index.translog.retention.size`:: -When soft deletes is disabled (enabled by default in 7.0 or later), -`index.translog.retention.size` controls the total size of translog files to keep. -Keeping more translog files increases the chance of performing an operation based -sync when recovering replicas. If the translog files are not sufficient, -replica recovery will fall back to a file based sync. Defaults to `512mb` - -Both `index.translog.retention.size` and `index.translog.retention.age` should not -be specified unless soft deletes is disabled as they will be ignored. - + This controls the total size of translog files to keep for each shard. + Keeping more translog files increases the chance of performing an operation + based sync when recovering a replica. If the translog files are not + sufficient, replica recovery will fall back to a file based sync. Defaults to + `512mb`. This setting is ignored, and should not be set, if soft deletes are + enabled. Soft deletes are enabled by default in indices created in {es} + versions 7.0.0 and later. `index.translog.retention.age`:: -When soft deletes is disabled (enabled by default in 7.0 or later), -`index.translog.retention.age` controls the maximum duration for which translog -files to keep. Keeping more translog files increases the chance of performing an -operation based sync when recovering replicas. If the translog files are not sufficient, -replica recovery will fall back to a file based sync. Defaults to `12h` - -Both `index.translog.retention.size` and `index.translog.retention.age` should not -be specified unless soft deletes is disabled as they will be ignored. + This controls the maximum duration for which translog files are kept by each + shard. Keeping more translog files increases the chance of performing an + operation based sync when recovering replicas. If the translog files are not + sufficient, replica recovery will fall back to a file based sync. Defaults to + `12h`. This setting is ignored, and should not be set, if soft deletes are + enabled. Soft deletes are enabled by default in indices created in {es} + versions 7.0.0 and later. diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 29a5b6d4d28a..333cbc98b8d3 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -1,13 +1,26 @@ [[indices-flush]] === Flush -The flush API allows to flush one or more indices through an API. The -flush process of an index makes sure that any data that is currently only -persisted in the <> is also permanently -persisted in Lucene. This reduces recovery times as that data doesn't need to be -reindexed from the transaction logs after the Lucene indexed is opened. By -default, Elasticsearch uses heuristics in order to automatically -trigger flushes as required. It is rare for users to need to call the API directly. +Flushing an index is the process of making sure that any data that is currently +only stored in the <> is also +permanently stored in the Lucene index. When restarting, {es} replays any +unflushed operations from the transaction log into the Lucene index to bring it +back into the state that it was in before the restart. {es} automatically +triggers flushes as needed, using heuristics that trade off the size of the +unflushed transaction log against the cost of performing each flush. + +Once each operation has been flushed it is permanently stored in the Lucene +index. This may mean that there is no need to maintain an additional copy of it +in the transaction log, unless <>. The transaction log is made up of multiple files, +called _generations_, and {es} will delete any generation files once they are no +longer needed, freeing up disk space. + +It is also possible to trigger a flush on one or more indices using the flush +API, although it is rare for users to need to call this API directly. If you +call the flush API after indexing some documents then a successful response +indicates that {es} has flushed all the documents that were indexed before the +flush API was called. [source,js] -------------------------------------------------- @@ -23,20 +36,22 @@ POST twitter/_flush The flush API accepts the following request parameters: [horizontal] -`wait_if_ongoing`:: If set to `true`(the default value) the flush operation will -block until the flush can be executed if another flush operation is already executing. +`wait_if_ongoing`:: If set to `true` the flush operation will block until the +flush can be executed if another flush operation is already executing. If set to +`false` then an exception will be thrown on the shard level if another flush +operation is already running. Defaults to `true`. -`force`:: Whether a flush should be forced even if it is not necessarily needed i.e. -if no changes will be committed to the index. This is useful if transaction log IDs -should be incremented even if no uncommitted changes are present. -(This setting can be considered as internal) +`force`:: Whether a flush should be forced even if it is not necessarily needed +i.e. if no changes will be committed to the index. This can be used to force +the generation number of the transaction log to be incremented even if no +uncommitted changes are present. This parameter should be considered internal. [float] [[flush-multi-index]] ==== Multi Index -The flush API can be applied to more than one index with a single call, -or even on `_all` the indices. +The flush API can be applied to more than one index with a single call, or even +on `_all` the indices. [source,js] -------------------------------------------------- @@ -50,26 +65,28 @@ POST _flush [[synced-flush-api]] ==== Synced Flush -Elasticsearch tracks the indexing activity of each shard. Shards that have not -received any indexing operations for 5 minutes are automatically marked as inactive. This presents -an opportunity for Elasticsearch to reduce shard resources and also perform -a special kind of flush, called `synced flush`. A synced flush performs a normal flush, then adds -a generated unique marker (sync_id) to all shards. +{es} keeps track of which shards have received indexing activity recently, and +considers shards that have not received any indexing operations for 5 minutes to +be inactive. When a shard becomes inactive {es} performs a special kind of flush +known as a _synced flush_. A synced flush performs a normal +<> on each copy of the shard, and then adds a marker known +as the `sync_id` to each copy to indicate that these copies have identical +Lucene indices. Comparing the `sync_id` markers of the two copies is a very +efficient way to check whether they have identical contents. -Since the sync id marker was added when there were no ongoing indexing operations, it can -be used as a quick way to check if the two shards' lucene indices are identical. This quick sync id -comparison (if present) is used during recovery or restarts to skip the first and -most costly phase of the process. In that case, no segment files need to be copied and -the transaction log replay phase of the recovery can start immediately. Note that since the sync id -marker was applied together with a flush, it is very likely that the transaction log will be empty, -speeding up recoveries even more. +When allocating shard copies, {es} must ensure that each replica contains the +same data as the primary. If the shard copies have been synced-flushed and the +replica shares a `sync_id` with the primary then {es} knows that the two copies +have identical contents. This means there is no need to copy any segment files +from the primary to the replica, which saves a good deal of time during +recoveries and restarts. -This is particularly useful for use cases having lots of indices which are -never or very rarely updated, such as time based data. This use case typically generates lots of indices whose -recovery without the synced flush marker would take a long time. +This is particularly useful for clusters having lots of indices which are very +rarely updated, such as with time-based indices. Without the synced flush +marker, recovery of this kind of cluster would be much slower. -To check whether a shard has a marker or not, look for the `commit` section of shard stats returned by -the <> API: +To check whether a shard has a `sync_id` marker or not, look for the `commit` +section of the shard stats returned by the <> API: [source,sh] -------------------------------------------------- @@ -118,26 +135,26 @@ which returns something similar to: // TESTRESPONSE[s/"sync_id" : "AVvFY-071siAOuFGEO9P"/"sync_id": $body.indices.twitter.shards.0.0.commit.user_data.sync_id/] <1> the `sync id` marker +NOTE: The `sync_id` marker is removed as soon as the shard is flushed again, and +{es} may trigger an automatic flush of a shard at any time if there are +unflushed operations in the shard's translog. In practice this means that one +should consider any indexing operation on an index as having removed its +`sync_id` markers. + [float] ==== Synced Flush API -The Synced Flush API allows an administrator to initiate a synced flush manually. This can be particularly useful for -a planned (rolling) cluster restart where you can stop indexing and don't want to wait the default 5 minutes for -idle indices to be sync-flushed automatically. - -While handy, there are a couple of caveats for this API: - -1. Synced flush is a best effort operation. Any ongoing indexing operations will cause -the synced flush to fail on that shard. This means that some shards may be synced flushed while others aren't. See below for more. -2. The `sync_id` marker is removed as soon as the shard is flushed again. That is because a flush replaces the low level -lucene commit point where the marker is stored. Uncommitted operations in the transaction log do not remove the marker. -In practice, one should consider any indexing operation on an index as removing the marker as a flush can be triggered by Elasticsearch -at any time. - - -NOTE: It is harmless to request a synced flush while there is ongoing indexing. Shards that are idle will succeed and shards - that are not will fail. Any shards that succeeded will have faster recovery times. +The Synced Flush API allows an administrator to initiate a synced flush +manually. This can be particularly useful for a planned cluster restart where +you can stop indexing but don't want to wait for 5 minutes until all indices +are marked as inactive and automatically sync-flushed. +You can request a synced flush even if there is ongoing indexing activity, and +{es} will perform the synced flush on a "best-effort" basis: shards that do not +have any ongoing indexing activity will be successfully sync-flushed, and other +shards will fail to sync-flush. The successfully sync-flushed shards will have +faster recovery times as long as the `sync_id` marker is not removed by a +subsequent flush. [source,sh] -------------------------------------------------- @@ -146,10 +163,11 @@ POST twitter/_flush/synced // CONSOLE // TEST[setup:twitter] -The response contains details about how many shards were successfully sync-flushed and information about any failure. +The response contains details about how many shards were successfully +sync-flushed and information about any failure. -Here is what it looks like when all shards of a two shards and one replica index successfully -sync-flushed: +Here is what it looks like when all shards of a two shards and one replica +index successfully sync-flushed: [source,js] -------------------------------------------------- @@ -168,7 +186,8 @@ sync-flushed: -------------------------------------------------- // TESTRESPONSE[s/"successful": 2/"successful": 1/] -Here is what it looks like when one shard group failed due to pending operations: +Here is what it looks like when one shard group failed due to pending +operations: [source,js] -------------------------------------------------- @@ -193,11 +212,12 @@ Here is what it looks like when one shard group failed due to pending operations -------------------------------------------------- // NOTCONSOLE -NOTE: The above error is shown when the synced flush fails due to concurrent indexing operations. The HTTP -status code in that case will be `409 CONFLICT`. +NOTE: The above error is shown when the synced flush fails due to concurrent +indexing operations. The HTTP status code in that case will be `409 Conflict`. -Sometimes the failures are specific to a shard copy. The copies that failed will not be eligible for -fast recovery but those that succeeded still will be. This case is reported as follows: +Sometimes the failures are specific to a shard copy. The copies that failed +will not be eligible for fast recovery but those that succeeded still will be. +This case is reported as follows: [source,js] -------------------------------------------------- @@ -230,7 +250,8 @@ fast recovery but those that succeeded still will be. This case is reported as f -------------------------------------------------- // NOTCONSOLE -NOTE: When a shard copy fails to sync-flush, the HTTP status code returned will be `409 CONFLICT`. +NOTE: When a shard copy fails to sync-flush, the HTTP status code returned will +be `409 Conflict`. The synced flush API can be applied to more than one index with a single call, or even on `_all` the indices. From bf0b3ce8855c8e6e38e566c51d0ef81a138ca5b7 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Wed, 4 Sep 2019 11:47:32 -0400 Subject: [PATCH 045/103] [DOCS] Reformat "put index template" API docs (#46297) --- docs/reference/indices.asciidoc | 6 +- docs/reference/indices/create-index.asciidoc | 17 +- docs/reference/indices/templates.asciidoc | 181 +++++++++--------- docs/reference/rest-api/common-parms.asciidoc | 27 +++ 4 files changed, 126 insertions(+), 105 deletions(-) diff --git a/docs/reference/indices.asciidoc b/docs/reference/indices.asciidoc index 9eb0bf7c4766..20b1b77d5fd6 100644 --- a/docs/reference/indices.asciidoc +++ b/docs/reference/indices.asciidoc @@ -50,10 +50,10 @@ index settings, aliases, mappings, and index templates. [float] [[index-templates]] === Index templates: +* <> * <> * <> * <> -* <> [float] [[monitoring]] @@ -119,14 +119,14 @@ include::indices/get-settings.asciidoc[] include::indices/analyze.asciidoc[] +include::indices/templates.asciidoc[] + include::indices/delete-index-template.asciidoc[] include::indices/get-index-template.asciidoc[] include::indices/template-exists.asciidoc[] -include::indices/templates.asciidoc[] - include::indices/stats.asciidoc[] include::indices/segments.asciidoc[] diff --git a/docs/reference/indices/create-index.asciidoc b/docs/reference/indices/create-index.asciidoc index 17e629024b7b..a40327be2120 100644 --- a/docs/reference/indices/create-index.asciidoc +++ b/docs/reference/indices/create-index.asciidoc @@ -64,22 +64,9 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=timeoutparms] (Optional, <>) Index aliases which include the index. See <>. -`mappings`:: -+ --- -(Optional, <>) Mapping for fields in the index. If -specified, this mapping can include: +include::{docdir}/rest-api/common-parms.asciidoc[tag=mappings] -* Field names -* <> -* <> - -See <>. --- - -`settings`:: -(Optional, <>) Configuration -options for the index. See <>. +include::{docdir}/rest-api/common-parms.asciidoc[tag=settings] [[indices-create-api-example]] ==== {api-examples-title} diff --git a/docs/reference/indices/templates.asciidoc b/docs/reference/indices/templates.asciidoc index ddd310e10c27..191b2c8f9223 100644 --- a/docs/reference/indices/templates.asciidoc +++ b/docs/reference/indices/templates.asciidoc @@ -1,19 +1,10 @@ [[indices-templates]] -=== Index Templates +=== Put index template API +++++ +Put index template +++++ -// tag::index-template-def[] -Index templates define <> and <> -that you can automatically apply when creating new indices. -{es} applies templates to new indices -based on an index pattern that matches the index name. -// end::index-template-def[] - -NOTE: Templates are only applied at index creation time. Changing a template -will have no impact on existing indices. When using the create index API, the -settings/mappings defined as part of the create index call will take precedence -over any matching settings/mappings defined in the template. - -For example: +Creates or updates an index template. [source,js] -------------------------------------------------- @@ -42,14 +33,96 @@ PUT _template/template_1 // CONSOLE // TESTSETUP -NOTE: Index templates provide C-style /* */ block comments. Comments are allowed -everywhere in the JSON document except before the initial opening curly bracket. -Defines a template named `template_1`, with a template pattern of `te*` or `bar*`. -The settings and mappings will be applied to any index name that matches -the `te*` or `bar*` pattern. +[[put-index-template-api-request]] +==== {api-request-title} -It is also possible to include aliases in an index template as follows: +`PUT /_template/` + + +[[put-index-template-api-desc]] +==== {api-description-title} + +Use the PUT index template API +to create or update an index template. + +// tag::index-template-def[] +Index templates define <> and <> +that you can automatically apply when creating new indices. +{es} applies templates to new indices +based on an index pattern that matches the index name. +// end::index-template-def[] + +Index templates are only applied during index creation. +Changes to index templates do not affect existing indices. +Settings and mappings specified in <> API requests +override any settings or mappings specified in an index template. + +===== Comments in index templates +You can use C-style /* */ block comments in index templates. +You can includes comments anywhere in the request body, +except before the opening curly bracket. + +[[getting]] +===== Getting templates + +See <>. + + +[[put-index-template-api-path-params]] +==== {api-path-parms-title} + +``:: +(Required, string) +Name of the index template to create. + + +[[put-index-template-api-query-params]] +==== {api-query-parms-title} + +`create`:: +(Optional, boolean) +If `true`, this request cannot replace or update existing index templates. +Defaults to `false`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=flat-settings] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=include-type-name] + +`order`:: +(Optional,integer) +Order in which {es} applies this template +if index matches multiple templates. ++ +Templates with lower `order` values are merged first. +Templates with higher `order` values are merged later, +overriding templates with lower values. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=timeoutparms] + + +[[put-index-template-api-request-body]] +==== {api-request-body-title} + +`index_patterns`:: +(Required, array of strings) +Array of wildcard expressions +used to match the names of indices during creation. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=aliases] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=mappings] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=settings] + + + +[[put-index-template-api-example]] +==== {api-examples-title} + +===== Index template with index aliases + +You can include <> in an index template. [source,js] -------------------------------------------------- @@ -77,28 +150,9 @@ PUT _template/template_1 <1> the `{index}` placeholder in the alias name will be replaced with the actual index name that the template gets applied to, during index creation. -[float] -[[delete]] -==== Deleting a Template -Index templates are identified by a name (in the above case -`template_1`) and can be deleted as well: - -[source,js] --------------------------------------------------- -DELETE /_template/template_1 --------------------------------------------------- -// CONSOLE - -[float] -[[getting]] -==== Getting templates - -See <>. - -[float] [[multiple-templates]] -==== Multiple Templates Matching +===== Indices matching multiple templates Multiple index templates can potentially match an index, in this case, both the settings and mappings are merged into the final configuration @@ -143,50 +197,3 @@ order templates, with lower order templates providing the basis. NOTE: Multiple matching templates with the same order value will result in a non-deterministic merging order. - -[float] -[[versioning-templates]] -==== Template Versioning - -Templates can optionally add a `version` number, which can be any integer value, -in order to simplify template management by external systems. The `version` -field is completely optional and it is meant solely for external management of -templates. To unset a `version`, simply replace the template without specifying -one. - -[source,js] --------------------------------------------------- -PUT /_template/template_1 -{ - "index_patterns" : ["*"], - "order" : 0, - "settings" : { - "number_of_shards" : 1 - }, - "version": 123 -} --------------------------------------------------- -// CONSOLE - -To check the `version`, you can -<> -using `filter_path` to limit the response to just the `version`: - -[source,js] --------------------------------------------------- -GET /_template/template_1?filter_path=*.version --------------------------------------------------- -// CONSOLE -// TEST[continued] - -This should give a small response that makes it both easy and inexpensive to parse: - -[source,js] --------------------------------------------------- -{ - "template_1" : { - "version" : 123 - } -} --------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index f281c2112fec..7d8a347507de 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -4,6 +4,12 @@ Comma-separated list or wildcard expression of index alias names used to limit the request. end::index-alias[] +tag::aliases[] +`aliases`:: +(Optional, <>) Index aliases which include the +index. See <>. +end::aliases[] + tag::allow-no-indices[] `allow_no_indices`:: (Optional, boolean) If `true`, the request returns an error if a wildcard @@ -161,6 +167,21 @@ node only. Defaults to `false`, which means information is retrieved from the master node. end::local[] +tag::mappings[] +`mappings`:: ++ +-- +(Optional, <>) Mapping for fields in the index. If +specified, this mapping can include: + +* Field names +* <> +* <> + +See <>. +-- +end::mappings[] + tag::max_docs[] `max_docs`:: (Optional, integer) Maximum number of documents to process. Defaults to all @@ -250,6 +271,12 @@ tag::search_type[] * `dfs_query_then_fetch` end::search_type[] +tag::settings[] +`settings`:: +(Optional, <>) Configuration +options for the index. See <>. +end::settings[] + tag::slices[] `slices`:: (Optional, integer) The number of slices this task should be divided into. From ba90ad9049c58df80fd68856255846505bb58ce2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 4 Sep 2019 11:52:42 -0400 Subject: [PATCH 046/103] Add test that get triggers shard search active (#46317) This commit is a follow-up to a change that fixed that multi-get was not triggering a shard to become search active. In that change, we added a test that multi-get properly triggers a shard to become search active. This commit is a follow-up to that change which adds a test for the get case. While get is already handled correctly in production code, there was not a test for it. This commit adds one. Additionally, we factor all the search idle tests from IndexShardIT into a separate test class, as an effort to keep related tests together instead of a single large test class containing a jumble of tests, and also to keep test classes smaller for better parallelization. --- .../index/shard/IndexShardIT.java | 136 ------------ .../index/shard/SearchIdleIT.java | 193 ++++++++++++++++++ 2 files changed, 193 insertions(+), 136 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/index/shard/SearchIdleIT.java diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index d513c80fcf25..71b6cd1124e8 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -26,9 +26,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; -import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -84,7 +82,6 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.InternalSettingsPlugin; -import org.elasticsearch.threadpool.ThreadPool; import org.junit.Assert; import java.io.IOException; @@ -101,15 +98,11 @@ import java.util.Locale; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.Phaser; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.IntToLongFunction; import java.util.function.Predicate; import java.util.stream.Stream; @@ -127,7 +120,6 @@ import static org.elasticsearch.index.shard.IndexShardTestCase.getTranslog; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoSearchHits; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.either; @@ -689,134 +681,6 @@ public class IndexShardIT extends ESSingleNodeTestCase { return shardRouting; } - public void testAutomaticRefreshSearch() throws InterruptedException { - runTestAutomaticRefresh(numDocs -> client().prepareSearch("test").get().getHits().getTotalHits().value); - } - - public void testAutomaticRefreshMultiGet() throws InterruptedException { - runTestAutomaticRefresh( - numDocs -> { - final MultiGetRequest request = new MultiGetRequest(); - request.realtime(false); - for (int i = 0; i < numDocs; i++) { - request.add("test", "" + i); - } - return Arrays.stream(client().multiGet(request).actionGet().getResponses()).filter(r -> r.getResponse().isExists()).count(); - }); - } - - private void runTestAutomaticRefresh(final IntToLongFunction count) throws InterruptedException { - TimeValue randomTimeValue = randomFrom(random(), null, TimeValue.ZERO, TimeValue.timeValueMillis(randomIntBetween(0, 1000))); - Settings.Builder builder = Settings.builder(); - if (randomTimeValue != null) { - builder.put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), randomTimeValue); - } - IndexService indexService = createIndex("test", builder.build()); - assertFalse(indexService.getIndexSettings().isExplicitRefresh()); - ensureGreen(); - AtomicInteger totalNumDocs = new AtomicInteger(Integer.MAX_VALUE); - assertNoSearchHits(client().prepareSearch().get()); - int numDocs = scaledRandomIntBetween(25, 100); - totalNumDocs.set(numDocs); - CountDownLatch indexingDone = new CountDownLatch(numDocs); - client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); - indexingDone.countDown(); // one doc is indexed above blocking - IndexShard shard = indexService.getShard(0); - boolean hasRefreshed = shard.scheduledRefresh(); - if (randomTimeValue == TimeValue.ZERO) { - // with ZERO we are guaranteed to see the doc since we will wait for a refresh in the background - assertFalse(hasRefreshed); - assertTrue(shard.isSearchIdle()); - } else { - if (randomTimeValue == null) { - assertFalse(shard.isSearchIdle()); - } - // we can't assert on hasRefreshed since it might have been refreshed in the background on the shard concurrently. - // and if the background refresh wins the refresh race (both call maybeRefresh), the document might not be visible - // until the background refresh is done. - if (hasRefreshed == false) { - ensureNoPendingScheduledRefresh(indexService.getThreadPool()); - } - } - - CountDownLatch started = new CountDownLatch(1); - Thread t = new Thread(() -> { - started.countDown(); - do { - - } while (count.applyAsLong(totalNumDocs.get()) != totalNumDocs.get()); - }); - t.start(); - started.await(); - assertThat(count.applyAsLong(totalNumDocs.get()), equalTo(1L)); - for (int i = 1; i < numDocs; i++) { - client().prepareIndex("test", "test", "" + i).setSource("{\"foo\" : \"bar\"}", XContentType.JSON) - .execute(new ActionListener() { - @Override - public void onResponse(IndexResponse indexResponse) { - indexingDone.countDown(); - } - - @Override - public void onFailure(Exception e) { - indexingDone.countDown(); - throw new AssertionError(e); - } - }); - } - indexingDone.await(); - t.join(); - } - - public void testPendingRefreshWithIntervalChange() throws Exception { - Settings.Builder builder = Settings.builder(); - builder.put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), TimeValue.ZERO); - IndexService indexService = createIndex("test", builder.build()); - assertFalse(indexService.getIndexSettings().isExplicitRefresh()); - ensureGreen(); - client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); - IndexShard shard = indexService.getShard(0); - assertFalse(shard.scheduledRefresh()); - assertTrue(shard.isSearchIdle()); - CountDownLatch refreshLatch = new CountDownLatch(1); - client().admin().indices().prepareRefresh() - .execute(ActionListener.wrap(refreshLatch::countDown));// async on purpose to make sure it happens concurrently - assertHitCount(client().prepareSearch().get(), 1); - client().prepareIndex("test", "test", "1").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); - assertFalse(shard.scheduledRefresh()); - - // now disable background refresh and make sure the refresh happens - CountDownLatch updateSettingsLatch = new CountDownLatch(1); - client().admin().indices() - .prepareUpdateSettings("test") - .setSettings(Settings.builder().put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), -1).build()) - .execute(ActionListener.wrap(updateSettingsLatch::countDown)); - assertHitCount(client().prepareSearch().get(), 2); - // wait for both to ensure we don't have in-flight operations - updateSettingsLatch.await(); - refreshLatch.await(); - // We need to ensure a `scheduledRefresh` triggered by the internal refresh setting update is executed before we index a new doc; - // otherwise, it will compete to call `Engine#maybeRefresh` with the `scheduledRefresh` that we are going to verify. - ensureNoPendingScheduledRefresh(indexService.getThreadPool()); - client().prepareIndex("test", "test", "2").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); - assertTrue(shard.scheduledRefresh()); - assertTrue(shard.isSearchIdle()); - assertHitCount(client().prepareSearch().get(), 3); - } - - private void ensureNoPendingScheduledRefresh(ThreadPool threadPool) { - // We can make sure that all scheduled refresh tasks are done by submitting *maximumPoolSize* blocking tasks, - // then wait until all of them completed. Note that using ThreadPoolStats is not watertight as both queue and - // active count can be 0 when ThreadPoolExecutor just takes a task out the queue but before marking it active. - ThreadPoolExecutor refreshThreadPoolExecutor = (ThreadPoolExecutor) threadPool.executor(ThreadPool.Names.REFRESH); - int maximumPoolSize = refreshThreadPoolExecutor.getMaximumPoolSize(); - Phaser barrier = new Phaser(maximumPoolSize + 1); - for (int i = 0; i < maximumPoolSize; i++) { - refreshThreadPoolExecutor.execute(barrier::arriveAndAwaitAdvance); - } - barrier.arriveAndAwaitAdvance(); - } - public void testGlobalCheckpointListeners() throws Exception { createIndex("test", Settings.builder() .put("index.number_of_shards", 1) diff --git a/server/src/test/java/org/elasticsearch/index/shard/SearchIdleIT.java b/server/src/test/java/org/elasticsearch/index/shard/SearchIdleIT.java new file mode 100644 index 000000000000..96f10df0084e --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/shard/SearchIdleIT.java @@ -0,0 +1,193 @@ +/* + * 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.index.shard; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.MultiGetRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Phaser; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.IntToLongFunction; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoSearchHits; +import static org.hamcrest.Matchers.equalTo; + +public class SearchIdleIT extends ESSingleNodeTestCase { + + public void testAutomaticRefreshSearch() throws InterruptedException { + runTestAutomaticRefresh(numDocs -> client().prepareSearch("test").get().getHits().getTotalHits().value); + } + + public void testAutomaticRefreshGet() throws InterruptedException { + runTestAutomaticRefresh( + numDocs -> { + int count = 0; + for (int i = 0; i < numDocs; i++) { + final GetRequest request = new GetRequest(); + request.realtime(false); + request.index("test"); + request.id("" + i); + if (client().get(request).actionGet().isExists()) { + count++; + } + } + return count; + }); + } + + public void testAutomaticRefreshMultiGet() throws InterruptedException { + runTestAutomaticRefresh( + numDocs -> { + final MultiGetRequest request = new MultiGetRequest(); + request.realtime(false); + for (int i = 0; i < numDocs; i++) { + request.add("test", "" + i); + } + return Arrays.stream(client().multiGet(request).actionGet().getResponses()).filter(r -> r.getResponse().isExists()).count(); + }); + } + + private void runTestAutomaticRefresh(final IntToLongFunction count) throws InterruptedException { + TimeValue randomTimeValue = randomFrom(random(), null, TimeValue.ZERO, TimeValue.timeValueMillis(randomIntBetween(0, 1000))); + Settings.Builder builder = Settings.builder(); + if (randomTimeValue != null) { + builder.put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), randomTimeValue); + } + IndexService indexService = createIndex("test", builder.build()); + assertFalse(indexService.getIndexSettings().isExplicitRefresh()); + ensureGreen(); + AtomicInteger totalNumDocs = new AtomicInteger(Integer.MAX_VALUE); + assertNoSearchHits(client().prepareSearch().get()); + int numDocs = scaledRandomIntBetween(25, 100); + totalNumDocs.set(numDocs); + CountDownLatch indexingDone = new CountDownLatch(numDocs); + client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); + indexingDone.countDown(); // one doc is indexed above blocking + IndexShard shard = indexService.getShard(0); + boolean hasRefreshed = shard.scheduledRefresh(); + if (randomTimeValue == TimeValue.ZERO) { + // with ZERO we are guaranteed to see the doc since we will wait for a refresh in the background + assertFalse(hasRefreshed); + assertTrue(shard.isSearchIdle()); + } else { + if (randomTimeValue == null) { + assertFalse(shard.isSearchIdle()); + } + // we can't assert on hasRefreshed since it might have been refreshed in the background on the shard concurrently. + // and if the background refresh wins the refresh race (both call maybeRefresh), the document might not be visible + // until the background refresh is done. + if (hasRefreshed == false) { + ensureNoPendingScheduledRefresh(indexService.getThreadPool()); + } + } + + CountDownLatch started = new CountDownLatch(1); + Thread t = new Thread(() -> { + started.countDown(); + do { + + } while (count.applyAsLong(totalNumDocs.get()) != totalNumDocs.get()); + }); + t.start(); + started.await(); + assertThat(count.applyAsLong(totalNumDocs.get()), equalTo(1L)); + for (int i = 1; i < numDocs; i++) { + client().prepareIndex("test", "test", "" + i).setSource("{\"foo\" : \"bar\"}", XContentType.JSON) + .execute(new ActionListener() { + @Override + public void onResponse(IndexResponse indexResponse) { + indexingDone.countDown(); + } + + @Override + public void onFailure(Exception e) { + indexingDone.countDown(); + throw new AssertionError(e); + } + }); + } + indexingDone.await(); + t.join(); + } + + + public void testPendingRefreshWithIntervalChange() throws Exception { + Settings.Builder builder = Settings.builder(); + builder.put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), TimeValue.ZERO); + IndexService indexService = createIndex("test", builder.build()); + assertFalse(indexService.getIndexSettings().isExplicitRefresh()); + ensureGreen(); + client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); + IndexShard shard = indexService.getShard(0); + assertFalse(shard.scheduledRefresh()); + assertTrue(shard.isSearchIdle()); + CountDownLatch refreshLatch = new CountDownLatch(1); + client().admin().indices().prepareRefresh() + .execute(ActionListener.wrap(refreshLatch::countDown));// async on purpose to make sure it happens concurrently + assertHitCount(client().prepareSearch().get(), 1); + client().prepareIndex("test", "test", "1").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); + assertFalse(shard.scheduledRefresh()); + + // now disable background refresh and make sure the refresh happens + CountDownLatch updateSettingsLatch = new CountDownLatch(1); + client().admin().indices() + .prepareUpdateSettings("test") + .setSettings(Settings.builder().put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), -1).build()) + .execute(ActionListener.wrap(updateSettingsLatch::countDown)); + assertHitCount(client().prepareSearch().get(), 2); + // wait for both to ensure we don't have in-flight operations + updateSettingsLatch.await(); + refreshLatch.await(); + // We need to ensure a `scheduledRefresh` triggered by the internal refresh setting update is executed before we index a new doc; + // otherwise, it will compete to call `Engine#maybeRefresh` with the `scheduledRefresh` that we are going to verify. + ensureNoPendingScheduledRefresh(indexService.getThreadPool()); + client().prepareIndex("test", "test", "2").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).get(); + assertTrue(shard.scheduledRefresh()); + assertTrue(shard.isSearchIdle()); + assertHitCount(client().prepareSearch().get(), 3); + } + + private void ensureNoPendingScheduledRefresh(ThreadPool threadPool) { + // We can make sure that all scheduled refresh tasks are done by submitting *maximumPoolSize* blocking tasks, + // then wait until all of them completed. Note that using ThreadPoolStats is not watertight as both queue and + // active count can be 0 when ThreadPoolExecutor just takes a task out the queue but before marking it active. + ThreadPoolExecutor refreshThreadPoolExecutor = (ThreadPoolExecutor) threadPool.executor(ThreadPool.Names.REFRESH); + int maximumPoolSize = refreshThreadPoolExecutor.getMaximumPoolSize(); + Phaser barrier = new Phaser(maximumPoolSize + 1); + for (int i = 0; i < maximumPoolSize; i++) { + refreshThreadPoolExecutor.execute(barrier::arriveAndAwaitAdvance); + } + barrier.arriveAndAwaitAdvance(); + } + +} From d528da3764ad23e1aca0f6117afa5cce948a5490 Mon Sep 17 00:00:00 2001 From: Aleh Zasypkin Date: Wed, 4 Sep 2019 18:20:57 +0200 Subject: [PATCH 047/103] Document support of OIDC Implicit flow in Kibana. (#45693) --- .../settings/security-settings.asciidoc | 6 ++--- .../authentication/oidc-guide.asciidoc | 23 +++++++++---------- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/docs/reference/settings/security-settings.asciidoc b/docs/reference/settings/security-settings.asciidoc index bcde84940208..963b00de2077 100644 --- a/docs/reference/settings/security-settings.asciidoc +++ b/docs/reference/settings/security-settings.asciidoc @@ -1280,9 +1280,9 @@ The OAuth 2.0 Client Secret that was assigned to {es} during registration at the OpenID Connect Provider `rp.redirect_uri`:: -The Redirect URI within {kib}. Typically this is the -"api/security/v1/oidc" endpoint of your Kibana server. For example, -`https://kibana.example.com/api/security/v1/oidc`. +The Redirect URI within {kib}. If you want to use the authorization code flow, this is the +"api/security/v1/oidc" endpoint of your {kib} server. If you want to use the implicit flow, it is the "api/security/v1/oidc/implicit" endpoint. +For example, `https://kibana.example.com/api/security/v1/oidc`. `rp.response_type`:: OAuth 2.0 Response Type value that determines the authorization diff --git a/x-pack/docs/en/security/authentication/oidc-guide.asciidoc b/x-pack/docs/en/security/authentication/oidc-guide.asciidoc index 6cc59f823165..9a73cf072451 100644 --- a/x-pack/docs/en/security/authentication/oidc-guide.asciidoc +++ b/x-pack/docs/en/security/authentication/oidc-guide.asciidoc @@ -5,8 +5,7 @@ The Elastic Stack supports single sign-on (SSO) using OpenID Connect via {kib} using {es} as the backend service that holds most of the functionality. {kib} and {es} -together represent an OpenID Connect Relying Party (RP) that supports the Authorization -Code Flow as this is defined in the OpenID Connect specification. +together represent an OpenID Connect Relying Party (RP) that supports the authorization code flow and implicit flow as these are defined in the OpenID Connect specification. This guide assumes that you have an OpenID Connect Provider where the Elastic Stack Relying Party will be registered. @@ -39,8 +38,8 @@ RP that you commonly need to provide for registration are the following: nor the Elastic Stack implementation impose any constraints on this value. - `Redirect URI`: This is the URI where the OP will redirect the user's browser after authentication. The appropriate value for this will depend on your setup and whether or not {kib} sits behind a proxy or -load balancer. It will typically be +$\{kibana-url}/api/security/v1/oidc+ where _$\{kibana-url}_ -is the base URL for your {kib} instance. You might also see this called `Callback URI`. +load balancer. It will typically be +$\{kibana-url}/api/security/v1/oidc+ (for the authorization code flow) or +$\{kibana-url}/api/security/v1/oidc/implicit+ (for the implicit flow) where _$\{kibana-url}_ is the base URL for your {kib} instance. You might also see this +called `Callback URI`. At the end of the registration process, the OP will assign a Client Identifier and a Client Secret for the RP ({stack}) to use. Note these two values as they will be used in the {es} configuration. @@ -158,8 +157,8 @@ op.authorization_endpoint:: op.token_endpoint:: The URL for the Token Endpoint in the OpenID Connect Provider. This is the endpoint where - {es} will send a request to exchange the code for an ID Token, in the case where the Authorization Code - flow is used. The value for this setting should be provided by your OpenID Connect Provider. + {es} will send a request to exchange the code for an ID Token. This setting is optional when + you use the implicit flow. The value for this setting should be provided by your OpenID Connect Provider. op.jwkset_path:: The path to a file or a URL containing a JSON Web Key Set with the key material that the OpenID Connect @@ -522,23 +521,23 @@ The three additional settings that are required for OpenID Connect support are s [source, yaml] ------------------------------------------------------------ -xpack.security.authProviders: [oidc] +xpack.security.authc.providers: [oidc] xpack.security.authc.oidc.realm: "oidc1" server.xsrf.whitelist: [/api/security/v1/oidc] ------------------------------------------------------------ The configuration values used in the example above are: -`xpack.security.authProviders`:: +`xpack.security.authc.providers`:: Set this to `[ oidc ]` to instruct {kib} to use OpenID Connect single sign-on as the authentication method. This instructs Kibana to attempt to initiate an SSO flow everytime a user attempts to access a URL in Kibana, if the user is not already authenticated. If you also want to allow users to login with a username and password, -you must enable the `basic` authProvider too. For example: +you must enable the `basic` authentication provider too. For example: [source, yaml] ------------------------------------------------------------ -xpack.security.authProviders: [oidc, basic] +xpack.security.authc.providers: [oidc, basic] ------------------------------------------------------------ This will allow users that haven't already authenticated with OpenID Connect to @@ -552,8 +551,8 @@ for this Kibana instance. {kib} has in-built protection against _Cross Site Request Forgery_ attacks, which is designed to prevent the {kib} server from processing requests that originated from outside the {kib} application. -In order to support OpenID Connect messages that originate from your -OP or a third party (see <>, we need to explicitly _whitelist_ the +If you use the authorization code flow, {kib} needs to support unsolicited messages that originate from your +OP or a third party (see <>). In order to do so, you must explicitly _whitelist_ the OpenID Connect authentication endpoint within {kib}, so that the {kib} server will not reject these external messages. From f5827ba0aed662c90648dcae168750396d59877d Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Wed, 4 Sep 2019 12:51:02 -0400 Subject: [PATCH 048/103] [DOCS] Replace "// CONSOLE" comments with [source,console] (#46159) --- ...ackup-and-restore-security-config.asciidoc | 15 ++--- .../adjacency-matrix-aggregation.asciidoc | 3 +- .../autodatehistogram-aggregation.asciidoc | 18 ++---- .../bucket/children-aggregation.asciidoc | 12 ++-- .../bucket/composite-aggregation.asciidoc | 39 ++++------- .../bucket/datehistogram-aggregation.asciidoc | 33 ++++------ .../bucket/daterange-aggregation.asciidoc | 15 ++--- .../diversified-sampler-aggregation.asciidoc | 6 +- .../bucket/filter-aggregation.asciidoc | 3 +- .../bucket/filters-aggregation.asciidoc | 9 +-- .../bucket/geodistance-aggregation.asciidoc | 15 ++--- .../bucket/geohashgrid-aggregation.asciidoc | 9 +-- .../bucket/geotilegrid-aggregation.asciidoc | 6 +- .../bucket/global-aggregation.asciidoc | 3 +- .../bucket/histogram-aggregation.asciidoc | 15 ++--- .../bucket/iprange-aggregation.asciidoc | 12 ++-- .../bucket/missing-aggregation.asciidoc | 3 +- .../bucket/nested-aggregation.asciidoc | 6 +- .../bucket/parent-aggregation.asciidoc | 12 ++-- .../bucket/range-aggregation.asciidoc | 27 +++----- .../bucket/rare-terms-aggregation.asciidoc | 15 ++--- .../reverse-nested-aggregation.asciidoc | 10 ++- .../bucket/sampler-aggregation.asciidoc | 6 +- .../significantterms-aggregation.asciidoc | 21 +++--- .../significanttext-aggregation.asciidoc | 16 ++--- .../bucket/terms-aggregation.asciidoc | 64 +++++++------------ .../matrix/stats-aggregation.asciidoc | 6 +- .../metrics/avg-aggregation.asciidoc | 15 ++--- .../metrics/cardinality-aggregation.asciidoc | 15 ++--- .../extendedstats-aggregation.asciidoc | 18 ++---- .../metrics/geobounds-aggregation.asciidoc | 3 +- .../metrics/geocentroid-aggregation.asciidoc | 6 +- .../metrics/max-aggregation.asciidoc | 15 ++--- ...an-absolute-deviation-aggregation.asciidoc | 15 ++--- .../metrics/min-aggregation.asciidoc | 15 ++--- .../metrics/percentile-aggregation.asciidoc | 24 +++---- .../percentile-rank-aggregation.asciidoc | 22 +++---- .../scripted-metric-aggregation.asciidoc | 9 +-- .../metrics/stats-aggregation.asciidoc | 15 ++--- .../metrics/sum-aggregation.asciidoc | 15 ++--- .../metrics/tophits-aggregation.asciidoc | 16 ++--- .../metrics/valuecount-aggregation.asciidoc | 9 +-- .../metrics/weighted-avg-aggregation.asciidoc | 12 ++-- docs/reference/aggregations/misc.asciidoc | 9 +-- docs/reference/aggregations/pipeline.asciidoc | 20 +++--- .../pipeline/avg-bucket-aggregation.asciidoc | 4 +- .../bucket-script-aggregation.asciidoc | 3 +- .../bucket-selector-aggregation.asciidoc | 3 +- .../pipeline/bucket-sort-aggregation.asciidoc | 7 +- ...umulative-cardinality-aggregation.asciidoc | 6 +- .../cumulative-sum-aggregation.asciidoc | 3 +- .../pipeline/derivative-aggregation.asciidoc | 9 +-- ...extended-stats-bucket-aggregation.asciidoc | 3 +- .../pipeline/max-bucket-aggregation.asciidoc | 3 +- .../pipeline/min-bucket-aggregation.asciidoc | 3 +- .../pipeline/movfn-aggregation.asciidoc | 33 ++++------ .../percentiles-bucket-aggregation.asciidoc | 3 +- .../pipeline/serial-diff-aggregation.asciidoc | 3 +- .../stats-bucket-aggregation.asciidoc | 3 +- .../pipeline/sum-bucket-aggregation.asciidoc | 3 +- .../reference/search/suggesters/misc.asciidoc | 3 +- .../search/suggesters/phrase-suggest.asciidoc | 16 ++--- docs/reference/search/uri-request.asciidoc | 6 +- docs/reference/search/validate.asciidoc | 21 ++---- .../setup/install/check-running.asciidoc | 3 +- docs/reference/setup/logging-config.asciidoc | 3 +- docs/reference/setup/secure-settings.asciidoc | 5 +- .../setup/sysconfig/file-descriptors.asciidoc | 3 +- docs/reference/setup/sysconfig/swap.asciidoc | 3 +- docs/reference/sql/endpoints/rest.asciidoc | 33 ++++------ .../sql/endpoints/translate.asciidoc | 3 +- docs/reference/sql/getting-started.asciidoc | 6 +- docs/reference/upgrade/close-ml.asciidoc | 6 +- .../upgrade/cluster_restart.asciidoc | 9 +-- .../upgrade/disable-shard-alloc.asciidoc | 3 +- docs/reference/upgrade/open-ml.asciidoc | 3 +- .../upgrade/reindex_upgrade.asciidoc | 3 +- .../upgrade/rolling_upgrade.asciidoc | 12 ++-- docs/reference/upgrade/synced-flush.asciidoc | 3 +- .../vectors/vector-functions.asciidoc | 28 +++----- 80 files changed, 318 insertions(+), 595 deletions(-) diff --git a/docs/reference/administering/backup-and-restore-security-config.asciidoc b/docs/reference/administering/backup-and-restore-security-config.asciidoc index 847557db4861..6b9ad8cd07e3 100644 --- a/docs/reference/administering/backup-and-restore-security-config.asciidoc +++ b/docs/reference/administering/backup-and-restore-security-config.asciidoc @@ -75,7 +75,7 @@ It is preferable to have a <> for this special index. If you wish, you can also snapshot the system indices for other {stack} components to this repository. + -- -[source,js] +[source,console] ----------------------------------- PUT /_snapshot/my_backup { @@ -85,7 +85,6 @@ PUT /_snapshot/my_backup } } ----------------------------------- -// CONSOLE The user calling this API must have the elevated `manage` cluster privilege to prevent non-administrators exfiltrating data. @@ -99,7 +98,7 @@ The following example creates a new user `snapshot_user` in the {stack-ov}/native-realm.html[native realm], but it is not important which realm the user is a member of: -[source,js] +[source,console] -------------------------------------------------- POST /_security/user/snapshot_user { @@ -107,7 +106,6 @@ POST /_security/user/snapshot_user "roles" : [ "snapshot_user" ] } -------------------------------------------------- -// CONSOLE // TEST[skip:security is not enabled in this fixture] -- @@ -118,7 +116,7 @@ POST /_security/user/snapshot_user The following example shows how to use the create snapshot API to backup the `.security` index to the `my_backup` repository: -[source,js] +[source,console] -------------------------------------------------- PUT /_snapshot/my_backup/snapshot_1 { @@ -126,7 +124,6 @@ PUT /_snapshot/my_backup/snapshot_1 "include_global_state": true <1> } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> This parameter value captures all the persistent settings stored in the @@ -189,18 +186,16 @@ the {security-features}. To restore your security configuration from a backup, first make sure that the repository holding `.security` snapshots is installed: -[source,js] +[source,console] -------------------------------------------------- GET /_snapshot/my_backup -------------------------------------------------- -// CONSOLE // TEST[continued] -[source,js] +[source,console] -------------------------------------------------- GET /_snapshot/my_backup/snapshot_1 -------------------------------------------------- -// CONSOLE // TEST[continued] Then log into one of the node hosts, navigate to {es} installation directory, diff --git a/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc b/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc index 5cd67877460b..8ed900a24fa0 100644 --- a/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc @@ -28,7 +28,7 @@ other than the default of the ampersand. Example: -[source,js] +[source,console] -------------------------------------------------- PUT /emails/_bulk?refresh { "index" : { "_id" : 1 } } @@ -54,7 +54,6 @@ GET emails/_search } } -------------------------------------------------- -// CONSOLE In the above example, we analyse email messages to see which groups of individuals have exchanged messages. diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index c09d1a67bf25..575a3e75894c 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -10,7 +10,7 @@ The buckets field is optional, and will default to 10 buckets if not specified. Requesting a target of 10 buckets. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -24,7 +24,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Keys @@ -37,7 +36,7 @@ date string using the format specified with the `format` parameter: TIP: If no `format` is specified, then it will use the first date <> specified in the field mapping. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -52,7 +51,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Supports expressive date <> @@ -119,7 +117,7 @@ Time zones may either be specified as an ISO 8601 UTC offset (e.g. `+01:00` or Consider the following example: -[source,js] +[source,console] --------------------------------- PUT my_index/log/1?refresh { @@ -148,7 +146,6 @@ GET my_index/_search?size=0 } } --------------------------------- -// CONSOLE UTC is used if no time zone is specified, three 1-hour buckets are returned starting at midnight UTC on 1 October 2015: @@ -186,7 +183,7 @@ starting at midnight UTC on 1 October 2015: If a `time_zone` of `-01:00` is specified, then midnight starts at one hour before midnight UTC: -[source,js] +[source,console] --------------------------------- GET my_index/_search?size=0 { @@ -201,7 +198,6 @@ GET my_index/_search?size=0 } } --------------------------------- -// CONSOLE // TEST[continued] @@ -273,7 +269,7 @@ The accepted units for `minimum_interval` are: * minute * second -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -288,7 +284,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Missing value @@ -297,7 +292,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -312,7 +307,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `publish_date` field will fall into the same bucket as documents that have the value `2000-01-01`. diff --git a/docs/reference/aggregations/bucket/children-aggregation.asciidoc b/docs/reference/aggregations/bucket/children-aggregation.asciidoc index d5ac6b5e09d1..67d75c39448b 100644 --- a/docs/reference/aggregations/bucket/children-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/children-aggregation.asciidoc @@ -9,7 +9,7 @@ This aggregation has a single option: For example, let's say we have an index of questions and answers. The answer type has the following `join` field in the mapping: -[source,js] +[source,console] -------------------------------------------------- PUT child_example { @@ -25,7 +25,6 @@ PUT child_example } } -------------------------------------------------- -// CONSOLE The `question` document contain a tag field and the `answer` documents contain an owner field. With the `children` aggregation the tag buckets can be mapped to the owner buckets in a single request even though the two fields exist in @@ -33,7 +32,7 @@ two different kinds of documents. An example of a question document: -[source,js] +[source,console] -------------------------------------------------- PUT child_example/_doc/1 { @@ -49,12 +48,11 @@ PUT child_example/_doc/1 ] } -------------------------------------------------- -// CONSOLE // TEST[continued] Examples of `answer` documents: -[source,js] +[source,console] -------------------------------------------------- PUT child_example/_doc/2?routing=1 { @@ -86,12 +84,11 @@ PUT child_example/_doc/3?routing=1&refresh "creation_date": "2009-05-05T13:45:37.030" } -------------------------------------------------- -// CONSOLE // TEST[continued] The following request can be built that connects the two together: -[source,js] +[source,console] -------------------------------------------------- POST child_example/_search?size=0 { @@ -120,7 +117,6 @@ POST child_example/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The `type` points to type / mapping with the name `answer`. diff --git a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc index 44a0ac79929c..2c50c844940f 100644 --- a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc @@ -112,7 +112,7 @@ The values are extracted from a field or a script exactly like the `terms` aggre Example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -127,11 +127,10 @@ GET /_search } } -------------------------------------------------- -// CONSOLE Like the `terms` aggregation it is also possible to use a script to create the values for the composite buckets: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -155,7 +154,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE ===== Histogram @@ -166,7 +164,7 @@ a value of `101` would be translated to `100` which is the key for the interval Example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -181,11 +179,10 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The values are built from a numeric field or a script that return numerical values: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -210,7 +207,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE ===== Date Histogram @@ -218,7 +214,7 @@ GET /_search The `date_histogram` is similar to the `histogram` value source except that the interval is specified by date/time expression: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -233,7 +229,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The example above creates an interval per day and translates all `timestamp` values to the start of its closest intervals. Available expressions for interval: `year`, `quarter`, `month`, `week`, `day`, `hour`, `minute`, `second` @@ -248,7 +243,7 @@ Internally, a date is represented as a 64 bit number representing a timestamp in These timestamps are returned as the bucket keys. It is possible to return a formatted date string instead using the format specified with the format parameter: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -271,7 +266,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> Supports expressive date <> @@ -291,7 +285,7 @@ The `sources` parameter accepts an array of values source. It is possible to mix different values source to create composite buckets. For example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -307,14 +301,13 @@ GET /_search } } -------------------------------------------------- -// CONSOLE This will create composite buckets from the values created by two values source, a `date_histogram` and a `terms`. Each bucket is composed of two values, one for each value source defined in the aggregation. Any type of combinations is allowed and the order in the array is preserved in the composite buckets. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -331,7 +324,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE ==== Order @@ -344,7 +336,7 @@ It is possible to define the direction of the sort for each value source by sett or `desc` (descending order) directly in the value source definition. For example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -360,7 +352,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE \... will sort the composite bucket in descending order when comparing values from the `date_histogram` source and in ascending order when comparing values from the `terms` source. @@ -371,7 +362,7 @@ By default documents without a value for a given source are ignored. It is possible to include them in the response by setting `missing_bucket` to `true` (defaults to `false`): -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -386,7 +377,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE In the example above the source `product_name` will emit an explicit `null` value for documents without a value for the field `product`. @@ -411,7 +401,7 @@ If all composite buckets should be retrieved it is preferable to use a small siz and then use the `after` parameter to retrieve the next results. For example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -428,7 +418,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] \... returns: @@ -477,7 +466,7 @@ the last composite buckets returned in a previous round. For the example below the last bucket can be found in `after_key` and the next round of result can be retrieved with: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -495,7 +484,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> Should restrict the aggregation to buckets that sort **after** the provided values. @@ -507,7 +495,7 @@ parent aggregation. For instance the following example computes the average value of a field per composite bucket: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -528,7 +516,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] \... returns: diff --git a/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc index 2ee9025b6ded..b5633cecd026 100644 --- a/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc @@ -103,7 +103,7 @@ specified timezone, so that the date and time are the same at the start and end. ===== Calendar Interval Examples As an example, here is an aggregation requesting bucket intervals of a month in calendar time: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -117,13 +117,12 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] If you attempt to use multiples of calendar units, the aggregation will fail because only singular calendar units are supported: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -137,7 +136,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] // TEST[catch:bad_request] @@ -199,7 +197,7 @@ Defined as 24 hours (86,400,000 milliseconds) If we try to recreate the "month" `calendar_interval` from earlier, we can approximate that with 30 fixed days: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -213,12 +211,11 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] But if we try to use a calendar unit that is not supported, such as weeks, we'll get an exception: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -232,7 +229,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] // TEST[catch:bad_request] @@ -290,7 +286,7 @@ date string using the `format` parameter specification: TIP: If you don't specify `format`, the first date <> specified in the field mapping is used. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -305,7 +301,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Supports expressive date <> @@ -353,7 +348,7 @@ such as`America/Los_Angeles`. Consider the following example: -[source,js] +[source,console] --------------------------------- PUT my_index/_doc/1?refresh { @@ -377,7 +372,6 @@ GET my_index/_search?size=0 } } --------------------------------- -// CONSOLE If you don't specify a timezone, UTC is used. This would result in both of these documents being placed into the same day bucket, which starts at midnight UTC @@ -405,7 +399,7 @@ on 1 October 2015: If you specify a `time_zone` of `-01:00`, midnight in that timezone is one hour before midnight UTC: -[source,js] +[source,console] --------------------------------- GET my_index/_search?size=0 { @@ -420,7 +414,6 @@ GET my_index/_search?size=0 } } --------------------------------- -// CONSOLE // TEST[continued] Now the first document falls into the bucket for 30 September 2015, while the @@ -474,7 +467,7 @@ For example, when using an interval of `day`, each bucket runs from midnight to midnight. Setting the `offset` parameter to `+6h` changes each bucket to run from 6am to 6am: -[source,js] +[source,console] ----------------------------- PUT my_index/_doc/1?refresh { @@ -499,7 +492,6 @@ GET my_index/_search?size=0 } } ----------------------------- -// CONSOLE Instead of a single bucket starting at midnight, the above request groups the documents into buckets starting at 6am: @@ -536,7 +528,7 @@ adjustments have been made. Setting the `keyed` flag to `true` associates a unique string key with each bucket and returns the ranges as a hash rather than an array: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -552,7 +544,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -606,7 +597,7 @@ The `missing` parameter defines how to treat documents that are missing a value. By default, they are ignored, but it is also possible to treat them as if they have a value. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -621,7 +612,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `publish_date` field will fall into the @@ -640,7 +630,7 @@ When you need to aggregate the results by day of the week, use a script that returns the day of the week: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -656,7 +646,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: diff --git a/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc b/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc index 828bafb8fb6e..c116bfcc785a 100644 --- a/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc @@ -12,7 +12,7 @@ for each range. Example: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -30,7 +30,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales s/now-10M\/M/10-2015/] <1> < now minus 10 months, rounded down to the start of the month. @@ -75,7 +74,7 @@ be treated. By default they will be ignored but it is also possible to treat them as if they had a value. This is done by adding a set of fieldname : value mappings to specify default values per field. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -100,7 +99,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `date` field will be added to the "Older" @@ -267,7 +265,7 @@ The `time_zone` parameter is also applied to rounding in date math expressions. As an example, to round to the beginning of the day in the CET time zone, you can do the following: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -286,7 +284,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> This date will be converted to `2016-02-01T00:00:00.000+01:00`. @@ -297,7 +294,7 @@ POST /sales/_search?size=0 Setting the `keyed` flag to `true` will associate a unique string key with each bucket and return the ranges as a hash rather than an array: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -316,7 +313,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales s/now-10M\/M/10-2015/] Response: @@ -347,7 +343,7 @@ Response: It is also possible to customize the key for each range: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -366,7 +362,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: diff --git a/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc b/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc index edd6c6163da8..7c825eebc646 100644 --- a/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc @@ -26,7 +26,7 @@ Example: We might want to see which tags are strongly associated with `#elasticsearch` on StackOverflow forum posts but ignoring the effects of some prolific users with a tendency to misspell #Kibana as #Cabana. -[source,js] +[source,console] -------------------------------------------------- POST /stackoverflow/_search?size=0 { @@ -53,7 +53,6 @@ POST /stackoverflow/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:stackoverflow] Response: @@ -92,7 +91,7 @@ Response: In this scenario we might want to diversify on a combination of field values. We can use a `script` to produce a hash of the multiple values in a tags field to ensure we don't have a sample that consists of the same repeated combinations of tags. -[source,js] +[source,console] -------------------------------------------------- POST /stackoverflow/_search?size=0 { @@ -123,7 +122,6 @@ POST /stackoverflow/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:stackoverflow] Response: diff --git a/docs/reference/aggregations/bucket/filter-aggregation.asciidoc b/docs/reference/aggregations/bucket/filter-aggregation.asciidoc index bff4096e2dd3..d65b1851a947 100644 --- a/docs/reference/aggregations/bucket/filter-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/filter-aggregation.asciidoc @@ -5,7 +5,7 @@ Defines a single bucket of all the documents in the current document set context Example: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -19,7 +19,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] In the above example, we calculate the average price of all the products that are of type t-shirt. diff --git a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc index 4e83c14a18ba..0e7b11a010c4 100644 --- a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc @@ -7,7 +7,7 @@ filter. Example: -[source,js] +[source,console] -------------------------------------------------- PUT /logs/_bulk?refresh { "index" : { "_id" : 1 } } @@ -32,7 +32,6 @@ GET logs/_search } } -------------------------------------------------- -// CONSOLE In the above example, we analyze log messages. The aggregation will build two collection (buckets) of log messages - one for all those containing an error, @@ -70,7 +69,7 @@ Response: The filters field can also be provided as an array of filters, as in the following request: -[source,js] +[source,console] -------------------------------------------------- GET logs/_search { @@ -87,7 +86,6 @@ GET logs/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] The filtered buckets are returned in the same order as provided in the @@ -133,7 +131,7 @@ this parameter will implicitly set the `other_bucket` parameter to `true`. The following snippet shows a response where the `other` bucket is requested to be named `other_messages`. -[source,js] +[source,console] -------------------------------------------------- PUT logs/_doc/4?refresh { @@ -156,7 +154,6 @@ GET logs/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] The response would be something like the following: diff --git a/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc b/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc index 21886686b67a..82838808a3e0 100644 --- a/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc @@ -3,7 +3,7 @@ A multi-bucket aggregation that works on `geo_point` fields and conceptually works very similar to the <> aggregation. The user can define a point of origin and a set of distance range buckets. The aggregation evaluate the distance of each document value from the origin point and determines the buckets it belongs to based on the ranges (a document belongs to a bucket if the distance between the document and the origin falls within the distance range of the bucket). -[source,js] +[source,console] -------------------------------------------------- PUT /museums { @@ -47,7 +47,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE Response: @@ -90,7 +89,7 @@ The specified field must be of type `geo_point` (which can only be set explicitl By default, the distance unit is `m` (meters) but it can also accept: `mi` (miles), `in` (inches), `yd` (yards), `km` (kilometers), `cm` (centimeters), `mm` (millimeters). -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -110,14 +109,13 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The distances will be computed in kilometers There are two distance calculation modes: `arc` (the default), and `plane`. The `arc` calculation is the most accurate. The `plane` is the fastest but least accurate. Consider using `plane` when your search context is "narrow", and spans smaller geographical areas (~5km). `plane` will return higher error margins for searches across very large areas (e.g. cross continent search). The distance calculation type can be set using the `distance_type` parameter: -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -138,14 +136,13 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] ==== Keyed Response Setting the `keyed` flag to `true` will associate a unique string key with each bucket and return the ranges as a hash rather than an array: -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -165,7 +162,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] Response: @@ -200,7 +196,7 @@ Response: It is also possible to customize the key for each range: -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -220,7 +216,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] Response: diff --git a/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc b/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc index d956ef4bfdc9..3c84f6549299 100644 --- a/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc @@ -17,7 +17,7 @@ The specified field must be of type `geo_point` (which can only be set explicitl ==== Simple low-precision request -[source,js] +[source,console] -------------------------------------------------- PUT /museums { @@ -56,7 +56,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE Response: @@ -90,7 +89,7 @@ Response: When requesting detailed buckets (typically for displaying a "zoomed in" map) a filter like <> should be applied to narrow the subject area otherwise potentially millions of buckets will be created and returned. -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -116,13 +115,12 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] The geohashes returned by the `geohash_grid` aggregation can be also used for zooming in. To zoom into the first geohash `u17` returned in the previous example, it should be specified as both `top_left` and `bottom_right` corner: -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -148,7 +146,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] diff --git a/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc b/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc index ac173ec2b002..43ca6b0176b1 100644 --- a/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc @@ -30,7 +30,7 @@ fields, in which case all points will be taken into account during aggregation. ==== Simple low-precision request -[source,js] +[source,console] -------------------------------------------------- PUT /museums { @@ -69,7 +69,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE Response: @@ -106,7 +105,7 @@ a filter like <> should be applied to narrow the subject area otherwise potentially millions of buckets will be created and returned. -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -132,7 +131,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] diff --git a/docs/reference/aggregations/bucket/global-aggregation.asciidoc b/docs/reference/aggregations/bucket/global-aggregation.asciidoc index affabc5067e6..9a17b7755d67 100644 --- a/docs/reference/aggregations/bucket/global-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/global-aggregation.asciidoc @@ -11,7 +11,7 @@ NOTE: Global aggregators can only be placed as top level aggregators because Example: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -29,7 +29,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> The `global` aggregation has an empty body diff --git a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc index 32a540130ef3..33dcecb450ad 100644 --- a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc @@ -19,7 +19,7 @@ The `interval` must be a positive decimal, while the `offset` must be a decimal The following snippet "buckets" the products based on their `price` by interval of `50`: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -33,7 +33,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] And the following may be the response: @@ -78,7 +77,7 @@ The response above show that no documents has a price that falls within the rang response will fill gaps in the histogram with empty buckets. It is possible change that and request buckets with a higher minimum count thanks to the `min_doc_count` setting: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -93,7 +92,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -154,7 +152,7 @@ under a range `filter` aggregation with the appropriate `from`/`to` settings. Example: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -175,7 +173,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Order @@ -199,7 +196,7 @@ documents. By default, the buckets are returned as an ordered array. It is also possible to request the response as a hash instead keyed by the buckets keys: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -214,7 +211,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -259,7 +255,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -274,7 +270,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `quantity` field will fall into the same bucket as documents that have the value `0`. diff --git a/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc b/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc index 0aabd3a71ed3..67578af3694d 100644 --- a/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc @@ -5,7 +5,7 @@ Just like the dedicated < Example: -[source,js] +[source,console] -------------------------------------------------- GET /ip_addresses/_search { @@ -23,7 +23,6 @@ GET /ip_addresses/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:iprange] Response: @@ -55,7 +54,7 @@ Response: IP ranges can also be defined as CIDR masks: -[source,js] +[source,console] -------------------------------------------------- GET /ip_addresses/_search { @@ -73,7 +72,6 @@ GET /ip_addresses/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:iprange] Response: @@ -109,7 +107,7 @@ Response: Setting the `keyed` flag to `true` will associate a unique string key with each bucket and return the ranges as a hash rather than an array: -[source,js] +[source,console] -------------------------------------------------- GET /ip_addresses/_search { @@ -128,7 +126,6 @@ GET /ip_addresses/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:iprange] Response: @@ -158,7 +155,7 @@ Response: It is also possible to customize the key for each range: -[source,js] +[source,console] -------------------------------------------------- GET /ip_addresses/_search { @@ -177,7 +174,6 @@ GET /ip_addresses/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:iprange] Response: diff --git a/docs/reference/aggregations/bucket/missing-aggregation.asciidoc b/docs/reference/aggregations/bucket/missing-aggregation.asciidoc index b67052911967..e37b73ed97af 100644 --- a/docs/reference/aggregations/bucket/missing-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/missing-aggregation.asciidoc @@ -5,7 +5,7 @@ A field data based single bucket aggregation, that creates a bucket of all docum Example: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -16,7 +16,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] In the above example, we get the total number of products that do not have a price. diff --git a/docs/reference/aggregations/bucket/nested-aggregation.asciidoc b/docs/reference/aggregations/bucket/nested-aggregation.asciidoc index d323eb1c134e..329b937bbc35 100644 --- a/docs/reference/aggregations/bucket/nested-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/nested-aggregation.asciidoc @@ -6,7 +6,7 @@ A special single bucket aggregation that enables aggregating nested documents. For example, lets say we have an index of products, and each product holds the list of resellers - each having its own price for the product. The mapping could look like: -[source,js] +[source,console] -------------------------------------------------- PUT /index { @@ -23,13 +23,12 @@ PUT /index } } -------------------------------------------------- -// CONSOLE // TESTSETUP <1> The `resellers` is an array that holds nested documents under the `product` object. The following aggregations will return the minimum price products can be purchased in: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -48,7 +47,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/GET \/_search/GET \/_search\?filter_path=aggregations/] // TEST[s/^/PUT index\/_doc\/0\?refresh\n{"name":"led", "resellers": [{"name": "foo", "price": 350.00}, {"name": "bar", "price": 500.00}]}\n/] diff --git a/docs/reference/aggregations/bucket/parent-aggregation.asciidoc b/docs/reference/aggregations/bucket/parent-aggregation.asciidoc index 37d98f8b9be3..208c643b31a0 100644 --- a/docs/reference/aggregations/bucket/parent-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/parent-aggregation.asciidoc @@ -9,7 +9,7 @@ This aggregation has a single option: For example, let's say we have an index of questions and answers. The answer type has the following `join` field in the mapping: -[source,js] +[source,console] -------------------------------------------------- PUT parent_example { @@ -25,7 +25,6 @@ PUT parent_example } } -------------------------------------------------- -// CONSOLE The `question` document contain a tag field and the `answer` documents contain an owner field. With the `parent` aggregation the owner buckets can be mapped to the tag buckets in a single request even though the two fields exist in @@ -33,7 +32,7 @@ two different kinds of documents. An example of a question document: -[source,js] +[source,console] -------------------------------------------------- PUT parent_example/_doc/1 { @@ -49,12 +48,11 @@ PUT parent_example/_doc/1 ] } -------------------------------------------------- -// CONSOLE // TEST[continued] Examples of `answer` documents: -[source,js] +[source,console] -------------------------------------------------- PUT parent_example/_doc/2?routing=1 { @@ -86,12 +84,11 @@ PUT parent_example/_doc/3?routing=1&refresh "creation_date": "2009-05-05T13:45:37.030" } -------------------------------------------------- -// CONSOLE // TEST[continued] The following request can be built that connects the two together: -[source,js] +[source,console] -------------------------------------------------- POST parent_example/_search?size=0 { @@ -120,7 +117,6 @@ POST parent_example/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The `type` points to type / mapping with the name `answer`. diff --git a/docs/reference/aggregations/bucket/range-aggregation.asciidoc b/docs/reference/aggregations/bucket/range-aggregation.asciidoc index 8ff26c7c92f5..b27af0b3be5f 100644 --- a/docs/reference/aggregations/bucket/range-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/range-aggregation.asciidoc @@ -6,7 +6,7 @@ Note that this aggregation includes the `from` value and excludes the `to` value Example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -24,7 +24,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] // TEST[s/GET \/_search/GET \/_search\?filter_path=aggregations/] @@ -64,7 +63,7 @@ Response: Setting the `keyed` flag to `true` will associate a unique string key with each bucket and return the ranges as a hash rather than an array: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -83,7 +82,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] // TEST[s/GET \/_search/GET \/_search\?filter_path=aggregations/] @@ -118,7 +116,7 @@ Response: It is also possible to customize the key for each range: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -137,7 +135,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] // TEST[s/GET \/_search/GET \/_search\?filter_path=aggregations/] @@ -177,7 +174,7 @@ will be executed during aggregation execution. The following example shows how to use an `inline` script with the `painless` script language and no script parameters: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -198,11 +195,10 @@ GET /_search } } -------------------------------------------------- -// CONSOLE It is also possible to use stored scripts. Here is a simple stored script: -[source,js] +[source,console] -------------------------------------------------- POST /_scripts/convert_currency { @@ -212,12 +208,11 @@ POST /_scripts/convert_currency } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] And this new stored script can be used in the range aggregation like this: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -240,7 +235,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/GET \/_search/GET \/_search\?filter_path=aggregations/] // TEST[continued] <1> Id of the stored script @@ -278,7 +272,7 @@ GET /_search Lets say the product prices are in USD but we would like to get the price ranges in EURO. We can use value script to convert the prices prior the aggregation (assuming conversion rate of 0.8) -[source,js] +[source,console] -------------------------------------------------- GET /sales/_search { @@ -302,14 +296,13 @@ GET /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Sub Aggregations The following example, not only "bucket" the documents to the different buckets but also computes statistics over the prices in each price range -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -332,7 +325,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] // TEST[s/GET \/_search/GET \/_search\?filter_path=aggregations/] @@ -391,7 +383,7 @@ Response: If a sub aggregation is also based on the same value source as the range aggregation (like the `stats` aggregation in the example above) it is possible to leave out the value source definition for it. The following will return the same response as above: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -414,5 +406,4 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> We don't need to specify the `price` as we "inherit" it by default from the parent `range` aggregation diff --git a/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc index e2537b61aefd..7757883a40fe 100644 --- a/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc @@ -85,7 +85,7 @@ better approximation, but higher memory usage. Cannot be smaller than `0.00001` Example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -98,7 +98,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] Response: @@ -124,7 +123,7 @@ Response: In this example, the only bucket that we see is the "swing" bucket, because it is the only term that appears in one document. If we increase the `max_doc_count` to `2`, we'll see some more buckets: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -138,7 +137,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] This now shows the "jazz" term which has a `doc_count` of 2": @@ -275,7 +273,7 @@ It is possible to filter the values for which buckets will be created. This can ===== Filtering Values with regular expressions -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -290,7 +288,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE In the above example, buckets will be created for all the tags that starts with `swi`, except those starting with `electro` (so the tag `swing` will be aggregated but not `electro_swing`). The `include` regular expression will determine what @@ -304,7 +301,7 @@ The syntax is the same as <>. For matching based on exact values the `include` and `exclude` parameters can simply take an array of strings that represent the terms as they are found in the index: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -319,7 +316,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE ==== Missing value @@ -328,7 +324,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -342,7 +338,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> Documents without a value in the `tags` field will fall into the same bucket as documents that have the value `N/A`. diff --git a/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc b/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc index f922d90331fd..8e3b65bfe71e 100644 --- a/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc @@ -15,7 +15,7 @@ a nested object field that falls outside the `nested` aggregation's nested struc For example, lets say we have an index for a ticket system with issues and comments. The comments are inlined into the issue documents as nested documents. The mapping could look like: -[source,js] +[source,console] -------------------------------------------------- PUT /issues { @@ -33,7 +33,7 @@ PUT /issues } } -------------------------------------------------- -// CONSOLE + <1> The `comments` is an array that holds nested documents under the `issue` object. The following aggregations will return the top commenters' username that have commented and per top commenter the top @@ -41,17 +41,16 @@ tags of the issues the user has commented on: ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- POST /issues/_doc/0?refresh {"tags": ["tag_1"], "comments": [{"username": "username_1"}]} -------------------------------------------------- -// CONSOLE // TEST[continued] ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- GET /issues/_search { @@ -86,7 +85,6 @@ GET /issues/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[s/_search/_search\?filter_path=aggregations/] diff --git a/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc b/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc index d3a255be1d5f..4a6c7bf8c7cc 100644 --- a/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc @@ -15,7 +15,7 @@ A query on StackOverflow data for the popular term `javascript` OR the rarer ter the `significant_terms` aggregation on top-scoring documents that are more likely to match the most interesting parts of our query we use a sample. -[source,js] +[source,console] -------------------------------------------------- POST /stackoverflow/_search?size=0 { @@ -41,7 +41,6 @@ POST /stackoverflow/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:stackoverflow] Response: @@ -85,7 +84,7 @@ Without the `sampler` aggregation the request query considers the full "long tai less significant terms such as `jquery` and `angular` rather than focusing on the more insightful Kibana-related terms. -[source,js] +[source,console] -------------------------------------------------- POST /stackoverflow/_search?size=0 { @@ -105,7 +104,6 @@ POST /stackoverflow/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:stackoverflow] Response: diff --git a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc index 50c238edb93c..11e0c28323bb 100644 --- a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc @@ -64,7 +64,7 @@ set used for statistical comparisons is the index or indices from which the resu Example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -78,12 +78,11 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] Response: -[source,js] +[source,console] -------------------------------------------------- { ... @@ -125,7 +124,7 @@ A simpler way to perform analysis across multiple categories is to use a parent- Example using a parent aggregation for segmentation: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -141,7 +140,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] Response: @@ -203,7 +201,7 @@ Now we have anomaly detection for each of the police forces using a single reque We can use other forms of top-level aggregations to segment our data, for example segmenting by geographic area to identify unusual hot-spots of a particular crime type: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -222,7 +220,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE This example uses the `geohash_grid` aggregation to create result buckets that represent geographic areas, and inside each bucket we can identify anomalous levels of a crime type in these tightly-focused areas e.g. @@ -464,7 +461,7 @@ NOTE: `shard_size` cannot be smaller than `size` (as it doesn't make much sens It is possible to only return terms that match more than a configured number of hits using the `min_doc_count` option: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -478,7 +475,7 @@ GET /_search } } -------------------------------------------------- -// CONSOLE + The above aggregation would only return tags which have been found in 10 hits or more. Default value is `3`. @@ -507,7 +504,7 @@ The default source of statistical information for background term frequencies is scope can be narrowed through the use of a `background_filter` to focus in on significant terms within a narrower context: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -528,7 +525,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The above filter would help focus in on terms that were peculiar to the city of Madrid rather than revealing terms like "Spanish" that are unusual in the full index's worldwide context but commonplace in the subset of documents containing the @@ -566,7 +562,7 @@ is significantly faster. By default, `map` is only used when running an aggregat ordinals. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -580,7 +576,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> the possible values are `map`, `global_ordinals` diff --git a/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc b/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc index 3a9221c3c437..95655f60ccbe 100644 --- a/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc @@ -32,7 +32,7 @@ and the _background_set used for statistical comparisons is the index or indices Example: -[source,js] +[source,console] -------------------------------------------------- GET news/_search { @@ -53,7 +53,6 @@ GET news/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:news] @@ -147,7 +146,7 @@ The uncleansed documents have thrown up some odd-looking terms that are, on the correlated with appearances of our search term "elasticsearch" e.g. "pozmantier". We can drill down into examples of these documents to see why pozmantier is connected using this query: -[source,js] +[source,console] -------------------------------------------------- GET news/_search { @@ -167,8 +166,8 @@ GET news/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:news] + The results show a series of very similar news articles about a judging panel for a number of tech projects: [source,js] @@ -215,7 +214,7 @@ Fortunately similar documents tend to rank similarly so as part of examining the aggregation can apply a filter to remove sequences of any 6 or more tokens that have already been seen. Let's try this same query now but with the `filter_duplicate_text` setting turned on: -[source,js] +[source,console] -------------------------------------------------- GET news/_search { @@ -241,7 +240,6 @@ GET news/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:news] The results from analysing our deduplicated text are obviously of higher quality to anyone familiar with the elastic stack: @@ -418,7 +416,7 @@ The default source of statistical information for background term frequencies is scope can be narrowed through the use of a `background_filter` to focus in on significant terms within a narrower context: -[source,js] +[source,console] -------------------------------------------------- GET news/_search { @@ -439,7 +437,6 @@ GET news/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:news] The above filter would help focus in on terms that were peculiar to the city of Madrid rather than revealing @@ -457,7 +454,7 @@ JSON field(s) and the indexed field being aggregated can differ. In these cases it is possible to list the JSON _source fields from which text will be analyzed using the `source_fields` parameter: -[source,js] +[source,console] -------------------------------------------------- GET news/_search { @@ -476,7 +473,6 @@ GET news/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:news] diff --git a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc index b40302b6120e..763a7074ee71 100644 --- a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc @@ -53,7 +53,7 @@ POST /products/_bulk?refresh Example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -64,8 +64,8 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] + <1> `terms` aggregation should be a field of type `keyword` or any other data type suitable for bucket aggregations. In order to use it with `text` you will need to enable <>. @@ -130,7 +130,7 @@ combined to give a final view. Consider the following scenario: A request is made to obtain the top 5 terms in the field product, ordered by descending document count from an index with 3 shards. In this case each shard is asked to give its top 5 terms. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -144,7 +144,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] The terms for each of the three shards are shown below with their @@ -260,7 +259,7 @@ could have the 4th highest document count. The second error value can be enabled by setting the `show_term_doc_count_error` parameter to true: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -275,7 +274,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] @@ -338,7 +336,7 @@ but at least the top buckets will be correctly picked. Ordering the buckets by their doc `_count` in an ascending manner: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -352,11 +350,10 @@ GET /_search } } -------------------------------------------------- -// CONSOLE Ordering the buckets alphabetically by their terms in an ascending manner: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -370,13 +367,12 @@ GET /_search } } -------------------------------------------------- -// CONSOLE deprecated[6.0.0, Use `_key` instead of `_term` to order buckets by their term] Ordering the buckets by single value metrics sub-aggregation (identified by the aggregation name): -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -393,11 +389,10 @@ GET /_search } } -------------------------------------------------- -// CONSOLE Ordering the buckets by multi value metrics sub-aggregation (identified by the aggregation name): -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -414,7 +409,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE [NOTE] .Pipeline aggs cannot be used for sorting @@ -444,7 +438,7 @@ METRIC = [ , ]* [ , ] ; -------------------------------------------------- -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -466,13 +460,12 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The above will sort the artist's countries buckets based on the average play count among the rock songs. Multiple criteria can be used to order the buckets by providing an array of order criteria such as the following: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -494,7 +487,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The above will sort the artist's countries buckets based on the average play count among the rock songs and then by their `doc_count` in descending order. @@ -506,7 +498,7 @@ tie-breaker in ascending alphabetical order to prevent non-deterministic orderin It is possible to only return terms that match more than a configured number of hits using the `min_doc_count` option: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -520,7 +512,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The above aggregation would only return tags which have been found in 10 hits or more. Default value is `1`. @@ -548,7 +539,7 @@ WARNING: When NOT sorting on `doc_count` descending, high values of `min_doc_cou Generating the terms using a script: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -564,13 +555,12 @@ GET /_search } } -------------------------------------------------- -// CONSOLE This will interpret the `script` parameter as an `inline` script with the default script language and no script parameters. To use a stored script use the following syntax: ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- POST /_scripts/my_script { @@ -580,11 +570,10 @@ POST /_scripts/my_script } } -------------------------------------------------- -// CONSOLE ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -602,12 +591,11 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] ==== Value Script -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -624,7 +612,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE ==== Filtering Values @@ -634,7 +621,7 @@ It is possible to filter the values for which buckets will be created. This can ===== Filtering Values with regular expressions -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -649,7 +636,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE In the above example, buckets will be created for all the tags that has the word `sport` in them, except those starting with `water_` (so the tag `water_sports` will not be aggregated). The `include` regular expression will determine what @@ -663,7 +649,7 @@ The syntax is the same as <>. For matching based on exact values the `include` and `exclude` parameters can simply take an array of strings that represent the terms as they are found in the index: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -683,7 +669,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE ===== Filtering Values with partitions @@ -693,7 +678,7 @@ This can be achieved by grouping the field's values into a number of partitions only one partition in each request. Consider this request which is looking for accounts that have not logged any access recently: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -722,7 +707,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE This request is finding the last logged access date for a subset of customer accounts because we might want to expire some customer accounts who haven't been seen for a long while. @@ -786,7 +770,7 @@ are expanded in one depth-first pass and only then any pruning occurs. In some scenarios this can be very wasteful and can hit memory constraints. An example problem scenario is querying a movie database for the 10 most popular actors and their 5 most common co-stars: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -808,7 +792,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE Even though the number of actors may be comparatively small and we want only 50 result buckets there is a combinatorial explosion of buckets during calculation - a single actor can produce n² buckets where n is the number of actors. The sane option would be to first determine @@ -818,7 +801,7 @@ mode as opposed to the `depth_first` mode. NOTE: The `breadth_first` is the default mode for fields with a cardinality bigger than the requested size or when the cardinality is unknown (numeric fields or scripts for instance). It is possible to override the default heuristic and to provide a collect mode directly in the request: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -841,7 +824,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> the possible values are `breadth_first` and `depth_first` @@ -870,7 +852,7 @@ so memory usage is linear to the number of values of the documents that are part is significantly faster. By default, `map` is only used when running an aggregation on scripts, since they don't have ordinals. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -884,7 +866,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> The possible values are `map`, `global_ordinals` @@ -896,7 +877,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -910,7 +891,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> Documents without a value in the `tags` field will fall into the same bucket as documents that have the value `N/A`. diff --git a/docs/reference/aggregations/matrix/stats-aggregation.asciidoc b/docs/reference/aggregations/matrix/stats-aggregation.asciidoc index df5afc82523f..ea1cd8063e6d 100644 --- a/docs/reference/aggregations/matrix/stats-aggregation.asciidoc +++ b/docs/reference/aggregations/matrix/stats-aggregation.asciidoc @@ -35,7 +35,7 @@ POST /_refresh The following example demonstrates the use of matrix stats to describe the relationship between income and poverty. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -48,7 +48,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE // TEST[s/_search/_search\?filter_path=aggregations/] The aggregation type is `matrix_stats` and the `fields` setting defines the set of fields (as an array) for computing @@ -119,7 +118,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. This is done by adding a set of fieldname : value mappings to specify default values per field. -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -133,7 +132,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE <1> Documents without a value in the `income` field will have the default value `50000`. diff --git a/docs/reference/aggregations/metrics/avg-aggregation.asciidoc b/docs/reference/aggregations/metrics/avg-aggregation.asciidoc index 0607dadf8572..ca7328859c97 100644 --- a/docs/reference/aggregations/metrics/avg-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/avg-aggregation.asciidoc @@ -6,7 +6,7 @@ A `single-value` metrics aggregation that computes the average of numeric values Assuming the data consists of documents representing exams grades (between 0 and 100) of students we can average their scores with: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -15,7 +15,6 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] The above aggregation computes the average grade over all documents. The aggregation type is `avg` and the `field` setting defines the numeric field of the documents the average will be computed on. The above will return the following: @@ -39,7 +38,7 @@ The name of the aggregation (`avg_grade` above) also serves as the key by which Computing the average grade based on a script: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -54,12 +53,11 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -77,14 +75,13 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams,stored_example_script] ===== Value Script It turned out that the exam was way above the level of the students and a grade correction needs to be applied. We can use value script to get the new average: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -104,7 +101,6 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] ==== Missing value @@ -113,7 +109,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -127,7 +123,6 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] <1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc index 405a9d89107f..534c69e704d9 100644 --- a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc @@ -7,7 +7,7 @@ document or generated by a script. Assume you are indexing store sales and would like to count the unique number of sold products that match a query: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -20,7 +20,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -42,7 +41,7 @@ Response: This aggregation also supports the `precision_threshold` option: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -56,7 +55,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> The `precision_threshold` options allows to trade memory for accuracy, and @@ -183,7 +181,7 @@ make sure that hashes are computed at most once per unique value per segment. The `cardinality` metric supports scripting, with a noticeable performance hit however since hashes need to be computed on the fly. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -199,12 +197,11 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -223,7 +220,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[skip:no script] ==== Missing value @@ -232,7 +228,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -246,6 +242,5 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `tag` field will fall into the same bucket as documents that have the value `N/A`. diff --git a/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc b/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc index 6eb2f18928a8..17a313202cb0 100644 --- a/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc @@ -7,7 +7,7 @@ The `extended_stats` aggregations is an extended version of the < `sigma` controls how many standard deviations +/- from the mean should be displayed @@ -89,7 +87,7 @@ if your data is skewed heavily left or right, the value returned will be mislead Computing the grades stats based on a script: -[source,js] +[source,console] -------------------------------------------------- GET /exams/_search { @@ -106,12 +104,11 @@ GET /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- GET /exams/_search { @@ -130,14 +127,13 @@ GET /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams,stored_example_script] ===== Value Script It turned out that the exam was way above the level of the students and a grade correction needs to be applied. We can use value script to get the new stats: -[source,js] +[source,console] -------------------------------------------------- GET /exams/_search { @@ -158,7 +154,6 @@ GET /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] ==== Missing value @@ -167,7 +162,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- GET /exams/_search { @@ -182,7 +177,6 @@ GET /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] <1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `0`. diff --git a/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc b/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc index 3859f1977ecb..e012ae78159f 100644 --- a/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc @@ -6,7 +6,7 @@ A metric aggregation that computes the bounding box containing all geo_point val Example: -[source,js] +[source,console] -------------------------------------------------- PUT /museums { @@ -48,7 +48,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE <1> The `geo_bounds` aggregation specifies the field to use to obtain the bounds <2> `wrap_longitude` is an optional parameter which specifies whether the bounding box should be allowed to overlap the international date line. The default value is `true` diff --git a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc index 9ce69a3dd395..cd9b4acfe967 100644 --- a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc @@ -5,7 +5,7 @@ A metric aggregation that computes the weighted https://en.wikipedia.org/wiki/Ce Example: -[source,js] +[source,console] -------------------------------------------------- PUT /museums { @@ -43,7 +43,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE <1> The `geo_centroid` aggregation specifies the field to use for computing the centroid. (NOTE: field must be a <> type) @@ -72,7 +71,7 @@ The `geo_centroid` aggregation is more interesting when combined as a sub-aggreg Example: -[source,js] +[source,console] -------------------------------------------------- POST /museums/_search?size=0 { @@ -88,7 +87,6 @@ POST /museums/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[continued] The above example uses `geo_centroid` as a sub-aggregation to a diff --git a/docs/reference/aggregations/metrics/max-aggregation.asciidoc b/docs/reference/aggregations/metrics/max-aggregation.asciidoc index cd9f4282e4d7..70731dbb5157 100644 --- a/docs/reference/aggregations/metrics/max-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/max-aggregation.asciidoc @@ -12,7 +12,7 @@ whose absolute value is greater than +2^53+. Computing the max price value across all documents -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -21,7 +21,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -48,7 +47,7 @@ response. The `max` aggregation can also calculate the maximum of a script. The example below computes the maximum price: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -63,13 +62,12 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] This will use the <> scripting language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -87,7 +85,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales,stored_example_script] ==== Value Script @@ -97,7 +94,7 @@ would like to compute the max in EURO (and for the sake of this example, let's say the conversion rate is 1.2). We can use a value script to apply the conversion rate to every value before it is aggregated: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -116,7 +113,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Missing value @@ -125,7 +121,7 @@ The `missing` parameter defines how documents that are missing a value should be treated. By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -139,7 +135,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `grade` field will fall into the same diff --git a/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc b/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc index 2e88b12d92cd..336074fd2f48 100644 --- a/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc @@ -24,7 +24,7 @@ In this example we have a product which has an average rating of 3 stars. Let's look at its ratings' median absolute deviation to determine how much they vary -[source,js] +[source,console] --------------------------------------------------------- GET reviews/_search { @@ -43,7 +43,6 @@ GET reviews/_search } } --------------------------------------------------------- -// CONSOLE // TEST[setup:reviews] <1> `rating` must be a numeric field @@ -84,7 +83,7 @@ cost of higher memory usage. For more about the characteristics of the TDigest `compression` parameter see <>. -[source,js] +[source,console] --------------------------------------------------------- GET reviews/_search { @@ -99,7 +98,6 @@ GET reviews/_search } } --------------------------------------------------------- -// CONSOLE // TEST[setup:reviews] The default `compression` value for this aggregation is `1000`. At this @@ -114,7 +112,7 @@ of one to ten, we can using scripting. To provide an inline script: -[source,js] +[source,console] --------------------------------------------------------- GET reviews/_search { @@ -134,12 +132,11 @@ GET reviews/_search } } --------------------------------------------------------- -// CONSOLE // TEST[setup:reviews] To provide a stored script: -[source,js] +[source,console] --------------------------------------------------------- GET reviews/_search { @@ -158,7 +155,6 @@ GET reviews/_search } } --------------------------------------------------------- -// CONSOLE // TEST[setup:reviews,stored_example_script] ==== Missing value @@ -170,7 +166,7 @@ as if they had a value. Let's be optimistic and assume some reviewers loved the product so much that they forgot to give it a rating. We'll assign them five stars -[source,js] +[source,console] --------------------------------------------------------- GET reviews/_search { @@ -185,5 +181,4 @@ GET reviews/_search } } --------------------------------------------------------- -// CONSOLE // TEST[setup:reviews] diff --git a/docs/reference/aggregations/metrics/min-aggregation.asciidoc b/docs/reference/aggregations/metrics/min-aggregation.asciidoc index 4225beff3f77..bc904e7403b0 100644 --- a/docs/reference/aggregations/metrics/min-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/min-aggregation.asciidoc @@ -12,7 +12,7 @@ whose absolute value is greater than +2^53+. Computing the min price value across all documents: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -21,7 +21,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -49,7 +48,7 @@ response. The `min` aggregation can also calculate the minimum of a script. The example below computes the minimum price: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -64,13 +63,12 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] This will use the <> scripting language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -88,7 +86,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales,stored_example_script] ==== Value Script @@ -98,7 +95,7 @@ would like to compute the min in EURO (and for the sake of this example, let's say the conversion rate is 1.2). We can use a value script to apply the conversion rate to every value before it is aggregated: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -117,7 +114,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Missing value @@ -126,7 +122,7 @@ The `missing` parameter defines how documents that are missing a value should be treated. By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -140,7 +136,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> Documents without a value in the `grade` field will fall into the same diff --git a/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc b/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc index 1903bbc6bcad..aafc99a6a51b 100644 --- a/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc @@ -24,7 +24,7 @@ but it can be easily skewed by a single slow response. Let's look at a range of percentiles representing load time: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -38,7 +38,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] <1> The field `load_time` must be a numeric field @@ -76,7 +75,7 @@ Often, administrators are only interested in outliers -- the extreme percentiles We can specify just the percents we are interested in (requested percentiles must be a value between 0-100 inclusive): -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -91,7 +90,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] <1> Use the `percents` parameter to specify particular percentiles to calculate @@ -99,7 +97,7 @@ GET latency/_search By default the `keyed` flag is set to `true` which associates a unique string key with each bucket and returns the ranges as a hash rather than an array. Setting the `keyed` flag to `false` will disable this behavior: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -114,7 +112,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] Response: @@ -168,7 +165,7 @@ The percentile metric supports scripting. For example, if our load times are in milliseconds but we want percentiles calculated in seconds, we could use a script to convert them on-the-fly: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -188,7 +185,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] <1> The `field` parameter is replaced with a `script` parameter, which uses the @@ -197,7 +193,7 @@ script to generate values which percentiles are calculated on This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -216,7 +212,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency,stored_example_script] [[search-aggregations-metrics-percentile-aggregation-approximation]] @@ -262,7 +257,7 @@ it. It would not be the case on more skewed distributions. Approximate algorithms must balance memory utilization with estimation accuracy. This balance can be controlled using a `compression` parameter: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -279,7 +274,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] <1> Compression controls memory usage and approximation error @@ -313,7 +307,7 @@ for values up to 1 millisecond and 3.6 seconds (or better) for the maximum track The HDR Histogram can be used by specifying the `method` parameter in the request: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -331,7 +325,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] <1> `hdr` object indicates that HDR Histogram should be used to calculate the percentiles and specific settings for this algorithm can be specified inside the object @@ -346,7 +339,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -361,7 +354,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] <1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc b/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc index b9dac1dd0fb0..cb6624d25b2b 100644 --- a/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc @@ -22,7 +22,7 @@ Assume your data consists of website load times. You may have a service agreeme Let's look at a range of percentiles representing load time: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -37,8 +37,8 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] + <1> The field `load_time` must be a numeric field The response will look like this: @@ -67,7 +67,7 @@ hitting the 95% load time target By default the `keyed` flag is set to `true` associates a unique string key with each bucket and returns the ranges as a hash rather than an array. Setting the `keyed` flag to `false` will disable this behavior: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -83,7 +83,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] Response: @@ -118,7 +117,7 @@ The percentile rank metric supports scripting. For example, if our load times are in milliseconds but we want to specify values in seconds, we could use a script to convert them on-the-fly: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -139,15 +138,15 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] + <1> The `field` parameter is replaced with a `script` parameter, which uses the script to generate values which percentile ranks are calculated on <2> Scripting supports parameterized input just like any other script This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -167,7 +166,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency,stored_example_script] ==== HDR Histogram @@ -183,7 +181,7 @@ microseconds) in a histogram set to 3 significant digits, it will maintain a val The HDR Histogram can be used by specifying the `method` parameter in the request: -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -201,8 +199,8 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] + <1> `hdr` object indicates that HDR Histogram should be used to calculate the percentiles and specific settings for this algorithm can be specified inside the object <2> `number_of_significant_value_digits` specifies the resolution of values for the histogram in number of significant digits @@ -215,7 +213,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- GET latency/_search { @@ -231,6 +229,6 @@ GET latency/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:latency] + <1> Documents without a value in the `load_time` field will fall into the same bucket as documents that have the value `10`. diff --git a/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc b/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc index 1f2ec113d31f..cfded78bbdf5 100644 --- a/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc @@ -5,7 +5,7 @@ A metric aggregation that executes using scripts to provide a metric output. Example: -[source,js] +[source,console] -------------------------------------------------- POST ledger/_search?size=0 { @@ -24,7 +24,6 @@ POST ledger/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:ledger] <1> `init_script` is an optional parameter, all other scripts are required. @@ -50,7 +49,7 @@ The response for the above aggregation: The above example can also be specified using stored scripts as follows: -[source,js] +[source,console] -------------------------------------------------- POST ledger/_search?size=0 { @@ -77,7 +76,6 @@ POST ledger/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:ledger,stored_scripted_metric_script] <1> script parameters for `init`, `map` and `combine` scripts must be specified @@ -145,7 +143,7 @@ final combined profit which will be returned in the response of the aggregation. Imagine a situation where you index the following documents into an index with 2 shards: -[source,js] +[source,console] -------------------------------------------------- PUT /transactions/_bulk?refresh {"index":{"_id":1}} @@ -157,7 +155,6 @@ PUT /transactions/_bulk?refresh {"index":{"_id":4}} {"type": "sale","amount": 130} -------------------------------------------------- -// CONSOLE Lets say that documents 1 and 3 end up on shard A and documents 2 and 4 end up on shard B. The following is a breakdown of what the aggregation result is at each stage of the example above. diff --git a/docs/reference/aggregations/metrics/stats-aggregation.asciidoc b/docs/reference/aggregations/metrics/stats-aggregation.asciidoc index 770b9c523fec..233d8139ff44 100644 --- a/docs/reference/aggregations/metrics/stats-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/stats-aggregation.asciidoc @@ -7,7 +7,7 @@ The stats that are returned consist of: `min`, `max`, `sum`, `count` and `avg`. Assuming the data consists of documents representing exams grades (between 0 and 100) of students -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -16,7 +16,6 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] The above aggregation computes the grades statistics over all documents. The aggregation type is `stats` and the `field` setting defines the numeric field of the documents the stats will be computed on. The above will return the following: @@ -46,7 +45,7 @@ The name of the aggregation (`grades_stats` above) also serves as the key by whi Computing the grades stats based on a script: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -62,12 +61,11 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -85,14 +83,13 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams,stored_example_script] ===== Value Script It turned out that the exam was way above the level of the students and a grade correction needs to be applied. We can use a value script to get the new stats: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -112,7 +109,6 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] ==== Missing value @@ -121,7 +117,7 @@ The `missing` parameter defines how documents that are missing a value should be By default they will be ignored but it is also possible to treat them as if they had a value. -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search?size=0 { @@ -135,7 +131,6 @@ POST /exams/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] <1> Documents without a value in the `grade` field will fall into the same bucket as documents that have the value `0`. diff --git a/docs/reference/aggregations/metrics/sum-aggregation.asciidoc b/docs/reference/aggregations/metrics/sum-aggregation.asciidoc index 8825f0795215..eeff864e0ada 100644 --- a/docs/reference/aggregations/metrics/sum-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/sum-aggregation.asciidoc @@ -6,7 +6,7 @@ A `single-value` metrics aggregation that sums up numeric values that are extrac Assuming the data consists of documents representing sales records we can sum the sale price of all hats with: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -22,7 +22,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Resulting in: @@ -46,7 +45,7 @@ The name of the aggregation (`hat_prices` above) also serves as the key by which We could also use a script to fetch the sales price: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -68,12 +67,11 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -98,7 +96,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales,stored_example_script] ===== Value Script @@ -106,7 +103,7 @@ POST /sales/_search?size=0 It is also possible to access the field value from the script using `_value`. For example, this will sum the square of the prices for all hats: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -129,7 +126,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== Missing value @@ -139,7 +135,7 @@ be treated. By default documents missing the value will be ignored but it is also possible to treat them as if they had a value. For example, this treats all hat sales without a price as being `100`. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -160,5 +156,4 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] diff --git a/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc b/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc index 29949bd3582c..1b8f224e5f01 100644 --- a/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc @@ -32,7 +32,7 @@ The top_hits aggregation returns regular search hits, because of this many per h In the following example we group the sales by type and per type we show the last sale. For each sale only the date and price fields are being included in the source. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -63,7 +63,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Possible response: @@ -185,7 +184,7 @@ belong to. By defining a `terms` aggregator on the `domain` field we group the r Also a `max` aggregator is defined which is used by the `terms` aggregator's order feature to return the buckets by relevancy order of the most relevant document in a bucket. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -218,7 +217,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] At the moment the `max` (or `min`) aggregator is needed to make sure the buckets from the `terms` aggregator are @@ -239,7 +237,7 @@ and includes the array field and the offset in the array field the nested hit be Let's see how it works with a real sample. Considering the following mapping: -[source,js] +[source,console] -------------------------------------------------- PUT /sales { @@ -257,12 +255,12 @@ PUT /sales } } -------------------------------------------------- -// CONSOLE + <1> The `comments` is an array that holds nested documents under the `product` object. And some documents: -[source,js] +[source,console] -------------------------------------------------- PUT /sales/_doc/1?refresh { @@ -274,12 +272,11 @@ PUT /sales/_doc/1?refresh ] } -------------------------------------------------- -// CONSOLE // TEST[continued] It's now possible to execute the following `top_hits` aggregation (wrapped in a `nested` aggregation): -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -308,7 +305,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[s/_search/_search\?filter_path=aggregations.by_sale.by_user.buckets/] diff --git a/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc b/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc index 8d5c72994c4a..1a617c36ca47 100644 --- a/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc @@ -6,7 +6,7 @@ These values can be extracted either from specific fields in the documents, or b this aggregator will be used in conjunction with other single-value aggregations. For example, when computing the `avg` one might be interested in the number of values the average is computed over. -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -15,7 +15,6 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: @@ -40,7 +39,7 @@ retrieved from the returned response. Counting the values generated by a script: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -55,12 +54,11 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] This will interpret the `script` parameter as an `inline` script with the `painless` script language and no script parameters. To use a stored script use the following syntax: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search?size=0 { @@ -78,5 +76,4 @@ POST /sales/_search?size=0 } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales,stored_example_script] diff --git a/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc b/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc index 70804a7904f1..16c14b920527 100644 --- a/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc @@ -51,7 +51,7 @@ The `value` and `weight` objects have per-field specific configuration: If our documents have a `"grade"` field that holds a 0-100 numeric score, and a `"weight"` field which holds an arbitrary numeric weight, we can calculate the weighted average using: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search { @@ -70,7 +70,6 @@ POST /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] Which yields a response like: @@ -98,7 +97,7 @@ This single weight will be applied independently to each value extracted from th This example show how a single document with multiple values will be averaged with a single weight: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_doc?refresh { @@ -123,7 +122,6 @@ POST /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST The three values (`1`, `2`, and `3`) will be included as independent values, all with the weight of `2`: @@ -149,7 +147,7 @@ The aggregation returns `2.0` as the result, which matches what we would expect Both the value and the weight can be derived from a script, instead of a field. As a simple example, the following will add one to the grade and weight in the document using a script: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search { @@ -168,7 +166,6 @@ POST /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] @@ -182,7 +179,7 @@ If the `weight` field is missing, it is assumed to have a weight of `1` (like a Both of these defaults can be overridden with the `missing` parameter: -[source,js] +[source,console] -------------------------------------------------- POST /exams/_search { @@ -203,6 +200,5 @@ POST /exams/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:exams] diff --git a/docs/reference/aggregations/misc.asciidoc b/docs/reference/aggregations/misc.asciidoc index 678ebc0a8a4c..36bb2a9afd0c 100644 --- a/docs/reference/aggregations/misc.asciidoc +++ b/docs/reference/aggregations/misc.asciidoc @@ -15,7 +15,7 @@ See <> for more details. There are many occasions when aggregations are required but search hits are not. For these cases the hits can be ignored by setting `size=0`. For example: -[source,js] +[source,console] -------------------------------------------------- GET /twitter/_search { @@ -29,7 +29,6 @@ GET /twitter/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] Setting `size` to `0` avoids executing the fetch phase of the search making the request more efficient. @@ -42,7 +41,7 @@ at response time. Consider this example where we want to associate the color blue with our `terms` aggregation. -[source,js] +[source,console] -------------------------------------------------- GET /twitter/_search { @@ -59,7 +58,6 @@ GET /twitter/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] Then that piece of metadata will be returned in place for our `titles` terms aggregation @@ -94,7 +92,7 @@ Considering the following <> named `top_users`: -[source,js] +[source,console] -------------------------------------------------- GET /twitter/_search?typed_keys { @@ -115,7 +113,6 @@ GET /twitter/_search?typed_keys } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] In the response, the aggregations names will be changed to respectively `date_histogram#tweets_over_time` and diff --git a/docs/reference/aggregations/pipeline.asciidoc b/docs/reference/aggregations/pipeline.asciidoc index f31aaa08b9f3..b6ec7512647c 100644 --- a/docs/reference/aggregations/pipeline.asciidoc +++ b/docs/reference/aggregations/pipeline.asciidoc @@ -50,7 +50,7 @@ Paths are relative from the position of the pipeline aggregation; they are not a aggregation tree. For example, this derivative is embedded inside a date_histogram and refers to a "sibling" metric `"the_sum"`: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -72,7 +72,7 @@ POST /_search } } -------------------------------------------------- -// CONSOLE + <1> The metric is called `"the_sum"` <2> The `buckets_path` refers to the metric via a relative path `"the_sum"` @@ -80,7 +80,7 @@ POST /_search instead of embedded "inside" them. For example, the `max_bucket` aggregation uses the `buckets_path` to specify a metric embedded inside a sibling aggregation: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -106,8 +106,8 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] + <1> `buckets_path` instructs this max_bucket aggregation that we want the maximum value of the `sales` aggregation in the `sales_per_month` date histogram. @@ -115,7 +115,7 @@ If a Sibling pipeline agg references a multi-bucket aggregation, such as a `term select specific keys from the multi-bucket. For example, a `bucket_script` could select two specific buckets (via their bucket keys) to perform the calculation: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -152,8 +152,8 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] + <1> `buckets_path` selects the hats and bags buckets (via `['hat']`/`['bag']``) to use in the script specifically, instead of fetching all the buckets from `sale_type` aggregation @@ -164,7 +164,7 @@ Instead of pathing to a metric, `buckets_path` can use a special `"_count"` path the pipeline aggregation to use the document count as its input. For example, a derivative can be calculated on the document count of each bucket, instead of a specific metric: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -183,14 +183,14 @@ POST /_search } } -------------------------------------------------- -// CONSOLE + <1> By using `_count` instead of a metric name, we can calculate the derivative of document counts in the histogram The `buckets_path` can also use `"_bucket_count"` and path to a multi-bucket aggregation to use the number of buckets returned by that aggregation in the pipeline aggregation instead of a metric. for example a `bucket_selector` can be used here to filter out buckets which contain no buckets for an inner terms aggregation: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -222,8 +222,8 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] + <1> By using `_bucket_count` instead of a metric name, we can filter out `histo` buckets where they contain no buckets for the `categories` aggregation diff --git a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc index 33ccf505e5b8..12fced9c6dd2 100644 --- a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc @@ -33,7 +33,7 @@ An `avg_bucket` aggregation looks like this in isolation: The following snippet calculates the average of the total monthly `sales`: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -61,8 +61,8 @@ POST /_search } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] + <1> `buckets_path` instructs this avg_bucket aggregation that we want the (mean) average value of the `sales` aggregation in the `sales_per_month` date histogram. diff --git a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc index 6ecd1248a4e3..e3e6be8718d5 100644 --- a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc @@ -41,7 +41,7 @@ for more details) |Required | The following snippet calculates the ratio percentage of t-shirt sales compared to total sales each month: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -86,7 +86,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] And the following may be the response: diff --git a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc index 41ce04803fbe..a710901b943d 100644 --- a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc @@ -44,7 +44,7 @@ for more details) |Required | The following snippet only retains buckets where the total sales for the month is more than 200: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -74,7 +74,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] And the following may be the response: diff --git a/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc index a11da760edd0..25d4ccef4b01 100644 --- a/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc @@ -47,7 +47,7 @@ is ascending. The following snippet returns the buckets corresponding to the 3 months with the highest total sales in descending order: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -77,8 +77,8 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] + <1> `sort` is set to use the values of `total_sales` in descending order <2> `size` is set to `3` meaning only the top 3 months in `total_sales` will be returned @@ -135,7 +135,7 @@ without specifying `sort`. The following example simply truncates the result so that only the second bucket is returned: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -158,7 +158,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] Response: diff --git a/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc b/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc index 2e316164d64b..99dd32f85c42 100644 --- a/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc @@ -38,7 +38,7 @@ A `cumulative_cardinality` aggregation looks like this in isolation: The following snippet calculates the cumulative cardinality of the total daily `users`: -[source,js] +[source,console] -------------------------------------------------- GET /user_hits/_search { @@ -65,7 +65,6 @@ GET /user_hits/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:user_hits] <1> `buckets_path` instructs this aggregation to use the output of the `distinct_users` aggregation for the cumulative cardinality @@ -138,7 +137,7 @@ are added each day, rather than the total cumulative count. This can be accomplished by adding a `derivative` aggregation to our query: -[source,js] +[source,console] -------------------------------------------------- GET /user_hits/_search { @@ -170,7 +169,6 @@ GET /user_hits/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:user_hits] diff --git a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc index 226fd9c1bd10..6cc06d65bd53 100644 --- a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc @@ -31,7 +31,7 @@ A `cumulative_sum` aggregation looks like this in isolation: The following snippet calculates the cumulative sum of the total monthly `sales`: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -58,7 +58,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` instructs this cumulative sum aggregation to use the output of the `sales` aggregation for the cumulative sum diff --git a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc index d987294f9656..b0f2eb6cabb3 100644 --- a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc @@ -34,7 +34,7 @@ A `derivative` aggregation looks like this in isolation: The following snippet calculates the derivative of the total monthly `sales`: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -61,7 +61,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` instructs this derivative aggregation to use the output of the `sales` aggregation for the derivative @@ -128,7 +127,7 @@ A second order derivative can be calculated by chaining the derivative pipeline pipeline aggregation as in the following example which will calculate both the first and the second order derivative of the total monthly sales: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -160,7 +159,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` for the second derivative points to the name of the first derivative @@ -228,7 +226,7 @@ The derivative aggregation allows the units of the derivative values to be speci `normalized_value` which reports the derivative value in the desired x-axis units. In the below example we calculate the derivative of the total sales per month but ask for the derivative of the sales as in the units of sales per day: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -256,7 +254,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `unit` specifies what unit to use for the x-axis of the derivative calculation diff --git a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc index 8f9522ec3e5d..6ad9f61f3a16 100644 --- a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc @@ -35,7 +35,7 @@ A `extended_stats_bucket` aggregation looks like this in isolation: The following snippet calculates the extended stats for monthly `sales` bucket: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -62,7 +62,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `bucket_paths` instructs this `extended_stats_bucket` aggregation that we want the calculate stats for the `sales` aggregation in the diff --git a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc index 58bdab6128bf..07f72d655792 100644 --- a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc @@ -33,7 +33,7 @@ A `max_bucket` aggregation looks like this in isolation: The following snippet calculates the maximum of the total monthly `sales`: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -60,7 +60,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` instructs this max_bucket aggregation that we want the maximum value of the `sales` aggregation in the diff --git a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc index 8f075f7e071d..217c3dc0ad74 100644 --- a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc @@ -33,7 +33,7 @@ A `min_bucket` aggregation looks like this in isolation: The following snippet calculates the minimum of the total monthly `sales`: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -60,7 +60,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` instructs this min_bucket aggregation that we want the minimum value of the `sales` aggregation in the diff --git a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc index cdea58d45ae0..7f221d235900 100644 --- a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc @@ -38,7 +38,7 @@ A `moving_fn` aggregation looks like this in isolation: `moving_fn` aggregations must be embedded inside of a `histogram` or `date_histogram` aggregation. They can be embedded like any other metric aggregation: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -65,7 +65,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-day intervals @@ -140,7 +139,7 @@ kind of calculation and emit a single `double` as the result. Emitting `null` i For example, this script will simply return the first value from the window, or `NaN` if no values are available: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -167,7 +166,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] [[shift-parameter]] @@ -211,7 +209,7 @@ is only calculated over the real values. If the window is empty, or all values a |`values` |The window of values to find the maximum |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -238,7 +236,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ===== min Function @@ -254,7 +251,7 @@ is only calculated over the real values. If the window is empty, or all values a |`values` |The window of values to find the minimum |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -281,7 +278,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ===== sum Function @@ -297,7 +293,7 @@ the sum is only calculated over the real values. If the window is empty, or all |`values` |The window of values to find the sum of |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -324,7 +320,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ===== stdDev Function @@ -342,7 +337,7 @@ This function accepts a collection of doubles and average, then returns the stan |`avg` |The average of the window |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -369,7 +364,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] The `avg` parameter must be provided to the standard deviation function because different styles of averages can be computed on the window @@ -394,7 +388,7 @@ values. |`values` |The window of values to find the sum of |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -421,7 +415,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== linearWeightedAvg Function @@ -440,7 +433,7 @@ If the window is empty, or all values are `null`/`NaN`, `NaN` is returned as the |`values` |The window of values to find the sum of |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -467,7 +460,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] ==== ewma Function @@ -492,7 +484,7 @@ values. |`alpha` |Exponential decay |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -519,7 +511,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] @@ -550,7 +541,7 @@ values. |`beta` |Trend decay value |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -577,7 +568,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] In practice, the `alpha` value behaves very similarly in `holtMovAvg` as `ewmaMovAvg`: small values produce more smoothing @@ -616,7 +606,7 @@ values. |`multiplicative` |True if you wish to use multiplicative holt-winters, false to use additive |=== -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -643,7 +633,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] [WARNING] diff --git a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc index 6620e5689cb6..dd9017807d93 100644 --- a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc @@ -34,7 +34,7 @@ A `percentiles_bucket` aggregation looks like this in isolation: The following snippet calculates the percentiles for the total monthly `sales` buckets: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -62,7 +62,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` instructs this percentiles_bucket aggregation that we want to calculate percentiles for diff --git a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc index 7a414b95d402..9d2464a48ab4 100644 --- a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc @@ -60,7 +60,7 @@ A `serial_diff` aggregation looks like this in isolation: `serial_diff` aggregations must be embedded inside of a `histogram` or `date_histogram` aggregation: -[source,js] +[source,console] -------------------------------------------------- POST /_search { @@ -88,7 +88,6 @@ POST /_search } } -------------------------------------------------- -// CONSOLE <1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-day intervals <2> A `sum` metric is used to calculate the sum of a field. This could be any metric (sum, min, max, etc) diff --git a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc index 8c6359fb7765..827dc1b5559a 100644 --- a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc @@ -32,7 +32,7 @@ A `stats_bucket` aggregation looks like this in isolation: The following snippet calculates the stats for monthly `sales`: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -59,7 +59,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `bucket_paths` instructs this `stats_bucket` aggregation that we want the calculate stats for the `sales` aggregation in the diff --git a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc index f0a19f325412..0c88ca406675 100644 --- a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc @@ -32,7 +32,7 @@ A `sum_bucket` aggregation looks like this in isolation: The following snippet calculates the sum of all the total monthly `sales` buckets: -[source,js] +[source,console] -------------------------------------------------- POST /sales/_search { @@ -59,7 +59,6 @@ POST /sales/_search } } -------------------------------------------------- -// CONSOLE // TEST[setup:sales] <1> `buckets_path` instructs this sum_bucket aggregation that we want the sum of the `sales` aggregation in the diff --git a/docs/reference/search/suggesters/misc.asciidoc b/docs/reference/search/suggesters/misc.asciidoc index 1cd6e6fdbfc5..075b2a653233 100644 --- a/docs/reference/search/suggesters/misc.asciidoc +++ b/docs/reference/search/suggesters/misc.asciidoc @@ -6,7 +6,7 @@ Sometimes you need to know the exact type of a suggester in order to parse its r Considering the following example with two suggesters `term` and `phrase`: -[source,js] +[source,console] -------------------------------------------------- POST _search?typed_keys { @@ -25,7 +25,6 @@ POST _search?typed_keys } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] In the response, the suggester names will be changed to respectively `term#my-first-suggester` and diff --git a/docs/reference/search/suggesters/phrase-suggest.asciidoc b/docs/reference/search/suggesters/phrase-suggest.asciidoc index 46dfec5db1c1..9d49fc16bd00 100644 --- a/docs/reference/search/suggesters/phrase-suggest.asciidoc +++ b/docs/reference/search/suggesters/phrase-suggest.asciidoc @@ -21,7 +21,7 @@ In general the `phrase` suggester requires special mapping up front to work. The `phrase` suggester examples on this page need the following mapping to work. The `reverse` analyzer is used only in the last example. -[source,js] +[source,console] -------------------------------------------------- PUT test { @@ -74,13 +74,12 @@ POST test/_doc?refresh=true POST test/_doc?refresh=true {"title": "nobel prize"} -------------------------------------------------- -// CONSOLE // TESTSETUP Once you have the analyzers and mappings set up you can use the `phrase` suggester in the same spot you'd use the `term` suggester: -[source,js] +[source,console] -------------------------------------------------- POST test/_search { @@ -104,7 +103,6 @@ POST test/_search } } -------------------------------------------------- -// CONSOLE The response contains suggestions scored by the most likely spell correction first. In this case we received the expected correction "nobel prize". @@ -222,7 +220,7 @@ The response contains suggestions scored by the most likely spell correction fir matching documents for the phrase was found, `false` otherwise. The default value for `prune` is `false`. -[source,js] +[source,console] -------------------------------------------------- POST test/_search { @@ -253,7 +251,7 @@ POST test/_search } } -------------------------------------------------- -// CONSOLE + <1> This query will be run once for every suggestion. <2> The `{{suggestion}}` variable will be replaced by the text of each suggestion. @@ -291,7 +289,7 @@ properties that can be configured. All parameters (`trigram_lambda`, `bigram_lambda`, `unigram_lambda`) must be supplied. -[source,js] +[source,console] -------------------------------------------------- POST test/_search { @@ -311,7 +309,6 @@ POST test/_search } } -------------------------------------------------- -// CONSOLE ===== Candidate Generators @@ -412,7 +409,7 @@ of the direct generators to require a constant prefix to provide high-performance suggestions. The `pre_filter` and `post_filter` options accept ordinary analyzer names. -[source,js] +[source,console] -------------------------------------------------- POST test/_search { @@ -436,7 +433,6 @@ POST test/_search } } -------------------------------------------------- -// CONSOLE `pre_filter` and `post_filter` can also be used to inject synonyms after candidates are generated. For instance for the query `captain usq` we diff --git a/docs/reference/search/uri-request.asciidoc b/docs/reference/search/uri-request.asciidoc index 86e2ee4551b4..5e0a1fc8b7ba 100644 --- a/docs/reference/search/uri-request.asciidoc +++ b/docs/reference/search/uri-request.asciidoc @@ -3,11 +3,10 @@ Specifies search criteria as query parameters in the request URI. -[source,js] +[source,console] -------------------------------------------------- GET twitter/_search?q=user:kimchy -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] @@ -110,11 +109,10 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=terminate_after] [[search-uri-request-api-example]] ==== {api-examples-title} -[source,js] +[source,console] -------------------------------------------------- GET twitter/_search?q=user:kimchy -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] diff --git a/docs/reference/search/validate.asciidoc b/docs/reference/search/validate.asciidoc index 6e2705983707..94853c49e323 100644 --- a/docs/reference/search/validate.asciidoc +++ b/docs/reference/search/validate.asciidoc @@ -4,7 +4,7 @@ The validate API allows a user to validate a potentially expensive query without executing it. We'll use the following test data to explain _validate: -[source,js] +[source,console] -------------------------------------------------- PUT twitter/_bulk?refresh {"index":{"_id":1}} @@ -12,16 +12,14 @@ PUT twitter/_bulk?refresh {"index":{"_id":2}} {"user" : "kimchi", "post_date" : "2009-11-15T14:12:13", "message" : "My username is similar to @kimchy!"} -------------------------------------------------- -// CONSOLE // TESTSETUP When sent a valid query: -[source,js] +[source,console] -------------------------------------------------- GET twitter/_validate/query?q=user:foo -------------------------------------------------- -// CONSOLE The response contains `valid:true`: @@ -58,7 +56,7 @@ not. Defaults to `false`. The query may also be sent in the request body: -[source,js] +[source,console] -------------------------------------------------- GET twitter/_validate/query { @@ -76,7 +74,6 @@ GET twitter/_validate/query } } -------------------------------------------------- -// CONSOLE NOTE: The query being sent in the body must be nested in a `query` key, same as the <> works @@ -85,7 +82,7 @@ If the query is invalid, `valid` will be `false`. Here the query is invalid because Elasticsearch knows the post_date field should be a date due to dynamic mapping, and 'foo' does not correctly parse into a date: -[source,js] +[source,console] -------------------------------------------------- GET twitter/_validate/query { @@ -97,7 +94,6 @@ GET twitter/_validate/query } } -------------------------------------------------- -// CONSOLE [source,js] -------------------------------------------------- @@ -108,7 +104,7 @@ GET twitter/_validate/query An `explain` parameter can be specified to get more detailed information about why a query failed: -[source,js] +[source,console] -------------------------------------------------- GET twitter/_validate/query?explain=true { @@ -120,7 +116,6 @@ GET twitter/_validate/query?explain=true } } -------------------------------------------------- -// CONSOLE responds with: @@ -148,7 +143,7 @@ is more detailed showing the actual Lucene query that will be executed. For More Like This: -[source,js] +[source,console] -------------------------------------------------- GET twitter/_validate/query?rewrite=true { @@ -162,7 +157,6 @@ GET twitter/_validate/query?rewrite=true } } -------------------------------------------------- -// CONSOLE // TEST[skip:the output is randomized depending on which shard we hit] Response: @@ -195,7 +189,7 @@ all available shards. For Fuzzy Queries: -[source,js] +[source,console] -------------------------------------------------- GET twitter/_validate/query?rewrite=true&all_shards=true { @@ -209,7 +203,6 @@ GET twitter/_validate/query?rewrite=true&all_shards=true } } -------------------------------------------------- -// CONSOLE Response: diff --git a/docs/reference/setup/install/check-running.asciidoc b/docs/reference/setup/install/check-running.asciidoc index 6a9bab2ae094..e25d9f6a35fb 100644 --- a/docs/reference/setup/install/check-running.asciidoc +++ b/docs/reference/setup/install/check-running.asciidoc @@ -3,11 +3,10 @@ You can test that your Elasticsearch node is running by sending an HTTP request to port `9200` on `localhost`: -[source,js] +[source,console] -------------------------------------------- GET / -------------------------------------------- -// CONSOLE which should give you a response something like this: diff --git a/docs/reference/setup/logging-config.asciidoc b/docs/reference/setup/logging-config.asciidoc index ce747c76621e..29a125ec4722 100644 --- a/docs/reference/setup/logging-config.asciidoc +++ b/docs/reference/setup/logging-config.asciidoc @@ -151,7 +151,7 @@ PUT /_cluster/settings For example: -[source,js] +[source,console] ------------------------------- PUT /_cluster/settings { @@ -160,7 +160,6 @@ PUT /_cluster/settings } } ------------------------------- -// CONSOLE This is most appropriate when you need to dynamically need to adjust a logging level on an actively-running cluster. diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index 82b61848a846..7385d3146738 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -104,11 +104,12 @@ can be re-read and applied on a running node. The values of all secure settings, *reloadable* or not, must be identical across all cluster nodes. After making the desired secure settings changes, using the `bin/elasticsearch-keystore add` command, call: -[source,js] + +[source,console] ---- POST _nodes/reload_secure_settings ---- -// CONSOLE + This API will decrypt and re-read the entire keystore, on every cluster node, but only the *reloadable* secure settings will be applied. Changes to other settings will not go into effect until the next restart. Once the call returns, diff --git a/docs/reference/setup/sysconfig/file-descriptors.asciidoc b/docs/reference/setup/sysconfig/file-descriptors.asciidoc index 4c22a0021ff5..27d330b6a541 100644 --- a/docs/reference/setup/sysconfig/file-descriptors.asciidoc +++ b/docs/reference/setup/sysconfig/file-descriptors.asciidoc @@ -25,8 +25,7 @@ descriptors to 65535 and do not require further configuration. You can check the `max_file_descriptors` configured for each node using the <> API, with: -[source,js] +[source,console] -------------------------------------------------- GET _nodes/stats/process?filter_path=**.max_file_descriptors -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/setup/sysconfig/swap.asciidoc b/docs/reference/setup/sysconfig/swap.asciidoc index d41fe8abf8d2..5dbbb16a4f87 100644 --- a/docs/reference/setup/sysconfig/swap.asciidoc +++ b/docs/reference/setup/sysconfig/swap.asciidoc @@ -67,11 +67,10 @@ After starting Elasticsearch, you can see whether this setting was applied successfully by checking the value of `mlockall` in the output from this request: -[source,js] +[source,console] -------------- GET _nodes?filter_path=**.mlockall -------------- -// CONSOLE If you see that `mlockall` is `false`, then it means that the `mlockall` request has failed. You will also see a line with more information in the logs diff --git a/docs/reference/sql/endpoints/rest.asciidoc b/docs/reference/sql/endpoints/rest.asciidoc index 4fb644a62ba5..28e4d294cbdd 100644 --- a/docs/reference/sql/endpoints/rest.asciidoc +++ b/docs/reference/sql/endpoints/rest.asciidoc @@ -17,14 +17,13 @@ The SQL REST API accepts SQL in a JSON document, executes it, and returns the results. For example: -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=txt { "query": "SELECT * FROM library ORDER BY page_count DESC LIMIT 5" } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -107,7 +106,7 @@ Here are some examples for the human readable formats: ==== CSV -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=csv { @@ -115,7 +114,6 @@ POST /_sql?format=csv "fetch_size": 5 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -133,7 +131,7 @@ James S.A. Corey,Leviathan Wakes,561,2011-06-02T00:00:00.000Z ==== JSON -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=json { @@ -141,7 +139,6 @@ POST /_sql?format=json "fetch_size": 5 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -169,7 +166,7 @@ Which returns: ==== TSV -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=tsv { @@ -177,7 +174,6 @@ POST /_sql?format=tsv "fetch_size": 5 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -196,7 +192,7 @@ James S.A. Corey Leviathan Wakes 561 2011-06-02T00:00:00.000Z ==== TXT -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=txt { @@ -204,7 +200,6 @@ POST /_sql?format=txt "fetch_size": 5 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -224,7 +219,7 @@ James S.A. Corey |Leviathan Wakes |561 |2011-06-02T00:00:00.000Z ==== YAML -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=yaml { @@ -232,7 +227,6 @@ POST /_sql?format=yaml "fetch_size": 5 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -279,14 +273,13 @@ cursor: "sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWWWdrRlVfSS1TbDYtcW9lc1FJNmlYdw==:BAFmB Using the example above, one can continue to the next page by sending back the `cursor` field. In case of text format the cursor is returned as `Cursor` http header. -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=json { "cursor": "sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWYUpOYklQMHhRUEtld3RsNnFtYU1hQQ==:BAFmBGRhdGUBZgVsaWtlcwFzB21lc3NhZ2UBZgR1c2Vy9f///w8=" } -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[s/sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWYUpOYklQMHhRUEtld3RsNnFtYU1hQQ==:BAFmBGRhdGUBZgVsaWtlcwFzB21lc3NhZ2UBZgR1c2Vy9f\/\/\/w8=/$body.cursor/] @@ -317,14 +310,13 @@ Elasticsearch state is cleared. To clear the state earlier, you can use the clear cursor command: -[source,js] +[source,console] -------------------------------------------------- POST /_sql/close { "cursor": "sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWYUpOYklQMHhRUEtld3RsNnFtYU1hQQ==:BAFmBGRhdGUBZgVsaWtlcwFzB21lc3NhZ2UBZgR1c2Vy9f///w8=" } -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[s/sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWYUpOYklQMHhRUEtld3RsNnFtYU1hQQ==:BAFmBGRhdGUBZgVsaWtlcwFzB21lc3NhZ2UBZgR1c2Vy9f\/\/\/w8=/$body.cursor/] @@ -347,7 +339,7 @@ You can filter the results that SQL will run on using a standard {es} query DSL by specifying the query in the filter parameter. -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=txt { @@ -363,7 +355,6 @@ POST /_sql?format=txt "fetch_size": 5 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -386,7 +377,7 @@ in a columnar fashion: one row represents all the values of a certain column fro The following formats can be returned in columnar orientation: `json`, `yaml`, `cbor` and `smile`. -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=json { @@ -395,7 +386,6 @@ POST /_sql?format=json "columnar": true } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: @@ -423,7 +413,7 @@ Which returns: Any subsequent calls using a `cursor` still have to contain the `columnar` parameter to preserve the orientation, meaning the initial query will not _remember_ the columnar option. -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=json { @@ -431,7 +421,6 @@ POST /_sql?format=json "columnar": true } -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[s/sDXF1ZXJ5QW5kRmV0Y2gBAAAAAAAAAAEWWWdrRlVfSS1TbDYtcW9lc1FJNmlYdw==:BAFmBmF1dGhvcgFmBG5hbWUBZgpwYWdlX2NvdW50AWYMcmVsZWFzZV9kYXRl\+v\/\/\/w8=/$body.cursor/] diff --git a/docs/reference/sql/endpoints/translate.asciidoc b/docs/reference/sql/endpoints/translate.asciidoc index 978fae410390..403a8011ad4e 100644 --- a/docs/reference/sql/endpoints/translate.asciidoc +++ b/docs/reference/sql/endpoints/translate.asciidoc @@ -6,7 +6,7 @@ The SQL Translate API accepts SQL in a JSON document and translates it into native {es} queries. For example: -[source,js] +[source,console] -------------------------------------------------- POST /_sql/translate { @@ -14,7 +14,6 @@ POST /_sql/translate "fetch_size": 10 } -------------------------------------------------- -// CONSOLE // TEST[setup:library] Which returns: diff --git a/docs/reference/sql/getting-started.asciidoc b/docs/reference/sql/getting-started.asciidoc index ffd52060e2b0..dbcdb68d5e6a 100644 --- a/docs/reference/sql/getting-started.asciidoc +++ b/docs/reference/sql/getting-started.asciidoc @@ -6,7 +6,7 @@ To start using {es-sql}, create an index with some data to experiment with: -[source,js] +[source,console] -------------------------------------------------- PUT /library/book/_bulk?refresh {"index":{"_id": "Leviathan Wakes"}} @@ -16,18 +16,16 @@ PUT /library/book/_bulk?refresh {"index":{"_id": "Dune"}} {"name": "Dune", "author": "Frank Herbert", "release_date": "1965-06-01", "page_count": 604} -------------------------------------------------- -// CONSOLE And now you can execute SQL using the <> right away: -[source,js] +[source,console] -------------------------------------------------- POST /_sql?format=txt { "query": "SELECT * FROM library WHERE release_date < '2000-01-01'" } -------------------------------------------------- -// CONSOLE // TEST[continued] Which should return something along the lines of: diff --git a/docs/reference/upgrade/close-ml.asciidoc b/docs/reference/upgrade/close-ml.asciidoc index affcd2792667..92be1e4106d8 100644 --- a/docs/reference/upgrade/close-ml.asciidoc +++ b/docs/reference/upgrade/close-ml.asciidoc @@ -3,11 +3,10 @@ //////////// Take us out of upgrade mode after running any snippets on this page. -[source,js] +[source,console] -------------------------------------------------- POST _ml/set_upgrade_mode?enabled=false -------------------------------------------------- -// CONSOLE // TEARDOWN //////////// @@ -25,11 +24,10 @@ it puts increased load on the cluster. prevent new jobs from opening by using the <>: + -[source,js] +[source,console] -------------------------------------------------- POST _ml/set_upgrade_mode?enabled=true -------------------------------------------------- -// CONSOLE + When you disable upgrade mode, the jobs resume using the last model state that was automatically saved. This option avoids the overhead of managing diff --git a/docs/reference/upgrade/cluster_restart.asciidoc b/docs/reference/upgrade/cluster_restart.asciidoc index d5d78bc04c91..eeb533138d4d 100644 --- a/docs/reference/upgrade/cluster_restart.asciidoc +++ b/docs/reference/upgrade/cluster_restart.asciidoc @@ -79,13 +79,12 @@ cluster and elect a master. At that point, you can use <> and <> to monitor nodes joining the cluster: -[source,sh] +[source,console] -------------------------------------------------- GET _cat/health GET _cat/nodes -------------------------------------------------- -// CONSOLE The `status` column returned by `_cat/health` shows the health of each node in the cluster: `red`, `yellow`, or `green`. @@ -113,7 +112,7 @@ When all nodes have joined the cluster and recovered their primary shards, reenable allocation by restoring `cluster.routing.allocation.enable` to its default: -[source,js] +[source,console] ------------------------------------------------------ PUT _cluster/settings { @@ -122,7 +121,6 @@ PUT _cluster/settings } } ------------------------------------------------------ -// CONSOLE Once allocation is reenabled, the cluster starts allocating replica shards to the data nodes. At this point it is safe to resume indexing and searching, @@ -133,13 +131,12 @@ is `green`. You can monitor progress with the <> and <> APIs: -[source,sh] +[source,console] -------------------------------------------------- GET _cat/health GET _cat/recovery -------------------------------------------------- -// CONSOLE -- . *Restart machine learning jobs.* diff --git a/docs/reference/upgrade/disable-shard-alloc.asciidoc b/docs/reference/upgrade/disable-shard-alloc.asciidoc index 839488f541f5..8f238a2c2c6a 100644 --- a/docs/reference/upgrade/disable-shard-alloc.asciidoc +++ b/docs/reference/upgrade/disable-shard-alloc.asciidoc @@ -7,7 +7,7 @@ restarted, this I/O is unnecessary. You can avoid racing the clock by <> of replicas before shutting down the node: -[source,js] +[source,console] -------------------------------------------------- PUT _cluster/settings { @@ -16,5 +16,4 @@ PUT _cluster/settings } } -------------------------------------------------- -// CONSOLE // TEST[skip:indexes don't assign] diff --git a/docs/reference/upgrade/open-ml.asciidoc b/docs/reference/upgrade/open-ml.asciidoc index a8882e7f1345..59bcc651c7d4 100644 --- a/docs/reference/upgrade/open-ml.asciidoc +++ b/docs/reference/upgrade/open-ml.asciidoc @@ -3,11 +3,10 @@ If you temporarily halted the tasks associated with your {ml} jobs, use the <> to return them to active states: -[source,js] +[source,console] -------------------------------------------------- POST _ml/set_upgrade_mode?enabled=false -------------------------------------------------- -// CONSOLE If you closed all {ml} jobs before the upgrade, open the jobs and start the datafeeds from {kib} or with the <> and diff --git a/docs/reference/upgrade/reindex_upgrade.asciidoc b/docs/reference/upgrade/reindex_upgrade.asciidoc index faa8fbc1639b..4beda3b53204 100644 --- a/docs/reference/upgrade/reindex_upgrade.asciidoc +++ b/docs/reference/upgrade/reindex_upgrade.asciidoc @@ -156,7 +156,7 @@ cluster and remove nodes from the old one. remote index into the new {version} index: + -- -[source,js] +[source,console] -------------------------------------------------- POST _reindex { @@ -178,7 +178,6 @@ POST _reindex } } -------------------------------------------------- -// CONSOLE // TEST[setup:host] // TEST[s/^/PUT source\n/] // TEST[s/oldhost:9200",/\${host}"/] diff --git a/docs/reference/upgrade/rolling_upgrade.asciidoc b/docs/reference/upgrade/rolling_upgrade.asciidoc index 5ee75dd62b89..ddc1e55ba3b0 100644 --- a/docs/reference/upgrade/rolling_upgrade.asciidoc +++ b/docs/reference/upgrade/rolling_upgrade.asciidoc @@ -73,11 +73,10 @@ particular, the placement of the realm type changed. See Start the newly-upgraded node and confirm that it joins the cluster by checking the log file or by submitting a `_cat/nodes` request: -[source,sh] +[source,console] -------------------------------------------------- GET _cat/nodes -------------------------------------------------- -// CONSOLE -- . *Reenable shard allocation.* @@ -87,7 +86,7 @@ GET _cat/nodes Once the node has joined the cluster, remove the `cluster.routing.allocation.enable` setting to enable shard allocation and start using the node: -[source,js] +[source,console] -------------------------------------------------- PUT _cluster/settings { @@ -96,7 +95,6 @@ PUT _cluster/settings } } -------------------------------------------------- -// CONSOLE -- . *Wait for the node to recover.* @@ -106,11 +104,10 @@ PUT _cluster/settings Before upgrading the next node, wait for the cluster to finish shard allocation. You can check progress by submitting a <> request: -[source,sh] +[source,console] -------------------------------------------------- GET _cat/health?v -------------------------------------------------- -// CONSOLE Wait for the `status` column to switch from `yellow` to `green`. Once the node is `green`, all primary and replica shards have been allocated. @@ -137,11 +134,10 @@ Shards that were not <> might take longer to recover. You can monitor the recovery status of individual shards by submitting a <> request: -[source,sh] +[source,console] -------------------------------------------------- GET _cat/recovery -------------------------------------------------- -// CONSOLE If you stopped indexing, it is safe to resume indexing as soon as recovery completes. diff --git a/docs/reference/upgrade/synced-flush.asciidoc b/docs/reference/upgrade/synced-flush.asciidoc index d909688f6a43..59bdea15ba74 100644 --- a/docs/reference/upgrade/synced-flush.asciidoc +++ b/docs/reference/upgrade/synced-flush.asciidoc @@ -1,9 +1,8 @@ -[source,sh] +[source,console] -------------------------------------------------- POST _flush/synced -------------------------------------------------- -// CONSOLE When you perform a synced flush, check the response to make sure there are no failures. Synced flush operations that fail due to pending indexing diff --git a/docs/reference/vectors/vector-functions.asciidoc b/docs/reference/vectors/vector-functions.asciidoc index 31b6b211e6ae..1a4876096b0c 100644 --- a/docs/reference/vectors/vector-functions.asciidoc +++ b/docs/reference/vectors/vector-functions.asciidoc @@ -17,7 +17,7 @@ to limit the number of matched documents with a `query` parameter. Let's create an index with the following mapping and index a couple of documents into it. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -52,13 +52,12 @@ PUT my_index/_doc/2 } -------------------------------------------------- -// CONSOLE // TESTSETUP For dense_vector fields, `cosineSimilarity` calculates the measure of cosine similarity between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -83,7 +82,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> To restrict the number of documents on which script score calculation is applied, provide a filter. <2> The script adds 1.0 to the cosine similarity to prevent the score from being negative. <3> To take advantage of the script optimizations, provide a query vector as a script parameter. @@ -94,7 +93,7 @@ different from the query's vector, an error will be thrown. Similarly, for sparse_vector fields, `cosineSimilaritySparse` calculates cosine similarity between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -119,12 +118,11 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE For dense_vector fields, `dotProduct` calculates the measure of dot product between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -152,14 +150,13 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE <1> Using the standard sigmoid function prevents scores from being negative. Similarly, for sparse_vector fields, `dotProductSparse` calculates dot product between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -187,13 +184,12 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE For dense_vector fields, `l1norm` calculates L^1^ distance (Manhattan distance) between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -218,7 +214,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE <1> Unlike `cosineSimilarity` that represent similarity, `l1norm` and `l2norm` shown below represent distances or differences. This means, that @@ -232,7 +227,7 @@ we added `1` in the denominator. For sparse_vector fields, `l1normSparse` calculates L^1^ distance between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -257,13 +252,12 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE For dense_vector fields, `l2norm` calculates L^2^ distance (Euclidean distance) between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -288,12 +282,11 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE Similarly, for sparse_vector fields, `l2normSparse` calculates L^2^ distance between a given query vector and document vectors. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -318,7 +311,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE NOTE: If a document doesn't have a value for a vector field on which a vector function is executed, an error will be thrown. From 860aff0f3743f6c0f4f3f00ee87b341034d5e2ee Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Wed, 4 Sep 2019 10:12:03 -0700 Subject: [PATCH 049/103] [DOCS] Identify reloadable EC2 Discovery Plugin settings (#46102) --- docs/plugins/discovery-ec2.asciidoc | 22 +++++++++---------- docs/reference/setup/secure-settings.asciidoc | 5 +++++ 2 files changed, 15 insertions(+), 12 deletions(-) diff --git a/docs/plugins/discovery-ec2.asciidoc b/docs/plugins/discovery-ec2.asciidoc index 9808aff41ddc..27233d8e4b7a 100644 --- a/docs/plugins/discovery-ec2.asciidoc +++ b/docs/plugins/discovery-ec2.asciidoc @@ -56,24 +56,22 @@ bin/elasticsearch-keystore add discovery.ec2.secret_key The available settings for the EC2 discovery plugin are as follows. -`discovery.ec2.access_key`:: +`discovery.ec2.access_key` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: An EC2 access key. If set, you must also set `discovery.ec2.secret_key`. If unset, `discovery-ec2` will instead use the instance role. This setting - is sensitive and must be stored in the {ref}/secure-settings.html[{es} - keystore]. + is sensitive and must be stored in the {es} keystore. -`discovery.ec2.secret_key`:: +`discovery.ec2.secret_key` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: An EC2 secret key. If set, you must also set `discovery.ec2.access_key`. - This setting is sensitive and must be stored in the - {ref}/secure-settings.html[{es} keystore]. + This setting is sensitive and must be stored in the {es} keystore. -`discovery.ec2.session_token`:: +`discovery.ec2.session_token` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: An EC2 session token. If set, you must also set `discovery.ec2.access_key` and `discovery.ec2.secret_key`. This setting is sensitive and must be - stored in the {ref}/secure-settings.html[{es} keystore]. + stored in the {es} keystore. `discovery.ec2.endpoint`:: @@ -99,19 +97,19 @@ The available settings for the EC2 discovery plugin are as follows. this setting determines the port to use to connect to the proxy. Defaults to `80`. -`discovery.ec2.proxy.username`:: +`discovery.ec2.proxy.username` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: When the address of an HTTP proxy is given in `discovery.ec2.proxy.host`, this setting determines the username to use to connect to the proxy. When not set, no username is used. This setting is sensitive and must be stored - in the {ref}/secure-settings.html[{es} keystore]. + in the {es} keystore. -`discovery.ec2.proxy.password`:: +`discovery.ec2.proxy.password` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: When the address of an HTTP proxy is given in `discovery.ec2.proxy.host`, this setting determines the password to use to connect to the proxy. When not set, no password is used. This setting is sensitive and must be stored - in the {ref}/secure-settings.html[{es} keystore]. + in the {es} keystore. `discovery.ec2.read_timeout`:: diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index 7385d3146738..f795baa5ed9f 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -120,3 +120,8 @@ had the new value from the start. When changing multiple *reloadable* secure settings, modify all of them, on each cluster node, and then issue a `reload_secure_settings` call, instead of reloading after each modification. + +There are reloadable secure settings for: + +* {plugins}/discovery-ec2-usage.html#_configuring_ec2_discovery[The EC2 Discovery Plugin] + From 903753db61929d4f95118cd39cf227e03b606828 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 4 Sep 2019 13:05:41 -0500 Subject: [PATCH 050/103] [ML] testFullClusterRestart waiting for stable cluster (#46280) * [ML] waiting for ml indices before waiting task assignment testFullClusterRestart * waiting for a stable cluster after fullrestart * removing unused imports --- .../xpack/ml/integration/MlDistributedFailureIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlDistributedFailureIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlDistributedFailureIT.java index 711c9687144c..aedbda091770 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlDistributedFailureIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlDistributedFailureIT.java @@ -113,6 +113,7 @@ public class MlDistributedFailureIT extends BaseMlIntegTestCase { logger.info("Restarting all nodes"); internalCluster().fullRestart(); logger.info("Restarted all nodes"); + ensureStableClusterOnAllNodes(3); }); } From c0de698d340623deaf9f4c9b54627a90b826c36c Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 4 Sep 2019 13:12:23 -0500 Subject: [PATCH 051/103] [ML][Transforms] fixing rolling upgrade continuous transform test (#45823) * [ML][Transforms] fixing rolling upgrade continuous transform test * adjusting wait assert logic * adjusting wait conditions --- .../upgrades/DataFrameSurvivesUpgradeIT.java | 101 ++++++++++++++---- 1 file changed, 82 insertions(+), 19 deletions(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java index a4a5025a139f..24ae1ce99275 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java @@ -7,10 +7,10 @@ package org.elasticsearch.upgrades; import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; +import org.elasticsearch.client.core.IndexerState; import org.elasticsearch.client.dataframe.GetDataFrameTransformStatsResponse; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats; @@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; @@ -37,7 +38,9 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -48,7 +51,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.oneOf; -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/43662") public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); @@ -80,11 +82,18 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { */ public void testDataFramesRollingUpgrade() throws Exception { assumeTrue("Continuous data frames not supported until 7.3", UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_3_0)); + Request adjustLoggingLevels = new Request("PUT", "/_cluster/settings"); + adjustLoggingLevels.setJsonEntity( + "{\"transient\": {" + + "\"logger.org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer\": \"trace\"," + + "\"logger.org.elasticsearch.xpack.dataframe\": \"trace\"}}"); + client().performRequest(adjustLoggingLevels); Request waitForYellow = new Request("GET", "/_cluster/health"); waitForYellow.addParameter("wait_for_nodes", "3"); waitForYellow.addParameter("wait_for_status", "yellow"); switch (CLUSTER_TYPE) { case OLD: + client().performRequest(waitForYellow); createAndStartContinuousDataFrame(); break; case MIXED: @@ -113,15 +122,15 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { private void createAndStartContinuousDataFrame() throws Exception { createIndex(CONTINUOUS_DATA_FRAME_SOURCE); - long totalDocsWritten = 0; + long totalDocsWrittenSum = 0; for (TimeValue bucket : BUCKETS) { int docs = randomIntBetween(1, 25); putData(CONTINUOUS_DATA_FRAME_SOURCE, docs, bucket, ENTITIES); - totalDocsWritten += docs * ENTITIES.size(); + totalDocsWrittenSum += docs * ENTITIES.size(); } - + long totalDocsWritten = totalDocsWrittenSum; DataFrameTransformConfig config = DataFrameTransformConfig.builder() - .setSyncConfig(new TimeSyncConfig("timestamp", TimeValue.timeValueSeconds(30))) + .setSyncConfig(new TimeSyncConfig("timestamp", TimeValue.timeValueSeconds(1))) .setPivotConfig(PivotConfig.builder() .setAggregations(new AggregatorFactories.Builder().addAggregator(AggregationBuilders.avg("stars").field("stars"))) .setGroups(GroupConfig.builder().groupBy("user_id", TermsGroupSource.builder().setField("user_id").build()).build()) @@ -129,19 +138,28 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { .setDest(DestConfig.builder().setIndex(CONTINUOUS_DATA_FRAME_ID + "_idx").build()) .setSource(SourceConfig.builder().setIndex(CONTINUOUS_DATA_FRAME_SOURCE).build()) .setId(CONTINUOUS_DATA_FRAME_ID) + .setFrequency(TimeValue.timeValueSeconds(1)) .build(); putTransform(CONTINUOUS_DATA_FRAME_ID, config); startTransform(CONTINUOUS_DATA_FRAME_ID); waitUntilAfterCheckpoint(CONTINUOUS_DATA_FRAME_ID, 0L); - DataFrameTransformStats stateAndStats = getTransformStats(CONTINUOUS_DATA_FRAME_ID); + assertBusy(() -> { + DataFrameTransformStats stateAndStats = getTransformStats(CONTINUOUS_DATA_FRAME_ID); + assertThat(stateAndStats.getIndexerStats().getOutputDocuments(), equalTo((long)ENTITIES.size())); + assertThat(stateAndStats.getIndexerStats().getNumDocuments(), equalTo(totalDocsWritten)); + // Even if we get back to started, we may periodically get set back to `indexing` when triggered. + // Though short lived due to no changes on the source indices, it could result in flaky test behavior + assertThat(stateAndStats.getState(), oneOf(DataFrameTransformStats.State.STARTED, DataFrameTransformStats.State.INDEXING)); + }, 120, TimeUnit.SECONDS); - assertThat(stateAndStats.getIndexerStats().getOutputDocuments(), equalTo((long)ENTITIES.size())); - assertThat(stateAndStats.getIndexerStats().getNumDocuments(), equalTo(totalDocsWritten)); - assertThat(stateAndStats.getState(), oneOf(DataFrameTransformStats.State.STARTED, DataFrameTransformStats.State.INDEXING)); + + // We want to make sure our latest state is written before we turn the node off, this makes the testing more reliable + awaitWrittenIndexerState(CONTINUOUS_DATA_FRAME_ID, IndexerState.STARTED.value()); } + @SuppressWarnings("unchecked") private void verifyContinuousDataFrameHandlesData(long expectedLastCheckpoint) throws Exception { // A continuous data frame should automatically become started when it gets assigned to a node @@ -161,9 +179,9 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { List entities = new ArrayList<>(1); entities.add("user_" + ENTITIES.size() + expectedLastCheckpoint); int docs = 5; - // Index the data very recently in the past so that the transform sync delay can catch up to reading it in our spin - // wait later. - putData(CONTINUOUS_DATA_FRAME_SOURCE, docs, TimeValue.timeValueSeconds(1), entities); + // Index the data + // The frequency and delay should see the data once its indexed + putData(CONTINUOUS_DATA_FRAME_SOURCE, docs, TimeValue.timeValueSeconds(0), entities); waitUntilAfterCheckpoint(CONTINUOUS_DATA_FRAME_ID, expectedLastCheckpoint); @@ -176,10 +194,55 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { assertThat(stateAndStats.getState(), oneOf(DataFrameTransformStats.State.STARTED, DataFrameTransformStats.State.INDEXING)); - assertThat(stateAndStats.getIndexerStats().getOutputDocuments(), - greaterThan(previousStateAndStats.getIndexerStats().getOutputDocuments())); - assertThat(stateAndStats.getIndexerStats().getNumDocuments(), - greaterThanOrEqualTo(docs + previousStateAndStats.getIndexerStats().getNumDocuments())); + awaitWrittenIndexerState(CONTINUOUS_DATA_FRAME_ID, (responseBody) -> { + Map indexerStats = (Map)((List)XContentMapValues.extractValue("hits.hits._source.stats", + responseBody)) + .get(0); + assertThat((Integer)indexerStats.get("documents_indexed"), + greaterThan(Long.valueOf(previousStateAndStats.getIndexerStats().getOutputDocuments()).intValue())); + assertThat((Integer)indexerStats.get("documents_processed"), + greaterThan(Long.valueOf(previousStateAndStats.getIndexerStats().getNumDocuments()).intValue())); + }); + } + + private void awaitWrittenIndexerState(String id, Consumer> responseAssertion) throws Exception { + Request getStatsDocsRequest = new Request("GET", ".data-frame-internal-*/_search"); + getStatsDocsRequest.setJsonEntity("{\n" + + " \"query\": {\n" + + " \"bool\": {\n" + + " \"filter\": \n" + + " {\"term\": {\n" + + " \"_id\": \"data_frame_transform_state_and_stats-" + id + "\"\n" + + " }}\n" + + " }\n" + + " },\n" + + " \"sort\": [\n" + + " {\n" + + " \"_index\": {\n" + + " \"order\": \"desc\"\n" + + " }\n" + + " }\n" + + " ],\n" + + " \"size\": 1\n" + + "}"); + assertBusy(() -> { + // Want to make sure we get the latest docs + client().performRequest(new Request("POST", ".data-frame-internal-*/_refresh")); + Response response = client().performRequest(getStatsDocsRequest); + assertEquals(200, response.getStatusLine().getStatusCode()); + Map responseBody = entityAsMap(response); + assertEquals(1, XContentMapValues.extractValue("hits.total.value", responseBody)); + responseAssertion.accept(responseBody); + }, 60, TimeUnit.SECONDS); + } + + private void awaitWrittenIndexerState(String id, String indexerState) throws Exception { + awaitWrittenIndexerState(id, (responseBody) -> { + String storedState = ((List)XContentMapValues.extractValue("hits.hits._source.state.indexer_state", responseBody)) + .get(0) + .toString(); + assertThat(storedState, equalTo(indexerState)); + }); } private void putTransform(String id, DataFrameTransformConfig config) throws IOException { @@ -222,7 +285,7 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { } private void waitUntilAfterCheckpoint(String id, long currentCheckpoint) throws Exception { - assertBusy(() -> assertThat(getTransformStats(id).getCheckpointingInfo().getNext().getCheckpoint(), greaterThan(currentCheckpoint)), + assertBusy(() -> assertThat(getTransformStats(id).getCheckpointingInfo().getLast().getCheckpoint(), greaterThan(currentCheckpoint)), 60, TimeUnit.SECONDS); } @@ -249,7 +312,7 @@ public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { final StringEntity entity = new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON); Request req = new Request("PUT", indexName); req.setEntity(entity); - client().performRequest(req); + assertThat(client().performRequest(req).getStatusLine().getStatusCode(), equalTo(200)); } } From c4fa32d665f478ec118b2ca454aac3b72427ab54 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 4 Sep 2019 14:32:48 -0500 Subject: [PATCH 052/103] muting test (#46343) --- .../org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java index 24ae1ce99275..b307d93073cc 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java @@ -7,6 +7,7 @@ package org.elasticsearch.upgrades; import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -51,6 +52,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.oneOf; +@LuceneTestCase.AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/46341") public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); From a4ed7b1ca102267ecdda7c788713675ed3958f3f Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 4 Sep 2019 21:48:03 +0200 Subject: [PATCH 053/103] Decouple shard allocation awareness from search and get requests (#45735) With this commit, Elasticsearch will no longer prefer using shards in the same location (with the same awareness attribute values) to process `_search` and `_get` requests. Instead, adaptive replica selection (the default since 7.0) should route requests more efficiently using the service time of prior inter-node communications. Clusters with big latencies between nodes should switch to cross cluster replication to isolate nodes within the same zone. Note that this change only targets 8.0 since it is considered as breaking. However a follow up pr should add an option to activate this behavior in 7.x in order to allow users to opt-in early. Closes #43453 --- .../migration/migrate_8_0/search.asciidoc | 12 ++- .../cluster/allocation_awareness.asciidoc | 4 - .../routing/IndexShardRoutingTable.java | 88 ------------------- .../cluster/routing/OperationRouting.java | 26 +----- .../structure/RoutingIteratorTests.java | 55 ------------ 5 files changed, 14 insertions(+), 171 deletions(-) diff --git a/docs/reference/migration/migrate_8_0/search.asciidoc b/docs/reference/migration/migrate_8_0/search.asciidoc index 97796a10fca2..4c3be2b4fadd 100644 --- a/docs/reference/migration/migrate_8_0/search.asciidoc +++ b/docs/reference/migration/migrate_8_0/search.asciidoc @@ -12,16 +12,24 @@ The `/{index}/{type}/_termvectors`, `/{index}/{type}/{id}/_termvectors` and `/{i [float] ==== Removal of queries -The `common` query was deprecated in 7.x and has been removed in 8.0. +The `common` query, deprecated in 7.x, has been removed in 8.0. The same functionality can be achieved by the `match` query if the total number of hits is not tracked. [float] ===== Removal of query parameters -The `cutoff_frequency` parameter was deprecated in 7.x and has been removed in 8.0 from `match` and `multi_match` queries. +The `cutoff_frequency` parameter, deprecated in 7.x, has been removed in 8.0 from `match` and `multi_match` queries. The same functionality can be achieved without any configuration provided that the total number of hits is not tracked. [float] ===== Removal of sort parameters The `nested_filter` and `nested_path` options, deprecated in 6.x, have been removed in favor of the `nested` context. + + +[float] +===== Shard allocation awareness in Search and Get requests + +{es} will no longer prefer using shards in the same location (with the same awareness attribute values) to process +`_search` and `_get` requests. Adaptive replica selection (activated by default in this version) will route requests +more efficiently using the service time of prior inter-node communications. \ No newline at end of file diff --git a/docs/reference/modules/cluster/allocation_awareness.asciidoc b/docs/reference/modules/cluster/allocation_awareness.asciidoc index 5fc9197d449d..2d81be8a87ec 100644 --- a/docs/reference/modules/cluster/allocation_awareness.asciidoc +++ b/docs/reference/modules/cluster/allocation_awareness.asciidoc @@ -17,10 +17,6 @@ The allocation awareness settings can be configured in `elasticsearch.yml` and updated dynamically with the <> API. -{es} prefers using shards in the same location (with the same -awareness attribute values) to process search or GET requests. Using local -shards is usually faster than crossing rack or zone boundaries. - NOTE: The number of attribute values determines how many shard copies are allocated in each location. If the number of nodes in each location is unbalanced and there are a lot of replicas, replica shards might be left diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index 5a98e9456f43..9fe807c39428 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -19,13 +19,11 @@ package org.elasticsearch.cluster.routing; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -44,8 +42,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import static java.util.Collections.emptyMap; - /** * {@link IndexShardRoutingTable} encapsulates all instances of a single shard. * Each Elasticsearch index consists of multiple shards, each shard encapsulates @@ -67,10 +63,6 @@ public class IndexShardRoutingTable implements Iterable { final Set allAllocationIds; final boolean allShardsStarted; - private volatile Map activeShardsByAttributes = emptyMap(); - private volatile Map initializingShardsByAttributes = emptyMap(); - private final Object shardsByAttributeMutex = new Object(); - /** * The initializing list, including ones that are initializing on a target node because of relocation. * If we can come up with a better variable name, it would be nice... @@ -549,86 +541,6 @@ public class IndexShardRoutingTable implements Iterable { } } - static class AttributesRoutings { - - public final List withSameAttribute; - public final List withoutSameAttribute; - public final int totalSize; - - AttributesRoutings(List withSameAttribute, List withoutSameAttribute) { - this.withSameAttribute = withSameAttribute; - this.withoutSameAttribute = withoutSameAttribute; - this.totalSize = withoutSameAttribute.size() + withSameAttribute.size(); - } - } - - private AttributesRoutings getActiveAttribute(AttributesKey key, DiscoveryNodes nodes) { - AttributesRoutings shardRoutings = activeShardsByAttributes.get(key); - if (shardRoutings == null) { - synchronized (shardsByAttributeMutex) { - ArrayList from = new ArrayList<>(activeShards); - List to = collectAttributeShards(key, nodes, from); - - shardRoutings = new AttributesRoutings(to, Collections.unmodifiableList(from)); - activeShardsByAttributes = Maps.copyMapWithAddedEntry(activeShardsByAttributes, key, shardRoutings); - } - } - return shardRoutings; - } - - private AttributesRoutings getInitializingAttribute(AttributesKey key, DiscoveryNodes nodes) { - AttributesRoutings shardRoutings = initializingShardsByAttributes.get(key); - if (shardRoutings == null) { - synchronized (shardsByAttributeMutex) { - ArrayList from = new ArrayList<>(allInitializingShards); - List to = collectAttributeShards(key, nodes, from); - shardRoutings = new AttributesRoutings(to, Collections.unmodifiableList(from)); - initializingShardsByAttributes = - Maps.copyMapWithAddedEntry(initializingShardsByAttributes, key, shardRoutings); - } - } - return shardRoutings; - } - - private static List collectAttributeShards(AttributesKey key, DiscoveryNodes nodes, ArrayList from) { - final ArrayList to = new ArrayList<>(); - for (final String attribute : key.attributes) { - final String localAttributeValue = nodes.getLocalNode().getAttributes().get(attribute); - if (localAttributeValue != null) { - for (Iterator iterator = from.iterator(); iterator.hasNext(); ) { - ShardRouting fromShard = iterator.next(); - final DiscoveryNode discoveryNode = nodes.get(fromShard.currentNodeId()); - if (discoveryNode == null) { - iterator.remove(); // node is not present anymore - ignore shard - } else if (localAttributeValue.equals(discoveryNode.getAttributes().get(attribute))) { - iterator.remove(); - to.add(fromShard); - } - } - } - } - return Collections.unmodifiableList(to); - } - - public ShardIterator preferAttributesActiveInitializingShardsIt(List attributes, DiscoveryNodes nodes) { - return preferAttributesActiveInitializingShardsIt(attributes, nodes, shuffler.nextSeed()); - } - - public ShardIterator preferAttributesActiveInitializingShardsIt(List attributes, DiscoveryNodes nodes, int seed) { - AttributesKey key = new AttributesKey(attributes); - AttributesRoutings activeRoutings = getActiveAttribute(key, nodes); - AttributesRoutings initializingRoutings = getInitializingAttribute(key, nodes); - - // we now randomize, once between the ones that have the same attributes, and once for the ones that don't - // we don't want to mix between the two! - ArrayList ordered = new ArrayList<>(activeRoutings.totalSize + initializingRoutings.totalSize); - ordered.addAll(shuffler.shuffle(activeRoutings.withSameAttribute, seed)); - ordered.addAll(shuffler.shuffle(activeRoutings.withoutSameAttribute, seed)); - ordered.addAll(shuffler.shuffle(initializingRoutings.withSameAttribute, seed)); - ordered.addAll(shuffler.shuffle(initializingRoutings.withoutSameAttribute, seed)); - return new PlainShardIterator(shardId, ordered); - } - public ShardRouting primaryShard() { return primary; } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java b/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java index 9d0a081af4cb..6d9397db3b37 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java @@ -22,7 +22,6 @@ package org.elasticsearch.cluster.routing; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.ClusterSettings; @@ -36,7 +35,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -47,14 +45,10 @@ public class OperationRouting { Setting.boolSetting("cluster.routing.use_adaptive_replica_selection", true, Setting.Property.Dynamic, Setting.Property.NodeScope); - private List awarenessAttributes; private boolean useAdaptiveReplicaSelection; public OperationRouting(Settings settings, ClusterSettings clusterSettings) { - this.awarenessAttributes = AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.get(settings); this.useAdaptiveReplicaSelection = USE_ADAPTIVE_REPLICA_SELECTION_SETTING.get(settings); - clusterSettings.addSettingsUpdateConsumer(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING, - this::setAwarenessAttributes); clusterSettings.addSettingsUpdateConsumer(USE_ADAPTIVE_REPLICA_SELECTION_SETTING, this::setUseAdaptiveReplicaSelection); } @@ -62,10 +56,6 @@ public class OperationRouting { this.useAdaptiveReplicaSelection = useAdaptiveReplicaSelection; } - private void setAwarenessAttributes(List awarenessAttributes) { - this.awarenessAttributes = awarenessAttributes; - } - public ShardIterator indexShards(ClusterState clusterState, String index, String id, @Nullable String routing) { return shards(clusterState, index, id, routing).shardsIt(); } @@ -194,23 +184,15 @@ public class OperationRouting { } // if not, then use it as the index int routingHash = 31 * Murmur3HashFunction.hash(preference) + indexShard.shardId.hashCode(); - if (awarenessAttributes.isEmpty()) { - return indexShard.activeInitializingShardsIt(routingHash); - } else { - return indexShard.preferAttributesActiveInitializingShardsIt(awarenessAttributes, nodes, routingHash); - } + return indexShard.activeInitializingShardsIt(routingHash); } private ShardIterator shardRoutings(IndexShardRoutingTable indexShard, DiscoveryNodes nodes, @Nullable ResponseCollectorService collectorService, @Nullable Map nodeCounts) { - if (awarenessAttributes.isEmpty()) { - if (useAdaptiveReplicaSelection) { - return indexShard.activeInitializingShardsRankedIt(collectorService, nodeCounts); - } else { - return indexShard.activeInitializingShardsRandomIt(); - } + if (useAdaptiveReplicaSelection) { + return indexShard.activeInitializingShardsRankedIt(collectorService, nodeCounts); } else { - return indexShard.preferAttributesActiveInitializingShardsIt(awarenessAttributes, nodes); + return indexShard.activeInitializingShardsRandomIt(); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java b/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java index 993a8ce408b4..51f1b52e9f3e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java @@ -41,10 +41,8 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.ShardId; -import java.util.Arrays; import java.util.Collections; import java.util.Iterator; -import java.util.Map; import static java.util.Collections.singletonMap; import static org.hamcrest.Matchers.anyOf; @@ -224,59 +222,6 @@ public class RoutingIteratorTests extends ESAllocationTestCase { assertThat(shardRouting1, sameInstance(shardRouting3)); } - public void testAttributePreferenceRouting() { - Settings.Builder settings = Settings.builder() - .put("cluster.routing.allocation.node_concurrent_recoveries", 10) - .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always"); - if (randomBoolean()) { - settings.put("cluster.routing.allocation.awareness.attributes", " rack_id, zone "); - } else { - settings.putList("cluster.routing.allocation.awareness.attributes", "rack_id", "zone"); - } - - AllocationService strategy = createAllocationService(settings.build()); - - MetaData metaData = MetaData.builder() - .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) - .build(); - - RoutingTable routingTable = RoutingTable.builder() - .addAsNew(metaData.index("test")) - .build(); - - ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING - .getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); - - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder() - .add(newNode("node1", Map.of("rack_id", "rack_1", "zone", "zone1"))) - .add(newNode("node2", Map.of("rack_id", "rack_2", "zone", "zone2"))) - .localNodeId("node1") - ).build(); - clusterState = strategy.reroute(clusterState, "reroute"); - - clusterState = startInitializingShardsAndReroute(strategy, clusterState); - clusterState = startInitializingShardsAndReroute(strategy, clusterState); - - // after all are started, check routing iteration - ShardIterator shardIterator = clusterState.routingTable().index("test").shard(0) - .preferAttributesActiveInitializingShardsIt(Arrays.asList("rack_id"), clusterState.nodes()); - ShardRouting shardRouting = shardIterator.nextOrNull(); - assertThat(shardRouting, notNullValue()); - assertThat(shardRouting.currentNodeId(), equalTo("node1")); - shardRouting = shardIterator.nextOrNull(); - assertThat(shardRouting, notNullValue()); - assertThat(shardRouting.currentNodeId(), equalTo("node2")); - - shardIterator = clusterState.routingTable().index("test").shard(0) - .preferAttributesActiveInitializingShardsIt(Arrays.asList("rack_id"), clusterState.nodes()); - shardRouting = shardIterator.nextOrNull(); - assertThat(shardRouting, notNullValue()); - assertThat(shardRouting.currentNodeId(), equalTo("node1")); - shardRouting = shardIterator.nextOrNull(); - assertThat(shardRouting, notNullValue()); - assertThat(shardRouting.currentNodeId(), equalTo("node2")); - } - public void testNodeSelectorRouting(){ AllocationService strategy = createAllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 10) From 41d3eb31946a1f6f894fe1c46f57260e2195704e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 4 Sep 2019 16:50:34 -0400 Subject: [PATCH 054/103] Revert "Sync translog without lock when trim unreferenced readers (#46203)" Unfortunately, with this change, we won't clean up all unreferenced generations when reopening. We assume that there's at most one unreferenced generation when reopening translog. The previous implementation guarantees this assumption by syncing translog every time after we remove a translog reader. This change, however, only syncs translog once after we have removed all unreferenced readers (can be more than one) and breaks the assumption. Closes #46267 This reverts commit fd8183ee51d7cf08d9def58a2ae027714beb60de. --- .../index/translog/Translog.java | 29 ++++++------------- 1 file changed, 9 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 89ee29689c55..305fcf626891 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1670,7 +1670,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * required generation */ public void trimUnreferencedReaders() throws IOException { - List toDeleteReaderList = new ArrayList<>(); try (ReleasableLock ignored = writeLock.acquire()) { if (closed.get()) { // we're shutdown potentially on some tragic event, don't delete anything @@ -1684,14 +1683,22 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC "deletion policy requires a minReferenceGen of [" + minReferencedGen + "] which is higher than the current generation [" + currentFileGeneration() + "]"; + for (Iterator iterator = readers.iterator(); iterator.hasNext(); ) { TranslogReader reader = iterator.next(); if (reader.getGeneration() >= minReferencedGen) { break; } iterator.remove(); - toDeleteReaderList.add(reader); IOUtils.closeWhileHandlingException(reader); + final Path translogPath = reader.path(); + logger.trace("delete translog file [{}], not referenced and not current anymore", translogPath); + // The checkpoint is used when opening the translog to know which files should be recovered from. + // We now update the checkpoint to ignore the file we are going to remove. + // Note that there is a provision in recoverFromFiles to allow for the case where we synced the checkpoint + // but crashed before we could delete the file. + current.sync(); + deleteReaderFiles(reader); } assert readers.isEmpty() == false || current.generation == minReferencedGen : "all readers were cleaned but the minReferenceGen [" + minReferencedGen + "] is not the current writer's gen [" + @@ -1700,24 +1707,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC closeOnTragicEvent(ex); throw ex; } - // Do sync outside the writeLock to avoid blocking all writing thread. - if (toDeleteReaderList.isEmpty() == false) { - try { - // The checkpoint is used when opening the translog to know which files should be recovered from. - // We now update the checkpoint to ignore the file we are going to remove. - // Note that there is a provision in recoverFromFiles to allow for the case where we synced the checkpoint - // but crashed before we could delete the file. - sync(); - for (TranslogReader reader : toDeleteReaderList) { - final Path translogPath = reader.path(); - logger.trace("delete translog file [{}], not referenced and not current anymore", translogPath); - deleteReaderFiles(reader); - } - } catch (final Exception ex) { - closeOnTragicEvent(ex); - throw ex; - } - } } /** From f196e96bea96338a680ae87242570ce8b1f96eda Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Wed, 4 Sep 2019 14:43:58 -0700 Subject: [PATCH 055/103] [DOCS] Identify reloadable S3 repository plugin settings (#46349) --- docs/plugins/repository-s3.asciidoc | 10 +++++----- docs/reference/setup/secure-settings.asciidoc | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/plugins/repository-s3.asciidoc b/docs/plugins/repository-s3.asciidoc index 6d29b0313ba4..454357a5d4b2 100644 --- a/docs/plugins/repository-s3.asciidoc +++ b/docs/plugins/repository-s3.asciidoc @@ -101,16 +101,16 @@ The following list contains the available client settings. Those that must be stored in the keystore are marked as "secure" and are *reloadable*; the other settings belong in the `elasticsearch.yml` file. -`access_key` ({ref}/secure-settings.html[Secure]):: +`access_key` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: An S3 access key. If set, the `secret_key` setting must also be specified. If unset, the client will use the instance or container role instead. -`secret_key` ({ref}/secure-settings.html[Secure]):: +`secret_key` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: An S3 secret key. If set, the `access_key` setting must also be specified. -`session_token` ({ref}/secure-settings.html[Secure]):: +`session_token` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: An S3 session token. If set, the `access_key` and `secret_key` settings must also be specified. @@ -137,11 +137,11 @@ settings belong in the `elasticsearch.yml` file. The port of a proxy to connect to S3 through. -`proxy.username` ({ref}/secure-settings.html[Secure]):: +`proxy.username` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: The username to connect to the `proxy.host` with. -`proxy.password` ({ref}/secure-settings.html[Secure]):: +`proxy.password` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: The password to connect to the `proxy.host` with. diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index f795baa5ed9f..ac10cd6e3871 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -123,5 +123,5 @@ of reloading after each modification. There are reloadable secure settings for: -* {plugins}/discovery-ec2-usage.html#_configuring_ec2_discovery[The EC2 Discovery Plugin] - +* {plugins}/discovery-ec2-usage.html#_configuring_ec2_discovery[The EC2 discovery plugin] +* {plugins}/repository-s3-client.html[The S3 repository plugin] From 406db11c3e6a1aaff5a65671de40da8a6a88168e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 4 Sep 2019 17:23:41 -0400 Subject: [PATCH 056/103] Unmute testRecoveryFromFailureOnTrimming Tracked at #46267 --- .../java/org/elasticsearch/index/translog/TranslogTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index a1330a36912a..daa6fdfe625e 100644 --- a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2285,7 +2285,6 @@ public class TranslogTests extends ESTestCase { * Tests the situation where the node crashes after a translog gen was committed to lucene, but before the translog had the chance * to clean up its files. */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46267") public void testRecoveryFromFailureOnTrimming() throws IOException { Path tempDir = createTempDir(); final FailSwitch fail = new FailSwitch(); From 7e112ca0f26471a82c6949d476398faf47a464ab Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Wed, 4 Sep 2019 16:24:55 -0700 Subject: [PATCH 057/103] [DOCS] Identify reloadable GCS repository plugin settings (#46352) --- docs/plugins/repository-gcs.asciidoc | 4 ++-- docs/reference/setup/secure-settings.asciidoc | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/plugins/repository-gcs.asciidoc b/docs/plugins/repository-gcs.asciidoc index 387c2f83b9ff..bda5ca42629b 100644 --- a/docs/plugins/repository-gcs.asciidoc +++ b/docs/plugins/repository-gcs.asciidoc @@ -158,9 +158,9 @@ bin/elasticsearch-keystore add-file gcs.client.default.credentials_file /path/se The following are the available client settings. Those that must be stored in the keystore are marked as `Secure`. -`credentials_file`:: +`credentials_file` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: - The service account file that is used to authenticate to the Google Cloud Storage service. (Secure) + The service account file that is used to authenticate to the Google Cloud Storage service. `endpoint`:: diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index ac10cd6e3871..cd1e8e6aa16d 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -124,4 +124,5 @@ of reloading after each modification. There are reloadable secure settings for: * {plugins}/discovery-ec2-usage.html#_configuring_ec2_discovery[The EC2 discovery plugin] +* {plugins}/repository-gcs-client.html[The GCS repository plugin] * {plugins}/repository-s3-client.html[The S3 repository plugin] From 210b592f62d39dd5d12d9ec60f64b9fa1c39e7db Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Wed, 4 Sep 2019 17:03:05 -0700 Subject: [PATCH 058/103] [DOCS] Synchs Watcher API titles with better HLRC titles (#46328) --- .../high-level/watcher/ack-watch.asciidoc | 4 ++-- .../watcher/activate-watch.asciidoc | 4 ++-- .../watcher/deactivate-watch.asciidoc | 2 +- .../high-level/watcher/delete-watch.asciidoc | 4 ++-- .../high-level/watcher/execute-watch.asciidoc | 4 ++-- .../high-level/watcher/get-watch.asciidoc | 2 +- .../high-level/watcher/put-watch.asciidoc | 4 ++-- .../watcher/start-watch-service.asciidoc | 2 +- .../watcher/stop-watch-service.asciidoc | 2 +- .../high-level/watcher/watcher-stats.asciidoc | 2 +- x-pack/docs/en/rest-api/watcher.asciidoc | 19 ++++++++++++++----- .../docs/en/rest-api/watcher/start.asciidoc | 7 +++---- .../docs/en/rest-api/watcher/stats.asciidoc | 7 ++++--- x-pack/docs/en/rest-api/watcher/stop.asciidoc | 6 +++--- 14 files changed, 39 insertions(+), 30 deletions(-) diff --git a/docs/java-rest/high-level/watcher/ack-watch.asciidoc b/docs/java-rest/high-level/watcher/ack-watch.asciidoc index 46a516798594..3494993d87c2 100644 --- a/docs/java-rest/high-level/watcher/ack-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/ack-watch.asciidoc @@ -5,12 +5,12 @@ -- [id="{upid}-{api}"] -=== Ack Watch API +=== Ack watch API [id="{upid}-{api}-request"] ==== Execution -{xpack-ref}/actions.html#actions-ack-throttle[Acknowledging a watch] enables you +{stack-ov}/actions.html#actions-ack-throttle[Acknowledging a watch] enables you to manually throttle execution of a watch's actions. A watch can be acknowledged through the following request: diff --git a/docs/java-rest/high-level/watcher/activate-watch.asciidoc b/docs/java-rest/high-level/watcher/activate-watch.asciidoc index 52124ccb6edd..229e44e7a84e 100644 --- a/docs/java-rest/high-level/watcher/activate-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/activate-watch.asciidoc @@ -5,7 +5,7 @@ -- [id="{upid}-{api}"] -=== Activate Watch API +=== Activate watch API [id="{upid}-{api}-request"] ==== Execution @@ -29,7 +29,7 @@ include-tagged::{doc-tests-file}[{api}-response] <1> `watchStatus` contains status of the watch [id="{upid}-{api}-request-async"] -==== Asynchronous Execution +==== Asynchronous execution This request can be executed asynchronously: diff --git a/docs/java-rest/high-level/watcher/deactivate-watch.asciidoc b/docs/java-rest/high-level/watcher/deactivate-watch.asciidoc index ca2178e5c05e..2ba82be1d6f3 100644 --- a/docs/java-rest/high-level/watcher/deactivate-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/deactivate-watch.asciidoc @@ -5,6 +5,6 @@ :doc-tests-file: {doc-tests}/WatcherDocumentationIT.java -- [[java-rest-high-watcher-deactivate-watch]] -=== Deactivate Watch API +=== Deactivate watch API include::../execution.asciidoc[] diff --git a/docs/java-rest/high-level/watcher/delete-watch.asciidoc b/docs/java-rest/high-level/watcher/delete-watch.asciidoc index 615337ba317b..3edf2e83c9fe 100644 --- a/docs/java-rest/high-level/watcher/delete-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/delete-watch.asciidoc @@ -1,5 +1,5 @@ [[java-rest-high-x-pack-watcher-delete-watch]] -=== Delete Watch API +=== Delete watch API [[java-rest-high-x-pack-watcher-delete-watch-execution]] ==== Execution @@ -26,7 +26,7 @@ include-tagged::{doc-tests}/WatcherDocumentationIT.java[x-pack-put-watch-respons <3> `_version` returns the version of the deleted watch [[java-rest-high-x-pack-watcher-delete-watch-async]] -==== Asynchronous Execution +==== Asynchronous execution This request can be executed asynchronously: diff --git a/docs/java-rest/high-level/watcher/execute-watch.asciidoc b/docs/java-rest/high-level/watcher/execute-watch.asciidoc index ed5b4b1659dc..06a282228b46 100644 --- a/docs/java-rest/high-level/watcher/execute-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/execute-watch.asciidoc @@ -4,7 +4,7 @@ :response: ExecuteWatchResponse -- [id="{upid}-{api}"] -=== Execute Watch API +=== Execute watch API The execute watch API allows clients to immediately execute a watch, either one that has been previously added via the @@ -27,7 +27,7 @@ include-tagged::{doc-tests-file}[x-pack-execute-watch-by-id] <6> Enable debug mode [id="{upid}-{api}-response-by-id"] -==== Execute by id Response +==== Execute by id response The returned `Response` contains details of the execution: diff --git a/docs/java-rest/high-level/watcher/get-watch.asciidoc b/docs/java-rest/high-level/watcher/get-watch.asciidoc index 7321a66eeaaf..3ba232aa6694 100644 --- a/docs/java-rest/high-level/watcher/get-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/get-watch.asciidoc @@ -5,7 +5,7 @@ -- [id="{upid}-{api}"] -=== Get Watch API +=== Get watch API [id="{upid}-{api}-request"] ==== Execution diff --git a/docs/java-rest/high-level/watcher/put-watch.asciidoc b/docs/java-rest/high-level/watcher/put-watch.asciidoc index e5ee87bea34a..494fc3d40888 100644 --- a/docs/java-rest/high-level/watcher/put-watch.asciidoc +++ b/docs/java-rest/high-level/watcher/put-watch.asciidoc @@ -1,5 +1,5 @@ [[java-rest-high-x-pack-watcher-put-watch]] -=== Put Watch API +=== Put watch API [[java-rest-high-x-pack-watcher-put-watch-execution]] ==== Execution @@ -28,7 +28,7 @@ include-tagged::{doc-tests}/WatcherDocumentationIT.java[x-pack-put-watch-respons <3> `_version` returns the newly created version [[java-rest-high-x-pack-watcher-put-watch-async]] -==== Asynchronous Execution +==== Asynchronous execution This request can be executed asynchronously: diff --git a/docs/java-rest/high-level/watcher/start-watch-service.asciidoc b/docs/java-rest/high-level/watcher/start-watch-service.asciidoc index 9e3eaf1359f5..013349410332 100644 --- a/docs/java-rest/high-level/watcher/start-watch-service.asciidoc +++ b/docs/java-rest/high-level/watcher/start-watch-service.asciidoc @@ -4,7 +4,7 @@ :response: StartWatchServiceResponse -- [id="{upid}-{api}"] -=== Start Watch Service API +=== Start watch service API [id="{upid}-{api}-request"] ==== Execution diff --git a/docs/java-rest/high-level/watcher/stop-watch-service.asciidoc b/docs/java-rest/high-level/watcher/stop-watch-service.asciidoc index 173edf3cc7b2..b59db5a34f85 100644 --- a/docs/java-rest/high-level/watcher/stop-watch-service.asciidoc +++ b/docs/java-rest/high-level/watcher/stop-watch-service.asciidoc @@ -4,7 +4,7 @@ :response: StopWatchServiceResponse -- [id="{upid}-{api}"] -=== Stop Watch Service API +=== Stop watch service API [[java-rest-high-watcher-stop-watch-service-execution]] ==== Execution diff --git a/docs/java-rest/high-level/watcher/watcher-stats.asciidoc b/docs/java-rest/high-level/watcher/watcher-stats.asciidoc index 7fd27053fcba..ddc877c2c95c 100644 --- a/docs/java-rest/high-level/watcher/watcher-stats.asciidoc +++ b/docs/java-rest/high-level/watcher/watcher-stats.asciidoc @@ -4,7 +4,7 @@ :response: WatcherStatsResponse -- [id="{upid}-{api}"] -=== Watcher Stats API +=== Get Watcher stats API [id="{upid}-{api}-request"] ==== Execution diff --git a/x-pack/docs/en/rest-api/watcher.asciidoc b/x-pack/docs/en/rest-api/watcher.asciidoc index 2fcb9b9ca190..a2b919386c1d 100644 --- a/x-pack/docs/en/rest-api/watcher.asciidoc +++ b/x-pack/docs/en/rest-api/watcher.asciidoc @@ -13,13 +13,22 @@ * <> * <> -include::watcher/put-watch.asciidoc[] -include::watcher/get-watch.asciidoc[] -include::watcher/delete-watch.asciidoc[] -include::watcher/execute-watch.asciidoc[] +//ACK include::watcher/ack-watch.asciidoc[] +//ACTIVATE include::watcher/activate-watch.asciidoc[] +//DEACTIVATE include::watcher/deactivate-watch.asciidoc[] +//DELETE +include::watcher/delete-watch.asciidoc[] +//EXECUTE +include::watcher/execute-watch.asciidoc[] +//GET +include::watcher/get-watch.asciidoc[] include::watcher/stats.asciidoc[] -include::watcher/stop.asciidoc[] +//PUT +include::watcher/put-watch.asciidoc[] +//START include::watcher/start.asciidoc[] +//STOP +include::watcher/stop.asciidoc[] \ No newline at end of file diff --git a/x-pack/docs/en/rest-api/watcher/start.asciidoc b/x-pack/docs/en/rest-api/watcher/start.asciidoc index 54deb1f8068c..3ae1139cb9a9 100644 --- a/x-pack/docs/en/rest-api/watcher/start.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/start.asciidoc @@ -1,12 +1,11 @@ [role="xpack"] [[watcher-api-start]] -=== Start API +=== Start watch service API ++++ -Start +Start watch service ++++ -The `start` API starts the {watcher} service if the service is not already -running. +Starts the {watcher} service if it is not already running. [float] ==== Request diff --git a/x-pack/docs/en/rest-api/watcher/stats.asciidoc b/x-pack/docs/en/rest-api/watcher/stats.asciidoc index d5fa4c75bbf1..8ae06491ef4b 100644 --- a/x-pack/docs/en/rest-api/watcher/stats.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/stats.asciidoc @@ -1,11 +1,12 @@ [role="xpack"] [[watcher-api-stats]] -=== Stats API +=== Get {watcher} stats API +[subs="attributes"] ++++ -Stats +Get {watcher} stats ++++ -The `stats` API returns the current {watcher} metrics. +Retrieves the current {watcher} metrics. [float] ==== Request diff --git a/x-pack/docs/en/rest-api/watcher/stop.asciidoc b/x-pack/docs/en/rest-api/watcher/stop.asciidoc index 5cbc8a622672..782b983a88cf 100644 --- a/x-pack/docs/en/rest-api/watcher/stop.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/stop.asciidoc @@ -1,11 +1,11 @@ [role="xpack"] [[watcher-api-stop]] -=== Stop API +=== Stop watch service API ++++ -Stop +Stop watch service ++++ -The `stop` API stops the {watcher} service if the service is running. +Stops the {watcher} service if it is running. [float] ==== Request From dfc74c096b876e65de29365e58214c4594bae7b2 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 5 Sep 2019 09:22:42 +0200 Subject: [PATCH 059/103] Add repository integration tests for Azure (#46263) Similarly to what had been done for S3 (#46081) and GCS (#46255) this commit adds repository integration tests for Azure, based on an internal HTTP server instead of mocks. --- .../azure/AzureStorageSettings.java | 2 +- .../azure/AzureBlobStoreRepositoryTests.java | 190 ++++++++++++++++++ 2 files changed, 191 insertions(+), 1 deletion(-) create mode 100644 plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java index d87e48542d1c..600715209f52 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageSettings.java @@ -66,7 +66,7 @@ final class AzureStorageSettings { /** * Azure endpoint suffix. Default to core.windows.net (CloudStorageAccount.DEFAULT_DNS). */ - public static final Setting ENDPOINT_SUFFIX_SETTING = Setting.affixKeySetting(AZURE_CLIENT_PREFIX_KEY, "endpoint_suffix", + public static final AffixSetting ENDPOINT_SUFFIX_SETTING = Setting.affixKeySetting(AZURE_CLIENT_PREFIX_KEY, "endpoint_suffix", key -> Setting.simpleString(key, Property.NodeScope), ACCOUNT_SETTING, KEY_SETTING); public static final AffixSetting TIMEOUT_SETTING = Setting.affixKeySetting(AZURE_CLIENT_PREFIX_KEY, "timeout", diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java new file mode 100644 index 000000000000..6e3de9901519 --- /dev/null +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java @@ -0,0 +1,190 @@ +/* + * 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.repositories.azure; + +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import com.sun.net.httpserver.HttpServer; +import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.network.InetAddresses; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.mocksocket.MockHttpServer; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.blobstore.ESBlobStoreRepositoryIntegTestCase; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.RestUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +@SuppressForbidden(reason = "this test uses a HttpServer to emulate an Azure endpoint") +public class AzureBlobStoreRepositoryTests extends ESBlobStoreRepositoryIntegTestCase { + + private static HttpServer httpServer; + + @BeforeClass + public static void startHttpServer() throws Exception { + httpServer = MockHttpServer.createHttp(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0), 0); + httpServer.start(); + } + + @Before + public void setUpHttpServer() { + httpServer.createContext("/container", new InternalHttpHandler()); + } + + @AfterClass + public static void stopHttpServer() { + httpServer.stop(0); + httpServer = null; + } + + @After + public void tearDownHttpServer() { + httpServer.removeContext("/container"); + } + + @Override + protected String repositoryType() { + return AzureRepository.TYPE; + } + + @Override + protected Settings repositorySettings() { + return Settings.builder() + .put(AzureRepository.Repository.CONTAINER_SETTING.getKey(), "container") + .put(AzureStorageSettings.ACCOUNT_SETTING.getKey(), "test") + .build(); + } + + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(AzureRepositoryPlugin.class); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + final String key = Base64.getEncoder().encodeToString(randomAlphaOfLength(10).getBytes(StandardCharsets.UTF_8)); + final MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setString(AzureStorageSettings.ACCOUNT_SETTING.getConcreteSettingForNamespace("test").getKey(), "account"); + secureSettings.setString(AzureStorageSettings.KEY_SETTING.getConcreteSettingForNamespace("test").getKey(), key); + + final InetSocketAddress address = httpServer.getAddress(); + final String endpoint = "ignored;DefaultEndpointsProtocol=http;BlobEndpoint=http://" + + InetAddresses.toUriString(address.getAddress()) + ":" + address.getPort(); + + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(AzureStorageSettings.ENDPOINT_SUFFIX_SETTING.getConcreteSettingForNamespace("test").getKey(), endpoint) + .setSecureSettings(secureSettings) + .build(); + } + + /** + * Minimal HTTP handler that acts as an Azure compliant server + */ + @SuppressForbidden(reason = "this test uses a HttpServer to emulate an Azure endpoint") + private static class InternalHttpHandler implements HttpHandler { + + private final Map blobs = new ConcurrentHashMap<>(); + + @Override + public void handle(final HttpExchange exchange) throws IOException { + final String request = exchange.getRequestMethod() + " " + exchange.getRequestURI().toString(); + try { + if (Regex.simpleMatch("PUT /container/*", request)) { + blobs.put(exchange.getRequestURI().toString(), Streams.readFully(exchange.getRequestBody())); + exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); + + } else if (Regex.simpleMatch("HEAD /container/*", request)) { + BytesReference blob = blobs.get(exchange.getRequestURI().toString()); + if (blob == null) { + exchange.sendResponseHeaders(RestStatus.NOT_FOUND.getStatus(), -1); + return; + } + exchange.getResponseHeaders().add("x-ms-blob-content-length", String.valueOf(blob.length())); + exchange.getResponseHeaders().add("x-ms-blob-type", "blockblob"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), -1); + + } else if (Regex.simpleMatch("GET /container/*", request)) { + final BytesReference blob = blobs.get(exchange.getRequestURI().toString()); + if (blob == null) { + exchange.sendResponseHeaders(RestStatus.NOT_FOUND.getStatus(), -1); + return; + } + exchange.getResponseHeaders().add("Content-Type", "application/octet-stream"); + exchange.getResponseHeaders().add("x-ms-blob-content-length", String.valueOf(blob.length())); + exchange.getResponseHeaders().add("x-ms-blob-type", "blockblob"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), blob.length()); + blob.writeTo(exchange.getResponseBody()); + + } else if (Regex.simpleMatch("DELETE /container/*", request)) { + Streams.readFully(exchange.getRequestBody()); + blobs.entrySet().removeIf(blob -> blob.getKey().startsWith(exchange.getRequestURI().toString())); + exchange.sendResponseHeaders(RestStatus.ACCEPTED.getStatus(), -1); + + } else if (Regex.simpleMatch("GET /container?restype=container&comp=list*", request)) { + final Map params = new HashMap<>(); + RestUtils.decodeQueryString(exchange.getRequestURI().getQuery(), 0, params); + + final StringBuilder list = new StringBuilder(); + list.append(""); + list.append(""); + final String prefix = params.get("prefix"); + list.append(""); + for (Map.Entry blob : blobs.entrySet()) { + if (prefix == null || blob.getKey().startsWith("/container/" + prefix)) { + list.append("").append(blob.getKey().replace("/container/", "")).append(""); + list.append("").append(blob.getValue().length()).append(""); + list.append("BlockBlob"); + } + } + list.append(""); + list.append(""); + + byte[] response = list.toString().getBytes(StandardCharsets.UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/xml"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), response.length); + exchange.getResponseBody().write(response); + + } else { + exchange.sendResponseHeaders(RestStatus.BAD_REQUEST.getStatus(), -1); + } + } finally { + exchange.close(); + } + } + } +} From 4726e1e6b3637c746d501d5181dd7d815ac4c33d Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 5 Sep 2019 09:25:23 +0200 Subject: [PATCH 060/103] Replace mocked client in GCSBlobStoreRepositoryTests by HTTP server (#46255) This commit removes the usage of MockGoogleCloudStoragePlugin in GoogleCloudStorageBlobStoreRepositoryTests and replaces it by a HttpServer that emulates the Storage service. This allows the repository tests to use the real Google's client under the hood in tests and will allow us to test the behavior of the snapshot/restore feature for GCS repositories by simulating random server-side internal errors. The HTTP server used to emulate the Storage service is intentionally simple and minimal to keep things understandable and maintainable. Testing full client options on the server side (like authentication, chunked encoding etc) remains the responsibility of the GoogleCloudStorageFixture. --- ...eCloudStorageBlobStoreRepositoryTests.java | 309 ++++++++++++++++-- .../ESBlobStoreRepositoryIntegTestCase.java | 4 +- 2 files changed, 284 insertions(+), 29 deletions(-) diff --git a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStoreRepositoryTests.java b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStoreRepositoryTests.java index fa9631d1a001..b267e686e653 100644 --- a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStoreRepositoryTests.java +++ b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStoreRepositoryTests.java @@ -19,27 +19,95 @@ package org.elasticsearch.repositories.gcs; -import com.google.cloud.storage.Storage; +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import com.sun.net.httpserver.HttpServer; +import org.apache.http.HttpStatus; import org.elasticsearch.cluster.metadata.RepositoryMetaData; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.network.InetAddresses; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.mocksocket.MockHttpServer; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.blobstore.ESBlobStoreRepositoryIntegTestCase; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.RestUtils; import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import java.io.BufferedInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.URLDecoder; +import java.security.KeyPairGenerator; +import java.util.Arrays; +import java.util.Base64; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.zip.GZIPInputStream; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.elasticsearch.repositories.gcs.GoogleCloudStorageClientSettings.CREDENTIALS_FILE_SETTING; +import static org.elasticsearch.repositories.gcs.GoogleCloudStorageClientSettings.ENDPOINT_SETTING; +import static org.elasticsearch.repositories.gcs.GoogleCloudStorageClientSettings.TOKEN_URI_SETTING; +import static org.elasticsearch.repositories.gcs.GoogleCloudStorageRepository.BUCKET; +import static org.elasticsearch.repositories.gcs.GoogleCloudStorageRepository.CLIENT_NAME; + +@SuppressForbidden(reason = "this test uses a HttpServer to emulate a Google Cloud Storage endpoint") public class GoogleCloudStorageBlobStoreRepositoryTests extends ESBlobStoreRepositoryIntegTestCase { - private static final String BUCKET = "gcs-repository-test"; + private static HttpServer httpServer; + private static byte[] serviceAccount; - // Static list of blobs shared among all nodes in order to act like a remote repository service: - // all nodes must see the same content - private static final ConcurrentMap blobs = new ConcurrentHashMap<>(); + @BeforeClass + public static void startHttpServer() throws Exception { + httpServer = MockHttpServer.createHttp(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0), 0); + httpServer.start(); + serviceAccount = createServiceAccount(); + } + + @Before + public void setUpHttpServer() { + httpServer.createContext("/", new InternalHttpHandler()); + httpServer.createContext("/token", new FakeOAuth2HttpHandler()); + } + + @AfterClass + public static void stopHttpServer() { + httpServer.stop(0); + httpServer = null; + } + + @After + public void tearDownHttpServer() { + httpServer.removeContext("/"); + httpServer.removeContext("/token"); + } @Override protected String repositoryType() { @@ -50,38 +118,31 @@ public class GoogleCloudStorageBlobStoreRepositoryTests extends ESBlobStoreRepos protected Settings repositorySettings() { return Settings.builder() .put(super.repositorySettings()) - .put("bucket", BUCKET) - .put("base_path", GoogleCloudStorageBlobStoreRepositoryTests.class.getSimpleName()) + .put(BUCKET.getKey(), "bucket") + .put(CLIENT_NAME.getKey(), "test") .build(); } @Override protected Collection> nodePlugins() { - return Collections.singletonList(MockGoogleCloudStoragePlugin.class); + return Collections.singletonList(GoogleCloudStoragePlugin.class); } - @After - public void wipeRepository() { - blobs.clear(); - } + @Override + protected Settings nodeSettings(int nodeOrdinal) { + final Settings.Builder settings = Settings.builder(); + settings.put(super.nodeSettings(nodeOrdinal)); - public static class MockGoogleCloudStoragePlugin extends GoogleCloudStoragePlugin { + final InetSocketAddress address = httpServer.getAddress(); + final String endpoint = "http://" + InetAddresses.toUriString(address.getAddress()) + ":" + address.getPort(); + settings.put(ENDPOINT_SETTING.getConcreteSettingForNamespace("test").getKey(), endpoint); + settings.put(TOKEN_URI_SETTING.getConcreteSettingForNamespace("test").getKey(), endpoint + "/token"); - public MockGoogleCloudStoragePlugin(final Settings settings) { - super(settings); - } + final MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setFile(CREDENTIALS_FILE_SETTING.getConcreteSettingForNamespace("test").getKey(), serviceAccount); + settings.setSecureSettings(secureSettings); - @Override - protected GoogleCloudStorageService createStorageService() { - return new MockGoogleCloudStorageService(); - } - } - - public static class MockGoogleCloudStorageService extends GoogleCloudStorageService { - @Override - public Storage client(String clientName) { - return new MockStorage(BUCKET, blobs); - } + return settings.build(); } public void testChunkSize() { @@ -121,4 +182,198 @@ public class GoogleCloudStorageBlobStoreRepositoryTests extends ESBlobStoreRepos }); assertEquals("failed to parse value [101mb] for setting [chunk_size], must be <= [100mb]", e.getMessage()); } + + private static byte[] createServiceAccount() throws Exception { + final KeyPairGenerator keyPairGenerator = KeyPairGenerator.getInstance("RSA"); + keyPairGenerator.initialize(1024); + final String privateKey = Base64.getEncoder().encodeToString(keyPairGenerator.generateKeyPair().getPrivate().getEncoded()); + + final ByteArrayOutputStream out = new ByteArrayOutputStream(); + try (XContentBuilder builder = new XContentBuilder(XContentType.JSON.xContent(), out)) { + builder.startObject(); + { + builder.field("type", "service_account"); + builder.field("project_id", getTestClass().getName().toLowerCase(Locale.ROOT)); + builder.field("private_key_id", UUID.randomUUID().toString()); + builder.field("private_key", "-----BEGIN PRIVATE KEY-----\n" + privateKey + "\n-----END PRIVATE KEY-----\n"); + builder.field("client_email", "elastic@appspot.gserviceaccount.com"); + builder.field("client_id", String.valueOf(randomNonNegativeLong())); + } + builder.endObject(); + } + return out.toByteArray(); + } + + /** + * Minimal HTTP handler that acts as a Google Cloud Storage compliant server + * + * Note: it does not support resumable uploads + */ + @SuppressForbidden(reason = "this test uses a HttpServer to emulate a Google Cloud Storage endpoint") + private static class InternalHttpHandler implements HttpHandler { + + private final ConcurrentMap blobs = new ConcurrentHashMap<>(); + + @Override + public void handle(final HttpExchange exchange) throws IOException { + final String request = exchange.getRequestMethod() + " " + exchange.getRequestURI().toString(); + try { + if (Regex.simpleMatch("GET /storage/v1/b/bucket/o*", request)) { + final Map params = new HashMap<>(); + RestUtils.decodeQueryString(exchange.getRequestURI().getQuery(), 0, params); + final String prefix = params.get("prefix"); + + final List> listOfBlobs = blobs.entrySet().stream() + .filter(blob -> prefix == null || blob.getKey().startsWith(prefix)).collect(Collectors.toList()); + + final StringBuilder list = new StringBuilder(); + list.append("{\"kind\":\"storage#objects\",\"items\":["); + for (Iterator> it = listOfBlobs.iterator(); it.hasNext(); ) { + Map.Entry blob = it.next(); + list.append("{\"kind\":\"storage#object\","); + list.append("\"bucket\":\"bucket\","); + list.append("\"name\":\"").append(blob.getKey()).append("\","); + list.append("\"id\":\"").append(blob.getKey()).append("\","); + list.append("\"size\":\"").append(blob.getValue().length()).append("\""); + list.append('}'); + + if (it.hasNext()) { + list.append(','); + } + } + list.append("]}"); + + byte[] response = list.toString().getBytes(UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/json; charset=utf-8"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), response.length); + exchange.getResponseBody().write(response); + + } else if (Regex.simpleMatch("GET /storage/v1/b/bucket*", request)) { + byte[] response = ("{\"kind\":\"storage#bucket\",\"name\":\"bucket\",\"id\":\"0\"}").getBytes(UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/json; charset=utf-8"); + exchange.sendResponseHeaders(HttpStatus.SC_OK, response.length); + exchange.getResponseBody().write(response); + + } else if (Regex.simpleMatch("GET /download/storage/v1/b/bucket/o/*", request)) { + BytesReference blob = blobs.get(exchange.getRequestURI().getPath().replace("/download/storage/v1/b/bucket/o/", "")); + if (blob != null) { + exchange.getResponseHeaders().add("Content-Type", "application/octet-stream"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), blob.length()); + exchange.getResponseBody().write(blob.toBytesRef().bytes); + } else { + exchange.sendResponseHeaders(RestStatus.NOT_FOUND.getStatus(), -1); + } + + } else if (Regex.simpleMatch("DELETE /storage/v1/b/bucket/o/*", request)) { + int deletions = 0; + for (Iterator> iterator = blobs.entrySet().iterator(); iterator.hasNext(); ) { + Map.Entry blob = iterator.next(); + if (blob.getKey().equals(exchange.getRequestURI().toString())) { + iterator.remove(); + deletions++; + } + } + exchange.sendResponseHeaders((deletions > 0 ? RestStatus.OK : RestStatus.NO_CONTENT).getStatus(), -1); + + } else if (Regex.simpleMatch("POST /batch/storage/v1", request)) { + final String uri = "/storage/v1/b/bucket/o/"; + final StringBuilder batch = new StringBuilder(); + for (String line : Streams.readAllLines(new BufferedInputStream(exchange.getRequestBody()))) { + if (line.length() == 0 || line.startsWith("--") || line.toLowerCase(Locale.ROOT).startsWith("content")) { + batch.append(line).append('\n'); + } else if (line.startsWith("DELETE")) { + final String name = line.substring(line.indexOf(uri) + uri.length(), line.lastIndexOf(" HTTP")); + if (Strings.hasText(name)) { + if (blobs.entrySet().removeIf(blob -> blob.getKey().equals(URLDecoder.decode(name, UTF_8)))) { + batch.append("HTTP/1.1 204 NO_CONTENT").append('\n'); + batch.append('\n'); + } + } + } + } + byte[] response = batch.toString().getBytes(UTF_8); + exchange.getResponseHeaders().add("Content-Type", exchange.getRequestHeaders().getFirst("Content-Type")); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), response.length); + exchange.getResponseBody().write(response); + + } else if (Regex.simpleMatch("POST /upload/storage/v1/b/bucket/*uploadType=multipart*", request)) { + byte[] response = new byte[0]; + try (BufferedInputStream in = new BufferedInputStream(new GZIPInputStream(exchange.getRequestBody()))) { + String blob = null; + int read; + while ((read = in.read()) != -1) { + boolean markAndContinue = false; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + do { // search next consecutive {carriage return, new line} chars and stop + if ((char) read == '\r') { + int next = in.read(); + if (next != -1) { + if (next == '\n') { + break; + } + out.write(read); + out.write(next); + continue; + } + } + out.write(read); + } while ((read = in.read()) != -1); + + final String line = new String(out.toByteArray(), UTF_8); + if (line.length() == 0 || line.equals("\r\n") || line.startsWith("--") + || line.toLowerCase(Locale.ROOT).startsWith("content")) { + markAndContinue = true; + } else if (line.startsWith("{\"bucket\":\"bucket\"")) { + markAndContinue = true; + Matcher matcher = Pattern.compile("\"name\":\"([^\"]*)\"").matcher(line); + if (matcher.find()) { + blob = matcher.group(1); + response = line.getBytes(UTF_8); + } + } + if (markAndContinue) { + in.mark(Integer.MAX_VALUE); + continue; + } + } + if (blob != null) { + in.reset(); + try (ByteArrayOutputStream binary = new ByteArrayOutputStream()) { + while ((read = in.read()) != -1) { + binary.write(read); + } + binary.flush(); + byte[] tmp = binary.toByteArray(); + // removes the trailing end "\r\n--__END_OF_PART__--\r\n" which is 23 bytes long + blobs.put(blob, new BytesArray(Arrays.copyOf(tmp, tmp.length - 23))); + } finally { + blob = null; + } + } + } + } + exchange.getResponseHeaders().add("Content-Type", "application/json"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), response.length); + exchange.getResponseBody().write(response); + + } else { + exchange.sendResponseHeaders(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), -1); + } + } finally { + exchange.close(); + } + } + } + + @SuppressForbidden(reason = "this test uses a HttpServer to emulate a fake OAuth2 authentication service") + private static class FakeOAuth2HttpHandler implements HttpHandler { + @Override + public void handle(final HttpExchange exchange) throws IOException { + byte[] response = ("{\"access_token\":\"foo\",\"token_type\":\"Bearer\",\"expires_in\":3600}").getBytes(UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/json"); + exchange.sendResponseHeaders(HttpStatus.SC_OK, response.length); + exchange.getResponseBody().write(response); + exchange.close(); + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java index e78975fdab51..09814c33105d 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/ESBlobStoreRepositoryIntegTestCase.java @@ -66,8 +66,8 @@ public abstract class ESBlobStoreRepositoryIntegTestCase extends ESIntegTestCase final Settings.Builder settings = Settings.builder(); settings.put("compress", randomBoolean()); if (randomBoolean()) { - long size = 1 << randomIntBetween(7, 10); - settings.put("chunk_size", new ByteSizeValue(size, randomFrom(ByteSizeUnit.BYTES, ByteSizeUnit.KB))); + long size = 1 << randomInt(10); + settings.put("chunk_size", new ByteSizeValue(size, ByteSizeUnit.KB)); } return settings.build(); } From 610e84245d575ab4756fee87a894c9f72e395944 Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Thu, 5 Sep 2019 12:24:19 +0300 Subject: [PATCH 061/103] Mute failing SamlAuthenticationIT tests (#46369) see #44410 --- .../xpack/security/authc/saml/SamlAuthenticationIT.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java b/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java index 41f0362bd787..12b4c1849ad2 100644 --- a/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java +++ b/x-pack/qa/saml-idp-tests/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlAuthenticationIT.java @@ -240,6 +240,7 @@ public class SamlAuthenticationIT extends ESRestTestCase { *
  • Uses that token to verify the user details
  • * */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/44410") public void testLoginUserWithSamlRoleMapping() throws Exception { // this ACS comes from the config in build.gradle final Tuple authTokens = loginViaSaml("http://localhost:54321" + SP_ACS_PATH_1); @@ -248,6 +249,7 @@ public class SamlAuthenticationIT extends ESRestTestCase { verifyElasticsearchAccessTokenForRoleMapping(accessToken); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/44410") public void testLoginUserWithAuthorizingRealm() throws Exception { // this ACS comes from the config in build.gradle final Tuple authTokens = loginViaSaml("http://localhost:54321" + SP_ACS_PATH_2); @@ -256,6 +258,7 @@ public class SamlAuthenticationIT extends ESRestTestCase { verifyElasticsearchAccessTokenForAuthorizingRealms(accessToken); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/44410") public void testLoginWithWrongRealmFails() throws Exception { this.acs = new URI("http://localhost:54321" + SP_ACS_PATH_WRONG_REALM); final BasicHttpContext context = new BasicHttpContext(); From c3317fc3d28b2df4ee0b89f5ee7cdb551d37abf0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 5 Sep 2019 12:23:12 +0200 Subject: [PATCH 062/103] Enable Debug Logging for Master and Coordination Packages (#46363) In order to track down #46091: * Enables debug logging in REST tests for `master` and `coordination` packages since we suspect that issues are caused by failed and then retried publications --- .../elasticsearch/gradle/testclusters/ElasticsearchNode.java | 4 ++++ .../admin/cluster/reroute/TransportClusterRerouteAction.java | 3 +++ .../action/support/master/TransportMasterNodeAction.java | 4 ++++ 3 files changed, 11 insertions(+) diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java index c0d4c4baafc3..fa47c73d49c4 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java @@ -964,6 +964,10 @@ public class ElasticsearchNode implements TestClusterConfiguration { // Don't wait for state, just start up quickly. This will also allow new and old nodes in the BWC case to become the master defaultConfig.put("discovery.initial_state_timeout", "0s"); + // TODO: Remove these once https://github.com/elastic/elasticsearch/issues/46091 is fixed + defaultConfig.put("logger.org.elasticsearch.action.support.master", "DEBUG"); + defaultConfig.put("logger.org.elasticsearch.cluster.coordination", "DEBUG"); + HashSet overriden = new HashSet<>(defaultConfig.keySet()); overriden.retainAll(settings.keySet()); overriden.removeAll(OVERRIDABLE_SETTINGS); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java index f69002b47b47..af0f25093787 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.admin.cluster.reroute; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ExceptionsHelper; @@ -59,6 +60,8 @@ import java.util.Map; public class TransportClusterRerouteAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportClusterRerouteAction.class); + private final AllocationService allocationService; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java b/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java index 8dbf880e29f0..f325e09a9fbf 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.support.master; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; @@ -56,6 +58,8 @@ import java.util.function.Predicate; public abstract class TransportMasterNodeAction, Response extends ActionResponse> extends HandledTransportAction { + protected static final Logger logger = LogManager.getLogger(TransportMasterNodeAction.class); + protected final ThreadPool threadPool; protected final TransportService transportService; protected final ClusterService clusterService; From 2648ece98a4ccd40fb15f43304086b1e1bc6a44c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 5 Sep 2019 13:03:00 +0200 Subject: [PATCH 063/103] Quiet down shard lock failures (#46368) These were actually never intended to be logged at the warning level but made visible by a refactoring in #19991, which introduced a new exception type but forgot to adapt some of the consumers of the exception. --- .../main/java/org/elasticsearch/indices/IndicesService.java | 3 +-- .../indices/cluster/IndicesClusterStateService.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 05384655d864..23a9985b7f89 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -25,7 +25,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader.CacheHelper; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.ElasticsearchException; @@ -805,7 +804,7 @@ public class IndicesService extends AbstractLifecycleComponent nodeEnv.deleteIndexDirectorySafe(index, 0, indexSettings); } success = true; - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { logger.debug(() -> new ParameterizedMessage("{} failed to delete index store - at least one shards is still locked", index), ex); } catch (Exception ex) { diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 25a74d8e7773..157f6d7d05ca 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -23,7 +23,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.store.LockObtainFailedException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; @@ -408,7 +407,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple // holding on to the lock due to a "currently canceled recovery" or so. The shard will delete itself BEFORE the // lock is released so it's guaranteed to be deleted by the time we get the lock indicesService.processPendingDeletes(index, indexSettings, new TimeValue(30, TimeUnit.MINUTES)); - } catch (LockObtainFailedException exc) { + } catch (ShardLockObtainFailedException exc) { logger.warn("[{}] failed to lock all shards for index - timed out after 30 seconds", index); } catch (InterruptedException e) { logger.warn("[{}] failed to lock all shards for index - interrupted", index); From 8cdce05c6b0ec9ce19cd7f40b5e7775a991730c2 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 5 Sep 2019 06:45:14 -0500 Subject: [PATCH 064/103] [ML][Transforms] allow executor to call start on started task (#46347) --- ...FrameTransformPersistentTasksExecutor.java | 3 ++- .../transforms/DataFrameTransformTask.java | 25 +++++++++++-------- .../upgrades/DataFrameSurvivesUpgradeIT.java | 2 -- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java index 8374edbbf217..b1d864519368 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutor.java @@ -305,7 +305,8 @@ public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksEx ActionListener listener) { buildTask.initializeIndexer(indexerBuilder); // DataFrameTransformTask#start will fail if the task state is FAILED - buildTask.setNumFailureRetries(numFailureRetries).start(previousCheckpoint, false, listener); + // Will continue to attempt to start the indexer, even if the state is STARTED + buildTask.setNumFailureRetries(numFailureRetries).start(previousCheckpoint, false, false, listener); } private void setNumFailureRetries(int numFailureRetries) { diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java index bc39cbab0425..a31c148a1ed4 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java @@ -219,13 +219,8 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S )); } - /** - * Start the background indexer and set the task's state to started - * @param startingCheckpoint Set the current checkpoint to this value. If null the - * current checkpoint is not set - * @param listener Started listener - */ - public synchronized void start(Long startingCheckpoint, boolean force, ActionListener listener) { + // Here `failOnConflict` is usually true, except when the initial start is called when the task is assigned to the node + synchronized void start(Long startingCheckpoint, boolean force, boolean failOnConflict, ActionListener listener) { logger.debug("[{}] start called with force [{}] and state [{}].", getTransformId(), force, getState()); if (taskState.get() == DataFrameTransformTaskState.FAILED && force == false) { listener.onFailure(new ElasticsearchStatusException( @@ -249,7 +244,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S return; } // If we are already in a `STARTED` state, we should not attempt to call `.start` on the indexer again. - if (taskState.get() == DataFrameTransformTaskState.STARTED) { + if (taskState.get() == DataFrameTransformTaskState.STARTED && failOnConflict) { listener.onFailure(new ElasticsearchStatusException( "Cannot start transform [{}] as it is already STARTED.", RestStatus.CONFLICT, @@ -260,7 +255,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S final IndexerState newState = getIndexer().start(); if (Arrays.stream(RUNNING_STATES).noneMatch(newState::equals)) { listener.onFailure(new ElasticsearchException("Cannot start task for data frame transform [{}], because state was [{}]", - transform.getId(), newState)); + transform.getId(), newState)); return; } stateReason.set(null); @@ -298,10 +293,20 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S logger.error(new ParameterizedMessage("[{}] failed updating state to [{}].", getTransformId(), state), exc); getIndexer().stop(); listener.onFailure(new ElasticsearchException("Error while updating state for data frame transform [" - + transform.getId() + "] to [" + state.getIndexerState() + "].", exc)); + + transform.getId() + "] to [" + state.getIndexerState() + "].", exc)); } )); } + /** + * Start the background indexer and set the task's state to started + * @param startingCheckpoint Set the current checkpoint to this value. If null the + * current checkpoint is not set + * @param force Whether to force start a failed task or not + * @param listener Started listener + */ + public synchronized void start(Long startingCheckpoint, boolean force, ActionListener listener) { + start(startingCheckpoint, force, true, listener); + } public synchronized void stop(boolean force) { logger.debug("[{}] stop called with force [{}] and state [{}]", getTransformId(), force, getState()); diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java index b307d93073cc..24ae1ce99275 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataFrameSurvivesUpgradeIT.java @@ -7,7 +7,6 @@ package org.elasticsearch.upgrades; import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -52,7 +51,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.oneOf; -@LuceneTestCase.AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/46341") public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase { private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); From a5c60fb59a93d25d3534abd63a03057e514f186f Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 5 Sep 2019 08:33:17 -0400 Subject: [PATCH 065/103] [DOCS] Reformat index segments API docs (#46345) --- docs/reference/cat/segments.asciidoc | 42 ++-- docs/reference/indices/segments.asciidoc | 186 ++++++++++++------ docs/reference/rest-api/common-parms.asciidoc | 79 ++++++-- 3 files changed, 207 insertions(+), 100 deletions(-) diff --git a/docs/reference/cat/segments.asciidoc b/docs/reference/cat/segments.asciidoc index e67f48440ab8..59fefaa309b9 100644 --- a/docs/reference/cat/segments.asciidoc +++ b/docs/reference/cat/segments.asciidoc @@ -49,46 +49,40 @@ Valid columns are: (Default) IP address of the segment's shard, such as `127.0.1.1`. `segment`:: -(Default) Name of the segment, such as `_0`. The segment name is derived from -the segment generation and used internally to create file names in the directory -of the shard. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment] `generation`:: -(Default) Generation number, such as `0`. {es} increments this generation number -for each segment written. {es} then uses this number to derive the segment name. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=generation] `docs.count`:: -(Default) Number of non-deleted documents in the segment, such as `25`. This -number is based on Lucene documents and may include documents from -<> fields. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=docs-count] `docs.deleted`:: -(Default) Number of deleted documents in the segment, such as `0`. This number -is based on Lucene documents. {es} reclaims the disk space of deleted Lucene -documents when a segment is merged. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=docs-deleted] `size`:: -(Default) Disk space used by the segment, such as `50kb`. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment-size] `size.memory`:: -(Default) Bytes of segment data stored in memory for efficient search, such as -`1264`. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=memory] `committed`:: -(Default) If `true`, the segment is committed to disk. Segments committed to -disk would survive a hard reboot. -+ -If `false`, the data from uncommitted segments is also stored in the transaction -log. {es} replays those changes on the next start. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=committed] `searchable`:: -(Default) If `true`, the segment is searchable. -+ -If `false`, likely means the segment is written to disk but has not been -<>. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment-search] `version`:: -(Default) Version of Lucene used to write the segment. +(Default) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment-version] `compound`:: (Default) If `true`, the segment is stored in a compound file. This means Lucene diff --git a/docs/reference/indices/segments.asciidoc b/docs/reference/indices/segments.asciidoc index bc204a0a4a57..5500fba2d9f4 100644 --- a/docs/reference/indices/segments.asciidoc +++ b/docs/reference/indices/segments.asciidoc @@ -1,41 +1,136 @@ [[indices-segments]] -=== Indices Segments +=== Index segments API +++++ +Index segments +++++ -Provide low level segments information that a Lucene index (shard level) -is built with. Allows to be used to provide more information on the -state of a shard and an index, possibly optimization information, data -"wasted" on deletes, and so on. +Returns low-level information about the https://lucene.apache.org/core/[Lucene] +segments in index shards. -Endpoints include segments for a specific index: +[source,console] +---- +GET /twitter/_segments +---- +// TEST[setup:twitter] -[source,js] + +[[index-segments-api-request]] +==== {api-request-title} + +`GET //_segments` + +`GET /_segments` + +`GET /_cat/segments/` + + +[[index-segments-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index] + + +[[index-segments-api-query-params]] +==== {api-query-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=allow-no-indices] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] ++ +Defaults to `open`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] + +`verbose`:: +experimental:[] +(Optional, boolean) +If `true`, the response includes detailed information +about Lucene's memory usage. +Defaults to `false`. + + +[[index-segments-api-response-body]] +==== {api-response-body-title} + +``:: +(String) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment] + +`generation`:: +(Integer) +include::{docdir}/rest-api/common-parms.asciidoc[tag=generation] + +`num_docs`:: +(Integer) +include::{docdir}/rest-api/common-parms.asciidoc[tag=docs-count] + +`deleted_docs`:: +(Integer) +include::{docdir}/rest-api/common-parms.asciidoc[tag=docs-deleted] + +`size_in_bytes`:: +(Integer) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment-size] + +`memory_in_bytes`:: +(Integer) +include::{docdir}/rest-api/common-parms.asciidoc[tag=memory] + +`committed`:: +(Boolean) +include::{docdir}/rest-api/common-parms.asciidoc[tag=committed] + +`search`:: +(Boolean) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment-search] + +`version`:: +(String) +include::{docdir}/rest-api/common-parms.asciidoc[tag=segment-version] + +`compound`:: +(Boolean) +If `true`, Lucene merged all files from the segment +into a single file to save file descriptors. + +`attributes`:: +(Object) +Contains information about whether high compression was enabled. + + +[[index-segments-api-example]] +==== {api-examples-title} + + +===== Get segment information for a specific index + +[source,console] -------------------------------------------------- GET /test/_segments -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test\n{"settings":{"number_of_shards":1, "number_of_replicas": 0}}\nPOST test\/test\?refresh\n{"test": "test"}\n/] -// TESTSETUP -For several indices: -[source,js] +===== Get segment information for several indices + +[source,console] -------------------------------------------------- GET /test1,test2/_segments -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test1\nPUT test2\n/] -Or for all indices: -[source,js] +===== Get segment information for all indices + +[source,console] -------------------------------------------------- GET /_segments -------------------------------------------------- -// CONSOLE +// TEST[s/^/PUT test\n{"settings":{"number_of_shards":1, "number_of_replicas": 0}}\nPOST test\/test\?refresh\n{"test": "test"}\n/] -Response: +The API returns the following response: -[source,js] +[source,console-response] -------------------------------------------------- { "_shards": ... @@ -79,61 +174,23 @@ Response: // TESTRESPONSE[s/: (\-)?[0-9]+/: $body.$_path/] // TESTRESPONSE[s/7\.0\.0/$body.$_path/] -_0:: The key of the JSON document is the name of the segment. This name - is used to generate file names: all files starting with this - segment name in the directory of the shard belong to this segment. -generation:: A generation number that is basically incremented when needing to - write a new segment. The segment name is derived from this - generation number. +===== Verbose mode -num_docs:: The number of non-deleted documents that are stored in this segment. +To add additional information that can be used for debugging, +use the `verbose` flag. -deleted_docs:: The number of deleted documents that are stored in this segment. - It is perfectly fine if this number is greater than 0, space is - going to be reclaimed when this segment gets merged. +experimental::[] -size_in_bytes:: The amount of disk space that this segment uses, in bytes. - -memory_in_bytes:: Segments need to store some data into memory in order to be - searchable efficiently. This number returns the number of bytes - that are used for that purpose. A value of -1 indicates that - Elasticsearch was not able to compute this number. - -committed:: Whether the segment has been sync'ed on disk. Segments that are - committed would survive a hard reboot. No need to worry in case - of false, the data from uncommitted segments is also stored in - the transaction log so that Elasticsearch is able to replay - changes on the next start. - -search:: Whether the segment is searchable. A value of false would most - likely mean that the segment has been written to disk but no - refresh occurred since then to make it searchable. - -version:: The version of Lucene that has been used to write this segment. - -compound:: Whether the segment is stored in a compound file. When true, this - means that Lucene merged all files from the segment in a single - one in order to save file descriptors. - -attributes:: Contains information about whether high compression was enabled - -[float] -==== Verbose mode - -To add additional information that can be used for debugging, use the `verbose` flag. - -NOTE: The format of the additional detail information is labelled as experimental in Lucene and it may change in the future. - -[source,js] +[source,console] -------------------------------------------------- GET /test/_segments?verbose=true -------------------------------------------------- -// CONSOLE +// TEST[continued] -Response: +The API returns the following response: -[source,js] +[source,console-response] -------------------------------------------------- { ... @@ -159,5 +216,4 @@ Response: ... } -------------------------------------------------- -// NOTCONSOLE -//Response is too verbose to be fully shown in documentation, so we just show the relevant bit and don't test the response. +// TESTRESPONSE[skip:Response is too verbose to be fully shown in documentation, so we just show the relevant bit and don't test the response.] diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index 7d8a347507de..4704266d15c5 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -34,6 +34,16 @@ tag::bytes[] (Optional, <>) Unit used to display byte values. end::bytes[] +tag::committed[] +If `true`, +the segments is synced to disk. Segments that are synced can survive a hard reboot. ++ +If `false`, +the data from uncommitted segments is also stored in +the transaction log so that Elasticsearch is able to replay +changes on the next start. +end::committed[] + tag::default_operator[] `default_operator`:: (Optional, string) The default operator for query string query: AND or OR. @@ -46,6 +56,18 @@ tag::df[] given in the query string. end::df[] +tag::docs-count[] +Number of non-deleted documents in the segment, such as `25`. This +number is based on Lucene documents and may include documents from +<> fields. +end::docs-count[] + +tag::docs-deleted[] +Number of deleted documents in the segment, such as `0`. This number +is based on Lucene documents. {es} reclaims the disk space of deleted Lucene +documents when a segment is merged. +end::docs-deleted[] + tag::expand-wildcards[] `expand_wildcards`:: + @@ -73,6 +95,11 @@ tag::flat-settings[] `false`. end::flat-settings[] +tag::generation[] +Generation number, such as `0`. {es} increments this generation number +for each segment written. {es} then uses this number to derive the segment name. +end::generation[] + tag::index-alias-filter[] <> used to limit the index alias. @@ -188,6 +215,13 @@ tag::max_docs[] documents. end::max_docs[] +tag::memory[] +Bytes of segment data stored in memory for efficient search, +such as `1264`. ++ +A value of `-1` indicates {es} was unable to compute this number. +end::memory[] + tag::name[] ``:: (Optional, string) Comma-separated list of alias names to return. @@ -231,8 +265,8 @@ end::request_cache[] tag::requests_per_second[] `requests_per_second`:: - (Optional, integer) The throttle for this request in sub-requests per second. - -1 means no throttle. Defaults to 0. +(Optional, integer) The throttle for this request in sub-requests per second. +-1 means no throttle. Defaults to 0. end::requests_per_second[] tag::routing[] @@ -240,6 +274,13 @@ tag::routing[] (Optional, string) Target the specified primary shard. end::routing[] +tag::index-routing[] +`routing`:: +(Optional, string) +Custom <> +used to route operations to a specific shard. +end::index-routing[] + tag::cat-s[] `s`:: (Optional, string) Comma-separated list of column names or column aliases used @@ -258,6 +299,15 @@ tag::scroll_size[] Defaults to 100. end::scroll_size[] +tag::segment-search[] +If `true`, +the segment is searchable. ++ +If `false`, +the segment has most likely been written to disk +but needs a <> to be searchable. +end::segment-search[] + tag::search_timeout[] `search_timeout`:: (Optional, <> Explicit timeout for each search @@ -271,12 +321,22 @@ tag::search_type[] * `dfs_query_then_fetch` end::search_type[] +tag::segment[] +Name of the segment, such as `_0`. The segment name is derived from +the segment generation and used internally to create file names in the directory +of the shard. +end::segment[] + tag::settings[] `settings`:: (Optional, <>) Configuration options for the index. See <>. end::settings[] +tag::segment-size[] +Disk space used by the segment, such as `50kb`. +end::segment-size[] + tag::slices[] `slices`:: (Optional, integer) The number of slices this task should be divided into. @@ -308,8 +368,8 @@ end::source_includes[] tag::stats[] `stats`:: - (Optional, string) Specific `tag` of the request for logging and statistical - purposes. +(Optional, string) Specific `tag` of the request for logging and statistical +purposes. end::stats[] tag::terminate_after[] @@ -347,13 +407,6 @@ tag::version[] (Optional, boolean) If `true`, returns the document version as part of a hit. end::version[] -tag::index-routing[] -`routing`:: -(Optional, string) -Custom <> -used to route operations to a specific shard. -end::index-routing[] - tag::doc-version[] `version`:: (Optional, integer) Explicit version number for concurrency control. @@ -361,6 +414,10 @@ The specified version must match the current version of the document for the request to succeed. end::doc-version[] +tag::segment-version[] +Version of Lucene used to write the segment. +end::segment-version[] + tag::version_type[] `version_type`:: (Optional, enum) Specific version type: `internal`, `external`, From b336fbe38aa6d1396ad384d5eb50dfb7fd64d89a Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 5 Sep 2019 10:18:15 -0400 Subject: [PATCH 066/103] [DOCS] Re-add versioning to put template docs (#46384) Adds documentation for index template versioning accidentally removed with #46297. --- docs/reference/indices/templates.asciidoc | 57 +++++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/docs/reference/indices/templates.asciidoc b/docs/reference/indices/templates.asciidoc index 191b2c8f9223..1c410a3d8fd9 100644 --- a/docs/reference/indices/templates.asciidoc +++ b/docs/reference/indices/templates.asciidoc @@ -115,6 +115,10 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=mappings] include::{docdir}/rest-api/common-parms.asciidoc[tag=settings] +`version`:: +(Optional, integer) +Version number used to manage index templates externally. +This number is not automatically generated by {es}. [[put-index-template-api-example]] @@ -197,3 +201,56 @@ order templates, with lower order templates providing the basis. NOTE: Multiple matching templates with the same order value will result in a non-deterministic merging order. + + +[[versioning-templates]] +===== Template versioning + +You can use the `version` parameter +to add an optional version number to an index template. +External systems can use these version numbers +to simplify template management. + +The `version` parameter is completely optional +and not automatically generated by {es}. + +To unset a `version`, +replace the template without specifying one. + +[source,js] +-------------------------------------------------- +PUT /_template/template_1 +{ + "index_patterns" : ["*"], + "order" : 0, + "settings" : { + "number_of_shards" : 1 + }, + "version": 123 +} +-------------------------------------------------- +// CONSOLE + +To check the `version`, +you can use the <> API +with the <> query parameter +to return only the version number: + +[source,js] +-------------------------------------------------- +GET /_template/template_1?filter_path=*.version +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +The API returns the following response: + +[source,js] +-------------------------------------------------- +{ + "template_1" : { + "version" : 123 + } +} +-------------------------------------------------- +// TESTRESPONSE From ca8dbe5b7e1fbbef1745af8dab6ec84f37816021 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 5 Sep 2019 10:00:57 -0500 Subject: [PATCH 067/103] [ML][Transforms] update supported aggs docs (#46388) --- docs/reference/data-frames/apis/transformresource.asciidoc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/reference/data-frames/apis/transformresource.asciidoc b/docs/reference/data-frames/apis/transformresource.asciidoc index 4f6b43c43bc1..f51074bd4562 100644 --- a/docs/reference/data-frames/apis/transformresource.asciidoc +++ b/docs/reference/data-frames/apis/transformresource.asciidoc @@ -90,6 +90,7 @@ pivot function `group by` fields and the aggregation to reduce the data. * {ref}/search-aggregations-metrics-avg-aggregation.html[Average] * {ref}/search-aggregations-metrics-weight-avg-aggregation.html[Weighted Average] * {ref}/search-aggregations-metrics-cardinality-aggregation.html[Cardinality] +* {ref}/search-aggregations-metrics-geobounds-aggregation.html[Geo Bounds] * {ref}/search-aggregations-metrics-geocentroid-aggregation.html[Geo Centroid] * {ref}/search-aggregations-metrics-max-aggregation.html[Max] * {ref}/search-aggregations-metrics-min-aggregation.html[Min] @@ -97,6 +98,7 @@ pivot function `group by` fields and the aggregation to reduce the data. * {ref}/search-aggregations-metrics-sum-aggregation.html[Sum] * {ref}/search-aggregations-metrics-valuecount-aggregation.html[Value Count] * {ref}/search-aggregations-pipeline-bucket-script-aggregation.html[Bucket Script] +* {ref}/search-aggregations-pipeline-bucket-selector-aggregation.html[Bucket Selector] IMPORTANT: {dataframe-transforms-cap} support a subset of the functionality in composite aggregations. See From 9d3467ae307b71b12608e29d8c557f2317d27c2d Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 5 Sep 2019 10:36:34 -0500 Subject: [PATCH 068/103] Support geotile_grid aggregation in composite agg sources (#45810) Adds support for `geotile_grid` as a source in composite aggs. Part of this change includes adding a new docFormat of `GEOTILE` that formats a hashed `long` value into a geotile formatting string `zoom/x/y`. --- .../test/search.aggregation/230_composite.yml | 94 +++++++++++++- .../elasticsearch/search/DocValueFormat.java | 23 ++++ .../elasticsearch/search/SearchModule.java | 1 + .../bucket/composite/CompositeAggregator.java | 12 ++ .../CompositeValuesSourceParserHelper.java | 12 ++ .../GeoTileGridValuesSourceBuilder.java | 122 ++++++++++++++++++ .../bucket/composite/GeoTileValuesSource.java | 61 +++++++++ .../bucket/geogrid/CellIdSource.java | 4 +- .../GeoTileGridAggregationBuilder.java | 2 +- .../bucket/geogrid/GeoTileUtils.java | 62 ++++++--- .../search/DocValueFormatTests.java | 18 +++ .../search/SearchSortValuesTests.java | 7 +- .../CompositeAggregationBuilderTests.java | 13 +- .../composite/CompositeAggregatorTests.java | 57 +++++++- .../GeoTileGridValuesSourceBuilderTests.java | 31 +++++ .../bucket/geogrid/GeoTileUtilsTests.java | 25 ++++ 16 files changed, 514 insertions(+), 30 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilder.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileValuesSource.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilderTests.java diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml index fc0710fdb537..78277cd2b670 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml @@ -12,6 +12,8 @@ setup: type: keyword long: type: long + geo_point: + type: geo_point nested: type: nested properties: @@ -38,25 +40,25 @@ setup: index: index: test id: 1 - body: { "keyword": "foo", "long": [10, 20], "nested": [{"nested_long": 10}, {"nested_long": 20}] } + body: { "keyword": "foo", "long": [10, 20], "geo_point": "37.2343,-115.8067", "nested": [{"nested_long": 10}, {"nested_long": 20}] } - do: index: index: test id: 2 - body: { "keyword": ["foo", "bar"] } + body: { "keyword": ["foo", "bar"], "geo_point": "41.12,-71.34" } - do: index: index: test id: 3 - body: { "keyword": "bar", "long": [100, 0], "nested": [{"nested_long": 10}, {"nested_long": 0}] } + body: { "keyword": "bar", "long": [100, 0], "geo_point": "90.0,0.0", "nested": [{"nested_long": 10}, {"nested_long": 0}] } - do: index: index: test id: 4 - body: { "keyword": "bar", "long": [1000, 0], "nested": [{"nested_long": 1000}, {"nested_long": 20}] } + body: { "keyword": "bar", "long": [1000, 0], "geo_point": "41.12,-71.34", "nested": [{"nested_long": 1000}, {"nested_long": 20}] } - do: index: @@ -615,3 +617,87 @@ setup: } ] +--- +"Simple Composite aggregation with GeoTile grid": + - skip: + version: " - 7.99.99" + reason: geotile_grid is not supported until 8.0.0 + - do: + search: + rest_total_hits_as_int: true + index: test + body: + aggregations: + test: + composite: + sources: [ + "geo": { + "geotile_grid": { + "field": "geo_point", + "precision": 12 + } + }, + { + "kw": { + "terms": { + "field": "keyword" + } + } + } + ] + + - match: {hits.total: 6} + - length: { aggregations.test.buckets: 4 } + - match: { aggregations.test.buckets.0.key.geo: "12/730/1590" } + - match: { aggregations.test.buckets.0.key.kw: "foo" } + - match: { aggregations.test.buckets.0.doc_count: 1 } + - match: { aggregations.test.buckets.1.key.geo: "12/1236/1533" } + - match: { aggregations.test.buckets.1.key.kw: "bar" } + - match: { aggregations.test.buckets.1.doc_count: 2 } + - match: { aggregations.test.buckets.2.key.geo: "12/1236/1533" } + - match: { aggregations.test.buckets.2.key.kw: "foo" } + - match: { aggregations.test.buckets.2.doc_count: 1 } + - match: { aggregations.test.buckets.3.key.geo: "12/2048/0" } + - match: { aggregations.test.buckets.3.key.kw: "bar" } + - match: { aggregations.test.buckets.3.doc_count: 1 } +--- +"Simple Composite aggregation with geotile grid add aggregate after": + - skip: + version: " - 7.99.99" + reason: geotile_grid is not supported until 8.0.0 + - do: + search: + rest_total_hits_as_int: true + index: test + body: + aggregations: + test: + composite: + sources: [ + "geo": { + "geotile_grid": { + "field": "geo_point", + "precision": 12 + } + }, + { + "kw": { + "terms": { + "field": "keyword" + } + } + } + ] + after: { "geo": "12/730/1590", "kw": "foo" } + + - match: {hits.total: 6} + - length: { aggregations.test.buckets: 3 } + - match: { aggregations.test.buckets.0.key.geo: "12/1236/1533" } + - match: { aggregations.test.buckets.0.key.kw: "bar" } + - match: { aggregations.test.buckets.0.doc_count: 2 } + - match: { aggregations.test.buckets.1.key.geo: "12/1236/1533" } + - match: { aggregations.test.buckets.1.key.kw: "foo" } + - match: { aggregations.test.buckets.1.doc_count: 1 } + - match: { aggregations.test.buckets.2.key.geo: "12/2048/0" } + - match: { aggregations.test.buckets.2.key.kw: "bar" } + - match: { aggregations.test.buckets.2.doc_count: 1 } diff --git a/server/src/main/java/org/elasticsearch/search/DocValueFormat.java b/server/src/main/java/org/elasticsearch/search/DocValueFormat.java index ad7699d9f9c2..684b403a0f0f 100644 --- a/server/src/main/java/org/elasticsearch/search/DocValueFormat.java +++ b/server/src/main/java/org/elasticsearch/search/DocValueFormat.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateMathParser; import org.elasticsearch.geometry.utils.Geohash; import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileUtils; import java.io.IOException; import java.net.InetAddress; @@ -246,6 +247,28 @@ public interface DocValueFormat extends NamedWriteable { } }; + DocValueFormat GEOTILE = new DocValueFormat() { + + @Override + public String getWriteableName() { + return "geo_tile"; + } + + @Override + public void writeTo(StreamOutput out) { + } + + @Override + public String format(long value) { + return GeoTileUtils.stringEncode(value); + } + + @Override + public String format(double value) { + return format((long) value); + } + }; + DocValueFormat BOOLEAN = new DocValueFormat() { @Override diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 01a4aa66810c..025f15033099 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -667,6 +667,7 @@ public class SearchModule { registerValueFormat(DocValueFormat.DateTime.NAME, DocValueFormat.DateTime::new); registerValueFormat(DocValueFormat.Decimal.NAME, DocValueFormat.Decimal::new); registerValueFormat(DocValueFormat.GEOHASH.getWriteableName(), in -> DocValueFormat.GEOHASH); + registerValueFormat(DocValueFormat.GEOTILE.getWriteableName(), in -> DocValueFormat.GEOTILE); registerValueFormat(DocValueFormat.IP.getWriteableName(), in -> DocValueFormat.IP); registerValueFormat(DocValueFormat.RAW.getWriteableName(), in -> DocValueFormat.RAW); registerValueFormat(DocValueFormat.BINARY.getWriteableName(), in -> DocValueFormat.BINARY); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java index cd7fd6abe8ca..4effb22f30cb 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -42,6 +42,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.MultiBucketCollector; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.bucket.geogrid.CellIdSource; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; @@ -299,6 +300,17 @@ final class CompositeAggregator extends BucketsAggregator { reverseMul ); + } else if (config.valuesSource() instanceof CellIdSource) { + final CellIdSource cis = (CellIdSource) config.valuesSource(); + return new GeoTileValuesSource( + bigArrays, + config.fieldType(), + cis::longValues, + LongUnaryOperator.identity(), + config.format(), + config.missingBucket(), + size, + reverseMul); } else if (config.valuesSource() instanceof ValuesSource.Numeric) { final ValuesSource.Numeric vs = (ValuesSource.Numeric) config.valuesSource(); if (vs.isFloatingPoint()) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java index d773a09d645d..4ca96ea577c1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.aggregations.bucket.composite; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -67,6 +68,12 @@ public class CompositeValuesSourceParserHelper { code = 1; } else if (builder.getClass() == HistogramValuesSourceBuilder.class) { code = 2; + } else if (builder.getClass() == GeoTileGridValuesSourceBuilder.class) { + if (out.getVersion().before(Version.V_8_0_0)) { + throw new IOException("Attempting to serialize [" + builder.getClass().getSimpleName() + + "] to a node with unsupported version [" + out.getVersion() + "]"); + } + code = 3; } else { throw new IOException("invalid builder type: " + builder.getClass().getSimpleName()); } @@ -83,6 +90,8 @@ public class CompositeValuesSourceParserHelper { return new DateHistogramValuesSourceBuilder(in); case 2: return new HistogramValuesSourceBuilder(in); + case 3: + return new GeoTileGridValuesSourceBuilder(in); default: throw new IOException("Invalid code " + code); } @@ -112,6 +121,9 @@ public class CompositeValuesSourceParserHelper { case HistogramValuesSourceBuilder.TYPE: builder = HistogramValuesSourceBuilder.parse(name, parser); break; + case GeoTileGridValuesSourceBuilder.TYPE: + builder = GeoTileGridValuesSourceBuilder.parse(name, parser); + break; default: throw new ParsingException(parser.getTokenLocation(), "invalid source type: " + type); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilder.java new file mode 100644 index 000000000000..36debb1e40fb --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilder.java @@ -0,0 +1,122 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.bucket.geogrid.CellIdSource; +import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileGridAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileUtils; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Objects; + +public class GeoTileGridValuesSourceBuilder extends CompositeValuesSourceBuilder { + static final String TYPE = "geotile_grid"; + + private static final ObjectParser PARSER; + static { + PARSER = new ObjectParser<>(GeoTileGridValuesSourceBuilder.TYPE); + PARSER.declareInt(GeoTileGridValuesSourceBuilder::precision, new ParseField("precision")); + CompositeValuesSourceParserHelper.declareValuesSourceFields(PARSER, ValueType.NUMERIC); + } + + static GeoTileGridValuesSourceBuilder parse(String name, XContentParser parser) throws IOException { + return PARSER.parse(parser, new GeoTileGridValuesSourceBuilder(name), null); + } + + private int precision = GeoTileGridAggregationBuilder.DEFAULT_PRECISION; + + GeoTileGridValuesSourceBuilder(String name) { + super(name); + } + + GeoTileGridValuesSourceBuilder(StreamInput in) throws IOException { + super(in); + this.precision = in.readInt(); + } + + public GeoTileGridValuesSourceBuilder precision(int precision) { + this.precision = GeoTileUtils.checkPrecisionRange(precision); + return this; + } + + @Override + public GeoTileGridValuesSourceBuilder format(String format) { + throw new IllegalArgumentException("[format] is not supported for [" + TYPE + "]"); + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeInt(precision); + } + + @Override + protected void doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field("precision", precision); + } + + @Override + String type() { + return TYPE; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), precision); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + if (super.equals(obj) == false) return false; + GeoTileGridValuesSourceBuilder other = (GeoTileGridValuesSourceBuilder) obj; + return precision == other.precision; + } + + @Override + protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig config) throws IOException { + ValuesSource orig = config.toValuesSource(context.getQueryShardContext()); + if (orig == null) { + orig = ValuesSource.GeoPoint.EMPTY; + } + if (orig instanceof ValuesSource.GeoPoint) { + ValuesSource.GeoPoint geoPoint = (ValuesSource.GeoPoint) orig; + // is specified in the builder. + final MappedFieldType fieldType = config.fieldContext() != null ? config.fieldContext().fieldType() : null; + CellIdSource cellIdSource = new CellIdSource(geoPoint, precision, GeoTileUtils::longEncode); + return new CompositeValuesSourceConfig(name, fieldType, cellIdSource, DocValueFormat.GEOTILE, order(), missingBucket()); + } else { + throw new IllegalArgumentException("invalid source, expected geo_point, got " + orig.getClass().getSimpleName()); + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileValuesSource.java new file mode 100644 index 000000000000..b7517f057740 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileValuesSource.java @@ -0,0 +1,61 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileUtils; + +import java.io.IOException; +import java.util.function.LongUnaryOperator; + +/** + * A {@link SingleDimensionValuesSource} for geotile values. + * + * Since geotile values can be represented as long values, this class is almost the same as {@link LongValuesSource} + * The main differences is {@link GeoTileValuesSource#setAfter(Comparable)} as it needs to accept geotile string values i.e. "zoom/x/y". + */ +class GeoTileValuesSource extends LongValuesSource { + GeoTileValuesSource(BigArrays bigArrays, + MappedFieldType fieldType, + CheckedFunction docValuesFunc, + LongUnaryOperator rounding, + DocValueFormat format, + boolean missingBucket, + int size, + int reverseMul) { + super(bigArrays, fieldType, docValuesFunc, rounding, format, missingBucket, size, reverseMul); + } + + @Override + void setAfter(Comparable value) { + if (missingBucket && value == null) { + afterValue = null; + } else if (value instanceof Number) { + afterValue = ((Number) value).longValue(); + } else { + afterValue = GeoTileUtils.longEncode(value.toString()); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/CellIdSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/CellIdSource.java index 0cc7734ad768..4ebb689c7c44 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/CellIdSource.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/CellIdSource.java @@ -32,12 +32,12 @@ import java.io.IOException; * Wrapper class to help convert {@link MultiGeoPointValues} * to numeric long values for bucketing. */ -class CellIdSource extends ValuesSource.Numeric { +public class CellIdSource extends ValuesSource.Numeric { private final ValuesSource.GeoPoint valuesSource; private final int precision; private final GeoPointLongEncoder encoder; - CellIdSource(GeoPoint valuesSource, int precision, GeoPointLongEncoder encoder) { + public CellIdSource(GeoPoint valuesSource, int precision, GeoPointLongEncoder encoder) { this.valuesSource = valuesSource; //different GeoPoints could map to the same or different hashing cells. this.precision = precision; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java index 374b7ddf7073..966bb81a9f85 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileGridAggregationBuilder.java @@ -35,7 +35,7 @@ import java.util.Map; public class GeoTileGridAggregationBuilder extends GeoGridAggregationBuilder { public static final String NAME = "geotile_grid"; - private static final int DEFAULT_PRECISION = 7; + public static final int DEFAULT_PRECISION = 7; private static final int DEFAULT_MAX_NUM_CELLS = 10000; private static final ObjectParser PARSER = createParser(NAME, GeoTileUtils::parsePrecision); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtils.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtils.java index d85cf6b1a56c..c417be016288 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtils.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtils.java @@ -38,7 +38,7 @@ import static org.elasticsearch.common.geo.GeoUtils.normalizeLon; * bits 29..57 -- X tile index (0..2^zoom) * bits 0..28 -- Y tile index (0..2^zoom) */ -final class GeoTileUtils { +public final class GeoTileUtils { private GeoTileUtils() {} @@ -80,7 +80,7 @@ final class GeoTileUtils { /** * Assert the precision value is within the allowed range, and return it if ok, or throw. */ - static int checkPrecisionRange(int precision) { + public static int checkPrecisionRange(int precision) { if (precision < 0 || precision > MAX_ZOOM) { throw new IllegalArgumentException("Invalid geotile_grid precision of " + precision + ". Must be between 0 and " + MAX_ZOOM + "."); @@ -93,7 +93,7 @@ final class GeoTileUtils { * The resulting hash contains interleaved tile X and Y coordinates. * The precision itself is also encoded as a few high bits. */ - static long longEncode(double longitude, double latitude, int precision) { + public static long longEncode(double longitude, double latitude, int precision) { // Mathematics for this code was adapted from https://wiki.openstreetmap.org/wiki/Slippy_map_tilenames#Java // Number of tiles for the current zoom level along the X and Y axis @@ -119,10 +119,18 @@ final class GeoTileUtils { yTile = tiles - 1; } - // Zoom value is placed in front of all the bits used for the geotile - // e.g. when max zoom is 29, the largest index would use 58 bits (57th..0th), - // leaving 5 bits unused for zoom. See MAX_ZOOM comment above. - return ((long) precision << ZOOM_SHIFT) | (xTile << MAX_ZOOM) | yTile; + return longEncode((long) precision, xTile, yTile); + } + + /** + * Encode a geotile hash style string to a long. + * + * @param hashAsString String in format "zoom/x/y" + * @return long encoded value of the given string hash + */ + public static long longEncode(String hashAsString) { + int[] parsed = parseHash(hashAsString); + return longEncode((long)parsed[0], (long)parsed[1], (long)parsed[2]); } /** @@ -135,10 +143,34 @@ final class GeoTileUtils { return new int[]{zoom, xTile, yTile}; } + private static long longEncode(long precision, long xTile, long yTile) { + // Zoom value is placed in front of all the bits used for the geotile + // e.g. when max zoom is 29, the largest index would use 58 bits (57th..0th), + // leaving 5 bits unused for zoom. See MAX_ZOOM comment above. + return (precision << ZOOM_SHIFT) | (xTile << MAX_ZOOM) | yTile; + } + + /** + * Parse geotile String hash format in "zoom/x/y" into an array of integers + */ + private static int[] parseHash(String hashAsString) { + final String[] parts = hashAsString.split("/", 4); + if (parts.length != 3) { + throw new IllegalArgumentException("Invalid geotile_grid hash string of " + + hashAsString + ". Must be three integers in a form \"zoom/x/y\"."); + } + try { + return new int[]{Integer.parseInt(parts[0]), Integer.parseInt(parts[1]), Integer.parseInt(parts[2])}; + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid geotile_grid hash string of " + + hashAsString + ". Must be three integers in a form \"zoom/x/y\".", e); + } + } + /** * Encode to a geotile string from the geotile based long format */ - static String stringEncode(long hash) { + public static String stringEncode(long hash) { int[] res = parseHash(hash); validateZXY(res[0], res[1], res[2]); return "" + res[0] + "/" + res[1] + "/" + res[2]; @@ -156,18 +188,8 @@ final class GeoTileUtils { * Decode a string bucket key in "zoom/x/y" format to a GeoPoint (center of the tile) */ static GeoPoint keyToGeoPoint(String hashAsString) { - final String[] parts = hashAsString.split("/", 4); - if (parts.length != 3) { - throw new IllegalArgumentException("Invalid geotile_grid hash string of " + - hashAsString + ". Must be three integers in a form \"zoom/x/y\"."); - } - - try { - return zxyToGeoPoint(Integer.parseInt(parts[0]), Integer.parseInt(parts[1]), Integer.parseInt(parts[2])); - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid geotile_grid hash string of " + - hashAsString + ". Must be three integers in a form \"zoom/x/y\".", e); - } + int[] hashAsInts = parseHash(hashAsString); + return zxyToGeoPoint(hashAsInts[0], hashAsInts[1], hashAsInts[2]); } /** diff --git a/server/src/test/java/org/elasticsearch/search/DocValueFormatTests.java b/server/src/test/java/org/elasticsearch/search/DocValueFormatTests.java index cd31487aeb51..df948a5bff71 100644 --- a/server/src/test/java/org/elasticsearch/search/DocValueFormatTests.java +++ b/server/src/test/java/org/elasticsearch/search/DocValueFormatTests.java @@ -35,6 +35,8 @@ import java.time.ZoneOffset; import java.util.ArrayList; import java.util.List; +import static org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileUtils.longEncode; + public class DocValueFormatTests extends ESTestCase { public void testSerialization() throws Exception { @@ -43,6 +45,7 @@ public class DocValueFormatTests extends ESTestCase { entries.add(new Entry(DocValueFormat.class, DocValueFormat.DateTime.NAME, DocValueFormat.DateTime::new)); entries.add(new Entry(DocValueFormat.class, DocValueFormat.Decimal.NAME, DocValueFormat.Decimal::new)); entries.add(new Entry(DocValueFormat.class, DocValueFormat.GEOHASH.getWriteableName(), in -> DocValueFormat.GEOHASH)); + entries.add(new Entry(DocValueFormat.class, DocValueFormat.GEOTILE.getWriteableName(), in -> DocValueFormat.GEOTILE)); entries.add(new Entry(DocValueFormat.class, DocValueFormat.IP.getWriteableName(), in -> DocValueFormat.IP)); entries.add(new Entry(DocValueFormat.class, DocValueFormat.RAW.getWriteableName(), in -> DocValueFormat.RAW)); entries.add(new Entry(DocValueFormat.class, DocValueFormat.BINARY.getWriteableName(), in -> DocValueFormat.BINARY)); @@ -87,6 +90,11 @@ public class DocValueFormatTests extends ESTestCase { in = new NamedWriteableAwareStreamInput(out.bytes().streamInput(), registry); assertSame(DocValueFormat.GEOHASH, in.readNamedWriteable(DocValueFormat.class)); + out = new BytesStreamOutput(); + out.writeNamedWriteable(DocValueFormat.GEOTILE); + in = new NamedWriteableAwareStreamInput(out.bytes().streamInput(), registry); + assertSame(DocValueFormat.GEOTILE, in.readNamedWriteable(DocValueFormat.class)); + out = new BytesStreamOutput(); out.writeNamedWriteable(DocValueFormat.IP); in = new NamedWriteableAwareStreamInput(out.bytes().streamInput(), registry); @@ -147,6 +155,16 @@ public class DocValueFormatTests extends ESTestCase { assertEquals("859,802.354", formatter.format(0.8598023539251286d * 1_000_000)); } + public void testGeoTileFormat() { + assertEquals("0/0/0", DocValueFormat.GEOTILE.format(longEncode(0, 0, 0))); + assertEquals("15/19114/7333", DocValueFormat.GEOTILE.format(longEncode(30, 70, 15))); + assertEquals("29/536869420/0", DocValueFormat.GEOTILE.format(longEncode(179.999, 89.999, 29))); + assertEquals("29/1491/536870911", DocValueFormat.GEOTILE.format(longEncode(-179.999, -89.999, 29))); + assertEquals("2/2/1", DocValueFormat.GEOTILE.format(longEncode(1, 1, 2))); + assertEquals("1/1/0", DocValueFormat.GEOTILE.format(longEncode(13,95, 1))); + assertEquals("1/1/1", DocValueFormat.GEOTILE.format(longEncode(13,-95, 1))); + } + public void testRawParse() { assertEquals(-1L, DocValueFormat.RAW.parseLong("-1", randomBoolean(), null)); assertEquals(1L, DocValueFormat.RAW.parseLong("1", randomBoolean(), null)); diff --git a/server/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java b/server/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java index 940a34344a4f..8cec44618c90 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchSortValuesTests.java @@ -67,7 +67,12 @@ public class SearchSortValuesTests extends AbstractSerializingTestCase> sources = new ArrayList<>(); for (int i = 0; i < numSources; i++) { - int type = randomIntBetween(0, 2); + int type = randomIntBetween(0, 3); switch (type) { case 0: sources.add(randomTermsSourceBuilder()); @@ -95,6 +103,9 @@ public class CompositeAggregationBuilderTests extends BaseAggregationTestCase>> dataset = new ArrayList<>(); + dataset.addAll( + Arrays.asList( + createDocument("geo_point", new GeoPoint(48.934059, 41.610741)), + createDocument("geo_point", new GeoPoint(-23.065941, 113.610741)), + createDocument("geo_point", new GeoPoint(90.0, 0.0)), + createDocument("geo_point", new GeoPoint(37.2343, -115.8067)), + createDocument("geo_point", new GeoPoint(90.0, 0.0)) + ) + ); + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("geo_point")), dataset, + () -> { + GeoTileGridValuesSourceBuilder geoTile = new GeoTileGridValuesSourceBuilder("geo_point") + .field("geo_point"); + return new CompositeAggregationBuilder("name", Collections.singletonList(geoTile)); + }, (result) -> { + assertEquals(2, result.getBuckets().size()); + assertEquals("{geo_point=7/64/56}", result.afterKey().toString()); + assertEquals("{geo_point=7/32/56}", result.getBuckets().get(0).getKeyAsString()); + assertEquals(2L, result.getBuckets().get(0).getDocCount()); + assertEquals("{geo_point=7/64/56}", result.getBuckets().get(1).getKeyAsString()); + assertEquals(3L, result.getBuckets().get(1).getDocCount()); + } + ); + + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("geo_point")), dataset, + () -> { + GeoTileGridValuesSourceBuilder geoTile = new GeoTileGridValuesSourceBuilder("geo_point") + .field("geo_point"); + return new CompositeAggregationBuilder("name", Collections.singletonList(geoTile)) + .aggregateAfter(Collections.singletonMap("geo_point", "7/32/56")); + }, (result) -> { + assertEquals(1, result.getBuckets().size()); + assertEquals("{geo_point=7/64/56}", result.afterKey().toString()); + assertEquals("{geo_point=7/64/56}", result.getBuckets().get(0).getKeyAsString()); + assertEquals(3L, result.getBuckets().get(0).getDocCount()); + } + ); + } + private void testSearchCase(List queries, List>> dataset, Supplier create, @@ -1845,6 +1895,11 @@ public class CompositeAggregatorTests extends AggregatorTestCase { } else if (value instanceof InetAddress) { doc.add(new SortedSetDocValuesField(name, new BytesRef(InetAddressPoint.encode((InetAddress) value)))); doc.add(new InetAddressPoint(name, (InetAddress) value)); + } else if (value instanceof GeoPoint) { + GeoPoint point = (GeoPoint)value; + doc.add(new SortedNumericDocValuesField(name, + GeoTileUtils.longEncode(point.lon(), point.lat(), GeoTileGridAggregationBuilder.DEFAULT_PRECISION))); + doc.add(new LatLonPoint(name, point.lat(), point.lon())); } else { throw new AssertionError("invalid object: " + value.getClass().getSimpleName()); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilderTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilderTests.java new file mode 100644 index 000000000000..6f9e0f697da2 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/GeoTileGridValuesSourceBuilderTests.java @@ -0,0 +1,31 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.elasticsearch.test.ESTestCase; + +public class GeoTileGridValuesSourceBuilderTests extends ESTestCase { + + public void testSetFormat() { + CompositeValuesSourceBuilder builder = new GeoTileGridValuesSourceBuilder("name"); + expectThrows(IllegalArgumentException.class, () -> builder.format("format")); + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtilsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtilsTests.java index e2881fd9b914..fc5cf6cb910b 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoTileUtilsTests.java @@ -76,6 +76,31 @@ public class GeoTileUtilsTests extends ESTestCase { expectThrows(IllegalArgumentException.class, () -> longEncode(-1, 0, MAX_ZOOM + 1)); } + public void testLongEncodeFromString() { + assertEquals(0x0000000000000000L, longEncode(stringEncode(longEncode(0, 0, 0)))); + assertEquals(0x3C00095540001CA5L, longEncode(stringEncode(longEncode(30, 70, 15)))); + assertEquals(0x77FFFF4580000000L, longEncode(stringEncode(longEncode(179.999, 89.999, 29)))); + assertEquals(0x740000BA7FFFFFFFL, longEncode(stringEncode(longEncode(-179.999, -89.999, 29)))); + assertEquals(0x0800000040000001L, longEncode(stringEncode(longEncode(1, 1, 2)))); + assertEquals(0x0C00000060000000L, longEncode(stringEncode(longEncode(-20, 100, 3)))); + assertEquals(0x71127D27C8ACA67AL, longEncode(stringEncode(longEncode(13, -15, 28)))); + assertEquals(0x4C0077776003A9ACL, longEncode(stringEncode(longEncode(-12, 15, 19)))); + assertEquals(0x140000024000000EL, longEncode(stringEncode(longEncode(-328.231870,16.064082, 5)))); + assertEquals(0x6436F96B60000000L, longEncode(stringEncode(longEncode(-590.769588,89.549167, 25)))); + assertEquals(0x6411BD6BA0A98359L, longEncode(stringEncode(longEncode(999.787079,51.830093, 25)))); + assertEquals(0x751BD6BBCA983596L, longEncode(stringEncode(longEncode(999.787079,51.830093, 29)))); + assertEquals(0x77CF880A20000000L, longEncode(stringEncode(longEncode(-557.039740,-632.103969, 29)))); + assertEquals(0x7624FA4FA0000000L, longEncode(stringEncode(longEncode(13,88, 29)))); + assertEquals(0x7624FA4FBFFFFFFFL, longEncode(stringEncode(longEncode(13,-88, 29)))); + assertEquals(0x0400000020000000L, longEncode(stringEncode(longEncode(13,89, 1)))); + assertEquals(0x0400000020000001L, longEncode(stringEncode(longEncode(13,-89, 1)))); + assertEquals(0x0400000020000000L, longEncode(stringEncode(longEncode(13,95, 1)))); + assertEquals(0x0400000020000001L, longEncode(stringEncode(longEncode(13,-95, 1)))); + + expectThrows(IllegalArgumentException.class, () -> longEncode("12/asdf/1")); + expectThrows(IllegalArgumentException.class, () -> longEncode("foo")); + } + private void assertGeoPointEquals(GeoPoint gp, final double longitude, final double latitude) { assertThat(gp.lon(), closeTo(longitude, GEOTILE_TOLERANCE)); assertThat(gp.lat(), closeTo(latitude, GEOTILE_TOLERANCE)); From d999942c6dfd931266d01db24d3fb26b29cf8f64 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Thu, 5 Sep 2019 11:51:59 -0400 Subject: [PATCH 069/103] Refactor AllocatedPersistentTask#init(), move rollup logic out of ctor (#46288) This makes the AllocatedPersistentTask#init() method protected so that implementing classes can perform their initialization logic there, instead of the constructor. Rollup's task is adjusted to use this init method. It also slightly refactors the methods to se a static logger in the AllocatedTask instead of passing it in via an argument. This is simpler, logged messages come from the task instead of the service, and is easier for tests --- .../persistent/AllocatedPersistentTask.java | 8 +- .../PersistentTasksNodeService.java | 2 +- .../xpack/rollup/job/RollupJobTask.java | 49 ++++++++---- .../xpack/rollup/job/RollupJobTaskTests.java | 75 ++++++++++++++----- 4 files changed, 94 insertions(+), 40 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java b/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java index 54dcffab6e36..15fe23d58c7e 100644 --- a/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java +++ b/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.persistent; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; @@ -37,13 +38,13 @@ import java.util.function.Predicate; */ public class AllocatedPersistentTask extends CancellableTask { + private static final Logger logger = LogManager.getLogger(AllocatedPersistentTask.class); private final AtomicReference state; private volatile String persistentTaskId; private volatile long allocationId; private volatile @Nullable Exception failure; private volatile PersistentTasksService persistentTasksService; - private volatile Logger logger; private volatile TaskManager taskManager; public AllocatedPersistentTask(long id, String type, String action, String description, TaskId parentTask, @@ -85,10 +86,9 @@ public class AllocatedPersistentTask extends CancellableTask { return persistentTaskId; } - void init(PersistentTasksService persistentTasksService, TaskManager taskManager, Logger logger, String persistentTaskId, long - allocationId) { + protected void init(PersistentTasksService persistentTasksService, TaskManager taskManager, + String persistentTaskId, long allocationId) { this.persistentTasksService = persistentTasksService; - this.logger = logger; this.taskManager = taskManager; this.persistentTaskId = persistentTaskId; this.allocationId = allocationId; diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java index 17803bdb1958..02b326445966 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java @@ -183,7 +183,7 @@ public class PersistentTasksNodeService implements ClusterStateListener { boolean processed = false; try { - task.init(persistentTasksService, taskManager, logger, taskInProgress.getId(), taskInProgress.getAllocationId()); + task.init(persistentTasksService, taskManager, taskInProgress.getId(), taskInProgress.getAllocationId()); logger.trace("Persistent task [{}] with id [{}] and allocation id [{}] was created", task.getAction(), task.getPersistentTaskId(), task.getAllocationId()); try { diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index cd7ad4fdf07f..bc20b8d5549f 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -22,7 +22,9 @@ import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.persistent.PersistentTasksExecutor; +import org.elasticsearch.persistent.PersistentTasksService; import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.indexing.IndexerState; @@ -150,7 +152,10 @@ public class RollupJobTask extends AllocatedPersistentTask implements SchedulerE private final RollupJob job; private final SchedulerEngine schedulerEngine; private final ThreadPool threadPool; - private final RollupIndexer indexer; + private final Client client; + private final IndexerState initialIndexerState; + private final Map initialPosition; + private RollupIndexer indexer; RollupJobTask(long id, String type, String action, TaskId parentTask, RollupJob job, RollupJobStatus state, Client client, SchedulerEngine schedulerEngine, ThreadPool threadPool, Map headers) { @@ -158,36 +163,48 @@ public class RollupJobTask extends AllocatedPersistentTask implements SchedulerE this.job = job; this.schedulerEngine = schedulerEngine; this.threadPool = threadPool; + this.client = client; + if (state == null) { + this.initialIndexerState = null; + this.initialPosition = null; + } else { + this.initialIndexerState = state.getIndexerState(); + this.initialPosition = state.getPosition(); + } - // If status is not null, we are resuming rather than starting fresh. - Map initialPosition = null; - IndexerState initialState = IndexerState.STOPPED; - if (state != null) { - final IndexerState existingState = state.getIndexerState(); - logger.debug("We have existing state, setting state to [" + existingState + "] " + - "and current position to [" + state.getPosition() + "] for job [" + job.getConfig().getId() + "]"); - if (existingState.equals(IndexerState.INDEXING)) { + } + + @Override + protected void init(PersistentTasksService persistentTasksService, TaskManager taskManager, + String persistentTaskId, long allocationId) { + super.init(persistentTasksService, taskManager, persistentTaskId, allocationId); + + // If initial position is not null, we are resuming rather than starting fresh. + IndexerState indexerState = IndexerState.STOPPED; + if (initialPosition != null) { + logger.debug("We have existing state, setting state to [" + initialIndexerState + "] " + + "and current position to [" + initialPosition + "] for job [" + job.getConfig().getId() + "]"); + if (initialIndexerState.equals(IndexerState.INDEXING)) { /* * If we were indexing, we have to reset back to STARTED otherwise the indexer will be "stuck" thinking * it is indexing but without the actual indexing thread running. */ - initialState = IndexerState.STARTED; + indexerState = IndexerState.STARTED; - } else if (existingState.equals(IndexerState.ABORTING) || existingState.equals(IndexerState.STOPPING)) { + } else if (initialIndexerState.equals(IndexerState.ABORTING) || initialIndexerState.equals(IndexerState.STOPPING)) { // It shouldn't be possible to persist ABORTING, but if for some reason it does, // play it safe and restore the job as STOPPED. An admin will have to clean it up, // but it won't be running, and won't delete itself either. Safest option. // If we were STOPPING, that means it persisted but was killed before finally stopped... so ok // to restore as STOPPED - initialState = IndexerState.STOPPED; + indexerState = IndexerState.STOPPED; } else { - initialState = existingState; + indexerState = initialIndexerState; } - initialPosition = state.getPosition(); } - this.indexer = new ClientRollupPageManager(job, initialState, initialPosition, - new ParentTaskAssigningClient(client, new TaskId(getPersistentTaskId()))); + this.indexer = new ClientRollupPageManager(job, indexerState, initialPosition, + new ParentTaskAssigningClient(client, getParentTaskId())); } @Override diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java index a32cd975b641..6f24257ffe0b 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -67,8 +68,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -80,8 +83,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -93,8 +98,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -106,8 +113,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -119,8 +128,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -132,8 +143,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -144,8 +157,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, null, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); } @@ -156,8 +171,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -186,8 +203,9 @@ public class RollupJobTaskTests extends ESTestCase { SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); AtomicInteger counter = new AtomicInteger(0); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, - null, client, schedulerEngine, pool, Collections.emptyMap()) { + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, ActionListener> listener) { @@ -205,6 +223,7 @@ public class RollupJobTaskTests extends ESTestCase { counter.incrementAndGet(); } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -263,7 +282,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -274,6 +294,7 @@ public class RollupJobTaskTests extends ESTestCase { new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -301,7 +322,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -312,6 +334,7 @@ public class RollupJobTaskTests extends ESTestCase { new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -342,7 +365,8 @@ public class RollupJobTaskTests extends ESTestCase { when(client.settings()).thenReturn(Settings.EMPTY); when(client.threadPool()).thenReturn(pool); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -353,6 +377,7 @@ public class RollupJobTaskTests extends ESTestCase { new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -410,7 +435,8 @@ public class RollupJobTaskTests extends ESTestCase { }).when(client).execute(anyObject(), anyObject(), anyObject()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -427,6 +453,7 @@ public class RollupJobTaskTests extends ESTestCase { } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -494,7 +521,8 @@ public class RollupJobTaskTests extends ESTestCase { }).when(client).execute(anyObject(), anyObject(), anyObject()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -511,6 +539,7 @@ public class RollupJobTaskTests extends ESTestCase { } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -579,7 +608,8 @@ public class RollupJobTaskTests extends ESTestCase { SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -596,6 +626,7 @@ public class RollupJobTaskTests extends ESTestCase { } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -630,8 +661,10 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()); + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); CountDownLatch latch = new CountDownLatch(1); @@ -658,7 +691,8 @@ public class RollupJobTaskTests extends ESTestCase { SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); AtomicInteger counter = new AtomicInteger(0); - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -680,6 +714,7 @@ public class RollupJobTaskTests extends ESTestCase { } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -744,13 +779,15 @@ public class RollupJobTaskTests extends ESTestCase { // the task would end before stop could be called. But to help test out all pathways, // just in case, we can override markAsCompleted so it's a no-op and test how stop // handles the situation - RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + TaskId taskId = new TaskId("node", 123); + RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void markAsCompleted() { latch.countDown(); } }; + task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); task.onCancelled(); From 794199536597c3f256b9f83a158a667aec8fcee0 Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Thu, 5 Sep 2019 08:54:56 -0700 Subject: [PATCH 070/103] [DOCS] Update snippets in security APIs (#46191) --- .../rest-api/security/authenticate.asciidoc | 5 ++- .../security/change-password.asciidoc | 6 ++-- .../en/rest-api/security/clear-cache.asciidoc | 9 ++---- .../security/clear-roles-cache.asciidoc | 3 +- .../security/create-api-keys.asciidoc | 5 ++- .../security/create-role-mappings.asciidoc | 30 ++++++----------- .../rest-api/security/create-roles.asciidoc | 6 ++-- .../rest-api/security/create-users.asciidoc | 6 ++-- .../delegate-pki-authentication.asciidoc | 5 ++- .../security/delete-app-privileges.asciidoc | 7 ++-- .../security/delete-role-mappings.asciidoc | 6 ++-- .../rest-api/security/delete-roles.asciidoc | 6 ++-- .../rest-api/security/delete-users.asciidoc | 6 ++-- .../rest-api/security/disable-users.asciidoc | 3 +- .../rest-api/security/enable-users.asciidoc | 3 +- .../rest-api/security/get-api-keys.asciidoc | 30 ++++++----------- .../security/get-app-privileges.asciidoc | 12 +++---- .../security/get-builtin-privileges.asciidoc | 7 ++-- .../security/get-role-mappings.asciidoc | 6 ++-- .../en/rest-api/security/get-roles.asciidoc | 9 ++---- .../en/rest-api/security/get-tokens.asciidoc | 15 ++++----- .../en/rest-api/security/get-users.asciidoc | 10 ++---- .../rest-api/security/has-privileges.asciidoc | 5 ++- .../security/invalidate-api-keys.asciidoc | 32 ++++++------------- .../security/invalidate-tokens.asciidoc | 32 ++++++------------- .../security/oidc-authenticate-api.asciidoc | 3 +- .../security/oidc-logout-api.asciidoc | 3 +- .../oidc-prepare-authentication-api.asciidoc | 16 ++++------ .../security/put-app-privileges.asciidoc | 12 +++---- x-pack/docs/en/rest-api/security/ssl.asciidoc | 4 +-- 30 files changed, 99 insertions(+), 203 deletions(-) diff --git a/x-pack/docs/en/rest-api/security/authenticate.asciidoc b/x-pack/docs/en/rest-api/security/authenticate.asciidoc index f5f6e2d33a6c..fc12e1dfb0f0 100644 --- a/x-pack/docs/en/rest-api/security/authenticate.asciidoc +++ b/x-pack/docs/en/rest-api/security/authenticate.asciidoc @@ -31,15 +31,14 @@ If the user cannot be authenticated, this API returns a 401 status code. To authenticate a user, submit a GET request to the `/_security/_authenticate` endpoint: -[source,js] +[source,console] -------------------------------------------------- GET /_security/_authenticate -------------------------------------------------- -// CONSOLE The following example output provides information about the "rdeniro" user: -[source,js] +[source,console-result] -------------------------------------------------- { "username": "rdeniro", diff --git a/x-pack/docs/en/rest-api/security/change-password.asciidoc b/x-pack/docs/en/rest-api/security/change-password.asciidoc index 07947088048b..ed696d668bc9 100644 --- a/x-pack/docs/en/rest-api/security/change-password.asciidoc +++ b/x-pack/docs/en/rest-api/security/change-password.asciidoc @@ -51,20 +51,18 @@ For more information about the native realm, see The following example updates the password for the `jacknich` user: -[source,js] +[source,console] -------------------------------------------------- POST /_security/user/jacknich/_password { "password" : "s3cr3t" } -------------------------------------------------- -// CONSOLE // TEST[setup:jacknich_user] A successful call returns an empty JSON structure. -[source,js] +[source,console-result] -------------------------------------------------- {} -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/clear-cache.asciidoc b/x-pack/docs/en/rest-api/security/clear-cache.asciidoc index 2b2ed8f91686..a214163ed179 100644 --- a/x-pack/docs/en/rest-api/security/clear-cache.asciidoc +++ b/x-pack/docs/en/rest-api/security/clear-cache.asciidoc @@ -44,25 +44,22 @@ To evict roles from the role cache, see the For example, to evict all users cached by the `file` realm: -[source,js] +[source,console] -------------------------------------------------- POST /_security/realm/default_file/_clear_cache -------------------------------------------------- -// CONSOLE To evict selected users, specify the `usernames` parameter: -[source,js] +[source,console] -------------------------------------------------- POST /_security/realm/default_file/_clear_cache?usernames=rdeniro,alpacino -------------------------------------------------- -// CONSOLE To clear the caches for multiple realms, specify the realms as a comma-delimited list: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/realm/default_file,ldap1/_clear_cache ------------------------------------------------------------ -// CONSOLE diff --git a/x-pack/docs/en/rest-api/security/clear-roles-cache.asciidoc b/x-pack/docs/en/rest-api/security/clear-roles-cache.asciidoc index eb8a721c835b..8fc8f0b88390 100644 --- a/x-pack/docs/en/rest-api/security/clear-roles-cache.asciidoc +++ b/x-pack/docs/en/rest-api/security/clear-roles-cache.asciidoc @@ -36,8 +36,7 @@ For more information about the native realm, see The clear roles cache API evicts roles from the native role cache. For example, to clear the cache for `my_admin_role`: -[source,js] +[source,console] -------------------------------------------------- POST /_security/role/my_admin_role/_clear_cache -------------------------------------------------- -// CONSOLE diff --git a/x-pack/docs/en/rest-api/security/create-api-keys.asciidoc b/x-pack/docs/en/rest-api/security/create-api-keys.asciidoc index 73e2061a90a1..42541d0f3633 100644 --- a/x-pack/docs/en/rest-api/security/create-api-keys.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-api-keys.asciidoc @@ -65,7 +65,7 @@ expire. The following example creates an API key: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/api_key { @@ -93,7 +93,6 @@ POST /_security/api_key } } ------------------------------------------------------------ -// CONSOLE <1> optional expiration for the API key being generated. If expiration is not provided then the API keys do not expire. <2> optional role descriptors for this API key, if not provided then permissions @@ -102,7 +101,7 @@ POST /_security/api_key A successful call returns a JSON structure that provides API key information. -[source,js] +[source,console-result] -------------------------------------------------- { "id":"VuaCfGcBCdbkQm-e5aOx", <1> diff --git a/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc b/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc index f2a43427d9e9..de727a70b71e 100644 --- a/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc @@ -105,7 +105,7 @@ The Examples section below demonstrates the use of templated role names. The following example assigns the "user" role to all users: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping1 { @@ -119,7 +119,6 @@ POST /_security/role_mapping/mapping1 } } ------------------------------------------------------------ -// CONSOLE <1> Mappings that have `enabled` set to `false` are ignored when role mapping is performed. <2> Metadata is optional. @@ -127,7 +126,7 @@ POST /_security/role_mapping/mapping1 A successful call returns a JSON structure that shows whether the mapping has been created or updated. -[source,js] +[source,console-result] -------------------------------------------------- { "role_mapping" : { @@ -135,12 +134,11 @@ been created or updated. } } -------------------------------------------------- -// TESTRESPONSE <1> When an existing mapping is updated, `created` is set to false. The following example assigns the "user" and "admin" roles to specific users: -[source,js] +[source,console] -------------------------------------------------- POST /_security/role_mapping/mapping2 { @@ -151,10 +149,9 @@ POST /_security/role_mapping/mapping2 } } -------------------------------------------------- -// CONSOLE The following example matches users who authenticated against a specific realm: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping3 { @@ -165,12 +162,11 @@ POST /_security/role_mapping/mapping3 } } ------------------------------------------------------------ -// CONSOLE The following example matches any user where either the username is `esadmin` or the user is in the `cn=admin,dc=example,dc=com` group: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping4 { @@ -192,7 +188,6 @@ POST /_security/role_mapping/mapping4 } } ------------------------------------------------------------ -// CONSOLE The example above is useful when the group names in your identity management system (such as Active Directory, or a SAML Identity Provider) do not have a @@ -213,7 +208,7 @@ roles is inefficient and can have a negative effect on system performance. If you only need to map a subset of the groups, then you should do this using explicit mappings. -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping5 { @@ -229,7 +224,6 @@ POST /_security/role_mapping/mapping5 "enabled": true } ------------------------------------------------------------ -// CONSOLE <1> The `tojson` mustache function is used to convert the list of group names into a valid JSON array. <2> Because the template produces a JSON array, the format must be @@ -237,7 +231,7 @@ POST /_security/role_mapping/mapping5 The following example matches users within a specific LDAP sub-tree: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping6 { @@ -248,12 +242,11 @@ POST /_security/role_mapping/mapping6 } } ------------------------------------------------------------ -// CONSOLE The following example matches users within a particular LDAP sub-tree in a specific realm: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping7 { @@ -267,7 +260,6 @@ POST /_security/role_mapping/mapping7 } } ------------------------------------------------------------ -// CONSOLE The rules can be more complex and include wildcard matching. For example, the following mapping matches any user where *all* of these conditions are met: @@ -278,7 +270,7 @@ following mapping matches any user where *all* of these conditions are met: - the user does not have a `terminated_date` -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping8 { @@ -316,7 +308,6 @@ POST /_security/role_mapping/mapping8 } } ------------------------------------------------------------ -// CONSOLE A templated role can be used to automatically map every user to their own custom role. The role itself can be defined through the @@ -329,7 +320,7 @@ role that is their username prefixed with `_user_`. As an example, the user `nwong` would be assigned the `saml_user` and `_user_nwong` roles. -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/role_mapping/mapping9 { @@ -341,7 +332,6 @@ POST /_security/role_mapping/mapping9 "enabled": true } ------------------------------------------------------------ -// CONSOLE <1> Because it is not possible to specify both `roles` and `role_templates` in the same role mapping, we can apply a "fixed name" role by using a template that has no substitutions. diff --git a/x-pack/docs/en/rest-api/security/create-roles.asciidoc b/x-pack/docs/en/rest-api/security/create-roles.asciidoc index 4976675c698c..85f549d635f7 100644 --- a/x-pack/docs/en/rest-api/security/create-roles.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-roles.asciidoc @@ -82,7 +82,7 @@ For more information, see {stack-ov}/defining-roles.html[Defining roles]. The following example adds a role called `my_admin_role`: -[source,js] +[source,console] -------------------------------------------------- POST /_security/role/my_admin_role { @@ -110,12 +110,11 @@ POST /_security/role/my_admin_role } } -------------------------------------------------- -// CONSOLE A successful call returns a JSON structure that shows whether the role has been created or updated. -[source,js] +[source,console-result] -------------------------------------------------- { "role": { @@ -123,5 +122,4 @@ created or updated. } } -------------------------------------------------- -// TESTRESPONSE <1> When an existing role is updated, `created` is set to false. diff --git a/x-pack/docs/en/rest-api/security/create-users.asciidoc b/x-pack/docs/en/rest-api/security/create-users.asciidoc index 0135f230912b..e331c8724770 100644 --- a/x-pack/docs/en/rest-api/security/create-users.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-users.asciidoc @@ -104,7 +104,7 @@ access permissions. To create a user without any roles, specify an empty list: The following example creates a user `jacknich`: -[source,js] +[source,console] -------------------------------------------------- POST /_security/user/jacknich { @@ -117,18 +117,16 @@ POST /_security/user/jacknich } } -------------------------------------------------- -// CONSOLE A successful call returns a JSON structure that shows whether the user has been created or updated. -[source,js] +[source,console-result] -------------------------------------------------- { "created": true <1> } -------------------------------------------------- -// TESTRESPONSE <1> When an existing user is updated, `created` is set to false. After you add a user, requests from that user can be authenticated. For example: diff --git a/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc b/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc index b8d20b23953b..d1bec9ae8dcf 100644 --- a/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc +++ b/x-pack/docs/en/rest-api/security/delegate-pki-authentication.asciidoc @@ -73,19 +73,18 @@ client's certificate. The following is an example request: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/delegate_pki { "x509_certificate_chain": ["MIIDbTCCAlWgAwIBAgIJAIxTS7Qdho9jMA0GCSqGSIb3DQEBCwUAMFMxKzApBgNVBAMTIkVsYXN0aWNzZWFyY2ggVGVzdCBJbnRlcm1lZGlhdGUgQ0ExFjAUBgNVBAsTDUVsYXN0aWNzZWFyY2gxDDAKBgNVBAoTA29yZzAeFw0xOTA3MTkxMzMzNDFaFw0yMzA3MTgxMzMzNDFaMEoxIjAgBgNVBAMTGUVsYXN0aWNzZWFyY2ggVGVzdCBDbGllbnQxFjAUBgNVBAsTDUVsYXN0aWNzZWFyY2gxDDAKBgNVBAoTA29yZzCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBANHgMX2aX8t0nj4sGLNuKISmmXIYCj9RwRqS7L03l9Nng7kOKnhHu/nXDt7zMRJyHj+q6FAt5khlavYSVCQyrDybRuA5z31gOdqXerrjs2OXS5HSHNvoDAnHFsaYX/5geMewVTtc/vqpd7Ph/QtaKfmG2FK0JNQo0k24tcgCIcyMtBh6BA70yGBM0OT8GdOgd/d/mA7mRhaxIUMNYQzRYRsp4hMnnWoOTkR5Q8KSO3MKw9dPSpPe8EnwtJE10S3s5aXmgytru/xQqrFycPBNj4KbKVmqMP0G60CzXik5pr2LNvOFz3Qb6sYJtqeZF+JKgGWdaTC89m63+TEnUHqk0lcCAwEAAaNNMEswCQYDVR0TBAIwADAdBgNVHQ4EFgQU/+aAD6Q4mFq1vpHorC25/OY5zjcwHwYDVR0jBBgwFoAU8siFCiMiYZZm/95qFC75AG/LRE0wDQYJKoZIhvcNAQELBQADggEBAIRpCgDLpvXcgDHUk10uhxev21mlIbU+VP46ANnCuj0UELhTrdTuWvO1PAI4z+WbDUxryQfOOXO9R6D0dE5yR56L/J7d+KayW34zU7yRDZM7+rXpocdQ1Ex8mjP9HJ/Bf56YZTBQJpXeDrKow4FvtkI3bcIMkqmbG16LHQXeG3RS4ds4S4wCnE2nA6vIn9y+4R999q6y1VSBORrYULcDWxS54plHLEdiMr1vVallg82AGobS9GMcTL2U4Nx5IYZG7sbTk3LrDxVpVg/S2wLofEdOEwqCeHug/iOihNLJBabEW6z4TDLJAVW5KCY1DfhkYlBfHn7vxKkfKoCUK/yLWWI="] <1> } ------------------------------------------------------------ -// CONSOLE <1> A one element certificate chain. Which returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==", diff --git a/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc b/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc index 9d54739fe946..7f7d06289c12 100644 --- a/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc @@ -38,17 +38,16 @@ To use this API, you must have either: The following example deletes the `read` application privilege from the `myapp` application: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/privilege/myapp/read -------------------------------------------------- -// CONSOLE // TEST[setup:app0102_privileges] If the role is successfully deleted, the request returns `{"found": true}`. Otherwise, `found` is set to false. -[source,js] +[source,console-result] -------------------------------------------------- { "myapp": { @@ -58,5 +57,3 @@ Otherwise, `found` is set to false. } } -------------------------------------------------- -// TESTRESPONSE - diff --git a/x-pack/docs/en/rest-api/security/delete-role-mappings.asciidoc b/x-pack/docs/en/rest-api/security/delete-role-mappings.asciidoc index 8704a83c64de..9fe2d6013cd3 100644 --- a/x-pack/docs/en/rest-api/security/delete-role-mappings.asciidoc +++ b/x-pack/docs/en/rest-api/security/delete-role-mappings.asciidoc @@ -37,20 +37,18 @@ see {stack-ov}/mapping-roles.html[Mapping users and groups to roles]. The following example delete a role mapping: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/role_mapping/mapping1 -------------------------------------------------- -// CONSOLE // TEST[setup:role_mapping] If the mapping is successfully deleted, the request returns `{"found": true}`. Otherwise, `found` is set to false. -[source,js] +[source,console-result] -------------------------------------------------- { "found" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/delete-roles.asciidoc b/x-pack/docs/en/rest-api/security/delete-roles.asciidoc index 2c060cb08224..dec674b65776 100644 --- a/x-pack/docs/en/rest-api/security/delete-roles.asciidoc +++ b/x-pack/docs/en/rest-api/security/delete-roles.asciidoc @@ -38,21 +38,19 @@ file-based role management. For more information about the native realm, see The following example deletes a `my_admin_role` role: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/role/my_admin_role -------------------------------------------------- -// CONSOLE // TEST[setup:admin_role] If the role is successfully deleted, the request returns `{"found": true}`. Otherwise, `found` is set to false. -[source,js] +[source,console-result] -------------------------------------------------- { "found" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/delete-users.asciidoc b/x-pack/docs/en/rest-api/security/delete-users.asciidoc index 0db82d44f053..db48a754f07b 100644 --- a/x-pack/docs/en/rest-api/security/delete-users.asciidoc +++ b/x-pack/docs/en/rest-api/security/delete-users.asciidoc @@ -35,20 +35,18 @@ For more information about the native realm, see The following example deletes the user `jacknich`: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/user/jacknich -------------------------------------------------- -// CONSOLE // TEST[setup:jacknich_user] If the user is successfully deleted, the request returns `{"found": true}`. Otherwise, `found` is set to false. -[source,js] +[source,console-result] -------------------------------------------------- { "found" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/disable-users.asciidoc b/x-pack/docs/en/rest-api/security/disable-users.asciidoc index 9b8bbe9b687c..4f005a0673ff 100644 --- a/x-pack/docs/en/rest-api/security/disable-users.asciidoc +++ b/x-pack/docs/en/rest-api/security/disable-users.asciidoc @@ -41,9 +41,8 @@ For more information about the native realm, see The following example disables the user `jacknich`: -[source,js] +[source,console] -------------------------------------------------- PUT /_security/user/jacknich/_disable -------------------------------------------------- -// CONSOLE // TEST[setup:jacknich_user] diff --git a/x-pack/docs/en/rest-api/security/enable-users.asciidoc b/x-pack/docs/en/rest-api/security/enable-users.asciidoc index b586c11ce74c..34b759aa8d8d 100644 --- a/x-pack/docs/en/rest-api/security/enable-users.asciidoc +++ b/x-pack/docs/en/rest-api/security/enable-users.asciidoc @@ -40,9 +40,8 @@ For more information about the native realm, see The following example enables the user `jacknich`: -[source,js] +[source,console] -------------------------------------------------- PUT /_security/user/jacknich/_enable -------------------------------------------------- -// CONSOLE // TEST[setup:jacknich_user] diff --git a/x-pack/docs/en/rest-api/security/get-api-keys.asciidoc b/x-pack/docs/en/rest-api/security/get-api-keys.asciidoc index 1aa47ebab64c..b65523574468 100644 --- a/x-pack/docs/en/rest-api/security/get-api-keys.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-api-keys.asciidoc @@ -59,7 +59,7 @@ NOTE: At least one of "id", "name", "username" and "realm_name" must be specifie If you create an API key as follows: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/api_key { @@ -67,13 +67,11 @@ POST /_security/api_key "role_descriptors": {} } ------------------------------------------------------------ -// CONSOLE -// TEST A successful call returns a JSON structure that provides API key information. For example: -[source,js] +[source,console-result] -------------------------------------------------- { "id":"VuaCfGcBCdbkQm-e5aOx", @@ -86,80 +84,72 @@ API key information. For example: You can use the following example to retrieve the API key by ID: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?id=VuaCfGcBCdbkQm-e5aOx -------------------------------------------------- -// CONSOLE // TEST[s/VuaCfGcBCdbkQm-e5aOx/$body.id/] // TEST[continued] You can use the following example to retrieve the API key by name: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?name=my-api-key -------------------------------------------------- -// CONSOLE // TEST[continued] The following example retrieves all API keys for the `native1` realm: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?realm_name=native1 -------------------------------------------------- -// CONSOLE // TEST[continued] The following example retrieves all API keys for the user `myuser` in all realms: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?username=myuser -------------------------------------------------- -// CONSOLE // TEST[continued] The following example retrieves all API keys owned by the currently authenticated user: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?owner=true -------------------------------------------------- -// CONSOLE // TEST[continued] Following creates an API key -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/api_key { "name": "my-api-key-1" } ------------------------------------------------------------ -// CONSOLE The following example retrieves the API key identified by the specified `id` if it is owned by the currently authenticated user: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?id=VuaCfGcBCdbkQm-e5aOx&owner=true -------------------------------------------------- -// CONSOLE // TEST[s/VuaCfGcBCdbkQm-e5aOx/$body.id/] // TEST[continued] Finally, the following example retrieves all API keys for the user `myuser` in the `native1` realm immediately: -[source,js] +[source,console] -------------------------------------------------- GET /_security/api_key?username=myuser&realm_name=native1 -------------------------------------------------- -// CONSOLE // TEST[continued] A successful call returns a JSON structure that contains the information of one or more API keys that were retrieved. diff --git a/x-pack/docs/en/rest-api/security/get-app-privileges.asciidoc b/x-pack/docs/en/rest-api/security/get-app-privileges.asciidoc index 774091ba3985..bba361fd62c0 100644 --- a/x-pack/docs/en/rest-api/security/get-app-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-app-privileges.asciidoc @@ -55,17 +55,16 @@ To check a user's application privileges, use the The following example retrieves information about the `read` privilege for the `app01` application: -[source,js] +[source,console] -------------------------------------------------- GET /_security/privilege/myapp/read -------------------------------------------------- -// CONSOLE // TEST[setup:app0102_privileges] A successful call returns an object keyed by application name and privilege name. If the privilege is not defined, the request responds with a 404 status. -[source,js] +[source,console-result] -------------------------------------------------- { "myapp": { @@ -83,20 +82,17 @@ name. If the privilege is not defined, the request responds with a 404 status. } } -------------------------------------------------- -// TESTRESPONSE To retrieve all privileges for an application, omit the privilege name: -[source,js] +[source,console] -------------------------------------------------- GET /_security/privilege/myapp/ -------------------------------------------------- -// CONSOLE To retrieve every privilege, omit both the application and privilege names: -[source,js] +[source,console] -------------------------------------------------- GET /_security/privilege/ -------------------------------------------------- -// CONSOLE diff --git a/x-pack/docs/en/rest-api/security/get-builtin-privileges.asciidoc b/x-pack/docs/en/rest-api/security/get-builtin-privileges.asciidoc index b9390066a0d9..7da8bdce87b3 100644 --- a/x-pack/docs/en/rest-api/security/get-builtin-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-builtin-privileges.asciidoc @@ -51,16 +51,14 @@ The response is an object with two fields: The following example retrieves the names of all builtin privileges: -[source,js] +[source,console] -------------------------------------------------- GET /_security/privilege/_builtin -------------------------------------------------- -// CONSOLE -// TEST A successful call returns an object with "cluster" and "index" fields. -[source,js] +[source,console-result] -------------------------------------------------- { "cluster" : [ @@ -115,4 +113,3 @@ A successful call returns an object with "cluster" and "index" fields. ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/get-role-mappings.asciidoc b/x-pack/docs/en/rest-api/security/get-role-mappings.asciidoc index 8d493bc0ad12..c0a77dd59b81 100644 --- a/x-pack/docs/en/rest-api/security/get-role-mappings.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-role-mappings.asciidoc @@ -55,15 +55,14 @@ response will have status code `404`. The following example retrieves information about the `mapping1` role mapping: -[source,js] +[source,console] -------------------------------------------------- GET /_security/role_mapping/mapping1 -------------------------------------------------- -// CONSOLE // TEST[setup:role_mapping] -[source,js] +[source,console-result] -------------------------------------------------- { "mapping1": { @@ -80,4 +79,3 @@ GET /_security/role_mapping/mapping1 } } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/get-roles.asciidoc b/x-pack/docs/en/rest-api/security/get-roles.asciidoc index 281f4e1726a4..f014166362ee 100644 --- a/x-pack/docs/en/rest-api/security/get-roles.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-roles.asciidoc @@ -51,14 +51,13 @@ If the role is not defined in the native realm, the request returns 404. The following example retrieves information about the `my_admin_role` role in the native realm: -[source,js] +[source,console] -------------------------------------------------- GET /_security/role/my_admin_role -------------------------------------------------- -// CONSOLE // TEST[setup:admin_role] -[source,js] +[source,console-result] -------------------------------------------------- { "my_admin_role": { @@ -83,14 +82,12 @@ GET /_security/role/my_admin_role } } -------------------------------------------------- -// TESTRESPONSE To retrieve all roles, omit the role name: -[source,js] +[source,console] -------------------------------------------------- GET /_security/role -------------------------------------------------- -// CONSOLE // TEST[continued] diff --git a/x-pack/docs/en/rest-api/security/get-tokens.asciidoc b/x-pack/docs/en/rest-api/security/get-tokens.asciidoc index 6d5644fd38a0..44c486038f12 100644 --- a/x-pack/docs/en/rest-api/security/get-tokens.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-tokens.asciidoc @@ -98,19 +98,18 @@ other supported grant type. The following example obtains a token using the `client_credentials` grant type, which simply creates a token as the authenticated user: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oauth2/token { "grant_type" : "client_credentials" } -------------------------------------------------- -// CONSOLE The following example output contains the access token, the amount of time (in seconds) that the token expires in, and the type: -[source,js] +[source,console-result] -------------------------------------------------- { "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==", @@ -134,7 +133,7 @@ The following example obtains a token for the `test_admin` user using the `password` grant type. This request needs to be made by an authenticated user with sufficient privileges that may or may not be the same as the one whose username is passed in the `username` parameter: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oauth2/token { @@ -143,12 +142,11 @@ POST /_security/oauth2/token "password" : "x-pack-test-password" } -------------------------------------------------- -// CONSOLE The following example output contains the access token, the amount of time (in seconds) that the token expires in, the type, and the refresh token: -[source,js] +[source,console-result] -------------------------------------------------- { "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==", @@ -165,7 +163,7 @@ To extend the life of an existing token obtained using the `password` grant type you can call the API again with the refresh token within 24 hours of the token's creation. For example: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oauth2/token { @@ -173,14 +171,13 @@ POST /_security/oauth2/token "refresh_token": "vLBPvmAB6KvwvJZr27cS" } -------------------------------------------------- -// CONSOLE // TEST[s/vLBPvmAB6KvwvJZr27cS/$body.refresh_token/] // TEST[continued] The API will return a new token and refresh token. Each refresh token may only be used one time. -[source,js] +[source,console-result] -------------------------------------------------- { "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==", diff --git a/x-pack/docs/en/rest-api/security/get-users.asciidoc b/x-pack/docs/en/rest-api/security/get-users.asciidoc index 877035de203c..63ae8bc140da 100644 --- a/x-pack/docs/en/rest-api/security/get-users.asciidoc +++ b/x-pack/docs/en/rest-api/security/get-users.asciidoc @@ -52,14 +52,13 @@ If the user is not defined in the `native` realm, the request 404s. To retrieve a native user, submit a GET request to the `/_security/user/` endpoint: -[source,js] +[source,console] -------------------------------------------------- GET /_security/user/jacknich -------------------------------------------------- -// CONSOLE // TEST[setup:jacknich_user] -[source,js] +[source,console-result] -------------------------------------------------- { "jacknich": { @@ -74,14 +73,11 @@ GET /_security/user/jacknich } } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE Omit the username to retrieve all users: -[source,js] +[source,console] -------------------------------------------------- GET /_security/user -------------------------------------------------- -// CONSOLE // TEST[continued] diff --git a/x-pack/docs/en/rest-api/security/has-privileges.asciidoc b/x-pack/docs/en/rest-api/security/has-privileges.asciidoc index f211c8df6331..cce21965d6e3 100644 --- a/x-pack/docs/en/rest-api/security/has-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/has-privileges.asciidoc @@ -64,7 +64,7 @@ should be checked The following example checks whether the current user has a specific set of cluster, index, and application privileges: -[source,js] +[source,console] -------------------------------------------------- GET /_security/user/_has_privileges { @@ -88,11 +88,10 @@ GET /_security/user/_has_privileges ] } -------------------------------------------------- -// CONSOLE The following example output indicates which privileges the "rdeniro" user has: -[source,js] +[source,console-result] -------------------------------------------------- { "username": "rdeniro", diff --git a/x-pack/docs/en/rest-api/security/invalidate-api-keys.asciidoc b/x-pack/docs/en/rest-api/security/invalidate-api-keys.asciidoc index ecd79a0906cf..a9453cb53577 100644 --- a/x-pack/docs/en/rest-api/security/invalidate-api-keys.asciidoc +++ b/x-pack/docs/en/rest-api/security/invalidate-api-keys.asciidoc @@ -68,20 +68,18 @@ keys. If you create an API key as follows: -[source, js] +[source,console] ------------------------------------------------------------ POST /_security/api_key { "name": "my-api-key" } ------------------------------------------------------------ -// CONSOLE -// TEST A successful call returns a JSON structure that provides API key information. For example: -[source,js] +[source,console-result] -------------------------------------------------- { "id":"VuaCfGcBCdbkQm-e5aOx", @@ -95,60 +93,53 @@ API key information. For example: The following example invalidates the API key identified by specified `id` immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { "id" : "VuaCfGcBCdbkQm-e5aOx" } -------------------------------------------------- -// CONSOLE // TEST[s/VuaCfGcBCdbkQm-e5aOx/$body.id/] // TEST[continued] The following example invalidates the API key identified by specified `name` immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { "name" : "my-api-key" } -------------------------------------------------- -// CONSOLE -// TEST The following example invalidates all API keys for the `native1` realm immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { "realm_name" : "native1" } -------------------------------------------------- -// CONSOLE -// TEST The following example invalidates all API keys for the user `myuser` in all realms immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { "username" : "myuser" } -------------------------------------------------- -// CONSOLE -// TEST The following example invalidates the API key identified by the specified `id` if it is owned by the currently authenticated user immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { @@ -156,25 +147,22 @@ DELETE /_security/api_key "owner" : "true" } -------------------------------------------------- -// CONSOLE The following example invalidates all API keys owned by the currently authenticated user immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { "owner" : "true" } -------------------------------------------------- -// CONSOLE -// TEST Finally, the following example invalidates all API keys for the user `myuser` in the `native1` realm immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/api_key { @@ -182,8 +170,6 @@ DELETE /_security/api_key "realm_name" : "native1" } -------------------------------------------------- -// CONSOLE -// TEST [source,js] -------------------------------------------------- diff --git a/x-pack/docs/en/rest-api/security/invalidate-tokens.asciidoc b/x-pack/docs/en/rest-api/security/invalidate-tokens.asciidoc index 91f9b1e1bfd8..d4da53ebbe5c 100644 --- a/x-pack/docs/en/rest-api/security/invalidate-tokens.asciidoc +++ b/x-pack/docs/en/rest-api/security/invalidate-tokens.asciidoc @@ -68,19 +68,17 @@ and potentially a list of errors encountered while invalidating specific tokens. For example, if you create a token using the `client_credentials` grant type as follows: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oauth2/token { "grant_type" : "client_credentials" } -------------------------------------------------- -// CONSOLE -// TEST The get token API returns the following information about the access token: -[source,js] +[source,console-result] -------------------------------------------------- { "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==", @@ -88,27 +86,25 @@ The get token API returns the following information about the access token: "expires_in" : 1200 } -------------------------------------------------- -// CONSOLE // TESTRESPONSE[s/dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==/$body.access_token/] This access token can now be immediately invalidated, as shown in the following example: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/oauth2/token { "token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==" } -------------------------------------------------- -// CONSOLE // TEST[s/dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==/$body.access_token/] // TEST[continued] If you used the `password` grant type to obtain a token for a user, the response might also contain a refresh token. For example: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oauth2/token { @@ -117,12 +113,10 @@ POST /_security/oauth2/token "password" : "x-pack-test-password" } -------------------------------------------------- -// CONSOLE -// TEST The get token API returns the following information: -[source,js] +[source,console-result] -------------------------------------------------- { "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==", @@ -131,54 +125,48 @@ The get token API returns the following information: "refresh_token": "vLBPvmAB6KvwvJZr27cS" } -------------------------------------------------- -// CONSOLE // TESTRESPONSE[s/dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==/$body.access_token/] // TESTRESPONSE[s/vLBPvmAB6KvwvJZr27cS/$body.refresh_token/] The refresh token can now also be immediately invalidated as shown in the following example: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/oauth2/token { "refresh_token" : "vLBPvmAB6KvwvJZr27cS" } -------------------------------------------------- -// CONSOLE // TEST[s/vLBPvmAB6KvwvJZr27cS/$body.refresh_token/] // TEST[continued] The following example invalidates all access tokens and refresh tokens for the `saml1` realm immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/oauth2/token { "realm_name" : "saml1" } -------------------------------------------------- -// CONSOLE -// TEST The following example invalidates all access tokens and refresh tokens for the user `myuser` in all realms immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/oauth2/token { "username" : "myuser" } -------------------------------------------------- -// CONSOLE -// TEST Finally, the following example invalidates all access tokens and refresh tokens for the user `myuser` in the `saml1` realm immediately: -[source,js] +[source,console] -------------------------------------------------- DELETE /_security/oauth2/token { @@ -186,8 +174,6 @@ DELETE /_security/oauth2/token "realm_name" : "saml1" } -------------------------------------------------- -// CONSOLE -// TEST [source,js] -------------------------------------------------- diff --git a/x-pack/docs/en/rest-api/security/oidc-authenticate-api.asciidoc b/x-pack/docs/en/rest-api/security/oidc-authenticate-api.asciidoc index 67e6d68a109e..0814257d3aef 100644 --- a/x-pack/docs/en/rest-api/security/oidc-authenticate-api.asciidoc +++ b/x-pack/docs/en/rest-api/security/oidc-authenticate-api.asciidoc @@ -61,7 +61,7 @@ OpenID Connect Provider after a successful authentication, for an {es} access token and refresh token to be used in subsequent requests. This example is from an authentication that uses the authorization code grant flow. -[source,js] +[source,console] -------------------------------------------------- POST /_security/oidc/authenticate { @@ -71,7 +71,6 @@ POST /_security/oidc/authenticate "realm" : "oidc1" } -------------------------------------------------- -// CONSOLE // TEST[catch:unauthorized] The following example output contains the access token that was generated in diff --git a/x-pack/docs/en/rest-api/security/oidc-logout-api.asciidoc b/x-pack/docs/en/rest-api/security/oidc-logout-api.asciidoc index 47d7125b94d9..0b6ef7030262 100644 --- a/x-pack/docs/en/rest-api/security/oidc-logout-api.asciidoc +++ b/x-pack/docs/en/rest-api/security/oidc-logout-api.asciidoc @@ -40,7 +40,7 @@ and The following example performs logout -[source,js] +[source,console] -------------------------------------------------- POST /_security/oidc/logout { @@ -48,7 +48,6 @@ POST /_security/oidc/logout "refresh_token": "vLBPvmAB6KvwvJZr27cS" } -------------------------------------------------- -// CONSOLE // TEST[catch:unauthorized] The following example output of the response contains the URI pointing to the diff --git a/x-pack/docs/en/rest-api/security/oidc-prepare-authentication-api.asciidoc b/x-pack/docs/en/rest-api/security/oidc-prepare-authentication-api.asciidoc index daf1854e81f8..424ed51742f8 100644 --- a/x-pack/docs/en/rest-api/security/oidc-prepare-authentication-api.asciidoc +++ b/x-pack/docs/en/rest-api/security/oidc-prepare-authentication-api.asciidoc @@ -65,19 +65,18 @@ parameter is not valid when `realm` is specified The following example generates an authentication request for the OpenID Connect Realm `oidc1`: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oidc/prepare { "realm" : "oidc1" } -------------------------------------------------- -// CONSOLE The following example output of the response contains the URI pointing to the Authorization Endpoint of the OpenID Connect Provider with all the parameters of the Authentication Request, as HTTP GET parameters: -[source,js] +[source,console-result] -------------------------------------------------- { "redirect" : "http://127.0.0.1:8080/c2id-login?scope=openid&response_type=id_token&redirect_uri=https%3A%2F%2Fmy.fantastic.rp%2Fcb&state=4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I&nonce=WaBPH0KqPVdG5HHdSxPRjfoZbXMCicm5v1OiAj0DUFM&client_id=elasticsearch-rp", @@ -92,7 +91,7 @@ The following example generates an authentication request for the OpenID Connect Realm `oidc1`, where the values for the state and the nonce have been generated by the client: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oidc/prepare { @@ -101,12 +100,11 @@ POST /_security/oidc/prepare "nonce" : "zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5" } -------------------------------------------------- -// CONSOLE The following example output of the response contains the URI pointing to the Authorization Endpoint of the OpenID Connect Provider with all the parameters of the Authentication Request, as HTTP GET parameters: -[source,js] +[source,console-result] -------------------------------------------------- { "redirect" : "http://127.0.0.1:8080/c2id-login?scope=openid&response_type=id_token&redirect_uri=https%3A%2F%2Fmy.fantastic.rp%2Fcb&state=lGYK0EcSLjqH6pkT5EVZjC6eIW5YCGgywj2sxROO&nonce=zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5&client_id=elasticsearch-rp", @@ -114,13 +112,12 @@ the Authentication Request, as HTTP GET parameters: "nonce" : "zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5" } -------------------------------------------------- -// TESTRESPONSE The following example generates an authentication request for a 3rd party initiated single sign on, specifying the issuer that should be used for matching the appropriate OpenID Connect Authentication realm: -[source,js] +[source,console] -------------------------------------------------- POST /_security/oidc/prepare { @@ -128,12 +125,11 @@ POST /_security/oidc/prepare "login_hint": "this_is_an_opaque_string" } -------------------------------------------------- -// CONSOLE The following example output of the response contains the URI pointing to the Authorization Endpoint of the OpenID Connect Provider with all the parameters of the Authentication Request, as HTTP GET parameters: -[source,js] +[source,console-result] -------------------------------------------------- { "redirect" : "http://127.0.0.1:8080/c2id-login?login_hint=this_is_an_opaque_string&scope=openid&response_type=id_token&redirect_uri=https%3A%2F%2Fmy.fantastic.rp%2Fcb&state=4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I&nonce=WaBPH0KqPVdG5HHdSxPRjfoZbXMCicm5v1OiAj0DUFM&client_id=elasticsearch-rp", diff --git a/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc b/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc index 494171492127..c048e19a18c2 100644 --- a/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc @@ -86,7 +86,7 @@ been created or updated. To add a single privilege, submit a PUT or POST request to the `/_security/privilege//` endpoint. For example: -[source,js] +[source,console] -------------------------------------------------- PUT /_security/privilege { @@ -102,7 +102,6 @@ PUT /_security/privilege } } -------------------------------------------------- -// CONSOLE <1> These strings have significance within the "myapp" application. {es} does not assign any meaning to them. <2> The use of a wildcard here (`*`) means that this privilege grants access to @@ -113,7 +112,7 @@ PUT /_security/privilege wildcard and returns `true`. <3> The metadata object is optional. -[source,js] +[source,console-result] -------------------------------------------------- { "myapp": { @@ -123,13 +122,12 @@ PUT /_security/privilege } } -------------------------------------------------- -// TESTRESPONSE <1> When an existing privilege is updated, `created` is set to false. To add multiple privileges, submit a POST request to the `/_security/privilege/` endpoint. For example: -[source,js] +[source,console] -------------------------------------------------- PUT /_security/privilege { @@ -148,12 +146,11 @@ PUT /_security/privilege } } -------------------------------------------------- -// CONSOLE A successful call returns a JSON structure that shows whether the privileges have been created or updated. -[source,js] +[source,console-result] -------------------------------------------------- { "app02": { @@ -171,4 +168,3 @@ have been created or updated. } } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/security/ssl.asciidoc b/x-pack/docs/en/rest-api/security/ssl.asciidoc index ad0ff9e5a0f3..4263bce00677 100644 --- a/x-pack/docs/en/rest-api/security/ssl.asciidoc +++ b/x-pack/docs/en/rest-api/security/ssl.asciidoc @@ -80,12 +80,10 @@ certificate, this field has a value of `true`. The following example provides information about the certificates on a single node of {es}: -[source,js] +[source,console] -------------------------------------------------- GET /_ssl/certificates -------------------------------------------------- -// CONSOLE -// TEST The API returns the following results: From 06c4c5cca34a95a7091eae6ab5180c61f945c648 Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Thu, 5 Sep 2019 10:44:21 -0700 Subject: [PATCH 071/103] [DOCS] Identify reloadable Azure repository plugin settings (#46358) --- docs/plugins/repository-azure.asciidoc | 143 ++++++++++++------ docs/reference/setup/secure-settings.asciidoc | 1 + 2 files changed, 95 insertions(+), 49 deletions(-) diff --git a/docs/plugins/repository-azure.asciidoc b/docs/plugins/repository-azure.asciidoc index 86d6d2a716dc..cb0fa574f477 100644 --- a/docs/plugins/repository-azure.asciidoc +++ b/docs/plugins/repository-azure.asciidoc @@ -19,13 +19,6 @@ bin/elasticsearch-keystore add azure.client.default.account bin/elasticsearch-keystore add azure.client.default.key ---------------------------------------------------------------- -Where `account` is the azure account name and `key` the azure secret key. Instead of an azure secret key under `key`, you can alternatively -define a shared access signatures (SAS) token under `sas_token` to use for authentication instead. When using an SAS token instead of an -account key, the SAS token must have read (r), write (w), list (l), and delete (d) permissions for the repository base path and -all its contents. These permissions need to be granted for the blob service (b) and apply to resource types service (s), container (c), and -object (o). -These settings are used by the repository's internal azure client. - Note that you can also define more than one account: [source,sh] @@ -36,42 +29,8 @@ bin/elasticsearch-keystore add azure.client.secondary.account bin/elasticsearch-keystore add azure.client.secondary.sas_token ---------------------------------------------------------------- -`default` is the default account name which will be used by a repository, -unless you set an explicit one in the -<>. - -The `account`, `key`, and `sas_token` storage settings are -{ref}/secure-settings.html#reloadable-secure-settings[reloadable]. After you -reload the settings, the internal azure clients, which are used to transfer the -snapshot, will utilize the latest settings from the keystore. - -NOTE: In progress snapshot/restore jobs will not be preempted by a *reload* -of the storage secure settings. They will complete using the client as it was built -when the operation started. - -You can set the client side timeout to use when making any single request. It can be defined globally, per account or both. -It's not set by default which means that Elasticsearch is using the -http://azure.github.io/azure-storage-java/com/microsoft/azure/storage/RequestOptions.html#setTimeoutIntervalInMs(java.lang.Integer)[default value] -set by the azure client (known as 5 minutes). - -`max_retries` can help to control the exponential backoff policy. It will fix the number of retries -in case of failures before considering the snapshot is failing. Defaults to `3` retries. -The initial backoff period is defined by Azure SDK as `30s`. Which means `30s` of wait time -before retrying after a first timeout or failure. The maximum backoff period is defined by Azure SDK as -`90s`. - -`endpoint_suffix` can be used to specify Azure endpoint suffix explicitly. Defaults to `core.windows.net`. - -[source,yaml] ----- -azure.client.default.timeout: 10s -azure.client.default.max_retries: 7 -azure.client.default.endpoint_suffix: core.chinacloudapi.cn -azure.client.secondary.timeout: 30s ----- - -In this example, timeout will be `10s` per try for `default` with `7` retries before failing -and endpoint suffix will be `core.chinacloudapi.cn` and `30s` per try for `secondary` with `3` retries. +For more information about these settings, see +<>. [IMPORTANT] .Supported Azure Storage Account types @@ -86,18 +45,104 @@ The Azure Repository plugin works with all Standard storage accounts https://azure.microsoft.com/en-gb/documentation/articles/storage-premium-storage[Premium Locally Redundant Storage] (`Premium_LRS`) is **not supported** as it is only usable as VM disk storage, not as general storage. =============================================== -You can register a proxy per client using the following settings: +[[repository-azure-client-settings]] +==== Client settings + +The client that you use to connect to Azure has a number of settings available. +The settings have the form `azure.client.CLIENT_NAME.SETTING_NAME`. By default, +`azure` repositories use a client named `default`, but this can be modified using +the <> `client`. +For example: + +[source,js] +---- +PUT _snapshot/my_backup +{ + "type": "azure", + "settings": { + "client": "secondary" + } +} +---- +// CONSOLE +// TEST[skip:we don't have azure setup while testing this] + +Most client settings can be added to the `elasticsearch.yml` configuration file. +For example: [source,yaml] ---- -azure.client.default.proxy.host: proxy.host -azure.client.default.proxy.port: 8888 -azure.client.default.proxy.type: http +azure.client.default.timeout: 10s +azure.client.default.max_retries: 7 +azure.client.default.endpoint_suffix: core.chinacloudapi.cn +azure.client.secondary.timeout: 30s ---- -Supported values for `proxy.type` are `direct` (default), `http` or `socks`. -When `proxy.type` is set to `http` or `socks`, `proxy.host` and `proxy.port` must be provided. +In this example, the client side timeout is `10s` per try for the `default` +account with `7` retries before failing. The endpoint suffix is +`core.chinacloudapi.cn` and `30s` per try for the `secondary` account with `3` +retries. +The `account`, `key`, and `sas_token` storage settings are reloadable secure +settings, which you add to the {es} keystore. For more information about +creating and updating the {es} keystore, see +{ref}/secure-settings.html[Secure settings]. After you reload the settings, the +internal Azure clients, which are used to transfer the snapshot, utilize the +latest settings from the keystore. + +NOTE: In progress snapshot or restore jobs will not be preempted by a *reload* +of the storage secure settings. They will complete using the client as it was +built when the operation started. + +The following list contains the available client settings. Those that must be +stored in the keystore are marked as "secure"; the other settings belong in the +`elasticsearch.yml` file. + +`account` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: + The Azure account name, which is used by the repository's internal Azure client. + +`endpoint_suffix`:: + The Azure endpoint suffix to connect to. The default value is + `core.windows.net`. + +`key` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: + The Azure secret key, which is used by the repository's internal Azure client. Alternatively, use `sas_token`. + +`max_retries`:: + The number of retries to use when an Azure request fails. This setting helps + control the exponential backoff policy. It specifies the number of retries + that must occur before the snapshot fails. The default value is `3`. The + initial backoff period is defined by Azure SDK as `30s`. Thus there is `30s` + of wait time before retrying after a first timeout or failure. The maximum + backoff period is defined by Azure SDK as `90s`. + +`proxy.host`:: + The host name of a proxy to connect to Azure through. For example: `azure.client.default.proxy.host: proxy.host`. + +`proxy.port`:: + The port of a proxy to connect to Azure through. For example, `azure.client.default.proxy.port: 8888`. + +`proxy.type`:: + Register a proxy type for the client. Supported values are `direct`, `http`, + and `socks`. For example: `azure.client.default.proxy.type: http`. When + `proxy.type` is set to `http` or `socks`, `proxy.host` and `proxy.port` must + also be provided. The default value is `direct`. + +`sas_token` ({ref}/secure-settings.html[Secure], {ref}/secure-settings.html#reloadable-secure-settings[reloadable]):: + A shared access signatures (SAS) token, which the repository's internal Azure + client uses for authentication. The SAS token must have read (r), write (w), + list (l), and delete (d) permissions for the repository base path and all its + contents. These permissions must be granted for the blob service (b) and apply + to resource types service (s), container (c), and object (o). Alternatively, + use `key`. + +`timeout`:: + The client side timeout for any single request to Azure. The value should + specify the time unit. For example, a value of `5s` specifies a 5 second + timeout. There is no default value, which means that {es} uses the + http://azure.github.io/azure-storage-java/com/microsoft/azure/storage/RequestOptions.html#setTimeoutIntervalInMs(java.lang.Integer)[default value] + set by the Azure client (known as 5 minutes). This setting can be defined + globally, per account, or both. [[repository-azure-repository-settings]] ==== Repository settings diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index cd1e8e6aa16d..f65865c903a2 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -123,6 +123,7 @@ of reloading after each modification. There are reloadable secure settings for: +* {plugins}/repository-azure-client-settings.html[The Azure repository plugin] * {plugins}/discovery-ec2-usage.html#_configuring_ec2_discovery[The EC2 discovery plugin] * {plugins}/repository-gcs-client.html[The GCS repository plugin] * {plugins}/repository-s3-client.html[The S3 repository plugin] From 6e7e0b5492a7bb4944bb4fce06414173d7f91a55 Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Thu, 5 Sep 2019 11:19:35 -0700 Subject: [PATCH 072/103] [DOCS] Reformats Watcher APIs using template (#46152) --- .../en/rest-api/watcher/ack-watch.asciidoc | 57 +++++++---- .../rest-api/watcher/activate-watch.asciidoc | 41 +++++--- .../watcher/deactivate-watch.asciidoc | 42 +++++--- .../en/rest-api/watcher/delete-watch.asciidoc | 42 +++++--- .../rest-api/watcher/execute-watch.asciidoc | 68 +++++++------ .../en/rest-api/watcher/get-watch.asciidoc | 45 ++++++--- .../en/rest-api/watcher/put-watch.asciidoc | 65 +++++++------ .../docs/en/rest-api/watcher/start.asciidoc | 33 +++++-- .../docs/en/rest-api/watcher/stats.asciidoc | 95 ++++++++++--------- x-pack/docs/en/rest-api/watcher/stop.asciidoc | 34 +++++-- 10 files changed, 326 insertions(+), 196 deletions(-) diff --git a/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc index 9e97e8d754a0..742b17c0c998 100644 --- a/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc @@ -6,40 +6,55 @@ ++++ {stack-ov}/actions.html#actions-ack-throttle[Acknowledging a watch] enables you -to manually throttle execution of the watch's actions. An action's -_acknowledgement state_ is stored in the `status.actions..ack.state` -structure. +to manually throttle execution of the watch's actions. -IMPORTANT: If the specified watch is currently being executed, this API will return -an error. The reason for this is to prevent overwriting of the watch status from a watch -execution. - -[float] -==== Request +[[watcher-api-ack-watch-request]] +==== {api-request-title} `PUT _watcher/watch//_ack` + `PUT _watcher/watch//_ack/` -[float] -==== Path Parameters +[[watcher-api-ack-watch-prereqs]] +==== {api-prereq-title} -`action_id`:: - (list) A comma-separated list of the action IDs to acknowledge. If you omit +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. + +[[watcher-api-ack-watch-desc]] +==== {api-description-title} + +An action's _acknowledgement state_ is stored in the +`status.actions..ack.state` structure. + +IMPORTANT: If the specified watch is currently being executed, this API will +return an error. The reason for this is to prevent overwriting of the watch +status from a watch execution. + +[[watcher-api-ack-watch-path-params]] +==== {api-path-parms-title} + +``:: + (Optional, list) A comma-separated list of the action IDs to acknowledge. If you omit this parameter, all of the actions of the watch are acknowledged. -`watch_id` (required):: - (string) Identifier for the watch. +``:: + (Required, string) Identifier for the watch. -[float] -==== Authorization +//[[watcher-api-ack-watch-query-params]] +//==== {api-query-parms-title} -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {xpack-ref}/security-privileges.html[Security Privileges]. +//[[watcher-api-ack-watch-request-body]] +//==== {api-request-body-title} +//[[watcher-api-ack-watch-response-body]] +//==== {api-response-body-title} -[float] -==== Examples +//[[watcher-api-ack-watch-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-ack-watch-example]] +==== {api-examples-title} To demonstrate let's create a new watch: diff --git a/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc index 74a98a00fa42..aa78af4439b1 100644 --- a/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc @@ -9,28 +9,43 @@ A watch can be either {stack-ov}/how-watcher-works.html#watch-active-state[active or inactive]. This API enables you to activate a currently inactive watch. -[float] -==== Request +[[watcher-api-activate-watch-request]] +==== {api-request-title} `PUT _watcher/watch//_activate` -[float] -==== Path Parameters +[[watcher-api-activate-watch-prereqs]] +==== {api-prereq-title} -`watch_id` (required):: - (string) Identifier for the watch. +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. -[float] -==== Authorization +//[[watcher-api-activate-watch-desc]] +//==== {api-description-title} -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {xpack-ref}/security-privileges.html[Security Privileges]. +[[watcher-api-activate-watch-path-params]] +==== {api-path-parms-title} -[float] -==== Examples +``:: + (Required, string) Identifier for the watch. + +//[[watcher-api-activate-watch-query-params]] +//==== {api-query-parms-title} + +//[[watcher-api-activate-watch-request-body]] +//==== {api-request-body-title} + +//[[watcher-api-activate-watch-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-activate-watch-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-activate-watch-example]] +==== {api-examples-title} The status of an inactive watch is returned with the watch definition when you -call the <>: +call the <>: [source,js] -------------------------------------------------- diff --git a/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc index 59625c139111..6023d020a4a6 100644 --- a/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc @@ -9,27 +9,43 @@ A watch can be either {stack-ov}/how-watcher-works.html#watch-active-state[active or inactive]. This API enables you to deactivate a currently active watch. -[float] -==== Request +[[watcher-api-deactivate-watch-request]] +==== {api-request-title} `PUT _watcher/watch//_deactivate` -[float] -==== Path Parameters +[[watcher-api-deactivate-watch-prereqs]] +==== {api-prereq-title} -`watch_id` (required):: - (string) Identifier for the watch. +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. -[float] -==== Authorization -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {xpack-ref}/security-privileges.html[Security Privileges]. +//[[watcher-api-deactivate-watch-desc]] +//==== {api-description-title} -[float] -==== Examples +[[watcher-api-deactivate-watch-path-params]] +==== {api-path-parms-title} + +``:: + (Required, string) Identifier for the watch. + +//[[watcher-api-deactivate-watch-query-params]] +//==== {api-query-parms-title} + +//[[watcher-api-deactivate-watch-request-body]] +//==== {api-request-body-title} + +//[[watcher-api-deactivate-watch-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-deactivate-watch-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-deactivate-watch-example]] +==== {api-examples-title} The status of an active watch is returned with the watch definition when you -call the <>: +call the <>: [source,js] -------------------------------------------------- diff --git a/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc index e6c9d784ebc5..44d50a4e16d2 100644 --- a/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc @@ -5,15 +5,21 @@ Delete watch ++++ -The delete watch API removes a watch from {watcher}. +Removes a watch from {watcher}. -[float] -==== Request +[[watcher-api-delete-watch-request]] +==== {api-request-title} `DELETE _watcher/watch/` -[float] -==== Description +[[watcher-api-delete-watch-prereqs]] +==== {api-prereq-title} + +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. + +[[watcher-api-delete-watch-desc]] +==== {api-description-title} When the watch is removed, the document representing the watch in the `.watches` index is gone and it will never be run again. @@ -27,20 +33,26 @@ IMPORTANT: Deleting a watch must be done via this API only. Do not delete the sure no `write` privileges are granted to anyone over the `.watches` index. -[float] -==== Path Parameters +[[watcher-api-delete-watch-path-params]] +==== {api-path-parms-title} -`watch_id` (required):: - (string) Identifier for the watch. +``:: + (Required, string) Identifier for the watch. -[float] -==== Authorization +//[[watcher-api-delete-watch-query-params]] +//==== {api-query-parms-title} -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {xpack-ref}/security-privileges.html[Security Privileges]. +//[[watcher-api-delete-watch-request-body]] +//==== {api-request-body-title} -[float] -==== Examples +//[[watcher-api-delete-watch-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-delete-watch-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-delete-watch-example]] +==== {api-examples-title} The following example deletes a watch with the `my-watch` id: diff --git a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc index 222fc2c8939c..42df82ddc886 100644 --- a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc @@ -5,19 +5,26 @@ Execute watch ++++ -The execute watch API forces the execution of a stored watch. It can be used to -force execution of the watch outside of its triggering logic, or to simulate the -watch execution for debugging purposes. +Forces the execution of a stored watch. -[float] -==== Request +[[watcher-api-execute-watch-request]] +==== {api-request-title} `POST _watcher/watch//_execute` + `POST _watcher/watch/_execute` -[float] -==== Description +[[watcher-api-execute-watch-prereqs]] +==== {api-prereq-title} + +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. + +[[watcher-api-execute-watch-desc]] +==== {api-description-title} + +This API can be used to force execution of the watch outside of its triggering +logic or to simulate the watch execution for debugging purposes. For testing and debugging purposes, you also have fine-grained control on how the watch runs. You can execute the watch without executing all of its actions @@ -25,29 +32,28 @@ or alternatively by simulating them. You can also force execution by ignoring the watch condition and control whether a watch record would be written to the watch history after execution. -[float] [[watcher-api-execute-inline-watch]] -===== Inline Watch Execution +===== Inline watch execution You can use the Execute API to execute watches that are not yet registered by specifying the watch definition inline. This serves as great tool for testing and debugging your watches prior to adding them to {watcher}. -[float] -==== Path Parameters +[[watcher-api-execute-watch-path-params]] +==== {api-path-parms-title} -`watch_id`:: - (string) Identifier for the watch. +``:: + (Optional, string) Identifier for the watch. -[float] -==== Query Parameters +[[watcher-api-execute-watch-query-params]] +==== {api-query-parms-title} `debug`:: - (boolean) Defines whether the watch runs in debug mode. The default value is - `false`. + (Optional, boolean) Defines whether the watch runs in debug mode. The default + value is `false`. -[float] -==== Request Body +[[watcher-api-execute-watch-request-body]] +==== {api-request-body-title} This API supports the following fields: @@ -59,7 +65,7 @@ This API supports the following fields: that will be used during the watch execution | `ignore_condition` | no | false | When set to `true`, the watch execution uses the - {stack-ov}/condition-always.html[Always Condition]. + {stack-ov}/condition-always.html[Always condition]. This can also be specified as an HTTP parameter. | `alternative_input` | no | null | When present, the watch uses this object as a payload @@ -81,9 +87,8 @@ This API supports the following fields: not persisted to the index and record_execution cannot be set. |====== -[float] [[watcher-api-execute-watch-action-mode]] -===== Action Execution Modes +===== Action execution modes Action modes define how actions are handled during the watch execution. There are five possible modes an action can be associated with: @@ -116,13 +121,8 @@ are five possible modes an action can be associated with: Effectively forces the action to be throttled. |====== -[float] -==== Authorization -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {stack-ov}/security-privileges.html[Security Privileges]. - -[float] -==== Security Integration +[[watcher-api-execute-watch-security]] +===== Security integration When {es} {security-features} are enabled on your cluster, watches are executed with the privileges of the user that stored the watches. If your @@ -133,8 +133,14 @@ When using the execute watch API, the authorization data of the user that called the API will be used as a base, instead of of the information who stored the watch. -[float] -==== Examples +//[[watcher-api-execute-watch-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-execute-watch-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-execute-watch-example]] +==== {api-examples-title} The following example executes the `my_watch` watch: diff --git a/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc index bd26c6f26748..b24d6be70e70 100644 --- a/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc @@ -5,28 +5,43 @@ Get watch ++++ -This API retrieves a watch by its ID. +Retrieves a watch by its ID. -[float] -==== Request +[[watcher-api-get-watch-request]] +==== {api-request-title} `GET _watcher/watch/` -[float] -==== Path Parameters +[[watcher-api-get-watch-prereqs]] +==== {api-prereq-title} -`watch_id` (required):: - (string) Identifier for the watch. - -[float] -==== Authorization - -You must have `manage_watcher` or `monitor_watcher` cluster privileges to use +* You must have `manage_watcher` or `monitor_watcher` cluster privileges to use this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. +{stack-ov}/security-privileges.html[Security privileges]. -[float] -==== Examples +//[[watcher-api-get-watch-desc]] +//==== {api-description-title} + +[[watcher-api-get-watch-path-params]] +==== {api-path-parms-title} + +``:: + (Required, string) Identifier for the watch. + +//[[watcher-api-get-watch-query-params]] +//==== {api-query-parms-title} + +//[[watcher-api-get-watch-request-body]] +//==== {api-request-body-title} + +//[[watcher-api-get-watch-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-get-watch-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-get-watch-example]] +==== {api-examples-title} The following example gets a watch with `my-watch` id: diff --git a/x-pack/docs/en/rest-api/watcher/put-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/put-watch.asciidoc index a824afb49b27..8acbef6c5591 100644 --- a/x-pack/docs/en/rest-api/watcher/put-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/put-watch.asciidoc @@ -5,16 +5,21 @@ Put watch ++++ -The PUT watch API either registers a new watch in {watcher} or updates an -existing one. +Either registers a new watch in {watcher} or updates an existing one. -[float] -==== Request +[[watcher-api-put-watch-request]] +==== {api-request-title} `PUT _watcher/watch/` -[float] -==== Description +[[watcher-api-put-watch-prereqs]] +==== {api-prereq-title} + +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. + +[[watcher-api-put-watch-desc]] +==== {api-description-title} When a watch is registered, a new document that represents the watch is added to the `.watches` index and its trigger is immediately registered with the relevant @@ -30,21 +35,29 @@ When adding a watch you can also define its initial {stack-ov}/how-watcher-works.html#watch-active-state[active state]. You do that by setting the `active` parameter. -[float] -==== Path Parameters +[[watcher-api-put-watch-security]] +===== Security integration -`watch_id` (required):: - (string) Identifier for the watch. +When {es} {security-features} are enabled, your watch can index or search only +on indices for which the user that stored the watch has privileges. If the user +is able to read index `a`, but not index `b`, the same will apply, when the watch +is executed. -[float] -==== Query Parameters +[[watcher-api-put-watch-path-params]] +==== {api-path-parms-title} + +``:: + (Required, string) Identifier for the watch. + +[[watcher-api-put-watch-query-params]] +==== {api-query-parms-title} `active`:: - (boolean) Defines whether the watch is active or inactive by default. The - default value is `true`, which means the watch is active by default. + (Optional, boolean) Defines whether the watch is active or inactive by default. + The default value is `true`, which means the watch is active by default. -[float] -==== Request Body +[[watcher-api-put-watch-request-body]] +==== {api-request-body-title} A watch has the following fields: @@ -82,22 +95,14 @@ A watch has the following fields: |====== -[float] -==== Authorization +//[[watcher-api-put-watch-response-body]] +//==== {api-response-body-title} -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {stack-ov}/security-privileges.html[Security Privileges]. +//[[watcher-api-put-watch-response-codes]] +//==== {api-response-codes-title} -[float] -==== Security Integration - -When {es} {security-features} are enabled, your watch can index or search only -on indices for which the user that stored the watch has privileges. If the user -is able to read index `a`, but not index `b`, the same will apply, when the watch -is executed. - -[float] -==== Examples +[[watcher-api-put-watch-example]] +==== {api-examples-title} The following example adds a watch with the `my-watch` id that has the following characteristics: diff --git a/x-pack/docs/en/rest-api/watcher/start.asciidoc b/x-pack/docs/en/rest-api/watcher/start.asciidoc index 3ae1139cb9a9..b4d5a5b7a934 100644 --- a/x-pack/docs/en/rest-api/watcher/start.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/start.asciidoc @@ -7,18 +7,37 @@ Starts the {watcher} service if it is not already running. -[float] -==== Request +[[watcher-api-start-request]] +==== {api-request-title} `POST _watcher/_start` -==== Authorization +[[watcher-api-start-prereqs]] +==== {api-prereq-title} -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {xpack-ref}/security-privileges.html[Security Privileges]. +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. -[float] -==== Examples +//[[watcher-api-start-desc]] +//==== {api-description-title} + +//[[watcher-api-start-path-params]] +//==== {api-path-parms-title} + +//[[watcher-api-start-query-params]] +//==== {api-query-parms-title} + +//[[watcher-api-start-request-body]] +//==== {api-request-body-title} + +//[[watcher-api-start-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-start-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-start-example]] +==== {api-examples-title} [source,js] -------------------------------------------------- diff --git a/x-pack/docs/en/rest-api/watcher/stats.asciidoc b/x-pack/docs/en/rest-api/watcher/stats.asciidoc index 8ae06491ef4b..f4c915f56121 100644 --- a/x-pack/docs/en/rest-api/watcher/stats.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/stats.asciidoc @@ -8,66 +8,75 @@ Retrieves the current {watcher} metrics. -[float] -==== Request +[[watcher-api-stats-request]] +==== {api-request-title} `GET _watcher/stats` + `GET _watcher/stats/` -[float] -==== Description +[[watcher-api-stats-prereqs]] +==== {api-prereq-title} -This API always returns basic metrics. You retrieve more metrics by using -the `metric` parameter. +* You must have `manage_watcher` or `monitor_watcher` cluster privileges to use +this API. For more information, see +{stack-ov}/security-privileges.html[Security privileges]. -[float] -===== Current executing watches metric +//[[watcher-api-stats-desc]] +//==== {api-description-title} -The current executing watches metric gives insight into the watches that are -currently being executed by {watcher}. Additional information is shared per -watch that is currently executing. This information includes the `watch_id`, -the time its execution started and its current execution phase. - -To include this metric, the `metric` option should be set to `current_watches` -or `_all`. In addition you can also specify the `emit_stacktraces=true` -parameter, which adds stack traces for each watch that is being executed. These -stack traces can give you more insight into an execution of a watch. - -[float] -===== Queued watches metric - -{watcher} moderates the execution of watches such that their execution won't put -too much pressure on the node and its resources. If too many watches trigger -concurrently and there isn't enough capacity to execute them all, some of the -watches are queued, waiting for the current executing watches to finish their -execution. The queued watches metric gives insight on these queued watches. - -To include this metric, the `metric` option should include `queued_watches` or -`_all`. - -[float] -==== Path Parameters +[[watcher-api-stats-path-params]] +==== {api-path-parms-title} `emit_stacktraces`:: - (boolean) Defines whether stack traces are generated for each watch that is - running. The default value is `false`. + (Optional, boolean) Defines whether stack traces are generated for each watch + that is running. The default value is `false`. -`metric`:: - (enum) Defines which additional metrics are included in the response. +``:: + (Optional, enum) Defines which additional metrics are included in the response. `current_watches`::: Includes the current executing watches in the response. `queued_watches`::: Includes the watches queued for execution in the response. `_all`::: Includes all metrics in the response. -[float] -==== Authorization +//[[watcher-api-stats-query-params]] +//==== {api-query-parms-title} -You must have `manage_watcher` or `monitor_watcher` cluster privileges to use -this API. For more information, see -{xpack-ref}/security-privileges.html[Security Privileges]. +//[[watcher-api-stats-request-body]] +//==== {api-request-body-title} -[float] -==== Examples +[[watcher-api-stats-response-body]] +==== {api-response-body-title} + +This API always returns basic metrics. You retrieve more metrics by using +the `metric` parameter. + +`current_watches`:: + (list) The current executing watches metric gives insight into the watches + that are currently being executed by {watcher}. Additional information is + shared per watch that is currently executing. This information includes the + `watch_id`, the time its execution started and its current execution phase. + + To include this metric, the `metric` option should be set to `current_watches` + or `_all`. In addition you can also specify the `emit_stacktraces=true` + parameter, which adds stack traces for each watch that is being executed. These + stack traces can give you more insight into an execution of a watch. + +`queued_watches`:: + (list) {watcher} moderates the execution of watches such that their execution + won't put too much pressure on the node and its resources. If too many watches + trigger concurrently and there isn't enough capacity to execute them all, some + of the watches are queued, waiting for the current executing watches to finish + their execution. The queued watches metric gives insight on these queued + watches. + + To include this metric, the `metric` option should include `queued_watches` or + `_all`. + +//[[watcher-api-stats-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-stats-example]] +==== {api-examples-title} The following example calls the `stats` API to retrieve basic metrics: diff --git a/x-pack/docs/en/rest-api/watcher/stop.asciidoc b/x-pack/docs/en/rest-api/watcher/stop.asciidoc index 782b983a88cf..28f5be96f1e6 100644 --- a/x-pack/docs/en/rest-api/watcher/stop.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/stop.asciidoc @@ -7,19 +7,37 @@ Stops the {watcher} service if it is running. -[float] -==== Request +[[watcher-api-stop-request]] +==== {api-request-title} `POST _watcher/_stop` -[float] -==== Authorization +[[watcher-api-stop-prereqs]] +==== {api-prereq-title} -You must have `manage_watcher` cluster privileges to use this API. For more -information, see {xpack-ref}/security-privileges.html[Security Privileges]. +* You must have `manage_watcher` cluster privileges to use this API. For more +information, see {stack-ov}/security-privileges.html[Security privileges]. -[float] -==== Examples +//[[watcher-api-stop-desc]] +//==== {api-description-title} + +//[[watcher-api-stop-path-params]] +//==== {api-path-parms-title} + +//[[watcher-api-stop-query-params]] +//==== {api-query-parms-title} + +//[[watcher-api-stop-request-body]] +//==== {api-request-body-title} + +//[[watcher-api-stop-response-body]] +//==== {api-response-body-title} + +//[[watcher-api-stop-response-codes]] +//==== {api-response-codes-title} + +[[watcher-api-stop-example]] +==== {api-examples-title} [source,js] -------------------------------------------------- From 1936f64f3b943730ff81fa66f3c5bd2f85d80e35 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 5 Sep 2019 14:40:38 -0400 Subject: [PATCH 073/103] Add docs on upgrading the keystore (#46331) This commit adds a note to the docs regarding upgrading the keystore. --- docs/reference/setup/secure-settings.asciidoc | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/reference/setup/secure-settings.asciidoc b/docs/reference/setup/secure-settings.asciidoc index f65865c903a2..2e88986ea5eb 100644 --- a/docs/reference/setup/secure-settings.asciidoc +++ b/docs/reference/setup/secure-settings.asciidoc @@ -91,6 +91,23 @@ To remove a setting from the keystore, use the `remove` command: bin/elasticsearch-keystore remove the.setting.name.to.remove ---------------------------------------------------------------- +[float] +[[keystore-upgrade]] +=== Upgrading the keystore + +Occasionally, the internal format of the keystore changes. When Elasticsearch is +installed from a package manager, an upgrade of the on-disk keystore to the new +format is done during package upgrade. In other cases, Elasticsearch will +perform such an upgrade during node startup. This requires that Elasticsearch +have write permissions to the directory that contains the keystore. +Alternatively, you can manually perform such an upgrade by using the `upgrade` +command: + +[source,sh] +---------------------------------------------------------------- +bin/elasticsearch-keystore upgrade +---------------------------------------------------------------- + [float] [[reloadable-secure-settings]] === Reloadable secure settings From ee5683fecadfce461e98381fe465611311375785 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 5 Sep 2019 13:43:07 -0500 Subject: [PATCH 074/103] [ML] Fixing instance serialization version for bwc (#46403) --- .../elasticsearch/xpack/core/ml/calendars/ScheduledEvent.java | 4 ++-- .../xpack/core/ml/job/results/ForecastRequestStats.java | 4 ++-- .../elasticsearch/xpack/ml/process/logging/CppLogMessage.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEvent.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEvent.java index dab8a8e6dbcd..fe067d9969e6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEvent.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEvent.java @@ -85,7 +85,7 @@ public class ScheduledEvent implements ToXContentObject, Writeable { public ScheduledEvent(StreamInput in) throws IOException { description = in.readString(); - if (in.getVersion().onOrAfter(Version.CURRENT)) { + if (in.getVersion().onOrAfter(Version.V_7_4_0)) { startTime = in.readInstant(); endTime = in.readInstant(); } else { @@ -146,7 +146,7 @@ public class ScheduledEvent implements ToXContentObject, Writeable { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(description); - if (out.getVersion().onOrAfter(Version.CURRENT)) { + if (out.getVersion().onOrAfter(Version.V_7_4_0)) { out.writeInstant(startTime); out.writeInstant(endTime); } else { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java index 1278c16f394e..d2a39dad45ad 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ForecastRequestStats.java @@ -130,7 +130,7 @@ public class ForecastRequestStats implements ToXContentObject, Writeable { messages = null; } - if (in.getVersion().onOrAfter(Version.CURRENT)) { + if (in.getVersion().onOrAfter(Version.V_7_4_0)) { timestamp = in.readInstant(); startTime = in.readInstant(); endTime = in.readInstant(); @@ -161,7 +161,7 @@ public class ForecastRequestStats implements ToXContentObject, Writeable { } else { out.writeBoolean(false); } - if (out.getVersion().onOrAfter(Version.CURRENT)) { + if (out.getVersion().onOrAfter(Version.V_7_4_0)) { out.writeInstant(timestamp); out.writeInstant(startTime); out.writeInstant(endTime); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/logging/CppLogMessage.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/logging/CppLogMessage.java index 7301c2ed0451..cd23afd5e839 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/logging/CppLogMessage.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/logging/CppLogMessage.java @@ -75,7 +75,7 @@ public class CppLogMessage implements ToXContentObject, Writeable { public CppLogMessage(StreamInput in) throws IOException { logger = in.readString(); - if (in.getVersion().onOrAfter(Version.CURRENT)) { + if (in.getVersion().onOrAfter(Version.V_7_4_0)) { timestamp = in.readInstant(); } else { timestamp = Instant.ofEpochMilli(in.readVLong()); @@ -93,7 +93,7 @@ public class CppLogMessage implements ToXContentObject, Writeable { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(logger); - if (out.getVersion().onOrAfter(Version.CURRENT)) { + if (out.getVersion().onOrAfter(Version.V_7_4_0)) { out.writeInstant(timestamp); } else { out.writeVLong(timestamp.toEpochMilli()); From 565ba9cb88edd99eef788b86617c070bf4c2397e Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 5 Sep 2019 15:03:33 -0400 Subject: [PATCH 075/103] [DOCS] Reformat index stats API docs (#46322) --- docs/reference/cluster/nodes-stats.asciidoc | 25 +-- docs/reference/indices/stats.asciidoc | 202 +++++++++++------- docs/reference/rest-api/common-parms.asciidoc | 141 +++++++++++- 3 files changed, 263 insertions(+), 105 deletions(-) diff --git a/docs/reference/cluster/nodes-stats.asciidoc b/docs/reference/cluster/nodes-stats.asciidoc index b5f3c3bb2d4e..521b3f18329d 100644 --- a/docs/reference/cluster/nodes-stats.asciidoc +++ b/docs/reference/cluster/nodes-stats.asciidoc @@ -113,25 +113,15 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=node-id] [[cluster-nodes-stats-api-query-params]] ==== {api-query-parms-title} -`completion_fields`:: - (Optional, string) A comma-separated list of fields for `fielddata` and - `suggest` index metric (supports wildcards). +include::{docdir}/rest-api/common-parms.asciidoc[tag=completion-fields] -`fielddata_fields`:: - (Optional, string) A comma-separated list of fields for `fielddata` index - metric (supports wildcards). +include::{docdir}/rest-api/common-parms.asciidoc[tag=fielddata-fields] -`fields`:: - (Optional, string) A comma-separated list of fields for `fielddata` and - `completion` index metric (supports wildcards). +include::{docdir}/rest-api/common-parms.asciidoc[tag=fields] -`groups`:: - (Optional, string) A comma-separated list of search groups for `search` - index metric. +include::{docdir}/rest-api/common-parms.asciidoc[tag=groups] -`level`:: - (Optional, string) Returns indices stats aggregated at index, node or shard - level. Supported options: `indices`, `node`, `shards`. +include::{docdir}/rest-api/common-parms.asciidoc[tag=level] `types`:: (Optional, string) A comma-separated list of document types for the @@ -139,10 +129,7 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=node-id] include::{docdir}/rest-api/common-parms.asciidoc[tag=timeoutparms] -`include_segment_file_sizes`:: - (Optional, boolean) If `true`, the call reports the aggregated disk usage of - each one of the Lucene index files (only applies if segment stats are - requested). Defaults to `false`. +include::{docdir}/rest-api/common-parms.asciidoc[tag=include-segment-file-sizes] [[cluster-nodes-stats-api-response-body]] diff --git a/docs/reference/indices/stats.asciidoc b/docs/reference/indices/stats.asciidoc index c85d63c8e579..089704c96618 100644 --- a/docs/reference/indices/stats.asciidoc +++ b/docs/reference/indices/stats.asciidoc @@ -1,98 +1,138 @@ [[indices-stats]] -=== Indices Stats +=== Index stats API +++++ +Index stats +++++ -Indices level stats provide statistics on different operations happening -on an index. The API provides statistics on the index level scope -(though most stats can also be retrieved using node level scope). +Returns statistics for an index. -The following returns high level aggregation and index level stats for -all indices: +[source,console] +---- +GET /twitter/_stats +---- +// TEST[setup:twitter] -[source,js] --------------------------------------------------- -GET /_stats --------------------------------------------------- -// CONSOLE -Specific index stats can be retrieved using: +[[index-stats-api-request]] +==== {api-request-title} -[source,js] +`GET //_stats/` + +`GET //_stats` + +`GET /_stats` + + +[[index-stats-api-desc]] +==== {api-description-title} + +Use the index stats API to get high-level aggregation and statistics for an index. + +By default, +the returned statistics are index-level +with `primaries` and `total` aggregations. +`primaries` are the values for only the primary shards. +`total` are the accumulated values for both primary and replica shards. + +To get shard-level statistics, +set the `level` parameter to `shards`. + +[NOTE] +==== +When moving to another node, +the shard-level statistics for a shard are cleared. +Although the shard +is no longer part of the node, +that node retains any node-level statistics +to which the shard contributed. +==== + + +[[index-stats-api-path-params]] +==== {api-path-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index] ++ +To retrieve statistics for all indices, +use a value of `_all` or `*` or omit this parameter. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=index-metric] + + +[[index-stats-api-query-params]] +==== {api-query-parms-title} + +include::{docdir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] ++ +Defaults to `open`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=fields] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=completion-fields] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=fielddata-fields] + +`forbid_closed_indices`:: +(Optional, boolean) +If `true`, statistics are *not* collected from closed indices. +Defaults to `true`. + +include::{docdir}/rest-api/common-parms.asciidoc[tag=groups] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=level] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=include-segment-file-sizes] + +include::{docdir}/rest-api/common-parms.asciidoc[tag=include-unloaded-segments] + + +[[index-stats-api-example]] +==== {api-examples-title} + + +[[index-stats-api-multiple-ex]] +===== Get statistics for multiple indices + +[source,console] -------------------------------------------------- GET /index1,index2/_stats -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT index1\nPUT index2\n/] -By default, all stats are returned, returning only specific stats can be -specified as well in the URI. Those stats can be any of: -[horizontal] -`docs`:: The number of docs / deleted docs (docs not yet merged out). - Note, affected by refreshing the index. +[[index-stats-api-all-ex]] +===== Get statistics for all indices -`store`:: The size of the index. - -`indexing`:: Indexing statistics, can be combined with a comma - separated list of `types` to provide document type level stats. - -`get`:: Get statistics, including missing stats. - -`search`:: Search statistics including suggest statistics. - You can include statistics for custom groups by adding - an extra `groups` parameter (search operations can be associated with one or more - groups). The `groups` parameter accepts a comma separated list of group names. - Use `_all` to return statistics for all groups. - -`segments`:: Retrieve the memory use of the open segments. Optionally, setting the `include_segment_file_sizes` flag, report the aggregated disk usage of each one of the Lucene index files. - -`completion`:: Completion suggest statistics. -`fielddata`:: Fielddata statistics. -`flush`:: Flush statistics. -`merge`:: Merge statistics. -`request_cache`:: <> statistics. -`refresh`:: Refresh statistics. -`warmer`:: Warmer statistics. -`translog`:: Translog statistics. - -Some statistics allow per field granularity which accepts a list -comma-separated list of included fields. By default all fields are included: - -[horizontal] -`fields`:: - - List of fields to be included in the statistics. This is used as the - default list unless a more specific field list is provided (see below). - -`completion_fields`:: - - List of fields to be included in the Completion Suggest statistics. - -`fielddata_fields`:: - - List of fields to be included in the Fielddata statistics. - - -Here are some samples: - -[source,js] +[source,console] +-------------------------------------------------- +GET /_stats +-------------------------------------------------- +// TEST[setup:twitter] + + +[[index-stats-api-specific-stats-ex]] +===== Get specific statistics + +The following request returns +only the `merge` and `refresh` statistics +for all indices. + +[source,console] -------------------------------------------------- -# Get back stats for merge and refresh only for all indices GET /_stats/merge,refresh -# Get back stats for type1 and type2 documents for the my_index index -GET /my_index/_stats/indexing?types=type1,type2 -# Get back just search stats for group1 and group2 +-------------------------------------------------- +// TEST[setup:twitter] + + +[[index-stats-api-specific-groups-ex]] +===== Get statistics for specific search groups + +The following request returns +only search statistics +for the `group1` and `group2` search groups. + +[source,console] +-------------------------------------------------- GET /_stats/search?groups=group1,group2 -------------------------------------------------- -// CONSOLE -// TEST[s/^/PUT my_index\n/] - -The stats returned are aggregated on the index level, with -`primaries` and `total` aggregations, where `primaries` are the values for only the -primary shards, and `total` are the accumulated values for both primary and replica shards. - -In order to get back shard level stats, set the `level` parameter to `shards`. - -Note, as shards move around the cluster, their stats will be cleared as -they are created on other nodes. On the other hand, even though a shard -"left" a node, that node will still retain the stats that shard -contributed to. +// TEST[setup:twitter] diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index 4704266d15c5..7eb73305b359 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -44,6 +44,13 @@ the transaction log so that Elasticsearch is able to replay changes on the next start. end::committed[] +tag::completion-fields[] +`completion_fields`:: +(Optional, string) +Comma-separated list or wildcard expressions of fields +to include in `fielddata` and `suggest` statistics. +end::completion-fields[] + tag::default_operator[] `default_operator`:: (Optional, string) The default operator for query string query: AND or OR. @@ -89,11 +96,26 @@ Wildcard expressions are not accepted. -- end::expand-wildcards[] -tag::flat-settings[] -`flat_settings`:: -(Optional, boolean) If `true`, returns settings in flat format. Defaults to -`false`. -end::flat-settings[] +tag::fielddata-fields[] +`fielddata_fields`:: +(Optional, string) +Comma-separated list or wildcard expressions of fields +to include in `fielddata` statistics. +end::fielddata-fields[] + +tag::fields[] +`fields`:: ++ +-- +(Optional, string) +Comma-separated list or wildcard expressions of fields +to include in the statistics. + +Used as the default list +unless a specific field list is provided +in the `completion_fields` or `fielddata_fields` parameters. +-- +end::fields[] tag::generation[] Generation number, such as `0`. {es} increments this generation number @@ -108,6 +130,12 @@ If specified, the index alias only applies to documents returned by the filter. end::index-alias-filter[] +tag::flat-settings[] +`flat_settings`:: +(Optional, boolean) If `true`, returns settings in flat format. Defaults to +`false`. +end::flat-settings[] + tag::http-format[] `format`:: (Optional, string) Short version of the @@ -120,6 +148,13 @@ tag::from[] (Optional, integer) Starting document offset. Defaults to `0`. end::from[] +tag::groups[] +`groups`:: +(Optional, string) +Comma-separated list of search groups +to include in the `search` statistics. +end::groups[] + tag::cat-h[] `h`:: (Optional, string) Comma-separated list of column names to display. @@ -162,6 +197,14 @@ tag::index-ignore-unavailable[] response. Defaults to `false`. end::index-ignore-unavailable[] +tag::include-segment-file-sizes[] +`include_segment_file_sizes`:: +(Optional, boolean) +If `true`, the call reports the aggregated disk usage of +each one of the Lucene index files (only applies if segment stats are +requested). Defaults to `false`. +end::include-segment-file-sizes[] + tag::include-unloaded-segments[] `include_unloaded_segments`:: (Optional, boolean) If `true`, the response includes information from segments @@ -174,6 +217,78 @@ tag::index[] used to limit the request. end::index[] +tag::index-metric[] +``:: ++ +-- +(Optional, string) +Comma-separated list of metrics used to limit the request. +Supported metrics are: + +`_all`:: +Return all statistics. + +`completion`:: +<> statistics. + +`docs`:: +Number of documents and deleted docs, which have not yet merged out. +<> can affect this statistic. + +`fielddata`:: +<> statistics. + +`flush`:: +<> statistics. + +`get`:: +Get statistics, +including missing stats. + +`indexing`:: +<> statistics. + +`merge`:: +<> statistics. + +`query_cache`:: +<> statistics. + +`refresh`:: +<> statistics. + +`request_cache`:: +<> statistics. + +`search`:: +Search statistics including suggest statistics. +You can include statistics for custom groups +by adding an extra `groups` parameter +(search operations can be associated with one or more groups). +The `groups` parameter accepts a comma separated list of group names. +Use `_all` to return statistics for all groups. + +`segments`:: +Memory use of all open segments. ++ +If the `include_segment_file_sizes` parameter is `true`, +this metric includes the aggregated disk usage +of each Lucene index file. + +`store`:: +Size of the index in <>. + +`suggest`:: +<> statistics. + +`translog`:: +<> statistics. + +`warmer`:: +<> statistics. +-- +end::index-metric[] + tag::index-template[] ``:: (Required, string) @@ -187,6 +302,22 @@ tag::lenient[] providing text to a numeric field) will be ignored. Defaults to `false`. end::lenient[] +tag::level[] +`level`:: ++ +-- +(Optional, string) +Indicates whether statistics are aggregated +at the cluster, index, or shard level. + +Valid values are: + +* `cluster` +* `indices` +* `shards` +-- +end::level[] + tag::local[] `local`:: (Optional, boolean) If `true`, the request retrieves information from the local From 995be007468cda5b0d1c27cefc051d2aae2f448a Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 5 Sep 2019 14:06:00 -0500 Subject: [PATCH 076/103] Adjusting bwc serialization after backport (#46400) --- .../test/search.aggregation/230_composite.yml | 8 ++++---- .../composite/CompositeValuesSourceParserHelper.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml index 78277cd2b670..b5e5b8bf4171 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml @@ -620,8 +620,8 @@ setup: --- "Simple Composite aggregation with GeoTile grid": - skip: - version: " - 7.99.99" - reason: geotile_grid is not supported until 8.0.0 + version: " - 7.4.99" + reason: geotile_grid is not supported until 7.5.0 - do: search: rest_total_hits_as_int: true @@ -663,8 +663,8 @@ setup: --- "Simple Composite aggregation with geotile grid add aggregate after": - skip: - version: " - 7.99.99" - reason: geotile_grid is not supported until 8.0.0 + version: " - 7.4.99" + reason: geotile_grid is not supported until 7.5.0 - do: search: rest_total_hits_as_int: true diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java index 4ca96ea577c1..cf8c951a6c5d 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceParserHelper.java @@ -69,7 +69,7 @@ public class CompositeValuesSourceParserHelper { } else if (builder.getClass() == HistogramValuesSourceBuilder.class) { code = 2; } else if (builder.getClass() == GeoTileGridValuesSourceBuilder.class) { - if (out.getVersion().before(Version.V_8_0_0)) { + if (out.getVersion().before(Version.V_7_5_0)) { throw new IOException("Attempting to serialize [" + builder.getClass().getSimpleName() + "] to a node with unsupported version [" + out.getVersion() + "]"); } From cde501198714e6b0c9fbadf82ac101cc4eedf081 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 5 Sep 2019 15:10:56 -0400 Subject: [PATCH 077/103] Clarify error message on keystore write permissions (#46321) When the Elasticsearch process does not have write permissions to upgrade the Elasticsearch keystore, we bail with an error message that indicates there is a filesystem permissions problem. This commit clarifies that error message by pointing out the directory where write permissions are required, or that the user can also run the elasticsearch-keystore upgrade command manually before starting the Elasticsearch process. In this case, the upgrade would not be needed at runtime, so the permissions would not be needed then. --- .../settings/EvilKeyStoreWrapperTests.java | 68 +++++++++++++++++++ .../common/settings/KeyStoreWrapper.java | 7 +- 2 files changed, 72 insertions(+), 3 deletions(-) create mode 100644 qa/evil-tests/src/test/java/org/elasticsearch/common/settings/EvilKeyStoreWrapperTests.java diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/common/settings/EvilKeyStoreWrapperTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/common/settings/EvilKeyStoreWrapperTests.java new file mode 100644 index 000000000000..c99e9d5cf732 --- /dev/null +++ b/qa/evil-tests/src/test/java/org/elasticsearch/common/settings/EvilKeyStoreWrapperTests.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.settings; + +import org.apache.lucene.util.Constants; +import org.apache.lucene.util.LuceneTestCase; +import org.elasticsearch.cli.ExitCodes; +import org.elasticsearch.cli.UserException; +import org.elasticsearch.test.ESTestCase; + +import java.nio.file.AccessDeniedException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.PosixFileAttributeView; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.Locale; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasToString; +import static org.hamcrest.Matchers.instanceOf; + +@LuceneTestCase.SuppressFileSystems("ExtrasFS") +public class EvilKeyStoreWrapperTests extends ESTestCase { + + public void testWritePermissions() throws Exception { + assumeFalse("requires POSIX file permissions", Constants.WINDOWS); + final Path configDir = createTempDir(); + PosixFileAttributeView attrs = Files.getFileAttributeView(configDir, PosixFileAttributeView.class); + if (attrs != null) { + // don't rely on umask: ensure the keystore has minimal permissions + attrs.setPermissions(PosixFilePermissions.fromString("r--r-----")); + } + try { + final KeyStoreWrapper wrapper = KeyStoreWrapper.create(); + final UserException e = expectThrows(UserException.class, () -> wrapper.save(configDir, new char[0])); + final String expected = String.format( + Locale.ROOT, + "unable to create temporary keystore at [%s], write permissions required for [%s] or run [elasticsearch-keystore upgrade]", + configDir.resolve("elasticsearch.keystore.tmp"), + configDir); + assertThat(e, hasToString(containsString(expected))); + assertThat(e.exitCode, equalTo(ExitCodes.CONFIG)); + assertThat(e.getCause(), instanceOf(AccessDeniedException.class)); + } finally { + // so the test framework can cleanup + attrs.setPermissions(PosixFilePermissions.fromString("rw-rw----")); + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java b/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java index 39d45e6e970d..44bcd2498186 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java +++ b/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java @@ -502,9 +502,10 @@ public class KeyStoreWrapper implements SecureSettings { } catch (final AccessDeniedException e) { final String message = String.format( - Locale.ROOT, - "unable to create temporary keystore at [%s], please check filesystem permissions", - configDir.resolve(tmpFile)); + Locale.ROOT, + "unable to create temporary keystore at [%s], write permissions required for [%s] or run [elasticsearch-keystore upgrade]", + configDir.resolve(tmpFile), + configDir); throw new UserException(ExitCodes.CONFIG, message, e); } From f336c74788f202e9eb0ce6e99da41eebb17a1768 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Thu, 5 Sep 2019 15:45:26 -0400 Subject: [PATCH 078/103] Revert "Refactor AllocatedPersistentTask#init(), move rollup logic out of ctor (#46288)" This reverts commit d999942c6dfd931266d01db24d3fb26b29cf8f64. --- .../persistent/AllocatedPersistentTask.java | 8 +- .../PersistentTasksNodeService.java | 2 +- .../xpack/rollup/job/RollupJobTask.java | 49 ++++-------- .../xpack/rollup/job/RollupJobTaskTests.java | 75 +++++-------------- 4 files changed, 40 insertions(+), 94 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java b/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java index 15fe23d58c7e..54dcffab6e36 100644 --- a/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java +++ b/server/src/main/java/org/elasticsearch/persistent/AllocatedPersistentTask.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.persistent; -import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; @@ -38,13 +37,13 @@ import java.util.function.Predicate; */ public class AllocatedPersistentTask extends CancellableTask { - private static final Logger logger = LogManager.getLogger(AllocatedPersistentTask.class); private final AtomicReference state; private volatile String persistentTaskId; private volatile long allocationId; private volatile @Nullable Exception failure; private volatile PersistentTasksService persistentTasksService; + private volatile Logger logger; private volatile TaskManager taskManager; public AllocatedPersistentTask(long id, String type, String action, String description, TaskId parentTask, @@ -86,9 +85,10 @@ public class AllocatedPersistentTask extends CancellableTask { return persistentTaskId; } - protected void init(PersistentTasksService persistentTasksService, TaskManager taskManager, - String persistentTaskId, long allocationId) { + void init(PersistentTasksService persistentTasksService, TaskManager taskManager, Logger logger, String persistentTaskId, long + allocationId) { this.persistentTasksService = persistentTasksService; + this.logger = logger; this.taskManager = taskManager; this.persistentTaskId = persistentTaskId; this.allocationId = allocationId; diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java index 02b326445966..17803bdb1958 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksNodeService.java @@ -183,7 +183,7 @@ public class PersistentTasksNodeService implements ClusterStateListener { boolean processed = false; try { - task.init(persistentTasksService, taskManager, taskInProgress.getId(), taskInProgress.getAllocationId()); + task.init(persistentTasksService, taskManager, logger, taskInProgress.getId(), taskInProgress.getAllocationId()); logger.trace("Persistent task [{}] with id [{}] and allocation id [{}] was created", task.getAction(), task.getPersistentTaskId(), task.getAllocationId()); try { diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index bc20b8d5549f..cd7ad4fdf07f 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -22,9 +22,7 @@ import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.persistent.PersistentTasksExecutor; -import org.elasticsearch.persistent.PersistentTasksService; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.indexing.IndexerState; @@ -152,10 +150,7 @@ public class RollupJobTask extends AllocatedPersistentTask implements SchedulerE private final RollupJob job; private final SchedulerEngine schedulerEngine; private final ThreadPool threadPool; - private final Client client; - private final IndexerState initialIndexerState; - private final Map initialPosition; - private RollupIndexer indexer; + private final RollupIndexer indexer; RollupJobTask(long id, String type, String action, TaskId parentTask, RollupJob job, RollupJobStatus state, Client client, SchedulerEngine schedulerEngine, ThreadPool threadPool, Map headers) { @@ -163,48 +158,36 @@ public class RollupJobTask extends AllocatedPersistentTask implements SchedulerE this.job = job; this.schedulerEngine = schedulerEngine; this.threadPool = threadPool; - this.client = client; - if (state == null) { - this.initialIndexerState = null; - this.initialPosition = null; - } else { - this.initialIndexerState = state.getIndexerState(); - this.initialPosition = state.getPosition(); - } - } - - @Override - protected void init(PersistentTasksService persistentTasksService, TaskManager taskManager, - String persistentTaskId, long allocationId) { - super.init(persistentTasksService, taskManager, persistentTaskId, allocationId); - - // If initial position is not null, we are resuming rather than starting fresh. - IndexerState indexerState = IndexerState.STOPPED; - if (initialPosition != null) { - logger.debug("We have existing state, setting state to [" + initialIndexerState + "] " + - "and current position to [" + initialPosition + "] for job [" + job.getConfig().getId() + "]"); - if (initialIndexerState.equals(IndexerState.INDEXING)) { + // If status is not null, we are resuming rather than starting fresh. + Map initialPosition = null; + IndexerState initialState = IndexerState.STOPPED; + if (state != null) { + final IndexerState existingState = state.getIndexerState(); + logger.debug("We have existing state, setting state to [" + existingState + "] " + + "and current position to [" + state.getPosition() + "] for job [" + job.getConfig().getId() + "]"); + if (existingState.equals(IndexerState.INDEXING)) { /* * If we were indexing, we have to reset back to STARTED otherwise the indexer will be "stuck" thinking * it is indexing but without the actual indexing thread running. */ - indexerState = IndexerState.STARTED; + initialState = IndexerState.STARTED; - } else if (initialIndexerState.equals(IndexerState.ABORTING) || initialIndexerState.equals(IndexerState.STOPPING)) { + } else if (existingState.equals(IndexerState.ABORTING) || existingState.equals(IndexerState.STOPPING)) { // It shouldn't be possible to persist ABORTING, but if for some reason it does, // play it safe and restore the job as STOPPED. An admin will have to clean it up, // but it won't be running, and won't delete itself either. Safest option. // If we were STOPPING, that means it persisted but was killed before finally stopped... so ok // to restore as STOPPED - indexerState = IndexerState.STOPPED; + initialState = IndexerState.STOPPED; } else { - indexerState = initialIndexerState; + initialState = existingState; } + initialPosition = state.getPosition(); } - this.indexer = new ClientRollupPageManager(job, indexerState, initialPosition, - new ParentTaskAssigningClient(client, getParentTaskId())); + this.indexer = new ClientRollupPageManager(job, initialState, initialPosition, + new ParentTaskAssigningClient(client, new TaskId(getPersistentTaskId()))); } @Override diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java index 6f24257ffe0b..a32cd975b641 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java @@ -17,7 +17,6 @@ import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -68,10 +67,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -83,10 +80,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -98,10 +93,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -113,10 +106,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -128,10 +119,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -143,10 +132,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -157,10 +144,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, null, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); } @@ -171,10 +156,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -203,9 +186,8 @@ public class RollupJobTaskTests extends ESTestCase { SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); AtomicInteger counter = new AtomicInteger(0); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, - null, client, schedulerEngine, pool, Collections.emptyMap()) { + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, + null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, ActionListener> listener) { @@ -223,7 +205,6 @@ public class RollupJobTaskTests extends ESTestCase { counter.incrementAndGet(); } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -282,8 +263,7 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -294,7 +274,6 @@ public class RollupJobTaskTests extends ESTestCase { new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -322,8 +301,7 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -334,7 +312,6 @@ public class RollupJobTaskTests extends ESTestCase { new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); @@ -365,8 +342,7 @@ public class RollupJobTaskTests extends ESTestCase { when(client.settings()).thenReturn(Settings.EMPTY); when(client.threadPool()).thenReturn(pool); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -377,7 +353,6 @@ public class RollupJobTaskTests extends ESTestCase { new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -435,8 +410,7 @@ public class RollupJobTaskTests extends ESTestCase { }).when(client).execute(anyObject(), anyObject(), anyObject()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -453,7 +427,6 @@ public class RollupJobTaskTests extends ESTestCase { } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -521,8 +494,7 @@ public class RollupJobTaskTests extends ESTestCase { }).when(client).execute(anyObject(), anyObject(), anyObject()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -539,7 +511,6 @@ public class RollupJobTaskTests extends ESTestCase { } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -608,8 +579,7 @@ public class RollupJobTaskTests extends ESTestCase { SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -626,7 +596,6 @@ public class RollupJobTaskTests extends ESTestCase { } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -661,10 +630,8 @@ public class RollupJobTaskTests extends ESTestCase { Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()); - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); CountDownLatch latch = new CountDownLatch(1); @@ -691,8 +658,7 @@ public class RollupJobTaskTests extends ESTestCase { SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); AtomicInteger counter = new AtomicInteger(0); - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, null, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void updatePersistentTaskState(PersistentTaskState taskState, @@ -714,7 +680,6 @@ public class RollupJobTaskTests extends ESTestCase { } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); @@ -779,15 +744,13 @@ public class RollupJobTaskTests extends ESTestCase { // the task would end before stop could be called. But to help test out all pathways, // just in case, we can override markAsCompleted so it's a no-op and test how stop // handles the situation - TaskId taskId = new TaskId("node", 123); - RollupJobTask task = new RollupJobTask(1, "type", "action", taskId, job, + RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override public void markAsCompleted() { latch.countDown(); } }; - task.init(null, mock(TaskManager.class), taskId.toString(), 123); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); task.onCancelled(); From 27fcf7b48d3ac73ed7c70d389d2bd72fb04c0549 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Thu, 5 Sep 2019 21:59:37 +0200 Subject: [PATCH 079/103] reuse mock client to avoid probles with thread context closed errors (#46398) --- ...ameTransformCheckpointServiceNodeTests.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java index fe6cc7c7525a..288250516301 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java @@ -50,7 +50,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgr import org.elasticsearch.xpack.dataframe.DataFrameSingleNodeTestCase; import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; -import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import java.nio.file.Path; @@ -67,8 +67,12 @@ import static org.mockito.Mockito.when; public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingleNodeTestCase { + // re-use the mock client for the whole test suite as the underlying thread pool and the + // corresponding context if recreated cause unreliable test execution + // see https://github.com/elastic/elasticsearch/issues/45238 and https://github.com/elastic/elasticsearch/issues/42577 + private static MockClientForCheckpointing mockClientForCheckpointing = null; + private DataFrameTransformsConfigManager transformsConfigManager; - private MockClientForCheckpointing mockClientForCheckpointing; private DataFrameTransformsCheckpointService transformsCheckpointService; private class MockClientForCheckpointing extends NoOpClient { @@ -121,18 +125,22 @@ public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingl @Before public void createComponents() { + // it's not possible to run it as @BeforeClass as clients aren't initialized + if (mockClientForCheckpointing == null) { + mockClientForCheckpointing = new MockClientForCheckpointing("DataFrameTransformCheckpointServiceNodeTests"); + } + transformsConfigManager = new DataFrameTransformsConfigManager(client(), xContentRegistry()); // use a mock for the checkpoint service - mockClientForCheckpointing = new MockClientForCheckpointing(getTestName()); DataFrameAuditor mockAuditor = mock(DataFrameAuditor.class); transformsCheckpointService = new DataFrameTransformsCheckpointService(mockClientForCheckpointing, transformsConfigManager, mockAuditor); } - @After - public void tearDownClient() { + @AfterClass + public static void tearDownClient() { mockClientForCheckpointing.close(); } From 466c59a4a79381f45c69a29dca30bff716a86a45 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 5 Sep 2019 16:47:18 -0400 Subject: [PATCH 080/103] [DOCS] Replace "// TESTRESPONSE" magic comments with "[source,console-result] (#46295) --- docs/plugins/analysis-icu.asciidoc | 4 ++-- docs/plugins/analysis-kuromoji.asciidoc | 19 +++++++++---------- docs/plugins/analysis-nori.asciidoc | 14 ++++++-------- docs/plugins/analysis-smartcn.asciidoc | 3 +-- docs/plugins/analysis-stempel.asciidoc | 3 +-- .../bucket/range-aggregation.asciidoc | 3 +-- .../analysis/analyzers/configuring.asciidoc | 3 +-- .../analyzers/custom-analyzer.asciidoc | 6 ++---- .../analyzers/fingerprint-analyzer.asciidoc | 6 ++---- .../analyzers/keyword-analyzer.asciidoc | 3 +-- .../analyzers/pattern-analyzer.asciidoc | 9 +++------ .../analyzers/simple-analyzer.asciidoc | 3 +-- .../analyzers/standard-analyzer.asciidoc | 6 ++---- .../analysis/analyzers/stop-analyzer.asciidoc | 6 ++---- .../analyzers/whitespace-analyzer.asciidoc | 3 +-- .../charfilters/htmlstrip-charfilter.asciidoc | 6 ++---- .../charfilters/mapping-charfilter.asciidoc | 7 +++---- .../pattern-replace-charfilter.asciidoc | 3 +-- .../common-grams-tokenfilter.asciidoc | 3 +-- .../condition-tokenfilter.asciidoc | 4 ++-- .../keep-types-tokenfilter.asciidoc | 6 ++---- .../keyword-marker-tokenfilter.asciidoc | 6 ++---- .../keyword-repeat-tokenfilter.asciidoc | 3 +-- .../multiplexer-tokenfilter.asciidoc | 3 +-- .../predicate-tokenfilter.asciidoc | 3 +-- .../tokenizers/chargroup-tokenizer.asciidoc | 4 +--- .../tokenizers/classic-tokenizer.asciidoc | 6 ++---- .../tokenizers/edgengram-tokenizer.asciidoc | 6 ++---- .../tokenizers/keyword-tokenizer.asciidoc | 3 +-- .../tokenizers/letter-tokenizer.asciidoc | 3 +-- .../tokenizers/lowercase-tokenizer.asciidoc | 3 +-- .../tokenizers/ngram-tokenizer.asciidoc | 6 ++---- .../pathhierarchy-tokenizer.asciidoc | 6 ++---- .../tokenizers/pattern-tokenizer.asciidoc | 9 +++------ .../simplepattern-tokenizer.asciidoc | 3 +-- .../simplepatternsplit-tokenizer.asciidoc | 3 +-- .../tokenizers/standard-tokenizer.asciidoc | 6 ++---- .../tokenizers/thai-tokenizer.asciidoc | 3 +-- .../tokenizers/uaxurlemail-tokenizer.asciidoc | 6 ++---- .../tokenizers/whitespace-tokenizer.asciidoc | 3 +-- docs/reference/api-conventions.asciidoc | 18 ++++++------------ .../delete-auto-follow-pattern.asciidoc | 3 +-- .../get-auto-follow-pattern.asciidoc | 3 +-- .../put-auto-follow-pattern.asciidoc | 3 +-- .../ccr/apis/follow-request-body.asciidoc | 3 +-- .../ccr/apis/follow/get-follow-info.asciidoc | 3 +-- .../apis/follow/post-pause-follow.asciidoc | 3 +-- .../apis/follow/post-resume-follow.asciidoc | 3 +-- .../ccr/apis/follow/post-unfollow.asciidoc | 3 +-- .../ccr/apis/follow/put-follow.asciidoc | 3 +-- docs/reference/ccr/getting-started.asciidoc | 11 ++++------- docs/reference/cluster/tasks.asciidoc | 6 ++---- .../apis/delete-transform.asciidoc | 4 ++-- .../apis/get-transform-stats.asciidoc | 4 ++-- .../data-frames/apis/get-transform.asciidoc | 4 ++-- .../data-frames/apis/put-transform.asciidoc | 4 ++-- .../data-frames/apis/start-transform.asciidoc | 4 ++-- .../data-frames/apis/stop-transform.asciidoc | 4 ++-- docs/reference/docs/delete-by-query.asciidoc | 10 ++++------ docs/reference/docs/reindex.asciidoc | 10 ++++------ docs/reference/docs/termvectors.asciidoc | 3 +-- docs/reference/docs/update-by-query.asciidoc | 15 +++++---------- docs/reference/docs/update.asciidoc | 3 +-- .../ilm/apis/delete-lifecycle.asciidoc | 4 +--- docs/reference/ilm/apis/get-status.asciidoc | 4 +--- docs/reference/ilm/apis/move-to-step.asciidoc | 4 +--- .../reference/ilm/apis/put-lifecycle.asciidoc | 5 ++--- .../apis/remove-policy-from-index.asciidoc | 4 +--- docs/reference/ilm/apis/start.asciidoc | 4 +--- docs/reference/ilm/apis/stop.asciidoc | 4 +--- docs/reference/ilm/start-stop-ilm.asciidoc | 16 ++++------------ docs/reference/indices/analyze.asciidoc | 4 ++-- docs/reference/indices/close.asciidoc | 3 +-- docs/reference/indices/create-index.asciidoc | 3 +-- docs/reference/indices/get-alias.asciidoc | 10 ++++------ .../indices/get-field-mapping.asciidoc | 9 +++------ docs/reference/indices/open-close.asciidoc | 3 +-- docs/reference/indices/recovery.asciidoc | 3 +-- .../reference/indices/rollover-index.asciidoc | 13 +++++-------- .../ingest/apis/delete-pipeline.asciidoc | 6 ++---- .../ingest/apis/get-pipeline.asciidoc | 9 +++------ .../ingest/apis/put-pipeline.asciidoc | 3 +-- docs/reference/ingest/ingest-node.asciidoc | 3 +-- docs/reference/mapping.asciidoc | 6 ++---- .../mapping/removal_of_types.asciidoc | 10 ++++------ .../mapping/types/percolator.asciidoc | 3 +-- .../migration/migrate_8_0/snapshots.asciidoc | 5 ++--- .../anomaly-detection/apis/close-job.asciidoc | 4 ++-- .../apis/delete-calendar-job.asciidoc | 3 +-- .../apis/delete-calendar.asciidoc | 4 ++-- .../apis/delete-datafeed.asciidoc | 4 ++-- .../apis/delete-expired-data.asciidoc | 4 ++-- .../apis/delete-filter.asciidoc | 4 ++-- .../apis/delete-job.asciidoc | 4 ++-- .../apis/delete-snapshot.asciidoc | 4 ++-- .../anomaly-detection/apis/flush-job.asciidoc | 4 ++-- .../apis/get-calendar.asciidoc | 4 ++-- .../apis/get-filter.asciidoc | 4 ++-- .../anomaly-detection/apis/open-job.asciidoc | 4 ++-- .../apis/post-calendar-event.asciidoc | 3 +-- .../apis/preview-datafeed.asciidoc | 4 ++-- .../apis/put-calendar-job.asciidoc | 3 +-- .../apis/put-calendar.asciidoc | 4 ++-- .../apis/put-filter.asciidoc | 4 ++-- .../apis/set-upgrade-mode.asciidoc | 3 +-- .../apis/start-datafeed.asciidoc | 4 ++-- .../apis/stop-datafeed.asciidoc | 3 +-- .../apis/update-filter.asciidoc | 3 +-- .../apis/validate-detector.asciidoc | 4 ++-- .../apis/validate-job.asciidoc | 4 ++-- .../apis/delete-dfanalytics.asciidoc | 3 +-- ...estimate-memory-usage-dfanalytics.asciidoc | 3 +-- .../apis/evaluate-dfanalytics.asciidoc | 3 +-- .../apis/get-dfanalytics-stats.asciidoc | 3 +-- .../apis/get-dfanalytics.asciidoc | 3 +-- .../apis/start-dfanalytics.asciidoc | 3 +-- .../apis/stop-dfanalytics.asciidoc | 3 +-- docs/reference/modules/snapshots.asciidoc | 9 +++------ .../query-dsl/percolate-query.asciidoc | 3 +-- .../reference/rollup/apis/delete-job.asciidoc | 3 +-- docs/reference/rollup/apis/get-job.asciidoc | 3 +-- docs/reference/rollup/apis/put-job.asciidoc | 5 ++--- .../rollup/apis/rollup-caps.asciidoc | 6 ++---- .../rollup/apis/rollup-index-caps.asciidoc | 3 +-- docs/reference/rollup/apis/start-job.asciidoc | 5 ++--- .../rollup/rollup-getting-started.asciidoc | 3 +-- docs/reference/search/count.asciidoc | 3 +-- docs/reference/search/explain.asciidoc | 3 +-- .../search/request/track-total-hits.asciidoc | 5 ++--- .../reference/search/search-template.asciidoc | 15 ++++++--------- docs/reference/search/validate.asciidoc | 13 +++++-------- docs/reference/sql/endpoints/rest.asciidoc | 4 +--- .../sql/endpoints/translate.asciidoc | 3 +-- .../security/create-role-mappings.asciidoc | 1 + .../rest-api/security/create-roles.asciidoc | 1 + .../rest-api/security/create-users.asciidoc | 1 + .../security/delete-app-privileges.asciidoc | 1 + .../security/put-app-privileges.asciidoc | 1 + .../en/rest-api/watcher/delete-watch.asciidoc | 3 +-- .../docs/en/rest-api/watcher/start.asciidoc | 3 +-- x-pack/docs/en/rest-api/watcher/stop.asciidoc | 3 +-- 141 files changed, 254 insertions(+), 423 deletions(-) diff --git a/docs/plugins/analysis-icu.asciidoc b/docs/plugins/analysis-icu.asciidoc index 69d741fa79a9..a189bd1adadd 100644 --- a/docs/plugins/analysis-icu.asciidoc +++ b/docs/plugins/analysis-icu.asciidoc @@ -185,7 +185,7 @@ GET icu_sample/_analyze The above `analyze` request returns the following: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -199,7 +199,7 @@ The above `analyze` request returns the following: ] } -------------------------------------------------- -// TESTRESPONSE + [[analysis-icu-normalization]] ==== ICU Normalization Token Filter diff --git a/docs/plugins/analysis-kuromoji.asciidoc b/docs/plugins/analysis-kuromoji.asciidoc index 8ccea28beda9..452173104524 100644 --- a/docs/plugins/analysis-kuromoji.asciidoc +++ b/docs/plugins/analysis-kuromoji.asciidoc @@ -191,7 +191,7 @@ GET kuromoji_sample/_analyze The above `analyze` request returns the following: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -209,7 +209,7 @@ The above `analyze` request returns the following: } ] } -------------------------------------------------- -// TESTRESPONSE + [[analysis-kuromoji-baseform]] ==== `kuromoji_baseform` token filter @@ -247,7 +247,7 @@ GET kuromoji_sample/_analyze which responds with: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -259,7 +259,7 @@ which responds with: } ] } -------------------------------------------------- -// TESTRESPONSE + [[analysis-kuromoji-speech]] ==== `kuromoji_part_of_speech` token filter @@ -313,7 +313,7 @@ GET kuromoji_sample/_analyze Which responds with: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -331,7 +331,7 @@ Which responds with: } ] } -------------------------------------------------- -// TESTRESPONSE + [[analysis-kuromoji-readingform]] ==== `kuromoji_readingform` token filter @@ -504,7 +504,7 @@ GET kuromoji_sample/_analyze The above request returns: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -516,7 +516,7 @@ The above request returns: } ] } -------------------------------------------------- -// TESTRESPONSE + [[analysis-kuromoji-number]] ==== `kuromoji_number` token filter @@ -554,7 +554,7 @@ GET kuromoji_sample/_analyze Which results in: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -566,4 +566,3 @@ Which results in: } ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/plugins/analysis-nori.asciidoc b/docs/plugins/analysis-nori.asciidoc index 68ec943533aa..7ed7d4c857ab 100644 --- a/docs/plugins/analysis-nori.asciidoc +++ b/docs/plugins/analysis-nori.asciidoc @@ -125,7 +125,7 @@ GET nori_sample/_analyze The above `analyze` request returns the following: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -150,7 +150,6 @@ The above `analyze` request returns the following: }] } -------------------------------------------------- -// TESTRESPONSE <1> This is a compound token that spans two positions (`mixed` mode). -- @@ -210,7 +209,7 @@ GET _analyze Which responds with: -[source,js] +[source,console-result] -------------------------------------------------- { "detail": { @@ -297,7 +296,7 @@ Which responds with: } } -------------------------------------------------- -// TESTRESPONSE + [[analysis-nori-speech]] ==== `nori_part_of_speech` token filter @@ -371,7 +370,7 @@ GET nori_sample/_analyze Which responds with: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -389,7 +388,7 @@ Which responds with: } ] } -------------------------------------------------- -// TESTRESPONSE + [[analysis-nori-readingform]] ==== `nori_readingform` token filter @@ -426,7 +425,7 @@ GET nori_sample/_analyze Which responds with: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ { @@ -438,6 +437,5 @@ Which responds with: }] } -------------------------------------------------- -// TESTRESPONSE <1> The Hanja form is replaced by the Hangul translation. diff --git a/docs/plugins/analysis-smartcn.asciidoc b/docs/plugins/analysis-smartcn.asciidoc index 3786cbc5e2b8..a67ac3edd572 100644 --- a/docs/plugins/analysis-smartcn.asciidoc +++ b/docs/plugins/analysis-smartcn.asciidoc @@ -99,7 +99,7 @@ GET smartcn_example/_analyze The above request returns: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -428,4 +428,3 @@ The above request returns: ] } -------------------------------------------------- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/plugins/analysis-stempel.asciidoc b/docs/plugins/analysis-stempel.asciidoc index cd234c8d4275..ea37cf0228e8 100644 --- a/docs/plugins/analysis-stempel.asciidoc +++ b/docs/plugins/analysis-stempel.asciidoc @@ -94,7 +94,7 @@ GET polish_stop_example/_analyze The above request returns: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ @@ -115,4 +115,3 @@ The above request returns: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/aggregations/bucket/range-aggregation.asciidoc b/docs/reference/aggregations/bucket/range-aggregation.asciidoc index b27af0b3be5f..525d8689efc8 100644 --- a/docs/reference/aggregations/bucket/range-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/range-aggregation.asciidoc @@ -242,7 +242,7 @@ GET /_search ////////////////////////// -[source,js] +[source,console-result] -------------------------------------------------- { "aggregations": { @@ -264,7 +264,6 @@ GET /_search } } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/analysis/analyzers/configuring.asciidoc b/docs/reference/analysis/analyzers/configuring.asciidoc index 994842508e8f..21af29313100 100644 --- a/docs/reference/analysis/analyzers/configuring.asciidoc +++ b/docs/reference/analysis/analyzers/configuring.asciidoc @@ -63,7 +63,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -91,6 +91,5 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/custom-analyzer.asciidoc b/docs/reference/analysis/analyzers/custom-analyzer.asciidoc index 4b601a0b9bd3..cc938b0cb131 100644 --- a/docs/reference/analysis/analyzers/custom-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/custom-analyzer.asciidoc @@ -89,7 +89,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -124,7 +124,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -216,7 +215,7 @@ are defined later in the request. ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -251,7 +250,6 @@ are defined later in the request. ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc b/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc index 4d053884a4b6..8736b5054436 100644 --- a/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc @@ -24,7 +24,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -38,7 +38,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -110,7 +109,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -124,7 +123,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc b/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc index 954b514ced60..e3f533812257 100644 --- a/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc @@ -19,7 +19,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -33,7 +33,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc index 22fa534288dd..899be69d2606 100644 --- a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc @@ -34,7 +34,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -125,7 +125,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -205,7 +204,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -247,7 +246,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -290,7 +288,7 @@ GET my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -339,7 +337,6 @@ GET my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/simple-analyzer.asciidoc b/docs/reference/analysis/analyzers/simple-analyzer.asciidoc index babe566154d1..68ee22543704 100644 --- a/docs/reference/analysis/analyzers/simple-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/simple-analyzer.asciidoc @@ -19,7 +19,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -103,7 +103,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/standard-analyzer.asciidoc b/docs/reference/analysis/analyzers/standard-analyzer.asciidoc index 5117763b6681..4096560dfebd 100644 --- a/docs/reference/analysis/analyzers/standard-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/standard-analyzer.asciidoc @@ -22,7 +22,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -106,7 +106,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -176,7 +175,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -253,7 +252,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/stop-analyzer.asciidoc b/docs/reference/analysis/analyzers/stop-analyzer.asciidoc index 1b84797d9476..3176b6e87f99 100644 --- a/docs/reference/analysis/analyzers/stop-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/stop-analyzer.asciidoc @@ -20,7 +20,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -90,7 +90,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -154,7 +153,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -217,7 +216,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc b/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc index 31ba8d9ce8f2..531751c63e29 100644 --- a/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc @@ -19,7 +19,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -96,7 +96,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc b/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc index 6c1a1875d67c..33741ee6a6cb 100644 --- a/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc @@ -22,7 +22,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -36,7 +36,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -103,7 +102,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -117,7 +116,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc b/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc index 30e565d443a3..ee95d1798d17 100644 --- a/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc @@ -76,7 +76,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -90,7 +90,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -143,7 +142,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -185,7 +184,7 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE + ///////////////////// diff --git a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc index 7386af902fbc..fe76d4edb06b 100644 --- a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc @@ -144,7 +144,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -186,7 +186,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc index 80ef7c2ce79b..3f5ba6a70157 100644 --- a/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc @@ -87,7 +87,7 @@ POST /common_grams_example/_analyze And the response will be: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens" : [ @@ -168,4 +168,3 @@ And the response will be: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc index 0aeadded0d4d..38ee975601a4 100644 --- a/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc @@ -63,7 +63,7 @@ POST /condition_example/_analyze And it'd respond: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -84,7 +84,7 @@ And it'd respond: ] } -------------------------------------------------- -// TESTRESPONSE + <1> The term `What` has been lowercased, because it is only 4 characters long <2> The term `Flapdoodle` has been left in its original case, because it doesn't pass the predicate diff --git a/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc index f8dce95f1b0f..0b2f232112a4 100644 --- a/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc @@ -56,7 +56,7 @@ POST /keep_types_example/_analyze The response will be: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -70,7 +70,6 @@ The response will be: ] } -------------------------------------------------- -// TESTRESPONSE Note how only the `` token is in the output. @@ -118,7 +117,7 @@ POST /keep_types_exclude_example/_analyze The response will be: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -139,4 +138,3 @@ The response will be: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc index 1f1e4e655c55..bf79167bdf2c 100644 --- a/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc @@ -66,7 +66,7 @@ POST /keyword_marker_example/_analyze And it'd respond: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -94,7 +94,6 @@ And it'd respond: ] } -------------------------------------------------- -// TESTRESPONSE As compared to the `normal` analyzer which has `cats` stemmed to `cat`: @@ -111,7 +110,7 @@ POST /keyword_marker_example/_analyze Response: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -139,4 +138,3 @@ Response: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc index 044e8c147695..f8f298c16086 100644 --- a/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc @@ -52,7 +52,7 @@ POST /keyword_repeat_example/_analyze And it'd respond: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -87,7 +87,6 @@ And it'd respond: ] } -------------------------------------------------- -// TESTRESPONSE Which preserves both the `cat` and `cats` tokens. Compare this to the example on the <>. diff --git a/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc index 50462cc2871d..9e82c84df33c 100644 --- a/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc @@ -68,7 +68,7 @@ POST /multiplexer_example/_analyze And it'd respond: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -110,7 +110,6 @@ And it'd respond: ] } -------------------------------------------------- -// TESTRESPONSE <1> The stemmer has also emitted a token `home` at position 1, but because it is a duplicate of this token it has been removed from the token stream diff --git a/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc index 44ead824a84f..91e4ea2bfff4 100644 --- a/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc @@ -58,7 +58,7 @@ POST /condition_example/_analyze And it'd respond: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -72,7 +72,6 @@ And it'd respond: ] } -------------------------------------------------- -// TESTRESPONSE <1> The token 'What' has been removed from the tokenstream because it does not match the predicate. diff --git a/docs/reference/analysis/tokenizers/chargroup-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/chargroup-tokenizer.asciidoc index 2c18e94878fb..1db4a4347ba3 100644 --- a/docs/reference/analysis/tokenizers/chargroup-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/chargroup-tokenizer.asciidoc @@ -41,7 +41,7 @@ POST _analyze returns -[source,js] +[source,console-result] --------------------------- { "tokens": [ @@ -76,5 +76,3 @@ returns ] } --------------------------- -// TESTRESPONSE - diff --git a/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc index 52bdcbd77322..83236b03eefe 100644 --- a/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc @@ -30,7 +30,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -114,7 +114,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -174,7 +173,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -251,7 +250,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc index 94f5930e88cf..9d9b7f83e5e7 100644 --- a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc @@ -33,7 +33,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -54,7 +54,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -138,7 +137,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -201,7 +200,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/keyword-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/keyword-tokenizer.asciidoc index 27515516fe5f..ff560904818a 100644 --- a/docs/reference/analysis/tokenizers/keyword-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/keyword-tokenizer.asciidoc @@ -20,7 +20,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -34,7 +34,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/letter-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/letter-tokenizer.asciidoc index 7423a68732d2..c21a913986df 100644 --- a/docs/reference/analysis/tokenizers/letter-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/letter-tokenizer.asciidoc @@ -21,7 +21,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -105,7 +105,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/lowercase-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/lowercase-tokenizer.asciidoc index 15620dbfef4e..b88d8d47e8dc 100644 --- a/docs/reference/analysis/tokenizers/lowercase-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/lowercase-tokenizer.asciidoc @@ -26,7 +26,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -110,7 +110,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc index c182ffacd1cf..509f088b6a34 100644 --- a/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc @@ -29,7 +29,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -155,7 +155,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -243,7 +242,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -292,7 +291,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc index 8d425197a2a6..d5cba041308b 100644 --- a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc @@ -20,7 +20,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -48,7 +48,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -124,7 +123,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -152,7 +151,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc index 5c19fcf59cc9..488e3209ef0a 100644 --- a/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc @@ -37,7 +37,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -86,7 +86,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -154,7 +153,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -182,7 +181,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -245,7 +243,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -266,7 +264,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc index adc5fc05deeb..5e6a4289398f 100644 --- a/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc @@ -65,7 +65,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens" : [ @@ -93,7 +93,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc index fc2e186f9726..21d7e1b8cedc 100644 --- a/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc @@ -66,7 +66,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens" : [ @@ -94,7 +94,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc index 9f77a0e13dc8..5156bc324703 100644 --- a/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc @@ -21,7 +21,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -105,7 +105,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -165,7 +164,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -256,7 +255,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/thai-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/thai-tokenizer.asciidoc index 3e9904d116e3..ee680f20897a 100644 --- a/docs/reference/analysis/tokenizers/thai-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/thai-tokenizer.asciidoc @@ -25,7 +25,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -88,7 +88,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc index 7fea0f1e8d8a..b474c396572a 100644 --- a/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc @@ -19,7 +19,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -54,7 +54,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// @@ -121,7 +120,7 @@ POST my_index/_analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -184,7 +183,6 @@ POST my_index/_analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/analysis/tokenizers/whitespace-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/whitespace-tokenizer.asciidoc index 149fe421c0a3..f1595b090d78 100644 --- a/docs/reference/analysis/tokenizers/whitespace-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/whitespace-tokenizer.asciidoc @@ -19,7 +19,7 @@ POST _analyze ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "tokens": [ @@ -96,7 +96,6 @@ POST _analyze ] } ---------------------------- -// TESTRESPONSE ///////////////////// diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index 85546df08516..518d71fc5bcd 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -251,7 +251,7 @@ GET /_cluster/state?filter_path=metadata.indices.*.stat* Responds: -[source,sh] +[source,console-result] -------------------------------------------------- { "metadata" : { @@ -261,7 +261,6 @@ Responds: } } -------------------------------------------------- -// TESTRESPONSE And the `**` wildcard can be used to include fields without knowing the exact path of the field. For example, we can return the Lucene version @@ -276,7 +275,7 @@ GET /_cluster/state?filter_path=routing_table.indices.**.state Responds: -[source,js] +[source,console-result] -------------------------------------------------- { "routing_table": { @@ -290,7 +289,6 @@ Responds: } } -------------------------------------------------- -// TESTRESPONSE It is also possible to exclude one or more fields by prefixing the filter with the char `-`: @@ -303,13 +301,12 @@ GET /_count?filter_path=-_shards Responds: -[source,js] +[source,console-result] -------------------------------------------------- { "count" : 5 } -------------------------------------------------- -// TESTRESPONSE And for more control, both inclusive and exclusive filters can be combined in the same expression. In this case, the exclusive filters will be applied first and the result will be filtered again using the @@ -324,7 +321,7 @@ GET /_cluster/state?filter_path=metadata.indices.*.state,-metadata.indices.logst Responds: -[source,js] +[source,console-result] -------------------------------------------------- { "metadata" : { @@ -336,7 +333,6 @@ Responds: } } -------------------------------------------------- -// TESTRESPONSE Note that Elasticsearch sometimes returns directly the raw value of a field, like the `_source` field. If you want to filter `_source` fields, you should @@ -356,7 +352,7 @@ GET /_search?filter_path=hits.hits._source&_source=title&sort=rating:desc -------------------------------------------------- // CONSOLE -[source,js] +[source,console-result] -------------------------------------------------- { "hits" : { @@ -370,7 +366,6 @@ GET /_search?filter_path=hits.hits._source&_source=title&sort=rating:desc } } -------------------------------------------------- -// TESTRESPONSE [float] @@ -585,7 +580,7 @@ POST /twitter/_search?size=surprise_me The response looks like: -[source,js] +[source,console-result] ---------------------------------------------------------------------- { "error" : { @@ -605,7 +600,6 @@ The response looks like: "status" : 400 } ---------------------------------------------------------------------- -// TESTRESPONSE But if you set `error_trace=true`: diff --git a/docs/reference/ccr/apis/auto-follow/delete-auto-follow-pattern.asciidoc b/docs/reference/ccr/apis/auto-follow/delete-auto-follow-pattern.asciidoc index ffe0e9e2fde9..4bbf9cbda60a 100644 --- a/docs/reference/ccr/apis/auto-follow/delete-auto-follow-pattern.asciidoc +++ b/docs/reference/ccr/apis/auto-follow/delete-auto-follow-pattern.asciidoc @@ -73,10 +73,9 @@ DELETE /_ccr/auto_follow/my_auto_follow_pattern The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/apis/auto-follow/get-auto-follow-pattern.asciidoc b/docs/reference/ccr/apis/auto-follow/get-auto-follow-pattern.asciidoc index 23e0cc665da0..f621c19564ac 100644 --- a/docs/reference/ccr/apis/auto-follow/get-auto-follow-pattern.asciidoc +++ b/docs/reference/ccr/apis/auto-follow/get-auto-follow-pattern.asciidoc @@ -88,7 +88,7 @@ GET /_ccr/auto_follow/my_auto_follow_pattern The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "patterns": [ @@ -106,4 +106,3 @@ The API returns the following result: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/apis/auto-follow/put-auto-follow-pattern.asciidoc b/docs/reference/ccr/apis/auto-follow/put-auto-follow-pattern.asciidoc index 89e2799f8e96..010390806841 100644 --- a/docs/reference/ccr/apis/auto-follow/put-auto-follow-pattern.asciidoc +++ b/docs/reference/ccr/apis/auto-follow/put-auto-follow-pattern.asciidoc @@ -114,13 +114,12 @@ PUT /_ccr/auto_follow/my_auto_follow_pattern The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/ccr/apis/follow-request-body.asciidoc b/docs/reference/ccr/apis/follow-request-body.asciidoc index e9c8892998f0..1cae2908fb94 100644 --- a/docs/reference/ccr/apis/follow-request-body.asciidoc +++ b/docs/reference/ccr/apis/follow-request-body.asciidoc @@ -79,7 +79,7 @@ GET /follower_index/_ccr/info?filter_path=follower_indices.parameters The following output from the follow info api describes all the default values for the above described index follow request parameters: -[source,js] +[source,console-result] -------------------------------------------------- { "follower_indices" : [ @@ -101,4 +101,3 @@ values for the above described index follow request parameters: } -------------------------------------------------- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/reference/ccr/apis/follow/get-follow-info.asciidoc b/docs/reference/ccr/apis/follow/get-follow-info.asciidoc index c1111af8404f..3f702f6dcacc 100644 --- a/docs/reference/ccr/apis/follow/get-follow-info.asciidoc +++ b/docs/reference/ccr/apis/follow/get-follow-info.asciidoc @@ -146,7 +146,7 @@ GET /follower_index/_ccr/info The API returns the following results: -[source,js] +[source,console-result] -------------------------------------------------- { "follower_indices" : [ @@ -171,4 +171,3 @@ The API returns the following results: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/apis/follow/post-pause-follow.asciidoc b/docs/reference/ccr/apis/follow/post-pause-follow.asciidoc index f1926b8778db..856b5c1b8d07 100644 --- a/docs/reference/ccr/apis/follow/post-pause-follow.asciidoc +++ b/docs/reference/ccr/apis/follow/post-pause-follow.asciidoc @@ -70,10 +70,9 @@ POST /follower_index/_ccr/pause_follow The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/apis/follow/post-resume-follow.asciidoc b/docs/reference/ccr/apis/follow/post-resume-follow.asciidoc index ac39e4d28517..b6aa6b5d9387 100644 --- a/docs/reference/ccr/apis/follow/post-resume-follow.asciidoc +++ b/docs/reference/ccr/apis/follow/post-resume-follow.asciidoc @@ -99,10 +99,9 @@ POST /follower_index/_ccr/resume_follow The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/apis/follow/post-unfollow.asciidoc b/docs/reference/ccr/apis/follow/post-unfollow.asciidoc index d2b3391767bc..cc7023babbb5 100644 --- a/docs/reference/ccr/apis/follow/post-unfollow.asciidoc +++ b/docs/reference/ccr/apis/follow/post-unfollow.asciidoc @@ -77,10 +77,9 @@ POST /follower_index/_ccr/unfollow The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/apis/follow/put-follow.asciidoc b/docs/reference/ccr/apis/follow/put-follow.asciidoc index 7582b2375766..6bb038b8326a 100644 --- a/docs/reference/ccr/apis/follow/put-follow.asciidoc +++ b/docs/reference/ccr/apis/follow/put-follow.asciidoc @@ -110,7 +110,7 @@ PUT /follower_index/_ccr/follow?wait_for_active_shards=1 The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "follow_index_created" : true, @@ -118,4 +118,3 @@ The API returns the following result: "index_following_started" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ccr/getting-started.asciidoc b/docs/reference/ccr/getting-started.asciidoc index 24304fea7642..5d726082205f 100644 --- a/docs/reference/ccr/getting-started.asciidoc +++ b/docs/reference/ccr/getting-started.asciidoc @@ -126,7 +126,7 @@ GET /_remote/info The API will respond by showing that the local cluster is connected to the remote cluster. -[source,js] +[source,console-result] -------------------------------------------------- { "leader" : { @@ -141,8 +141,7 @@ remote cluster. } } -------------------------------------------------- -// TESTRESPONSE -// TEST[s/127.0.0.1:9300/$body.leader.seeds.0/] +// TESTRESPONSE[s/127.0.0.1:9300/$body.leader.seeds.0/] // TEST[s/"connected" : true/"connected" : $body.leader.connected/] // TEST[s/"num_nodes_connected" : 1/"num_nodes_connected" : $body.leader.num_nodes_connected/] <1> This shows the local cluster is connected to the remote cluster with cluster @@ -226,7 +225,7 @@ PUT /server-metrics-copy/_ccr/follow?wait_for_active_shards=1 ////////////////////////// -[source,js] +[source,console-result] -------------------------------------------------- { "follow_index_created" : true, @@ -234,7 +233,6 @@ PUT /server-metrics-copy/_ccr/follow?wait_for_active_shards=1 "index_following_started" : true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// @@ -300,13 +298,12 @@ PUT /_ccr/auto_follow/beats ////////////////////////// -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/cluster/tasks.asciidoc b/docs/reference/cluster/tasks.asciidoc index 7db58055f20f..02586cb9677d 100644 --- a/docs/reference/cluster/tasks.asciidoc +++ b/docs/reference/cluster/tasks.asciidoc @@ -64,7 +64,7 @@ GET _tasks?nodes=nodeId1,nodeId2&actions=cluster:* <3> The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes" : { @@ -98,7 +98,6 @@ The API returns the following result: } } -------------------------------------------------- -// TESTRESPONSE ===== Retrieve information from a particular task @@ -141,7 +140,7 @@ GET _tasks?actions=*search&detailed The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes" : { @@ -166,7 +165,6 @@ The API returns the following result: } } -------------------------------------------------- -// TESTRESPONSE The new `description` field contains human readable text that identifies the particular request that the task is performing such as identifying the search diff --git a/docs/reference/data-frames/apis/delete-transform.asciidoc b/docs/reference/data-frames/apis/delete-transform.asciidoc index 7c3e4e53c288..c01d18379ce0 100644 --- a/docs/reference/data-frames/apis/delete-transform.asciidoc +++ b/docs/reference/data-frames/apis/delete-transform.asciidoc @@ -53,10 +53,10 @@ DELETE _data_frame/transforms/ecommerce_transform // TEST[skip:setup kibana sample data] When the {dataframe-transform} is deleted, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged" : true } ---- -// TESTRESPONSE diff --git a/docs/reference/data-frames/apis/get-transform-stats.asciidoc b/docs/reference/data-frames/apis/get-transform-stats.asciidoc index df1e7749273f..9b558b6b3c25 100644 --- a/docs/reference/data-frames/apis/get-transform-stats.asciidoc +++ b/docs/reference/data-frames/apis/get-transform-stats.asciidoc @@ -119,7 +119,8 @@ GET _data_frame/transforms/ecommerce_transform/_stats // TEST[skip:todo] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count" : 1, @@ -172,4 +173,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/data-frames/apis/get-transform.asciidoc b/docs/reference/data-frames/apis/get-transform.asciidoc index 0acf991402bf..27ccfcfe0a72 100644 --- a/docs/reference/data-frames/apis/get-transform.asciidoc +++ b/docs/reference/data-frames/apis/get-transform.asciidoc @@ -113,7 +113,8 @@ GET _data_frame/transforms/ecommerce_transform // TEST[skip:setup kibana sample data] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count" : 1, @@ -158,4 +159,3 @@ The API returns the following results: } ---- -// TESTRESPONSE diff --git a/docs/reference/data-frames/apis/put-transform.asciidoc b/docs/reference/data-frames/apis/put-transform.asciidoc index 8c13da7b4eb4..f9e622488bc2 100644 --- a/docs/reference/data-frames/apis/put-transform.asciidoc +++ b/docs/reference/data-frames/apis/put-transform.asciidoc @@ -187,10 +187,10 @@ PUT _data_frame/transforms/ecommerce_transform // TEST[setup:kibana_sample_data_ecommerce] When the transform is created, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged" : true } ---- -// TESTRESPONSE diff --git a/docs/reference/data-frames/apis/start-transform.asciidoc b/docs/reference/data-frames/apis/start-transform.asciidoc index 2c5f0ca6517a..b20c5c872530 100644 --- a/docs/reference/data-frames/apis/start-transform.asciidoc +++ b/docs/reference/data-frames/apis/start-transform.asciidoc @@ -69,10 +69,10 @@ POST _data_frame/transforms/ecommerce_transform/_start // TEST[skip:set up kibana samples] When the {dataframe-transform} starts, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged" : true } ---- -// TESTRESPONSE diff --git a/docs/reference/data-frames/apis/stop-transform.asciidoc b/docs/reference/data-frames/apis/stop-transform.asciidoc index f6f7784ebb91..234a43def1d7 100644 --- a/docs/reference/data-frames/apis/stop-transform.asciidoc +++ b/docs/reference/data-frames/apis/stop-transform.asciidoc @@ -99,10 +99,10 @@ POST _data_frame/transforms/ecommerce_transform/_stop // TEST[skip:set up kibana samples] When the {dataframe-transform} stops, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged" : true } ---- -// TESTRESPONSE diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index b9fd67722868..c8f18f2c4796 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -472,7 +472,7 @@ POST twitter/_search?size=0&filter_path=hits.total Which results in a sensible `total` like this one: -[source,js] +[source,console-result] ---------------------------------------------------------------- { "hits": { @@ -483,7 +483,6 @@ Which results in a sensible `total` like this one: } } ---------------------------------------------------------------- -// TESTRESPONSE [float] [[docs-delete-by-query-automatic-slice]] @@ -529,7 +528,7 @@ POST twitter/_search?size=0&filter_path=hits.total Which results in a sensible `total` like this one: -[source,js] +[source,console-result] ---------------------------------------------------------------- { "hits": { @@ -540,7 +539,6 @@ Which results in a sensible `total` like this one: } } ---------------------------------------------------------------- -// TESTRESPONSE Setting `slices` to `auto` will let {es} choose the number of slices to use. This setting will use one slice per shard, up to a certain limit. If @@ -605,7 +603,7 @@ GET _tasks?detailed=true&actions=*/delete/byquery The response looks like: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes" : { @@ -642,7 +640,7 @@ The response looks like: } } -------------------------------------------------- -// TESTRESPONSE + <1> This object contains the actual status. It is just like the response JSON with the important addition of the `total` field. `total` is the total number of operations that the reindex expects to perform. You can estimate the diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index d4a27b8fd149..5a2544260149 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -795,7 +795,7 @@ GET _tasks?detailed=true&actions=*reindex The response looks like: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes" : { @@ -841,7 +841,7 @@ The response looks like: } } -------------------------------------------------- -// TESTRESPONSE + <1> This object contains the actual status. It is identical to the response JSON except for the important addition of the `total` field. `total` is the total number of operations that the `_reindex` expects to perform. You can estimate the @@ -1036,7 +1036,7 @@ POST new_twitter/_search?size=0&filter_path=hits.total which results in a sensible `total` like this one: -[source,js] +[source,console-result] ---------------------------------------------------------------- { "hits": { @@ -1047,7 +1047,6 @@ which results in a sensible `total` like this one: } } ---------------------------------------------------------------- -// TESTRESPONSE [float] [[docs-reindex-automatic-slice]] @@ -1082,7 +1081,7 @@ POST new_twitter/_search?size=0&filter_path=hits.total which results in a sensible `total` like this one: -[source,js] +[source,console-result] ---------------------------------------------------------------- { "hits": { @@ -1093,7 +1092,6 @@ which results in a sensible `total` like this one: } } ---------------------------------------------------------------- -// TESTRESPONSE Setting `slices` to `auto` will let Elasticsearch choose the number of slices to use. This setting will use one slice per shard, up to a certain limit. If diff --git a/docs/reference/docs/termvectors.asciidoc b/docs/reference/docs/termvectors.asciidoc index 2f5e699dee34..515d75ff3baf 100644 --- a/docs/reference/docs/termvectors.asciidoc +++ b/docs/reference/docs/termvectors.asciidoc @@ -409,7 +409,7 @@ GET /imdb/_termvectors Response: -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "imdb", @@ -446,4 +446,3 @@ Response: } } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index fded2c95e5e6..c51964345150 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -365,7 +365,7 @@ GET _tasks?detailed=true&actions=*byquery The responses looks like: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes" : { @@ -405,7 +405,6 @@ The responses looks like: } } -------------------------------------------------- -// TESTRESPONSE <1> This object contains the actual status. It is just like the response JSON with the important addition of the `total` field. `total` is the total number @@ -524,7 +523,7 @@ POST twitter/_search?size=0&q=extra:test&filter_path=hits.total Which results in a sensible `total` like this one: -[source,js] +[source,console-result] ---------------------------------------------------------------- { "hits": { @@ -535,7 +534,6 @@ Which results in a sensible `total` like this one: } } ---------------------------------------------------------------- -// TESTRESPONSE [float] [[docs-update-by-query-automatic-slice]] @@ -568,7 +566,7 @@ POST twitter/_search?size=0&q=extra:test&filter_path=hits.total Which results in a sensible `total` like this one: -[source,js] +[source,console-result] ---------------------------------------------------------------- { "hits": { @@ -579,7 +577,6 @@ Which results in a sensible `total` like this one: } } ---------------------------------------------------------------- -// TESTRESPONSE Setting `slices` to `auto` will let Elasticsearch choose the number of slices to use. This setting will use one slice per shard, up to a certain limit. If @@ -690,7 +687,7 @@ POST test/_search?filter_path=hits.total // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "hits" : { @@ -701,7 +698,6 @@ POST test/_search?filter_path=hits.total } } -------------------------------------------------- -// TESTRESPONSE But you can issue an `_update_by_query` request to pick up the new mapping: @@ -720,7 +716,7 @@ POST test/_search?filter_path=hits.total // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "hits" : { @@ -731,6 +727,5 @@ POST test/_search?filter_path=hits.total } } -------------------------------------------------- -// TESTRESPONSE You can do the exact same thing when adding a field to a multifield. diff --git a/docs/reference/docs/update.asciidoc b/docs/reference/docs/update.asciidoc index 823c624c20e5..2e033d5f5b04 100644 --- a/docs/reference/docs/update.asciidoc +++ b/docs/reference/docs/update.asciidoc @@ -237,7 +237,7 @@ POST test/_update/1 If the value of `name` is already `new_name`, the update request is ignored and the `result` element in the response returns `noop`: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": { @@ -254,7 +254,6 @@ request is ignored and the `result` element in the response returns `noop`: "result": "noop" } -------------------------------------------------- -// TESTRESPONSE You can disable this behavior by setting `"detect_noop": false`: diff --git a/docs/reference/ilm/apis/delete-lifecycle.asciidoc b/docs/reference/ilm/apis/delete-lifecycle.asciidoc index 8d959049fb1b..db0faebf3112 100644 --- a/docs/reference/ilm/apis/delete-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/delete-lifecycle.asciidoc @@ -76,11 +76,9 @@ DELETE _ilm/policy/my_policy When the policy is successfully deleted, you receive the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE diff --git a/docs/reference/ilm/apis/get-status.asciidoc b/docs/reference/ilm/apis/get-status.asciidoc index 719bafe536c8..2f8a1ed43af9 100644 --- a/docs/reference/ilm/apis/get-status.asciidoc +++ b/docs/reference/ilm/apis/get-status.asciidoc @@ -42,11 +42,9 @@ GET _ilm/status If the request succeeds, the body of the response shows the operation mode: -[source,js] +[source,console-result] -------------------------------------------------- { "operation_mode": "RUNNING" } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE diff --git a/docs/reference/ilm/apis/move-to-step.asciidoc b/docs/reference/ilm/apis/move-to-step.asciidoc index f5da5370b1a3..7d132290cced 100644 --- a/docs/reference/ilm/apis/move-to-step.asciidoc +++ b/docs/reference/ilm/apis/move-to-step.asciidoc @@ -107,14 +107,12 @@ POST _ilm/move/my_index If the request succeeds, you receive the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE The request will fail if the index is not in the `new` phase as specified by the `current_step`. diff --git a/docs/reference/ilm/apis/put-lifecycle.asciidoc b/docs/reference/ilm/apis/put-lifecycle.asciidoc index 7e7c159c20dd..4dde6b7c05a9 100644 --- a/docs/reference/ilm/apis/put-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/put-lifecycle.asciidoc @@ -70,11 +70,10 @@ PUT _ilm/policy/my_policy // TEST If the request succeeds, you receive the following result: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// CONSOLE -// TESTRESPONSE diff --git a/docs/reference/ilm/apis/remove-policy-from-index.asciidoc b/docs/reference/ilm/apis/remove-policy-from-index.asciidoc index ef4e9e015726..6b97d014d98d 100644 --- a/docs/reference/ilm/apis/remove-policy-from-index.asciidoc +++ b/docs/reference/ilm/apis/remove-policy-from-index.asciidoc @@ -83,12 +83,10 @@ POST my_index/_ilm/remove If the request succeeds, you receive the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "has_failures" : false, "failed_indexes" : [] } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE diff --git a/docs/reference/ilm/apis/start.asciidoc b/docs/reference/ilm/apis/start.asciidoc index 7d3c923fc938..b8f1ba656c40 100644 --- a/docs/reference/ilm/apis/start.asciidoc +++ b/docs/reference/ilm/apis/start.asciidoc @@ -77,11 +77,9 @@ POST _ilm/start If the request succeeds, you receive the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE diff --git a/docs/reference/ilm/apis/stop.asciidoc b/docs/reference/ilm/apis/stop.asciidoc index 85acc059d2d1..aaf9f74c933a 100644 --- a/docs/reference/ilm/apis/stop.asciidoc +++ b/docs/reference/ilm/apis/stop.asciidoc @@ -80,14 +80,12 @@ POST _ilm/stop If the request does not encounter errors, you receive the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/ilm/start-stop-ilm.asciidoc b/docs/reference/ilm/start-stop-ilm.asciidoc index fd1ab654ab6c..6e2629b60ea2 100644 --- a/docs/reference/ilm/start-stop-ilm.asciidoc +++ b/docs/reference/ilm/start-stop-ilm.asciidoc @@ -61,14 +61,12 @@ GET _ilm/status If the request does not encounter errors, you receive the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "operation_mode": "RUNNING" } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE The operating modes of ILM: @@ -107,14 +105,12 @@ GET _ilm/status // TEST[continued] //// -[source,js] +[source,console-result] -------------------------------------------------- { "operation_mode": "STOPPING" } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE The ILM service will then, asynchronously, run all policies to a point where it is safe to stop. After ILM verifies that it is safe, it will @@ -131,14 +127,12 @@ GET _ilm/status // TEST[continued] //// -[source,js] +[source,console-result] -------------------------------------------------- { "operation_mode": "STOPPED" } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE [float] === Starting ILM @@ -165,11 +159,9 @@ GET _ilm/status The Start API will send a request to the ILM service to immediately begin normal operations. -[source,js] +[source,console-result] -------------------------------------------------- { "operation_mode": "RUNNING" } -------------------------------------------------- -// CONSOLE -// TESTRESPONSE diff --git a/docs/reference/indices/analyze.asciidoc b/docs/reference/indices/analyze.asciidoc index b48243c6b59f..75f0fb618ef0 100644 --- a/docs/reference/indices/analyze.asciidoc +++ b/docs/reference/indices/analyze.asciidoc @@ -308,7 +308,7 @@ GET /_analyze The request returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "detail" : { @@ -351,7 +351,7 @@ The request returns the following result: } } -------------------------------------------------- -// TESTRESPONSE + <1> Output only "keyword" attribute, since specify "attributes" in the request. [[tokens-limit-settings]] diff --git a/docs/reference/indices/close.asciidoc b/docs/reference/indices/close.asciidoc index 71a5f8b63420..d6997eca7450 100644 --- a/docs/reference/indices/close.asciidoc +++ b/docs/reference/indices/close.asciidoc @@ -70,7 +70,7 @@ POST /my_index/_close The API returns following response: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true, @@ -82,4 +82,3 @@ The API returns following response: } } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/indices/create-index.asciidoc b/docs/reference/indices/create-index.asciidoc index a40327be2120..144b3b00c143 100644 --- a/docs/reference/indices/create-index.asciidoc +++ b/docs/reference/indices/create-index.asciidoc @@ -169,7 +169,7 @@ By default, index creation will only return a response to the client when the pr each shard have been started, or the request times out. The index creation response will indicate what happened: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true, @@ -177,7 +177,6 @@ what happened: "index": "test" } -------------------------------------------------- -// TESTRESPONSE `acknowledged` indicates whether the index was successfully created in the cluster, while `shards_acknowledged` indicates whether the requisite number of shard copies were started for diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index 9a66eb65d4d5..ebc4a5db64cb 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -99,7 +99,7 @@ GET /logs_20302801/_alias/* The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "logs_20302801" : { @@ -117,7 +117,6 @@ The API returns the following response: } } -------------------------------------------------- -// TESTRESPONSE [[get-alias-api-named-ex]] @@ -134,7 +133,7 @@ GET /_alias/2030 The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "logs_20302801" : { @@ -150,7 +149,7 @@ The API returns the following response: } } -------------------------------------------------- -// TESTRESPONSE + [[get-alias-api-wildcard-ex]] ===== Get aliases based on a wildcard @@ -166,7 +165,7 @@ GET /_alias/20* The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "logs_20302801" : { @@ -182,4 +181,3 @@ The API returns the following response: } } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/indices/get-field-mapping.asciidoc b/docs/reference/indices/get-field-mapping.asciidoc index 9397052b2871..40ee958b22c5 100644 --- a/docs/reference/indices/get-field-mapping.asciidoc +++ b/docs/reference/indices/get-field-mapping.asciidoc @@ -94,7 +94,7 @@ GET publications/_mapping/field/title The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "publications": { @@ -111,7 +111,6 @@ The API returns the following response: } } -------------------------------------------------- -// TESTRESPONSE [[get-field-mapping-api-specific-fields-ex]] ===== Specifying fields @@ -129,7 +128,7 @@ GET publications/_mapping/field/author.id,abstract,name returns: -[source,js] +[source,console-result] -------------------------------------------------- { "publications": { @@ -154,7 +153,6 @@ returns: } } -------------------------------------------------- -// TESTRESPONSE The get field mapping API also supports wildcard notation. @@ -167,7 +165,7 @@ GET publications/_mapping/field/a* returns: -[source,js] +[source,console-result] -------------------------------------------------- { "publications": { @@ -200,7 +198,6 @@ returns: } } -------------------------------------------------- -// TESTRESPONSE [[get-field-mapping-api-multi-index-ex]] ===== Multiple indices and fields diff --git a/docs/reference/indices/open-close.asciidoc b/docs/reference/indices/open-close.asciidoc index 49d73060458f..2e43962c8fc8 100644 --- a/docs/reference/indices/open-close.asciidoc +++ b/docs/reference/indices/open-close.asciidoc @@ -107,11 +107,10 @@ POST /my_index/_open The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true, "shards_acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/indices/recovery.asciidoc b/docs/reference/indices/recovery.asciidoc index d6f7fafed2c5..354ce0c6fc1e 100644 --- a/docs/reference/indices/recovery.asciidoc +++ b/docs/reference/indices/recovery.asciidoc @@ -43,7 +43,7 @@ POST /_snapshot/my_repository/snap_1/_restore?wait_for_completion=true -------------------------------------------------- // CONSOLE -[source,js] +[source,console-result] -------------------------------------------------- { "snapshot": { @@ -59,7 +59,6 @@ POST /_snapshot/my_repository/snap_1/_restore?wait_for_completion=true } } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/indices/rollover-index.asciidoc b/docs/reference/indices/rollover-index.asciidoc index 17336fbc4585..94295632c2fc 100644 --- a/docs/reference/indices/rollover-index.asciidoc +++ b/docs/reference/indices/rollover-index.asciidoc @@ -66,7 +66,7 @@ POST /logs_write/_rollover <2> The above request might return the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true, @@ -82,7 +82,7 @@ The above request might return the following response: } } -------------------------------------------------- -// TESTRESPONSE + <1> Whether the index was rolled over. <2> Whether the rollover was dry run. <3> The result of each condition. @@ -162,7 +162,7 @@ GET _alias // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "logs-2016.10.31-000002": { @@ -175,7 +175,6 @@ GET _alias } } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// @@ -305,7 +304,7 @@ PUT logs/_doc/2 <2> <1> configures `my_logs_index` as the write index for the `logs` alias <2> newly indexed documents against the `logs` alias will write to the new index -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "my_logs_index-000002", @@ -322,7 +321,6 @@ PUT logs/_doc/2 <2> "_primary_term" : 1 } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// [source,js] @@ -336,7 +334,7 @@ GET _alias After the rollover, the alias metadata for the two indices will have the `is_write_index` setting reflect each index's role, with the newly created index as the write index. -[source,js] +[source,console-result] -------------------------------------------------- { "my_logs_index-000002": { @@ -351,4 +349,3 @@ reflect each index's role, with the newly created index as the write index. } } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/ingest/apis/delete-pipeline.asciidoc b/docs/reference/ingest/apis/delete-pipeline.asciidoc index 468a5bf5d686..bf7a667503ba 100644 --- a/docs/reference/ingest/apis/delete-pipeline.asciidoc +++ b/docs/reference/ingest/apis/delete-pipeline.asciidoc @@ -34,13 +34,12 @@ DELETE _ingest/pipeline/my-pipeline-id ////////////////////////// -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// TESTRESPONSE [source,js] -------------------------------------------------- @@ -68,12 +67,11 @@ DELETE _ingest/pipeline/* ////////////////////////// -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/ingest/apis/get-pipeline.asciidoc b/docs/reference/ingest/apis/get-pipeline.asciidoc index 2f798f9fbd76..4d68315cb0d4 100644 --- a/docs/reference/ingest/apis/get-pipeline.asciidoc +++ b/docs/reference/ingest/apis/get-pipeline.asciidoc @@ -33,7 +33,7 @@ GET _ingest/pipeline/my-pipeline-id Example response: -[source,js] +[source,console-result] -------------------------------------------------- { "my-pipeline-id" : { @@ -49,7 +49,6 @@ Example response: } } -------------------------------------------------- -// TESTRESPONSE For each returned pipeline, the source and the version are returned. The version is useful for knowing which version of the pipeline the node has. @@ -96,7 +95,7 @@ GET /_ingest/pipeline/my-pipeline-id?filter_path=*.version This should give a small response that makes it both easy and inexpensive to parse: -[source,js] +[source,console-result] -------------------------------------------------- { "my-pipeline-id" : { @@ -104,7 +103,6 @@ This should give a small response that makes it both easy and inexpensive to par } } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// @@ -115,12 +113,11 @@ DELETE /_ingest/pipeline/my-pipeline-id // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/ingest/apis/put-pipeline.asciidoc b/docs/reference/ingest/apis/put-pipeline.asciidoc index e27b4d1b50ee..9d2ba39dbaf5 100644 --- a/docs/reference/ingest/apis/put-pipeline.asciidoc +++ b/docs/reference/ingest/apis/put-pipeline.asciidoc @@ -29,13 +29,12 @@ DELETE /_ingest/pipeline/my-pipeline-id // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// diff --git a/docs/reference/ingest/ingest-node.asciidoc b/docs/reference/ingest/ingest-node.asciidoc index e0f5254f8f0c..b9afd2727a9a 100644 --- a/docs/reference/ingest/ingest-node.asciidoc +++ b/docs/reference/ingest/ingest-node.asciidoc @@ -196,7 +196,7 @@ POST test/_doc/1?pipeline=drop_guests_network Results in nothing indexed since the conditional evaluated to `true`. -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "test", @@ -211,7 +211,6 @@ Results in nothing indexed since the conditional evaluated to `true`. } } -------------------------------------------------- -// TESTRESPONSE [[ingest-conditional-nullcheck]] diff --git a/docs/reference/mapping.asciidoc b/docs/reference/mapping.asciidoc index de6951f96708..2cac5be9ccf3 100644 --- a/docs/reference/mapping.asciidoc +++ b/docs/reference/mapping.asciidoc @@ -196,7 +196,7 @@ GET /my-index/_mapping The API returns the following response: -[source,js] +[source,console-result] ---- { "my-index" : { @@ -220,7 +220,6 @@ The API returns the following response: } } ---- -// TESTRESPONSE [float] @@ -244,7 +243,7 @@ GET /my-index/_mapping/field/employee-id The API returns the following response: -[source,js] +[source,console-result] ---- { "my-index" : { @@ -263,7 +262,6 @@ The API returns the following response: } ---- -// TESTRESPONSE -- diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index 4c026c764d6f..e6362ea76811 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -494,7 +494,7 @@ GET index/_mappings?include_type_name=false The above call returns -[source,js] +[source,console-result] -------------------------------------------------- { "index": { @@ -511,7 +511,7 @@ The above call returns } } -------------------------------------------------- -// TESTRESPONSE + <1> Mappings are included directly under the `mappings` key, without a type name. [float] @@ -529,7 +529,7 @@ PUT index/_doc/1 -------------------------------------------------- // CONSOLE -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "index", @@ -546,7 +546,6 @@ PUT index/_doc/1 "_primary_term": 1 } -------------------------------------------------- -// TESTRESPONSE Similarly, the `get` and `delete` APIs use the path `{index}/_doc/{id}`: @@ -624,7 +623,7 @@ GET index/_doc/1 -------------------------------------------------- // CONSOLE -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "index", @@ -639,7 +638,6 @@ GET index/_doc/1 } } -------------------------------------------------- -// TESTRESPONSE [float] ==== Index templates diff --git a/docs/reference/mapping/types/percolator.asciidoc b/docs/reference/mapping/types/percolator.asciidoc index cdc10bcaa036..4c75f9c3068a 100644 --- a/docs/reference/mapping/types/percolator.asciidoc +++ b/docs/reference/mapping/types/percolator.asciidoc @@ -311,7 +311,7 @@ POST /test_index/_analyze This results the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "tokens": [ @@ -332,7 +332,6 @@ This results the following response: ] } -------------------------------------------------- -// TESTRESPONSE All the tokens in the returned order need to replace the query text in the percolator query: diff --git a/docs/reference/migration/migrate_8_0/snapshots.asciidoc b/docs/reference/migration/migrate_8_0/snapshots.asciidoc index 086d70a96bbe..6bf8eba824d5 100644 --- a/docs/reference/migration/migrate_8_0/snapshots.asciidoc +++ b/docs/reference/migration/migrate_8_0/snapshots.asciidoc @@ -25,7 +25,7 @@ GET _snapshot/repo1/snap1 produces the following response -[source,js] +[source,console-result] ----------------------------------- { "responses": [ @@ -57,8 +57,7 @@ produces the following response ] } ----------------------------------- -// TESTRESPONSE -// TEST[skip:no repo and snapshots are created] +// TESTRESPONSE[skip:no repo and snapshots are created] See <> for more information. diff --git a/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc index 69e1fbead790..81afe7d81e75 100644 --- a/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc @@ -106,10 +106,10 @@ POST _ml/anomaly_detectors/total-requests/_close // TEST[skip:setup:server_metrics_openjob] When the job is closed, you receive the following results: -[source,js] + +[source,console-result] ---- { "closed": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc index 7eb18772beb5..09df1f55a160 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc @@ -46,11 +46,10 @@ DELETE _ml/calendars/planned-outages/jobs/total-requests When the job is removed from the calendar, you receive the following results: -[source,js] +[source,console-result] ---- { "calendar_id": "planned-outages", "job_ids": [] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc index c38a15f1d11e..b9b1e24181e3 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc @@ -45,10 +45,10 @@ DELETE _ml/calendars/planned-outages // TEST[skip:setup:calendar_outages] When the calendar is deleted, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc index 92a9a9dc8217..d08b763836f3 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc @@ -50,10 +50,10 @@ DELETE _ml/datafeeds/datafeed-total-requests // TEST[skip:setup:server_metrics_datafeed] When the {dfeed} is deleted, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc index ada9ec1c8c34..2a2008d6a690 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc @@ -40,10 +40,10 @@ DELETE _ml/_delete_expired_data // TEST When the expired data is deleted, you receive the following response: -[source,js] + +[source,console-result] ---- { "deleted": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc index 75fe2533b88c..4943c96d45b2 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc @@ -46,10 +46,10 @@ DELETE _ml/filters/safe_domains // TEST[skip:setup:ml_filter_safe_domains] When the filter is deleted, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc index 506e224f9432..82d12ef83a3c 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc @@ -67,13 +67,13 @@ DELETE _ml/anomaly_detectors/total-requests // TEST[skip:setup:server_metrics_job] When the job is deleted, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE In the next example we delete the `total-requests` job asynchronously: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc index 62a223c4447a..58beffdd9761 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc @@ -49,10 +49,10 @@ DELETE _ml/anomaly_detectors/farequote/model_snapshots/1491948163 // TEST[skip:todo] When the snapshot is deleted, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc index 47b961b1d2a4..7d73cb96ad86 100644 --- a/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc @@ -106,11 +106,11 @@ POST _ml/anomaly_detectors/total-requests/_flush // TEST[skip:setup:server_metrics_openjob] When the operation succeeds, you receive the following results: -[source,js] + +[source,console-result] ---- { "flushed": true, "last_finalized_bucket_end": 1514804400000 } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc index bc4ac0f24a50..88b48752b68f 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc @@ -76,7 +76,8 @@ GET _ml/calendars/planned-outages // TEST[skip:setup:calendar_outages_addjob] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count": 1, @@ -90,4 +91,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc index ba2036bea5dc..b45ed11eb00b 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc @@ -74,7 +74,8 @@ GET _ml/filters/safe_domains // TEST[skip:setup:ml_filter_safe_domains] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count": 1, @@ -90,4 +91,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc index 8761bf834284..d73d8f93d392 100644 --- a/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc @@ -63,10 +63,10 @@ POST _ml/anomaly_detectors/total-requests/_open // TEST[skip:setup:server_metrics_job] When the job opens, you receive the following results: -[source,js] + +[source,console-result] ---- { "opened": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc b/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc index b247297456a3..39b91a00ba84 100644 --- a/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc @@ -76,7 +76,7 @@ POST _ml/calendars/planned-outages/events The API returns the following results: -[source,js] +[source,console-result] ---- { "events": [ @@ -101,4 +101,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc index dcf96d2297bb..7fe46d65f4df 100644 --- a/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc @@ -56,7 +56,8 @@ GET _ml/datafeeds/datafeed-farequote/_preview // TEST[skip:setup:farequote_datafeed] The data that is returned for this example is as follows: -[source,js] + +[source,console-result] ---- [ { @@ -79,4 +80,3 @@ The data that is returned for this example is as follows: } ] ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc index 927829e44e9c..d2ad602fcd71 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc @@ -42,7 +42,7 @@ PUT _ml/calendars/planned-outages/jobs/total-requests The API returns the following results: -[source,js] +[source,console-result] ---- { "calendar_id": "planned-outages", @@ -51,4 +51,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc index cefcb4e04118..4e9e0fd194c2 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc @@ -49,11 +49,11 @@ PUT _ml/calendars/planned-outages // TEST[skip:need-license] When the calendar is created, you receive the following results: -[source,js] + +[source,console-result] ---- { "calendar_id": "planned-outages", "job_ids": [] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc index da080ef700f8..38fa9424a4a7 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc @@ -61,7 +61,8 @@ PUT _ml/filters/safe_domains // TEST[skip:need-licence] When the filter is created, you receive the following response: -[source,js] + +[source,console-result] ---- { "filter_id": "safe_domains", @@ -69,4 +70,3 @@ When the filter is created, you receive the following response: "items": ["*.google.com", "wikipedia.org"] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/set-upgrade-mode.asciidoc b/docs/reference/ml/anomaly-detection/apis/set-upgrade-mode.asciidoc index 763ebb92d8e3..9e75078e1a4f 100644 --- a/docs/reference/ml/anomaly-detection/apis/set-upgrade-mode.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/set-upgrade-mode.asciidoc @@ -83,13 +83,12 @@ POST _ml/set_upgrade_mode?enabled=true&timeout=10m When the call is successful, an acknowledged response is returned. For example: -[source,js] +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE The acknowledged response will only be returned once all {ml} jobs and {dfeeds} have finished writing to the {ml} internal indices. This means it is safe to reindex those diff --git a/docs/reference/ml/anomaly-detection/apis/start-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/start-datafeed.asciidoc index 1d2cfb21e1ce..159656901137 100644 --- a/docs/reference/ml/anomaly-detection/apis/start-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/start-datafeed.asciidoc @@ -107,10 +107,10 @@ POST _ml/datafeeds/datafeed-total-requests/_start // TEST[skip:setup:server_metrics_openjob] When the {dfeed} starts, you receive the following results: -[source,js] + +[source,console-result] ---- { "started": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/stop-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/stop-datafeed.asciidoc index f849f0faf758..a18894d78679 100644 --- a/docs/reference/ml/anomaly-detection/apis/stop-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/stop-datafeed.asciidoc @@ -94,10 +94,9 @@ POST _ml/datafeeds/datafeed-total-requests/_stop When the {dfeed} stops, you receive the following results: -[source,js] +[source,console-result] ---- { "stopped": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/update-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/update-filter.asciidoc index 51897ce061a2..d7fe0122e904 100644 --- a/docs/reference/ml/anomaly-detection/apis/update-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/update-filter.asciidoc @@ -58,7 +58,7 @@ POST _ml/filters/safe_domains/_update The API returns the following results: -[source,js] +[source,console-result] ---- { "filter_id": "safe_domains", @@ -66,4 +66,3 @@ The API returns the following results: "items": ["*.google.com", "*.myorg.com"] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/validate-detector.asciidoc b/docs/reference/ml/anomaly-detection/apis/validate-detector.asciidoc index b1efc4dcb3b4..e4116d04f91e 100644 --- a/docs/reference/ml/anomaly-detection/apis/validate-detector.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/validate-detector.asciidoc @@ -50,10 +50,10 @@ POST _ml/anomaly_detectors/_validate/detector // TEST[skip:needs-licence] When the validation completes, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/anomaly-detection/apis/validate-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/validate-job.asciidoc index 0d1ece9cf82c..2517205ad859 100644 --- a/docs/reference/ml/anomaly-detection/apis/validate-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/validate-job.asciidoc @@ -61,10 +61,10 @@ POST _ml/anomaly_detectors/_validate // TEST[skip:needs-licence] When the validation is complete, you receive the following results: -[source,js] + +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/df-analytics/apis/delete-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/delete-dfanalytics.asciidoc index 0775bc889bee..d848271eea01 100644 --- a/docs/reference/ml/df-analytics/apis/delete-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/delete-dfanalytics.asciidoc @@ -43,10 +43,9 @@ DELETE _ml/data_frame/analytics/loganalytics The API returns the following result: -[source,js] +[source,console-result] ---- { "acknowledged" : true } ---- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/reference/ml/df-analytics/apis/estimate-memory-usage-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/estimate-memory-usage-dfanalytics.asciidoc index 4393a3365fe7..f91afb2196f3 100644 --- a/docs/reference/ml/df-analytics/apis/estimate-memory-usage-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/estimate-memory-usage-dfanalytics.asciidoc @@ -73,11 +73,10 @@ POST _ml/data_frame/analytics/_estimate_memory_usage The API returns the following results: -[source,js] +[source,console-result] ---- { "expected_memory_without_disk": "128MB", "expected_memory_with_disk": "32MB" } ---- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/reference/ml/df-analytics/apis/evaluate-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/evaluate-dfanalytics.asciidoc index 92729c3b0e2c..e4d02c3a892a 100644 --- a/docs/reference/ml/df-analytics/apis/evaluate-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/evaluate-dfanalytics.asciidoc @@ -92,7 +92,7 @@ POST _ml/data_frame/_evaluate The API returns the following results: -[source,js] +[source,console-result] ---- { "binary_soft_classification": { @@ -132,4 +132,3 @@ The API returns the following results: } } ---- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc b/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc index b1a8d4c194b6..77051d786942 100644 --- a/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc +++ b/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc @@ -92,7 +92,7 @@ GET _ml/data_frame/analytics/loganalytics/_stats The API returns the following results: -[source,js] +[source,console-result] ---- { "count": 1, @@ -122,4 +122,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/df-analytics/apis/get-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/get-dfanalytics.asciidoc index 09380a340310..fe79b4f52451 100644 --- a/docs/reference/ml/df-analytics/apis/get-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/get-dfanalytics.asciidoc @@ -99,7 +99,7 @@ GET _ml/data_frame/analytics/loganalytics The API returns the following results: -[source,js] +[source,console-result] ---- { "count": 1, @@ -126,4 +126,3 @@ The API returns the following results: ] } ---- -// TESTRESPONSE diff --git a/docs/reference/ml/df-analytics/apis/start-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/start-dfanalytics.asciidoc index 39ce13530285..8ca3dfaaafd2 100644 --- a/docs/reference/ml/df-analytics/apis/start-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/start-dfanalytics.asciidoc @@ -55,10 +55,9 @@ POST _ml/data_frame/analytics/loganalytics/_start When the {dfanalytics-job} starts, you receive the following results: -[source,js] +[source,console-result] ---- { "acknowledged" : true } ---- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/reference/ml/df-analytics/apis/stop-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/stop-dfanalytics.asciidoc index 7e5ee42234b7..87700245e583 100644 --- a/docs/reference/ml/df-analytics/apis/stop-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/stop-dfanalytics.asciidoc @@ -77,10 +77,9 @@ POST _ml/data_frame/analytics/loganalytics/_stop When the {dfanalytics-job} stops, you receive the following results: -[source,js] +[source,console-result] ---- { "stopped" : true } ---- -// TESTRESPONSE \ No newline at end of file diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index 7383dd5d1929..72921f05f763 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -114,7 +114,7 @@ GET /_snapshot/my_backup which returns: -[source,js] +[source,console-result] ----------------------------------- { "my_backup": { @@ -125,7 +125,6 @@ which returns: } } ----------------------------------- -// TESTRESPONSE To retrieve information about multiple repositories, specify a comma-delimited list of repositories. You can also use the * wildcard when @@ -349,7 +348,7 @@ POST /_snapshot/my_repository/_cleanup The response to a cleanup request looks as follows: -[source,js] +[source,console-result] -------------------------------------------------- { "results": { @@ -358,7 +357,6 @@ The response to a cleanup request looks as follows: } } -------------------------------------------------- -// TESTRESPONSE Depending on the concrete repository implementation the numbers shown for bytes free as well as the number of blobs removed will either be an approximation or an exact result. Any non-zero value for the number of blobs removed implies that unreferenced blobs were found and @@ -704,7 +702,7 @@ GET /_snapshot/my_backup/snapshot_1/_status The output looks similar to the following: -[source,js] +[source,console-result] -------------------------------------------------- { "snapshots": [ @@ -742,7 +740,6 @@ The output looks similar to the following: ] } -------------------------------------------------- -// TESTRESPONSE The output is composed of different sections. The `stats` sub-object provides details on the number and size of files that were snapshotted. As snapshots are incremental, copying only the Lucene segments that are not already in the repository, diff --git a/docs/reference/query-dsl/percolate-query.asciidoc b/docs/reference/query-dsl/percolate-query.asciidoc index 314c30bfc1a9..b32a7328acba 100644 --- a/docs/reference/query-dsl/percolate-query.asciidoc +++ b/docs/reference/query-dsl/percolate-query.asciidoc @@ -297,7 +297,7 @@ PUT /my-index/_doc/2 // TEST[continued] Index response: -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "my-index", @@ -314,7 +314,6 @@ Index response: "_primary_term" : 1 } -------------------------------------------------- -// TESTRESPONSE Percolating an existing document, using the index response as basis to build to new search request: diff --git a/docs/reference/rollup/apis/delete-job.asciidoc b/docs/reference/rollup/apis/delete-job.asciidoc index 85246d518348..1a36a146da6f 100644 --- a/docs/reference/rollup/apis/delete-job.asciidoc +++ b/docs/reference/rollup/apis/delete-job.asciidoc @@ -85,10 +85,9 @@ DELETE _rollup/job/sensor Which will return the response: -[source,js] +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE diff --git a/docs/reference/rollup/apis/get-job.asciidoc b/docs/reference/rollup/apis/get-job.asciidoc index e173cfa29942..def1dd9c71e5 100644 --- a/docs/reference/rollup/apis/get-job.asciidoc +++ b/docs/reference/rollup/apis/get-job.asciidoc @@ -88,7 +88,7 @@ GET _rollup/job/sensor The API yields the following response: -[source,js] +[source,console-result] ---- { "jobs" : [ @@ -149,7 +149,6 @@ The API yields the following response: ] } ---- -// TESTRESPONSE The `jobs` array contains a single job (`id: sensor`) since we requested a single job in the endpoint's URL. If we add another job, we can see how multi-job responses are handled: diff --git a/docs/reference/rollup/apis/put-job.asciidoc b/docs/reference/rollup/apis/put-job.asciidoc index 9f473d7f2023..7a0946436871 100644 --- a/docs/reference/rollup/apis/put-job.asciidoc +++ b/docs/reference/rollup/apis/put-job.asciidoc @@ -105,10 +105,9 @@ PUT _rollup/job/sensor When the job is created, you receive the following results: -[source,js] +[source,console-result] ---- { "acknowledged": true } ----- -// TESTRESPONSE \ No newline at end of file +---- \ No newline at end of file diff --git a/docs/reference/rollup/apis/rollup-caps.asciidoc b/docs/reference/rollup/apis/rollup-caps.asciidoc index e50806f3c1e0..3d055bf90def 100644 --- a/docs/reference/rollup/apis/rollup-caps.asciidoc +++ b/docs/reference/rollup/apis/rollup-caps.asciidoc @@ -95,7 +95,7 @@ GET _rollup/data/sensor-* Which will yield the following response: -[source,js] +[source,console-result] ---- { "sensor-*" : { @@ -140,7 +140,6 @@ Which will yield the following response: } } ---- -// TESTRESPONSE The response that is returned contains information that is similar to the original Rollup configuration, but formatted differently. First, there are some house-keeping details: the Rollup job's ID, the index that holds the rolled data, @@ -172,13 +171,12 @@ GET _rollup/data/sensor-1 // CONSOLE // TEST[continued] -[source,js] +[source,console-result] ---- { } ---- -// TESTRESPONSE Why is this? The original rollup job was configured against a specific index pattern (`sensor-*`) not a concrete index (`sensor-1`). So while the index belongs to the pattern, the rollup job is only valid across the entirety of the pattern diff --git a/docs/reference/rollup/apis/rollup-index-caps.asciidoc b/docs/reference/rollup/apis/rollup-index-caps.asciidoc index a0697ba70326..60776d8a9fd4 100644 --- a/docs/reference/rollup/apis/rollup-index-caps.asciidoc +++ b/docs/reference/rollup/apis/rollup-index-caps.asciidoc @@ -88,7 +88,7 @@ GET /sensor_rollup/_rollup/data Note how we are requesting the concrete rollup index name (`sensor_rollup`) as the first part of the URL. This will yield the following response: -[source,js] +[source,console-result] ---- { "sensor_rollup" : { @@ -133,7 +133,6 @@ This will yield the following response: } } ---- -// TESTRESPONSE The response that is returned contains information that is similar to the original Rollup configuration, but formatted diff --git a/docs/reference/rollup/apis/start-job.asciidoc b/docs/reference/rollup/apis/start-job.asciidoc index 29d413c63597..27183df64fc5 100644 --- a/docs/reference/rollup/apis/start-job.asciidoc +++ b/docs/reference/rollup/apis/start-job.asciidoc @@ -56,10 +56,9 @@ POST _rollup/job/sensor/_start Which will return the response: -[source,js] +[source,console-result] ---- { "started": true } ----- -// TESTRESPONSE \ No newline at end of file +---- \ No newline at end of file diff --git a/docs/reference/rollup/rollup-getting-started.asciidoc b/docs/reference/rollup/rollup-getting-started.asciidoc index 3e2d694464ee..7dcc36d29a9d 100644 --- a/docs/reference/rollup/rollup-getting-started.asciidoc +++ b/docs/reference/rollup/rollup-getting-started.asciidoc @@ -96,13 +96,12 @@ For more details about the job syntax, see <>. After you execute the above command and create the job, you'll receive the following response: -[source,js] +[source,console-result] ---- { "acknowledged": true } ---- -// TESTRESPONSE [float] === Starting the job diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index 93262a12f8ac..fa0c4d64d2ea 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -31,7 +31,7 @@ the <> works Both examples above do the same thing, which is count the number of tweets from the `twitter` index for a certain user. The result is: -[source,js] +[source,console-result] -------------------------------------------------- { "count" : 1, @@ -43,7 +43,6 @@ tweets from the `twitter` index for a certain user. The result is: } } -------------------------------------------------- -// TESTRESPONSE The query is optional, and when not provided, it will use `match_all` to count all the docs. diff --git a/docs/reference/search/explain.asciidoc b/docs/reference/search/explain.asciidoc index 061bae555bcf..f1710679805a 100644 --- a/docs/reference/search/explain.asciidoc +++ b/docs/reference/search/explain.asciidoc @@ -26,7 +26,7 @@ GET /twitter/_explain/0 This will yield the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "_index":"twitter", @@ -99,7 +99,6 @@ This will yield the following result: } } -------------------------------------------------- -// TESTRESPONSE There is also a simpler way of specifying the query via the `q` parameter. The specified `q` parameter value is then parsed as if the diff --git a/docs/reference/search/request/track-total-hits.asciidoc b/docs/reference/search/request/track-total-hits.asciidoc index 1e9ede3ae94d..24b1f1f39acf 100644 --- a/docs/reference/search/request/track-total-hits.asciidoc +++ b/docs/reference/search/request/track-total-hits.asciidoc @@ -118,7 +118,7 @@ If the total number of his that match the query is greater than the value set in `track_total_hits`, the total hits in the response will indicate that the returned value is a lower bound: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... @@ -132,8 +132,7 @@ will indicate that the returned value is a lower bound: } } -------------------------------------------------- -// TESTRESPONSE -// TEST[skip:response is already tested in the previous snippet] +// TESTRESPONSE[skip:response is already tested in the previous snippet] <1> There are at least 100 documents that match the query <2> This is a lower bound (`"gte"`). diff --git a/docs/reference/search/search-template.asciidoc b/docs/reference/search/search-template.asciidoc index fb6fe4d41a25..5a8b7f4e2519 100644 --- a/docs/reference/search/search-template.asciidoc +++ b/docs/reference/search/search-template.asciidoc @@ -131,13 +131,12 @@ POST _scripts/ The API returns the following result if the template has been successfully created: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged" : true } -------------------------------------------------- -// TESTRESPONSE ////////////////////////// @@ -153,7 +152,7 @@ GET _scripts/ The API returns the following result: -[source,js] +[source,console-result] ------------------------------------------ { "script" : { @@ -167,7 +166,6 @@ The API returns the following result: "found": true } ------------------------------------------ -// TESTRESPONSE This template can be deleted by calling @@ -180,7 +178,6 @@ DELETE _scripts/ // TEST[continued] - [[use-registered-templates]] ===== Using a stored search template @@ -224,7 +221,7 @@ GET _render/template The API returns the rendered template: -[source,js] +[source,console-result] ------------------------------------------ { "template_output": { @@ -239,7 +236,7 @@ The API returns the rendered template: } } ------------------------------------------ -// TESTRESPONSE + <1> `status` array has been populated with values from the `params` object. @@ -649,7 +646,7 @@ GET _render/template The previous query will be rendered as: -[source,js] +[source,console-result] ------------------------------------------ { "template_output" : { @@ -661,7 +658,7 @@ The previous query will be rendered as: } } ------------------------------------------ -// TESTRESPONSE + [[multi-search-template]] === Multi Search Template diff --git a/docs/reference/search/validate.asciidoc b/docs/reference/search/validate.asciidoc index 94853c49e323..4f6d948fd3ea 100644 --- a/docs/reference/search/validate.asciidoc +++ b/docs/reference/search/validate.asciidoc @@ -23,11 +23,11 @@ GET twitter/_validate/query?q=user:foo The response contains `valid:true`: -[source,js] +[source,console-result] -------------------------------------------------- {"valid":true,"_shards":{"total":1,"successful":1,"failed":0}} -------------------------------------------------- -// TESTRESPONSE + [float] === Request Parameters @@ -95,11 +95,10 @@ GET twitter/_validate/query } -------------------------------------------------- -[source,js] +[source,console-result] -------------------------------------------------- {"valid":false,"_shards":{"total":1,"successful":1,"failed":0}} -------------------------------------------------- -// TESTRESPONSE An `explain` parameter can be specified to get more detailed information about why a query failed: @@ -161,7 +160,7 @@ GET twitter/_validate/query?rewrite=true Response: -[source,js] +[source,console-result] -------------------------------------------------- { "valid": true, @@ -179,7 +178,6 @@ Response: ] } -------------------------------------------------- -// TESTRESPONSE By default, the request is executed on a single shard only, which is randomly selected. The detailed explanation of the query may depend on which shard is @@ -206,7 +204,7 @@ GET twitter/_validate/query?rewrite=true&all_shards=true Response: -[source,js] +[source,console-result] -------------------------------------------------- { "valid": true, @@ -225,4 +223,3 @@ Response: ] } -------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/sql/endpoints/rest.asciidoc b/docs/reference/sql/endpoints/rest.asciidoc index 28e4d294cbdd..650b834c4620 100644 --- a/docs/reference/sql/endpoints/rest.asciidoc +++ b/docs/reference/sql/endpoints/rest.asciidoc @@ -322,14 +322,12 @@ POST /_sql/close Which will like return the -[source,js] +[source,console-result] -------------------------------------------------- { "succeeded" : true } -------------------------------------------------- -// TESTRESPONSE - [[sql-rest-filtering]] diff --git a/docs/reference/sql/endpoints/translate.asciidoc b/docs/reference/sql/endpoints/translate.asciidoc index 403a8011ad4e..b146d0e542b0 100644 --- a/docs/reference/sql/endpoints/translate.asciidoc +++ b/docs/reference/sql/endpoints/translate.asciidoc @@ -18,7 +18,7 @@ POST /_sql/translate Which returns: -[source,js] +[source,console-result] -------------------------------------------------- { "size" : 10, @@ -47,7 +47,6 @@ Which returns: ] } -------------------------------------------------- -// TESTRESPONSE Which is the request that SQL will run to provide the results. In this case, SQL will use the <> diff --git a/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc b/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc index de727a70b71e..2a1e6a26b242 100644 --- a/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-role-mappings.asciidoc @@ -134,6 +134,7 @@ been created or updated. } } -------------------------------------------------- + <1> When an existing mapping is updated, `created` is set to false. The following example assigns the "user" and "admin" roles to specific users: diff --git a/x-pack/docs/en/rest-api/security/create-roles.asciidoc b/x-pack/docs/en/rest-api/security/create-roles.asciidoc index 85f549d635f7..7eda4c22b0dc 100644 --- a/x-pack/docs/en/rest-api/security/create-roles.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-roles.asciidoc @@ -122,4 +122,5 @@ created or updated. } } -------------------------------------------------- + <1> When an existing role is updated, `created` is set to false. diff --git a/x-pack/docs/en/rest-api/security/create-users.asciidoc b/x-pack/docs/en/rest-api/security/create-users.asciidoc index e331c8724770..47df7c01f503 100644 --- a/x-pack/docs/en/rest-api/security/create-users.asciidoc +++ b/x-pack/docs/en/rest-api/security/create-users.asciidoc @@ -127,6 +127,7 @@ created or updated. "created": true <1> } -------------------------------------------------- + <1> When an existing user is updated, `created` is set to false. After you add a user, requests from that user can be authenticated. For example: diff --git a/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc b/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc index 7f7d06289c12..6dd4325925ab 100644 --- a/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/delete-app-privileges.asciidoc @@ -57,3 +57,4 @@ Otherwise, `found` is set to false. } } -------------------------------------------------- + diff --git a/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc b/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc index c048e19a18c2..38383e22d48b 100644 --- a/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/put-app-privileges.asciidoc @@ -122,6 +122,7 @@ PUT /_security/privilege } } -------------------------------------------------- + <1> When an existing privilege is updated, `created` is set to false. To add multiple privileges, submit a POST request to the diff --git a/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc index 44d50a4e16d2..4f5098a185b8 100644 --- a/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/delete-watch.asciidoc @@ -65,7 +65,7 @@ DELETE _watcher/watch/my_watch Response: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -73,4 +73,3 @@ Response: "_version": 2 } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/watcher/start.asciidoc b/x-pack/docs/en/rest-api/watcher/start.asciidoc index b4d5a5b7a934..583a69306af8 100644 --- a/x-pack/docs/en/rest-api/watcher/start.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/start.asciidoc @@ -47,10 +47,9 @@ POST _watcher/_start {watcher} returns the following response if the request is successful: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// TESTRESPONSE diff --git a/x-pack/docs/en/rest-api/watcher/stop.asciidoc b/x-pack/docs/en/rest-api/watcher/stop.asciidoc index 28f5be96f1e6..d810744ad2f7 100644 --- a/x-pack/docs/en/rest-api/watcher/stop.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/stop.asciidoc @@ -47,10 +47,9 @@ POST _watcher/_stop {watcher} returns the following response if the request is successful: -[source,js] +[source,console-result] -------------------------------------------------- { "acknowledged": true } -------------------------------------------------- -// TESTRESPONSE From 2cef2a41140ee30c6c97dd10d455fad400b5fcc4 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 6 Sep 2019 08:35:24 +0200 Subject: [PATCH 081/103] [ML-DataFrame] improve error message for timeout case in stop (#46131) improve error message if stopping of transform times out. related #45610 --- ...TransportStopDataFrameTransformAction.java | 50 ++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformAction.java index 34fee0cc4a7d..3e073c90ee1b 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformAction.java @@ -254,7 +254,55 @@ public class TransportStopDataFrameTransformAction extends TransportTasksAction< listener.onFailure(new ElasticsearchStatusException(message, RestStatus.CONFLICT)); }, - listener::onFailure + e -> { + // waitForPersistentTasksCondition throws a IllegalStateException on timeout + if (e instanceof IllegalStateException && e.getMessage().startsWith("Timed out")) { + PersistentTasksCustomMetaData persistentTasksCustomMetaData = clusterService.state().metaData() + .custom(PersistentTasksCustomMetaData.TYPE); + + if (persistentTasksCustomMetaData == null) { + listener.onResponse(new Response(Boolean.TRUE)); + return; + } + + // collect which tasks are still running + Set stillRunningTasks = new HashSet<>(); + for (String persistentTaskId : persistentTaskIds) { + if (persistentTasksCustomMetaData.getTask(persistentTaskId) != null) { + stillRunningTasks.add(persistentTaskId); + } + } + + if (stillRunningTasks.isEmpty()) { + // should not happen + listener.onResponse(new Response(Boolean.TRUE)); + return; + } else { + StringBuilder message = new StringBuilder(); + if (persistentTaskIds.size() - stillRunningTasks.size() - exceptions.size() > 0) { + message.append("Successfully stopped ["); + message.append(persistentTaskIds.size() - stillRunningTasks.size() - exceptions.size()); + message.append("] transforms. "); + } + + if (exceptions.size() > 0) { + message.append("Could not stop the transforms "); + message.append(exceptions.keySet()); + message.append(" as they were failed. Use force stop to stop the transforms. "); + } + + if (stillRunningTasks.size() > 0) { + message.append("Could not stop the transforms "); + message.append(stillRunningTasks); + message.append(" as they timed out."); + } + + listener.onFailure(new ElasticsearchStatusException(message.toString(), RestStatus.REQUEST_TIMEOUT)); + return; + } + } + listener.onFailure(e); + } )); } } From 0321073ae6ab7d8c2a7d90e0c8b69e22c261e2e4 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 6 Sep 2019 09:10:06 +0200 Subject: [PATCH 082/103] Fix usage of randomIntBetween() in testWriteBlobWithRetries (#46380) This commit fixes the usage of randomIntBetween() in the test testWriteBlobWithRetries, when the test generates a random array of a single byte. --- .../repositories/s3/S3BlobContainerRetriesTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index da2fc588e4d5..2c97ae2b5fa0 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -206,7 +206,7 @@ public class S3BlobContainerRetriesTests extends ESTestCase { if (randomBoolean()) { if (randomBoolean()) { - Streams.readFully(exchange.getRequestBody(), new byte[randomIntBetween(1, bytes.length - 1)]); + Streams.readFully(exchange.getRequestBody(), new byte[randomIntBetween(1, Math.max(1, bytes.length - 1))]); } else { Streams.readFully(exchange.getRequestBody()); exchange.sendResponseHeaders(randomFrom(HttpStatus.SC_INTERNAL_SERVER_ERROR, HttpStatus.SC_BAD_GATEWAY, From da3c04f6ac887ada4cc19e4e6e1f0c4ea978a23f Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 6 Sep 2019 08:55:57 +0200 Subject: [PATCH 083/103] cleanup static member --- .../checkpoint/DataFrameTransformCheckpointServiceNodeTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java index 288250516301..ad45ab61772d 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/checkpoint/DataFrameTransformCheckpointServiceNodeTests.java @@ -142,6 +142,7 @@ public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingl @AfterClass public static void tearDownClient() { mockClientForCheckpointing.close(); + mockClientForCheckpointing = null; } public void testCreateReadDeleteCheckpoint() throws InterruptedException { From f509e5861f4e406710c694bf2b8b880580811f3c Mon Sep 17 00:00:00 2001 From: "Yunfeng,Wu" Date: Fri, 6 Sep 2019 15:44:55 +0800 Subject: [PATCH 084/103] Resolve the incorrect scroll_current when delete or close index (#45226) Resolve the incorrect current scroll for deleted or closed index --- .../index/search/stats/SearchStats.java | 23 +++++++++++++++++++ .../elasticsearch/indices/IndicesService.java | 3 ++- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java b/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java index 5f514b89b64a..7dc3ae4785cf 100644 --- a/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java +++ b/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java @@ -117,6 +117,22 @@ public class SearchStats implements Writeable, ToXContentFragment { suggestCurrent += stats.suggestCurrent; } + public void addForClosingShard(Stats stats) { + queryCount += stats.queryCount; + queryTimeInMillis += stats.queryTimeInMillis; + + fetchCount += stats.fetchCount; + fetchTimeInMillis += stats.fetchTimeInMillis; + + scrollCount += stats.scrollCount; + scrollTimeInMillis += stats.scrollTimeInMillis; + // need consider the count of the shard's current scroll + scrollCount += stats.scrollCurrent; + + suggestCount += stats.suggestCount; + suggestTimeInMillis += stats.suggestTimeInMillis; + } + public long getQueryCount() { return queryCount; } @@ -274,6 +290,13 @@ public class SearchStats implements Writeable, ToXContentFragment { totalStats.add(searchStats.totalStats); } + public void addTotalsForClosingShard(SearchStats searchStats) { + if (searchStats == null) { + return; + } + totalStats.addForClosingShard(searchStats.totalStats); + } + public Stats getTotal() { return this.totalStats; } diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 23a9985b7f89..bcc436405736 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -721,7 +721,8 @@ public class IndicesService extends AbstractLifecycleComponent if (indexShard != null) { getStats.addTotals(indexShard.getStats()); indexingStats.addTotals(indexShard.indexingStats()); - searchStats.addTotals(indexShard.searchStats()); + // if this index was closed or deleted, we should eliminate the effect of the current scroll for this shard + searchStats.addTotalsForClosingShard(indexShard.searchStats()); mergeStats.addTotals(indexShard.mergeStats()); refreshStats.addTotals(indexShard.refreshStats()); flushStats.addTotals(indexShard.flushStats()); From 94fec109c77819a731562cc6f827dfc3bf86c5b7 Mon Sep 17 00:00:00 2001 From: Dimitris Athanasiou Date: Fri, 6 Sep 2019 10:47:44 +0300 Subject: [PATCH 085/103] [ML] Extract DataFrameAnalyticsTask into its own class (#46402) This refactors `DataFrameAnalyticsTask` into its own class. The task has quite a lot of functionality now and I believe it would make code more readable to have it live as its own class rather than an inner class of the start action class. --- ...NativeDataFrameAnalyticsIntegTestCase.java | 4 +- ...ansportDeleteDataFrameAnalyticsAction.java | 3 +- ...sportGetDataFrameAnalyticsStatsAction.java | 2 +- ...ransportStartDataFrameAnalyticsAction.java | 242 +--------------- ...TransportStopDataFrameAnalyticsAction.java | 5 +- .../dataframe/DataFrameAnalyticsManager.java | 1 - .../ml/dataframe/DataFrameAnalyticsTask.java | 268 ++++++++++++++++++ .../process/AnalyticsProcessManager.java | 2 +- .../process/AnalyticsResultProcessor.java | 2 +- .../AnalyticsResultProcessorTests.java | 2 +- 10 files changed, 280 insertions(+), 251 deletions(-) create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeDataFrameAnalyticsIntegTestCase.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeDataFrameAnalyticsIntegTestCase.java index 4491fac98c38..7e9ea18a1b30 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeDataFrameAnalyticsIntegTestCase.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeDataFrameAnalyticsIntegTestCase.java @@ -24,7 +24,7 @@ import org.elasticsearch.xpack.core.ml.dataframe.analyses.OutlierDetection; import org.elasticsearch.xpack.core.ml.dataframe.analyses.Regression; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.utils.PhaseProgress; -import org.elasticsearch.xpack.ml.action.TransportStartDataFrameAnalyticsAction; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask; import java.util.ArrayList; import java.util.List; @@ -138,7 +138,7 @@ abstract class MlNativeDataFrameAnalyticsIntegTestCase extends MlNativeIntegTest protected SearchResponse searchStoredProgress(String id) { return client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) - .setQuery(QueryBuilders.idsQuery().addIds(TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask.progressDocId(id))) + .setQuery(QueryBuilders.idsQuery().addIds(DataFrameAnalyticsTask.progressDocId(id))) .get(); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteDataFrameAnalyticsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteDataFrameAnalyticsAction.java index 61348c8e2193..6b41ca4a2321 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteDataFrameAnalyticsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteDataFrameAnalyticsAction.java @@ -42,6 +42,7 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsConfig; import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsState; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask; import org.elasticsearch.xpack.ml.dataframe.persistence.DataFrameAnalyticsConfigProvider; import org.elasticsearch.xpack.ml.process.MlMemoryTracker; import org.elasticsearch.xpack.ml.utils.MlIndicesUtils; @@ -155,7 +156,7 @@ public class TransportDeleteDataFrameAnalyticsAction ActionListener listener) { DeleteByQueryRequest request = new DeleteByQueryRequest(AnomalyDetectorsIndex.jobStateIndexPattern()); request.setQuery(QueryBuilders.idsQuery().addIds( - TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask.progressDocId(analyticsId))); + DataFrameAnalyticsTask.progressDocId(analyticsId))); request.setIndicesOptions(MlIndicesUtils.addIgnoreUnavailable(IndicesOptions.lenientExpandOpen())); request.setSlices(AbstractBulkByScrollRequest.AUTO_SLICES); request.setAbortOnVersionConflict(false); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java index 875a0a8f4474..11bdeb33c8bf 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java @@ -46,7 +46,7 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsTaskState; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; import org.elasticsearch.xpack.core.ml.utils.PhaseProgress; -import org.elasticsearch.xpack.ml.action.TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask; import org.elasticsearch.xpack.ml.dataframe.StoredProgress; import java.io.IOException; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java index e7fa989447b8..e72d4467926f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java @@ -7,22 +7,14 @@ package org.elasticsearch.xpack.ml.action; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceAlreadyExistsException; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; -import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; -import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskRequest; -import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; @@ -33,16 +25,12 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.index.reindex.BulkByScrollTask; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.persistent.AllocatedPersistentTask; @@ -54,7 +42,6 @@ import org.elasticsearch.persistent.PersistentTasksService; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskResult; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.ClientHelper; @@ -62,7 +49,6 @@ import org.elasticsearch.xpack.core.XPackField; import org.elasticsearch.xpack.core.ml.MlMetadata; import org.elasticsearch.xpack.core.ml.MlTasks; import org.elasticsearch.xpack.core.ml.action.EstimateMemoryUsageAction; -import org.elasticsearch.xpack.core.ml.action.GetDataFrameAnalyticsStatsAction; import org.elasticsearch.xpack.core.ml.action.PutDataFrameAnalyticsAction; import org.elasticsearch.xpack.core.ml.action.StartDataFrameAnalyticsAction; import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsConfig; @@ -70,13 +56,11 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsState; import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsTaskState; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; -import org.elasticsearch.xpack.core.ml.utils.PhaseProgress; -import org.elasticsearch.xpack.core.watcher.watch.Payload; import org.elasticsearch.xpack.ml.MachineLearning; import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsManager; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask; import org.elasticsearch.xpack.ml.dataframe.MappingsMerger; import org.elasticsearch.xpack.ml.dataframe.SourceDestValidator; -import org.elasticsearch.xpack.ml.dataframe.StoredProgress; import org.elasticsearch.xpack.ml.dataframe.extractor.DataFrameDataExtractorFactory; import org.elasticsearch.xpack.ml.dataframe.persistence.DataFrameAnalyticsConfigProvider; import org.elasticsearch.xpack.ml.job.JobNodeSelector; @@ -84,16 +68,12 @@ import org.elasticsearch.xpack.ml.process.MlMemoryTracker; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; -import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; import static org.elasticsearch.xpack.core.ml.MlTasks.AWAITING_UPGRADE; import static org.elasticsearch.xpack.ml.MachineLearning.MAX_OPEN_JOBS_PER_NODE; @@ -410,226 +390,6 @@ public class TransportStartDataFrameAnalyticsAction ); } - public static class DataFrameAnalyticsTask extends AllocatedPersistentTask implements StartDataFrameAnalyticsAction.TaskMatcher { - - private final Client client; - private final ClusterService clusterService; - private final DataFrameAnalyticsManager analyticsManager; - private final StartDataFrameAnalyticsAction.TaskParams taskParams; - @Nullable - private volatile Long reindexingTaskId; - private volatile boolean isReindexingFinished; - private volatile boolean isStopping; - private final ProgressTracker progressTracker = new ProgressTracker(); - - public DataFrameAnalyticsTask(long id, String type, String action, TaskId parentTask, Map headers, - Client client, ClusterService clusterService, DataFrameAnalyticsManager analyticsManager, - StartDataFrameAnalyticsAction.TaskParams taskParams) { - super(id, type, action, MlTasks.DATA_FRAME_ANALYTICS_TASK_ID_PREFIX + taskParams.getId(), parentTask, headers); - this.client = Objects.requireNonNull(client); - this.clusterService = Objects.requireNonNull(clusterService); - this.analyticsManager = Objects.requireNonNull(analyticsManager); - this.taskParams = Objects.requireNonNull(taskParams); - } - - public StartDataFrameAnalyticsAction.TaskParams getParams() { - return taskParams; - } - - public void setReindexingTaskId(Long reindexingTaskId) { - this.reindexingTaskId = reindexingTaskId; - } - - public void setReindexingFinished() { - isReindexingFinished = true; - } - - public boolean isStopping() { - return isStopping; - } - - public ProgressTracker getProgressTracker() { - return progressTracker; - } - - @Override - protected void onCancelled() { - stop(getReasonCancelled(), TimeValue.ZERO); - } - - @Override - public void markAsCompleted() { - persistProgress(() -> super.markAsCompleted()); - } - - @Override - public void markAsFailed(Exception e) { - persistProgress(() -> super.markAsFailed(e)); - } - - public void stop(String reason, TimeValue timeout) { - isStopping = true; - - ActionListener reindexProgressListener = ActionListener.wrap( - aVoid -> doStop(reason, timeout), - e -> { - LOGGER.error(new ParameterizedMessage("[{}] Error updating reindexing progress", taskParams.getId()), e); - // We should log the error but it shouldn't stop us from stopping the task - doStop(reason, timeout); - } - ); - - // We need to update reindexing progress before we cancel the task - updateReindexTaskProgress(reindexProgressListener); - } - - private void doStop(String reason, TimeValue timeout) { - if (reindexingTaskId != null) { - cancelReindexingTask(reason, timeout); - } - analyticsManager.stop(this); - } - - private void cancelReindexingTask(String reason, TimeValue timeout) { - TaskId reindexTaskId = new TaskId(clusterService.localNode().getId(), reindexingTaskId); - LOGGER.debug("[{}] Cancelling reindex task [{}]", taskParams.getId(), reindexTaskId); - - CancelTasksRequest cancelReindex = new CancelTasksRequest(); - cancelReindex.setTaskId(reindexTaskId); - cancelReindex.setReason(reason); - cancelReindex.setTimeout(timeout); - CancelTasksResponse cancelReindexResponse = client.admin().cluster().cancelTasks(cancelReindex).actionGet(); - Throwable firstError = null; - if (cancelReindexResponse.getNodeFailures().isEmpty() == false) { - firstError = cancelReindexResponse.getNodeFailures().get(0).getRootCause(); - } - if (cancelReindexResponse.getTaskFailures().isEmpty() == false) { - firstError = cancelReindexResponse.getTaskFailures().get(0).getCause(); - } - // There is a chance that the task is finished by the time we cancel it in which case we'll get - // a ResourceNotFoundException which we can ignore. - if (firstError != null && firstError instanceof ResourceNotFoundException == false) { - throw ExceptionsHelper.serverError("[" + taskParams.getId() + "] Error cancelling reindex task", firstError); - } else { - LOGGER.debug("[{}] Reindex task was successfully cancelled", taskParams.getId()); - } - } - - public void updateState(DataFrameAnalyticsState state, @Nullable String reason) { - DataFrameAnalyticsTaskState newTaskState = new DataFrameAnalyticsTaskState(state, getAllocationId(), reason); - updatePersistentTaskState(newTaskState, ActionListener.wrap( - updatedTask -> LOGGER.info("[{}] Successfully update task state to [{}]", getParams().getId(), state), - e -> LOGGER.error(new ParameterizedMessage("[{}] Could not update task state to [{}] with reason [{}]", - getParams().getId(), state, reason), e) - )); - } - - public void updateReindexTaskProgress(ActionListener listener) { - TaskId reindexTaskId = getReindexTaskId(); - if (reindexTaskId == null) { - // The task is not present which means either it has not started yet or it finished. - // We keep track of whether the task has finished so we can use that to tell whether the progress 100. - if (isReindexingFinished) { - progressTracker.reindexingPercent.set(100); - } - listener.onResponse(null); - return; - } - - GetTaskRequest getTaskRequest = new GetTaskRequest(); - getTaskRequest.setTaskId(reindexTaskId); - client.admin().cluster().getTask(getTaskRequest, ActionListener.wrap( - taskResponse -> { - TaskResult taskResult = taskResponse.getTask(); - BulkByScrollTask.Status taskStatus = (BulkByScrollTask.Status) taskResult.getTask().getStatus(); - int progress = taskStatus.getTotal() == 0 ? 0 : (int) (taskStatus.getCreated() * 100.0 / taskStatus.getTotal()); - progressTracker.reindexingPercent.set(progress); - listener.onResponse(null); - }, - error -> { - if (error instanceof ResourceNotFoundException) { - // The task is not present which means either it has not started yet or it finished. - // We keep track of whether the task has finished so we can use that to tell whether the progress 100. - if (isReindexingFinished) { - progressTracker.reindexingPercent.set(100); - } - listener.onResponse(null); - } else { - listener.onFailure(error); - } - } - )); - } - - @Nullable - private TaskId getReindexTaskId() { - try { - return new TaskId(clusterService.localNode().getId(), reindexingTaskId); - } catch (NullPointerException e) { - // This may happen if there is no reindexing task id set which means we either never started the task yet or we're finished - return null; - } - } - - private void persistProgress(Runnable runnable) { - GetDataFrameAnalyticsStatsAction.Request getStatsRequest = new GetDataFrameAnalyticsStatsAction.Request(taskParams.getId()); - executeAsyncWithOrigin(client, ML_ORIGIN, GetDataFrameAnalyticsStatsAction.INSTANCE, getStatsRequest, ActionListener.wrap( - statsResponse -> { - GetDataFrameAnalyticsStatsAction.Response.Stats stats = statsResponse.getResponse().results().get(0); - IndexRequest indexRequest = new IndexRequest(AnomalyDetectorsIndex.jobStateIndexWriteAlias()); - indexRequest.id(progressDocId(taskParams.getId())); - indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - try (XContentBuilder jsonBuilder = JsonXContent.contentBuilder()) { - new StoredProgress(stats.getProgress()).toXContent(jsonBuilder, Payload.XContent.EMPTY_PARAMS); - indexRequest.source(jsonBuilder); - } - executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, ActionListener.wrap( - indexResponse -> { - LOGGER.debug("[{}] Successfully indexed progress document", taskParams.getId()); - runnable.run(); - }, - indexError -> { - LOGGER.error(new ParameterizedMessage( - "[{}] cannot persist progress as an error occurred while indexing", taskParams.getId()), indexError); - runnable.run(); - } - )); - }, - e -> { - LOGGER.error(new ParameterizedMessage( - "[{}] cannot persist progress as an error occurred while retrieving stats", taskParams.getId()), e); - runnable.run(); - } - )); - } - - public static String progressDocId(String id) { - return "data_frame_analytics-" + id + "-progress"; - } - - public static class ProgressTracker { - - public static final String REINDEXING = "reindexing"; - public static final String LOADING_DATA = "loading_data"; - public static final String ANALYZING = "analyzing"; - public static final String WRITING_RESULTS = "writing_results"; - - public final AtomicInteger reindexingPercent = new AtomicInteger(0); - public final AtomicInteger loadingDataPercent = new AtomicInteger(0); - public final AtomicInteger analyzingPercent = new AtomicInteger(0); - public final AtomicInteger writingResultsPercent = new AtomicInteger(0); - - public List report() { - return Arrays.asList( - new PhaseProgress(REINDEXING, reindexingPercent.get()), - new PhaseProgress(LOADING_DATA, loadingDataPercent.get()), - new PhaseProgress(ANALYZING, analyzingPercent.get()), - new PhaseProgress(WRITING_RESULTS, writingResultsPercent.get()) - ); - } - } - } - static List verifyIndicesPrimaryShardsAreActive(ClusterState clusterState, String... indexNames) { IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); String[] concreteIndices = resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexNames); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopDataFrameAnalyticsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopDataFrameAnalyticsAction.java index b02d89c17766..d6d67aeeddd8 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopDataFrameAnalyticsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopDataFrameAnalyticsAction.java @@ -34,6 +34,7 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsState; import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsTaskState; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; import org.elasticsearch.xpack.ml.MachineLearning; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask; import org.elasticsearch.xpack.ml.dataframe.persistence.DataFrameAnalyticsConfigProvider; import java.util.ArrayList; @@ -50,7 +51,7 @@ import java.util.stream.Collectors; * TODO Add to the upgrade mode action */ public class TransportStopDataFrameAnalyticsAction - extends TransportTasksAction { private static final Logger logger = LogManager.getLogger(TransportStopDataFrameAnalyticsAction.class); @@ -222,7 +223,7 @@ public class TransportStopDataFrameAnalyticsAction @Override protected void taskOperation(StopDataFrameAnalyticsAction.Request request, - TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask task, + DataFrameAnalyticsTask task, ActionListener listener) { DataFrameAnalyticsTaskState stoppingState = new DataFrameAnalyticsTaskState(DataFrameAnalyticsState.STOPPING, task.getAllocationId(), null); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsManager.java index 4b73a9188644..c9e1604bf211 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsManager.java @@ -31,7 +31,6 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsConfig; import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsState; import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsTaskState; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; -import org.elasticsearch.xpack.ml.action.TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask; import org.elasticsearch.xpack.ml.dataframe.extractor.DataFrameDataExtractorFactory; import org.elasticsearch.xpack.ml.dataframe.persistence.DataFrameAnalyticsConfigProvider; import org.elasticsearch.xpack.ml.dataframe.process.AnalyticsProcessManager; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java new file mode 100644 index 000000000000..1e3cbdf016af --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java @@ -0,0 +1,268 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.dataframe; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; +import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskRequest; +import org.elasticsearch.action.index.IndexAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.index.reindex.BulkByScrollTask; +import org.elasticsearch.persistent.AllocatedPersistentTask; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskResult; +import org.elasticsearch.xpack.core.ml.MlTasks; +import org.elasticsearch.xpack.core.ml.action.GetDataFrameAnalyticsStatsAction; +import org.elasticsearch.xpack.core.ml.action.StartDataFrameAnalyticsAction; +import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsState; +import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsTaskState; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; +import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.ml.utils.PhaseProgress; +import org.elasticsearch.xpack.core.watcher.watch.Payload; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; +import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; + +public class DataFrameAnalyticsTask extends AllocatedPersistentTask implements StartDataFrameAnalyticsAction.TaskMatcher { + + private static final Logger LOGGER = LogManager.getLogger(DataFrameAnalyticsTask.class); + + private final Client client; + private final ClusterService clusterService; + private final DataFrameAnalyticsManager analyticsManager; + private final StartDataFrameAnalyticsAction.TaskParams taskParams; + @Nullable + private volatile Long reindexingTaskId; + private volatile boolean isReindexingFinished; + private volatile boolean isStopping; + private final ProgressTracker progressTracker = new ProgressTracker(); + + public DataFrameAnalyticsTask(long id, String type, String action, TaskId parentTask, Map headers, + Client client, ClusterService clusterService, DataFrameAnalyticsManager analyticsManager, + StartDataFrameAnalyticsAction.TaskParams taskParams) { + super(id, type, action, MlTasks.DATA_FRAME_ANALYTICS_TASK_ID_PREFIX + taskParams.getId(), parentTask, headers); + this.client = Objects.requireNonNull(client); + this.clusterService = Objects.requireNonNull(clusterService); + this.analyticsManager = Objects.requireNonNull(analyticsManager); + this.taskParams = Objects.requireNonNull(taskParams); + } + + public StartDataFrameAnalyticsAction.TaskParams getParams() { + return taskParams; + } + + public void setReindexingTaskId(Long reindexingTaskId) { + this.reindexingTaskId = reindexingTaskId; + } + + public void setReindexingFinished() { + isReindexingFinished = true; + } + + public boolean isStopping() { + return isStopping; + } + + public ProgressTracker getProgressTracker() { + return progressTracker; + } + + @Override + protected void onCancelled() { + stop(getReasonCancelled(), TimeValue.ZERO); + } + + @Override + public void markAsCompleted() { + persistProgress(() -> super.markAsCompleted()); + } + + @Override + public void markAsFailed(Exception e) { + persistProgress(() -> super.markAsFailed(e)); + } + + public void stop(String reason, TimeValue timeout) { + isStopping = true; + + ActionListener reindexProgressListener = ActionListener.wrap( + aVoid -> doStop(reason, timeout), + e -> { + LOGGER.error(new ParameterizedMessage("[{}] Error updating reindexing progress", taskParams.getId()), e); + // We should log the error but it shouldn't stop us from stopping the task + doStop(reason, timeout); + } + ); + + // We need to update reindexing progress before we cancel the task + updateReindexTaskProgress(reindexProgressListener); + } + + private void doStop(String reason, TimeValue timeout) { + if (reindexingTaskId != null) { + cancelReindexingTask(reason, timeout); + } + analyticsManager.stop(this); + } + + private void cancelReindexingTask(String reason, TimeValue timeout) { + TaskId reindexTaskId = new TaskId(clusterService.localNode().getId(), reindexingTaskId); + LOGGER.debug("[{}] Cancelling reindex task [{}]", taskParams.getId(), reindexTaskId); + + CancelTasksRequest cancelReindex = new CancelTasksRequest(); + cancelReindex.setTaskId(reindexTaskId); + cancelReindex.setReason(reason); + cancelReindex.setTimeout(timeout); + CancelTasksResponse cancelReindexResponse = client.admin().cluster().cancelTasks(cancelReindex).actionGet(); + Throwable firstError = null; + if (cancelReindexResponse.getNodeFailures().isEmpty() == false) { + firstError = cancelReindexResponse.getNodeFailures().get(0).getRootCause(); + } + if (cancelReindexResponse.getTaskFailures().isEmpty() == false) { + firstError = cancelReindexResponse.getTaskFailures().get(0).getCause(); + } + // There is a chance that the task is finished by the time we cancel it in which case we'll get + // a ResourceNotFoundException which we can ignore. + if (firstError != null && firstError instanceof ResourceNotFoundException == false) { + throw ExceptionsHelper.serverError("[" + taskParams.getId() + "] Error cancelling reindex task", firstError); + } else { + LOGGER.debug("[{}] Reindex task was successfully cancelled", taskParams.getId()); + } + } + + public void updateState(DataFrameAnalyticsState state, @Nullable String reason) { + DataFrameAnalyticsTaskState newTaskState = new DataFrameAnalyticsTaskState(state, getAllocationId(), reason); + updatePersistentTaskState(newTaskState, ActionListener.wrap( + updatedTask -> LOGGER.info("[{}] Successfully update task state to [{}]", getParams().getId(), state), + e -> LOGGER.error(new ParameterizedMessage("[{}] Could not update task state to [{}] with reason [{}]", + getParams().getId(), state, reason), e) + )); + } + + public void updateReindexTaskProgress(ActionListener listener) { + TaskId reindexTaskId = getReindexTaskId(); + if (reindexTaskId == null) { + // The task is not present which means either it has not started yet or it finished. + // We keep track of whether the task has finished so we can use that to tell whether the progress 100. + if (isReindexingFinished) { + progressTracker.reindexingPercent.set(100); + } + listener.onResponse(null); + return; + } + + GetTaskRequest getTaskRequest = new GetTaskRequest(); + getTaskRequest.setTaskId(reindexTaskId); + client.admin().cluster().getTask(getTaskRequest, ActionListener.wrap( + taskResponse -> { + TaskResult taskResult = taskResponse.getTask(); + BulkByScrollTask.Status taskStatus = (BulkByScrollTask.Status) taskResult.getTask().getStatus(); + int progress = taskStatus.getTotal() == 0 ? 0 : (int) (taskStatus.getCreated() * 100.0 / taskStatus.getTotal()); + progressTracker.reindexingPercent.set(progress); + listener.onResponse(null); + }, + error -> { + if (error instanceof ResourceNotFoundException) { + // The task is not present which means either it has not started yet or it finished. + // We keep track of whether the task has finished so we can use that to tell whether the progress 100. + if (isReindexingFinished) { + progressTracker.reindexingPercent.set(100); + } + listener.onResponse(null); + } else { + listener.onFailure(error); + } + } + )); + } + + @Nullable + private TaskId getReindexTaskId() { + try { + return new TaskId(clusterService.localNode().getId(), reindexingTaskId); + } catch (NullPointerException e) { + // This may happen if there is no reindexing task id set which means we either never started the task yet or we're finished + return null; + } + } + + private void persistProgress(Runnable runnable) { + GetDataFrameAnalyticsStatsAction.Request getStatsRequest = new GetDataFrameAnalyticsStatsAction.Request(taskParams.getId()); + executeAsyncWithOrigin(client, ML_ORIGIN, GetDataFrameAnalyticsStatsAction.INSTANCE, getStatsRequest, ActionListener.wrap( + statsResponse -> { + GetDataFrameAnalyticsStatsAction.Response.Stats stats = statsResponse.getResponse().results().get(0); + IndexRequest indexRequest = new IndexRequest(AnomalyDetectorsIndex.jobStateIndexWriteAlias()); + indexRequest.id(progressDocId(taskParams.getId())); + indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + try (XContentBuilder jsonBuilder = JsonXContent.contentBuilder()) { + new StoredProgress(stats.getProgress()).toXContent(jsonBuilder, Payload.XContent.EMPTY_PARAMS); + indexRequest.source(jsonBuilder); + } + executeAsyncWithOrigin(client, ML_ORIGIN, IndexAction.INSTANCE, indexRequest, ActionListener.wrap( + indexResponse -> { + LOGGER.debug("[{}] Successfully indexed progress document", taskParams.getId()); + runnable.run(); + }, + indexError -> { + LOGGER.error(new ParameterizedMessage( + "[{}] cannot persist progress as an error occurred while indexing", taskParams.getId()), indexError); + runnable.run(); + } + )); + }, + e -> { + LOGGER.error(new ParameterizedMessage( + "[{}] cannot persist progress as an error occurred while retrieving stats", taskParams.getId()), e); + runnable.run(); + } + )); + } + + public static String progressDocId(String id) { + return "data_frame_analytics-" + id + "-progress"; + } + + public static class ProgressTracker { + + public static final String REINDEXING = "reindexing"; + public static final String LOADING_DATA = "loading_data"; + public static final String ANALYZING = "analyzing"; + public static final String WRITING_RESULTS = "writing_results"; + + public final AtomicInteger reindexingPercent = new AtomicInteger(0); + public final AtomicInteger loadingDataPercent = new AtomicInteger(0); + public final AtomicInteger analyzingPercent = new AtomicInteger(0); + public final AtomicInteger writingResultsPercent = new AtomicInteger(0); + + public List report() { + return Arrays.asList( + new PhaseProgress(REINDEXING, reindexingPercent.get()), + new PhaseProgress(LOADING_DATA, loadingDataPercent.get()), + new PhaseProgress(ANALYZING, analyzingPercent.get()), + new PhaseProgress(WRITING_RESULTS, writingResultsPercent.get()) + ); + } + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessManager.java index 6a6462c2a968..245afa1cbab7 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsProcessManager.java @@ -19,7 +19,7 @@ import org.elasticsearch.xpack.core.ml.dataframe.DataFrameAnalyticsState; import org.elasticsearch.xpack.core.ml.dataframe.analyses.DataFrameAnalysis; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; import org.elasticsearch.xpack.ml.MachineLearning; -import org.elasticsearch.xpack.ml.action.TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask; import org.elasticsearch.xpack.ml.dataframe.extractor.DataFrameDataExtractor; import org.elasticsearch.xpack.ml.dataframe.extractor.DataFrameDataExtractorFactory; import org.elasticsearch.xpack.ml.dataframe.process.customprocessing.CustomProcessor; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessor.java index 666817c0acf8..079feecbab63 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessor.java @@ -9,7 +9,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.common.Nullable; -import org.elasticsearch.xpack.ml.action.TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask.ProgressTracker; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask.ProgressTracker; import org.elasticsearch.xpack.ml.dataframe.process.results.AnalyticsResult; import org.elasticsearch.xpack.ml.dataframe.process.results.RowResults; diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessorTests.java index 6b4e54e19ff9..7f26293cabf8 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/process/AnalyticsResultProcessorTests.java @@ -8,7 +8,7 @@ package org.elasticsearch.xpack.ml.dataframe.process; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.ml.dataframe.analyses.DataFrameAnalysis; -import org.elasticsearch.xpack.ml.action.TransportStartDataFrameAnalyticsAction.DataFrameAnalyticsTask.ProgressTracker; +import org.elasticsearch.xpack.ml.dataframe.DataFrameAnalyticsTask.ProgressTracker; import org.elasticsearch.xpack.ml.dataframe.process.results.AnalyticsResult; import org.elasticsearch.xpack.ml.dataframe.process.results.RowResults; import org.junit.Before; From 0cdc3a31f7c199dd6c765adafefb5f216d282f1e Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 6 Sep 2019 10:41:42 +0200 Subject: [PATCH 086/103] Mute CcrRollingUpgradeIT.testUniDirectionalIndexFollowing and testUniDirectionalIndexFollowing (#46429) Relates #46416 --- .../java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java b/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java index e9e4b671f564..c14e250ebb12 100644 --- a/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade-multi-cluster/src/test/java/org/elasticsearch/upgrades/CcrRollingUpgradeIT.java @@ -23,6 +23,7 @@ import static org.hamcrest.Matchers.equalTo; public class CcrRollingUpgradeIT extends AbstractMultiClusterUpgradeTestCase { + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46416") public void testUniDirectionalIndexFollowing() throws Exception { logger.info("clusterName={}, upgradeState={}", clusterName, upgradeState); @@ -90,6 +91,7 @@ public class CcrRollingUpgradeIT extends AbstractMultiClusterUpgradeTestCase { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46416") public void testAutoFollowing() throws Exception { String leaderIndex1 = "logs-20200101"; String leaderIndex2 = "logs-20200102"; From f033c3fb8955d3ac01e5551556b67a5b7a172d95 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 6 Sep 2019 10:51:52 +0200 Subject: [PATCH 087/103] Mute SSLClientAuthTests.testThatHttpFailsWithoutSslClientAuth() Tracked in #46230 --- .../java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java index a50d1b5a9b81..8a53e674963d 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java @@ -98,6 +98,7 @@ public class SSLClientAuthTests extends SecurityIntegTestCase { return true; } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/46230") public void testThatHttpFailsWithoutSslClientAuth() throws IOException { SSLIOSessionStrategy sessionStrategy = new SSLIOSessionStrategy(SSLContexts.createDefault(), NoopHostnameVerifier.INSTANCE); try (RestClient restClient = createRestClient(httpClientBuilder -> httpClientBuilder.setSSLStrategy(sessionStrategy), "https")) { From 96b4f3df9067d579bf7e22e09613ea94f4f05462 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 6 Sep 2019 13:04:32 +0100 Subject: [PATCH 088/103] Add yet more logging around index creation (#46431) Further investigation into #46091, expanding on #46363, to add even more detailed logging around the retry behaviour during index creation. --- .../gradle/testclusters/ElasticsearchNode.java | 9 ++++++++- ...TransportClusterAllocationExplainAction.java | 4 ++++ ...ransportAddVotingConfigExclusionsAction.java | 4 ++++ ...nsportClearVotingConfigExclusionsAction.java | 4 ++++ .../TransportClusterUpdateSettingsAction.java | 4 ++++ .../status/TransportSnapshotsStatusAction.java | 4 ++++ .../TransportPendingClusterTasksAction.java | 4 ++++ .../alias/TransportIndicesAliasesAction.java | 4 ++++ .../close/TransportCloseIndexAction.java | 4 ++++ .../CreateIndexClusterStateUpdateRequest.java | 15 +++++++++++++++ .../delete/TransportDeleteIndexAction.java | 4 ++++ .../mapping/get/TransportGetMappingsAction.java | 4 ++++ .../mapping/put/TransportPutMappingAction.java | 4 ++++ .../indices/open/TransportOpenIndexAction.java | 4 ++++ .../put/TransportUpdateSettingsAction.java | 4 ++++ .../TransportIndicesShardStoresAction.java | 3 +++ .../TransportDeleteIndexTemplateAction.java | 4 ++++ .../put/TransportPutIndexTemplateAction.java | 4 ++++ .../post/TransportUpgradeSettingsAction.java | 4 ++++ .../master/TransportMasterNodeAction.java | 17 ++++++++++++----- .../metadata/MetaDataCreateIndexService.java | 7 +++++++ .../ccr/action/TransportUnfollowAction.java | 4 ++++ .../DataFrameUsageTransportAction.java | 4 ++++ .../action/TransportFreezeIndexAction.java | 4 ++++ .../ilm/action/TransportPutLifecycleAction.java | 4 ++++ .../xpack/ilm/action/TransportRetryAction.java | 4 ++++ .../ml/action/TransportDeleteJobAction.java | 4 ++++ .../TransportFinalizeJobExecutionAction.java | 4 ++++ .../ml/action/TransportGetDatafeedsAction.java | 4 ++++ .../TransportGetDatafeedsStatsAction.java | 4 ++++ .../xpack/ml/action/TransportGetJobsAction.java | 4 ++++ .../xpack/ml/action/TransportOpenJobAction.java | 2 ++ .../TransportRevertModelSnapshotAction.java | 4 ++++ .../ml/action/TransportStartDatafeedAction.java | 4 ++++ .../action/TransportPutRollupJobAction.java | 3 +++ .../service/TransportWatcherServiceAction.java | 4 ++++ 36 files changed, 166 insertions(+), 6 deletions(-) diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java index fa47c73d49c4..addcac2ad011 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java @@ -965,8 +965,15 @@ public class ElasticsearchNode implements TestClusterConfiguration { defaultConfig.put("discovery.initial_state_timeout", "0s"); // TODO: Remove these once https://github.com/elastic/elasticsearch/issues/46091 is fixed - defaultConfig.put("logger.org.elasticsearch.action.support.master", "DEBUG"); + defaultConfig.put("logger.org.elasticsearch.action.support.master.TransportMasterNodeAction", "TRACE"); + defaultConfig.put("logger.org.elasticsearch.cluster.metadata.MetaDataCreateIndexService", "TRACE"); + defaultConfig.put("logger.org.elasticsearch.cluster.service", "DEBUG"); defaultConfig.put("logger.org.elasticsearch.cluster.coordination", "DEBUG"); + defaultConfig.put("logger.org.elasticsearch.gateway.MetaStateService", "TRACE"); + if (getVersion().getMajor() >= 8) { + defaultConfig.put("cluster.service.slow_task_logging_threshold", "5s"); + defaultConfig.put("cluster.service.slow_master_task_logging_threshold", "5s"); + } HashSet overriden = new HashSet<>(defaultConfig.keySet()); overriden.retainAll(settings.keySet()); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java index 846f920b7b18..3533036209c1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportClusterAllocationExplainAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.cluster.allocation; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; @@ -56,6 +58,8 @@ import java.util.List; public class TransportClusterAllocationExplainAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportClusterAllocationExplainAction.class); + private final ClusterInfoService clusterInfoService; private final AllocationDeciders allocationDeciders; private final ShardsAllocator shardAllocator; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java index 13270075806c..7dcbd863d29e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsAction.java @@ -18,6 +18,8 @@ */ package org.elasticsearch.action.admin.cluster.configuration; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionListener; @@ -53,6 +55,8 @@ import java.util.stream.Collectors; public class TransportAddVotingConfigExclusionsAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportAddVotingConfigExclusionsAction.class); + public static final Setting MAXIMUM_VOTING_CONFIG_EXCLUSIONS_SETTING = Setting.intSetting("cluster.max_voting_config_exclusions", 10, 1, Property.Dynamic, Property.NodeScope); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java index 78709c3db09f..0be05b480c05 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsAction.java @@ -18,6 +18,8 @@ */ package org.elasticsearch.action.admin.cluster.configuration; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionListener; @@ -49,6 +51,8 @@ import java.util.function.Predicate; public class TransportClearVotingConfigExclusionsAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportClearVotingConfigExclusionsAction.class); + @Inject public TransportClearVotingConfigExclusionsAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index 777b2f4ca9d6..70fd23acfda9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.cluster.settings; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; @@ -47,6 +49,8 @@ import java.io.IOException; public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportClusterUpdateSettingsAction.class); + private final AllocationService allocationService; private final ClusterSettings clusterSettings; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 83bfc800037b..4cfc1bafe33b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -21,6 +21,8 @@ package org.elasticsearch.action.admin.cluster.snapshots.status; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.ActionFilters; @@ -62,6 +64,8 @@ import java.util.stream.Collectors; public class TransportSnapshotsStatusAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportSnapshotsStatusAction.class); + private final SnapshotsService snapshotsService; private final NodeClient client; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java index 4eada5d0bc6e..75e644776156 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.cluster.tasks; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; @@ -39,6 +41,8 @@ import java.util.List; public class TransportPendingClusterTasksAction extends TransportMasterNodeReadAction { + private static final Logger logger = LogManager.getLogger(TransportPendingClusterTasksAction.class); + private final ClusterService clusterService; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java index 978a5bc075a5..d0ed81f8b7ac 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/TransportIndicesAliasesAction.java @@ -20,6 +20,8 @@ package org.elasticsearch.action.admin.indices.alias; import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.RequestValidators; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; @@ -61,6 +63,8 @@ import static java.util.Collections.unmodifiableList; */ public class TransportIndicesAliasesAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportIndicesAliasesAction.class); + private final MetaDataIndexAliasesService indexAliasesService; private final RequestValidators requestValidators; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java index e28dddbbb017..d6b17e70ad25 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.close; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -49,6 +51,8 @@ import java.util.Collections; */ public class TransportCloseIndexAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportCloseIndexAction.class); + private final MetaDataIndexStateService indexStateService; private final DestructiveOperations destructiveOperations; private volatile boolean closeIndexEnabled; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexClusterStateUpdateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexClusterStateUpdateRequest.java index f9449bae8bef..af6b1e632df8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexClusterStateUpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexClusterStateUpdateRequest.java @@ -147,4 +147,19 @@ public class CreateIndexClusterStateUpdateRequest extends ClusterStateUpdateRequ return copySettings; } + @Override + public String toString() { + return "CreateIndexClusterStateUpdateRequest{" + + "cause='" + cause + '\'' + + ", index='" + index + '\'' + + ", providedName='" + providedName + '\'' + + ", recoverFrom=" + recoverFrom + + ", resizeType=" + resizeType + + ", copySettings=" + copySettings + + ", settings=" + settings + + ", aliases=" + aliases + + ", blocks=" + blocks + + ", waitForActiveShards=" + waitForActiveShards + + '}'; + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java index 19341b63f172..87a5362e04c8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.delete; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -48,6 +50,8 @@ import java.util.Set; */ public class TransportDeleteIndexAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportDeleteIndexAction.class); + private final MetaDataDeleteIndexService deleteIndexService; private final DestructiveOperations destructiveOperations; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetMappingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetMappingsAction.java index 0dda5fc8cfe1..6a5376246166 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetMappingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetMappingsAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.mapping.get; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.info.TransportClusterInfoAction; @@ -39,6 +41,8 @@ import java.io.IOException; public class TransportGetMappingsAction extends TransportClusterInfoAction { + private static final Logger logger = LogManager.getLogger(TransportGetMappingsAction.class); + private final IndicesService indicesService; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java index 41d807a0cf48..4a9d833a29c5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/TransportPutMappingAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.mapping.put; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.RequestValidators; @@ -49,6 +51,8 @@ import java.util.Optional; */ public class TransportPutMappingAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportPutMappingAction.class); + private final MetaDataMappingService metaDataMappingService; private final RequestValidators requestValidators; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java index 53c015596049..b3eb1bcb6412 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/open/TransportOpenIndexAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.open; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -45,6 +47,8 @@ import java.io.IOException; */ public class TransportOpenIndexAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportOpenIndexAction.class); + private final MetaDataIndexStateService indexStateService; private final DestructiveOperations destructiveOperations; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java index 7f69e8bc05bd..f58f9ab943e0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.settings.put; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -43,6 +45,8 @@ import java.io.IOException; public class TransportUpdateSettingsAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportUpdateSettingsAction.class); + private final MetaDataUpdateSettingsService updateSettingsService; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java index 5c23753d53bf..6efb9c8e89be 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.admin.indices.shards; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.action.ActionListener; @@ -70,6 +71,8 @@ import java.util.concurrent.ConcurrentLinkedQueue; public class TransportIndicesShardStoresAction extends TransportMasterNodeReadAction { + private static final Logger logger = LogManager.getLogger(TransportIndicesShardStoresAction.class); + private final NodeClient client; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java index c6b252067805..707d4ad71ebf 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/delete/TransportDeleteIndexTemplateAction.java @@ -18,6 +18,8 @@ */ package org.elasticsearch.action.admin.indices.template.delete; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -43,6 +45,8 @@ import java.io.IOException; public class TransportDeleteIndexTemplateAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportDeleteIndexTemplateAction.class); + private final MetaDataIndexTemplateService indexTemplateService; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java index ad18a4b8bc72..753b517a0cd9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/TransportPutIndexTemplateAction.java @@ -18,6 +18,8 @@ */ package org.elasticsearch.action.admin.indices.template.put; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -45,6 +47,8 @@ import java.io.IOException; */ public class TransportPutIndexTemplateAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportPutIndexTemplateAction.class); + private final MetaDataIndexTemplateService indexTemplateService; private final IndexScopedSettings indexScopedSettings; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java index a3283a1e3386..8a72d8ee8054 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/post/TransportUpgradeSettingsAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.action.admin.indices.upgrade.post; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -41,6 +43,8 @@ import java.io.IOException; public class TransportUpgradeSettingsAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportUpgradeSettingsAction.class); + private final MetaDataUpdateSettingsService updateSettingsService; @Inject diff --git a/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java b/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java index f325e09a9fbf..d4ac88c8b723 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/TransportMasterNodeAction.java @@ -58,7 +58,7 @@ import java.util.function.Predicate; public abstract class TransportMasterNodeAction, Response extends ActionResponse> extends HandledTransportAction { - protected static final Logger logger = LogManager.getLogger(TransportMasterNodeAction.class); + private static final Logger logger = LogManager.getLogger(TransportMasterNodeAction.class); protected final ThreadPool threadPool; protected final TransportService transportService; @@ -121,6 +121,7 @@ public abstract class TransportMasterNodeAction { try { ClusterBlockException newException = checkBlock(request, newState); return (newException == null || !newException.retryable()); } catch (Exception e) { // accept state as block will be rechecked by doStart() and listener.onFailure() then called - logger.trace("exception occurred during cluster block checking, accepting state", e); + logger.debug("exception occurred during cluster block checking, accepting state", e); return true; } }); @@ -156,6 +158,7 @@ public abstract class TransportMasterNodeAction(listener, TransportMasterNodeAction.this::read) { @Override @@ -181,6 +185,8 @@ public abstract class TransportMasterNodeAction listener) { + logger.trace("createIndex[{}]", request); onlyCreateIndex(request, ActionListener.wrap(response -> { if (response.isAcknowledged()) { + logger.trace("[{}] index creation acknowledged, waiting for active shards [{}]", + request.index(), request.waitForActiveShards()); activeShardsObserver.waitForActiveShards(new String[]{request.index()}, request.waitForActiveShards(), request.ackTimeout(), shardsAcknowledged -> { if (shardsAcknowledged == false) { logger.debug("[{}] index created, but the operation timed out while waiting for " + "enough shards to be started.", request.index()); + } else { + logger.trace("[{}] index created and shards acknowledged", request.index()); } listener.onResponse(new CreateIndexClusterStateUpdateResponse(response.isAcknowledged(), shardsAcknowledged)); }, listener::onFailure); } else { + logger.trace("index creation not acknowledged for [{}]", request); listener.onResponse(new CreateIndexClusterStateUpdateResponse(false, false)); } }, listener::onFailure)); @@ -278,6 +284,7 @@ public class MetaDataCreateIndexService { @Override public ClusterState execute(ClusterState currentState) throws Exception { + logger.trace("executing IndexCreationTask for [{}] against cluster state version [{}]", request, currentState.version()); Index createdIndex = null; String removalExtraInfo = null; IndexRemovalReason removalReason = IndexRemovalReason.FAILURE; diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowAction.java index b2d85f7dd751..fb49136bb234 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportUnfollowAction.java @@ -6,6 +6,8 @@ package org.elasticsearch.xpack.ccr.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchSecurityException; @@ -49,6 +51,8 @@ import java.util.Objects; public class TransportUnfollowAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportUnfollowAction.class); + private final Client client; @Inject diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrameUsageTransportAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrameUsageTransportAction.java index 7f0a79581907..5280b8801de1 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrameUsageTransportAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrameUsageTransportAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.dataframe; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.search.SearchRequest; @@ -45,6 +47,8 @@ import java.util.Map; public class DataFrameUsageTransportAction extends XPackUsageFeatureTransportAction { + private static final Logger logger = LogManager.getLogger(DataFrameUsageTransportAction.class); + private final boolean enabled; private final XPackLicenseState licenseState; private final Client client; diff --git a/x-pack/plugin/frozen-indices/src/main/java/org/elasticsearch/xpack/frozen/action/TransportFreezeIndexAction.java b/x-pack/plugin/frozen-indices/src/main/java/org/elasticsearch/xpack/frozen/action/TransportFreezeIndexAction.java index a787a7945fc8..b412a47ea25a 100644 --- a/x-pack/plugin/frozen-indices/src/main/java/org/elasticsearch/xpack/frozen/action/TransportFreezeIndexAction.java +++ b/x-pack/plugin/frozen-indices/src/main/java/org/elasticsearch/xpack/frozen/action/TransportFreezeIndexAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.frozen.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; @@ -48,6 +50,8 @@ import java.util.List; public final class TransportFreezeIndexAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportFreezeIndexAction.class); + private final DestructiveOperations destructiveOperations; private final MetaDataIndexStateService indexStateService; private final TransportCloseIndexAction transportCloseIndexAction; diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java index 9e344cefee22..e5139c1b02f2 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportPutLifecycleAction.java @@ -6,6 +6,8 @@ package org.elasticsearch.xpack.ilm.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; @@ -42,6 +44,8 @@ import java.util.stream.Collectors; */ public class TransportPutLifecycleAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportPutLifecycleAction.class); + @Inject public TransportPutLifecycleAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportRetryAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportRetryAction.java index e308bea9f1d4..1df695c47e5d 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportRetryAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportRetryAction.java @@ -6,6 +6,8 @@ package org.elasticsearch.xpack.ilm.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; @@ -32,6 +34,8 @@ import java.io.IOException; public class TransportRetryAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportRetryAction.class); + IndexLifecycleService indexLifecycleService; @Inject diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java index 45b4e1805174..70e0387d00f1 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java @@ -6,6 +6,8 @@ package org.elasticsearch.xpack.ml.action; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; @@ -90,6 +92,8 @@ import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; public class TransportDeleteJobAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportDeleteJobAction.class); + private static final int MAX_SNAPSHOTS_TO_DELETE = 10000; private final Client client; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java index e31ff5c26fd5..bba6d3db8a0f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.ml.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.WriteRequest; @@ -40,6 +42,8 @@ import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; public class TransportFinalizeJobExecutionAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportFinalizeJobExecutionAction.class); + private final Client client; @Inject diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsAction.java index c3a068329443..8332d3cbfc6c 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.ml.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; @@ -37,6 +39,8 @@ import java.util.Set; public class TransportGetDatafeedsAction extends TransportMasterNodeReadAction { + private static final Logger logger = LogManager.getLogger(TransportGetDatafeedsAction.class); + private final DatafeedConfigProvider datafeedConfigProvider; @Inject diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsStatsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsStatsAction.java index f50ce26132b9..f2da0f65a287 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsStatsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDatafeedsStatsAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.ml.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; @@ -36,6 +38,8 @@ import java.util.stream.Collectors; public class TransportGetDatafeedsStatsAction extends TransportMasterNodeReadAction { + private static final Logger logger = LogManager.getLogger(TransportGetDatafeedsStatsAction.class); + private final DatafeedConfigProvider datafeedConfigProvider; private final JobResultsProvider jobResultsProvider; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetJobsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetJobsAction.java index 98b7ece2dcb6..c6617d6d7dd7 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetJobsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetJobsAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.ml.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; @@ -25,6 +27,8 @@ import java.io.IOException; public class TransportGetJobsAction extends TransportMasterNodeReadAction { + private static final Logger logger = LogManager.getLogger(TransportGetJobsAction.class); + private final JobManager jobManager; @Inject diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java index b2f94020fbb9..e7a1ece44a82 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java @@ -81,6 +81,8 @@ import static org.elasticsearch.xpack.ml.MachineLearning.MAX_OPEN_JOBS_PER_NODE; */ public class TransportOpenJobAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportOpenJobAction.class); + static final PersistentTasksCustomMetaData.Assignment AWAITING_MIGRATION = new PersistentTasksCustomMetaData.Assignment(null, "job cannot be assigned until it has been migrated."); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java index 1a5ba7629ca8..7b394b580044 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportRevertModelSnapshotAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.ml.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -42,6 +44,8 @@ import java.util.function.Consumer; public class TransportRevertModelSnapshotAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportRevertModelSnapshotAction.class); + private final Client client; private final JobManager jobManager; private final JobResultsProvider jobResultsProvider; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java index a65f7cc615df..1f44d22c6681 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.ml.action; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceAlreadyExistsException; @@ -79,6 +81,8 @@ import java.util.function.Predicate; */ public class TransportStartDatafeedAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportStartDatafeedAction.class); + private final Client client; private final XPackLicenseState licenseState; private final PersistentTasksService persistentTasksService; diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportPutRollupJobAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportPutRollupJobAction.java index 99111d43edde..b18b91711380 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportPutRollupJobAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportPutRollupJobAction.java @@ -62,6 +62,9 @@ import java.util.Objects; import java.util.stream.Collectors; public class TransportPutRollupJobAction extends TransportMasterNodeAction { + + private static final Logger logger = LogManager.getLogger(TransportPutRollupJobAction.class); + private final XPackLicenseState licenseState; private final PersistentTasksService persistentTasksService; private final Client client; diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/service/TransportWatcherServiceAction.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/service/TransportWatcherServiceAction.java index 675d04dc3d0f..92200bcc4f37 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/service/TransportWatcherServiceAction.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/service/TransportWatcherServiceAction.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.watcher.transport.actions.service; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; @@ -34,6 +36,8 @@ import java.io.IOException; public class TransportWatcherServiceAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportWatcherServiceAction.class); + private AckedRequest ackedRequest = new AckedRequest() { @Override public TimeValue ackTimeout() { From 0da251f2c6b7180c8a56095dcc4ce1119dc6fc9f Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 6 Sep 2019 14:29:46 +0200 Subject: [PATCH 089/103] [Transform] simplify class structure of indexer (#46306) simplify transform task and indexer - remove redundant transform id - moving client data frame indexer (and builder) into a separate file --- .../transforms/ClientDataFrameIndexer.java | 591 +++++++++++++++ .../ClientDataFrameIndexerBuilder.java | 123 ++++ .../transforms/DataFrameIndexer.java | 5 + ...FrameTransformPersistentTasksExecutor.java | 6 +- .../transforms/DataFrameTransformTask.java | 689 +----------------- .../ClientDataFrameIndexerTests.java | 2 +- 6 files changed, 739 insertions(+), 677 deletions(-) create mode 100644 x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexer.java create mode 100644 x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerBuilder.java diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexer.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexer.java new file mode 100644 index 000000000000..a64de70dc5ff --- /dev/null +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexer.java @@ -0,0 +1,591 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.dataframe.transforms; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkAction; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.logging.LoggerMessageFormat; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPosition; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.dataframe.checkpoint.CheckpointProvider; +import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; +import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; +import org.elasticsearch.xpack.dataframe.persistence.SeqNoPrimaryTermAndIndex; +import org.elasticsearch.xpack.dataframe.transforms.pivot.AggregationResultUtils; + +import java.time.Instant; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +class ClientDataFrameIndexer extends DataFrameIndexer { + + private static final Logger logger = LogManager.getLogger(ClientDataFrameIndexer.class); + + private long logEvery = 1; + private long logCount = 0; + private final Client client; + private final DataFrameTransformsConfigManager transformsConfigManager; + private final CheckpointProvider checkpointProvider; + private final DataFrameTransformTask transformTask; + private final AtomicInteger failureCount; + private volatile boolean auditBulkFailures = true; + // Indicates that the source has changed for the current run + private volatile boolean hasSourceChanged = true; + // Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index + private volatile String lastAuditedExceptionMessage = null; + private final AtomicBoolean oldStatsCleanedUp = new AtomicBoolean(false); + private volatile Instant changesLastDetectedAt; + + ClientDataFrameIndexer(DataFrameTransformsConfigManager transformsConfigManager, + CheckpointProvider checkpointProvider, + AtomicReference initialState, + DataFrameIndexerPosition initialPosition, + Client client, + DataFrameAuditor auditor, + DataFrameIndexerTransformStats initialStats, + DataFrameTransformConfig transformConfig, + Map fieldMappings, + DataFrameTransformProgress transformProgress, + DataFrameTransformCheckpoint lastCheckpoint, + DataFrameTransformCheckpoint nextCheckpoint, + DataFrameTransformTask parentTask) { + super(ExceptionsHelper.requireNonNull(parentTask, "parentTask") + .getThreadPool() + .executor(ThreadPool.Names.GENERIC), + ExceptionsHelper.requireNonNull(auditor, "auditor"), + transformConfig, + fieldMappings, + ExceptionsHelper.requireNonNull(initialState, "initialState"), + initialPosition, + initialStats == null ? new DataFrameIndexerTransformStats() : initialStats, + transformProgress, + lastCheckpoint, + nextCheckpoint); + this.transformsConfigManager = ExceptionsHelper.requireNonNull(transformsConfigManager, "transformsConfigManager"); + this.checkpointProvider = ExceptionsHelper.requireNonNull(checkpointProvider, "checkpointProvider"); + + this.client = ExceptionsHelper.requireNonNull(client, "client"); + this.transformTask = parentTask; + this.failureCount = new AtomicInteger(0); + } + + @Override + protected void onStart(long now, ActionListener listener) { + if (transformTask.getTaskState() == DataFrameTransformTaskState.FAILED) { + logger.debug("[{}] attempted to start while failed.", getJobId()); + listener.onFailure(new ElasticsearchException("Attempted to start a failed transform [{}].", getJobId())); + return; + } + // On each run, we need to get the total number of docs and reset the count of processed docs + // Since multiple checkpoints can be executed in the task while it is running on the same node, we need to gather + // the progress here, and not in the executor. + ActionListener updateConfigListener = ActionListener.wrap( + updateConfigResponse -> { + if (initialRun()) { + createCheckpoint(ActionListener.wrap(cp -> { + nextCheckpoint = cp; + // If nextCheckpoint > 1, this means that we are now on the checkpoint AFTER the batch checkpoint + // Consequently, the idea of percent complete no longer makes sense. + if (nextCheckpoint.getCheckpoint() > 1) { + progress = new DataFrameTransformProgress(null, 0L, 0L); + super.onStart(now, listener); + return; + } + TransformProgressGatherer.getInitialProgress(this.client, buildFilterQuery(), getConfig(), ActionListener.wrap( + newProgress -> { + logger.trace("[{}] reset the progress from [{}] to [{}].", getJobId(), progress, newProgress); + progress = newProgress; + super.onStart(now, listener); + }, + failure -> { + progress = null; + logger.warn(new ParameterizedMessage("[{}] unable to load progress information for task.", + getJobId()), + failure); + super.onStart(now, listener); + } + )); + }, listener::onFailure)); + } else { + super.onStart(now, listener); + } + }, + listener::onFailure + ); + + // If we are continuous, we will want to verify we have the latest stored configuration + ActionListener changedSourceListener = ActionListener.wrap( + r -> { + if (isContinuous()) { + transformsConfigManager.getTransformConfiguration(getJobId(), ActionListener.wrap( + config -> { + transformConfig = config; + logger.debug("[{}] successfully refreshed data frame transform config from index.", getJobId()); + updateConfigListener.onResponse(null); + }, + failure -> { + String msg = DataFrameMessages.getMessage( + DataFrameMessages.FAILED_TO_RELOAD_TRANSFORM_CONFIGURATION, + getJobId()); + logger.error(msg, failure); + // If the transform config index or the transform config is gone, something serious occurred + // We are in an unknown state and should fail out + if (failure instanceof ResourceNotFoundException) { + updateConfigListener.onFailure(new TransformConfigReloadingException(msg, failure)); + } else { + auditor.warning(getJobId(), msg); + updateConfigListener.onResponse(null); + } + } + )); + } else { + updateConfigListener.onResponse(null); + } + }, + listener::onFailure + ); + + // If we are not on the initial batch checkpoint and its the first pass of whatever continuous checkpoint we are on, + // we should verify if there are local changes based on the sync config. If not, do not proceed further and exit. + if (transformTask.getCheckpoint() > 0 && initialRun()) { + sourceHasChanged(ActionListener.wrap( + hasChanged -> { + hasSourceChanged = hasChanged; + if (hasChanged) { + changesLastDetectedAt = Instant.now(); + logger.debug("[{}] source has changed, triggering new indexer run.", getJobId()); + changedSourceListener.onResponse(null); + } else { + logger.trace("[{}] source has not changed, finish indexer early.", getJobId()); + // No changes, stop executing + listener.onResponse(false); + } + }, + failure -> { + // If we failed determining if the source changed, it's safer to assume there were changes. + // We should allow the failure path to complete as normal + hasSourceChanged = true; + listener.onFailure(failure); + } + )); + } else { + hasSourceChanged = true; + changedSourceListener.onResponse(null); + } + } + + public CheckpointProvider getCheckpointProvider() { + return checkpointProvider; + } + + Instant getChangesLastDetectedAt() { + return changesLastDetectedAt; + } + + @Override + public synchronized boolean maybeTriggerAsyncJob(long now) { + if (transformTask.getTaskState() == DataFrameTransformTaskState.FAILED) { + logger.debug("[{}] schedule was triggered for transform but task is failed. Ignoring trigger.", getJobId()); + return false; + } + + // ignore trigger if indexer is running, prevents log spam in A2P indexer + IndexerState indexerState = getState(); + if (IndexerState.INDEXING.equals(indexerState) || IndexerState.STOPPING.equals(indexerState)) { + logger.debug("[{}] indexer for transform has state [{}]. Ignoring trigger.", getJobId(), indexerState); + return false; + } + + return super.maybeTriggerAsyncJob(now); + } + + @Override + protected void doNextSearch(SearchRequest request, ActionListener nextPhase) { + if (transformTask.getTaskState() == DataFrameTransformTaskState.FAILED) { + logger.debug("[{}] attempted to search while failed.", getJobId()); + nextPhase.onFailure(new ElasticsearchException("Attempted to do a search request for failed transform [{}].", + getJobId())); + return; + } + ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), ClientHelper.DATA_FRAME_ORIGIN, client, + SearchAction.INSTANCE, request, nextPhase); + } + + @Override + protected void doNextBulk(BulkRequest request, ActionListener nextPhase) { + if (transformTask.getTaskState() == DataFrameTransformTaskState.FAILED) { + logger.debug("[{}] attempted to bulk index while failed.", getJobId()); + nextPhase.onFailure(new ElasticsearchException("Attempted to do a bulk index request for failed transform [{}].", + getJobId())); + return; + } + ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), + ClientHelper.DATA_FRAME_ORIGIN, + client, + BulkAction.INSTANCE, + request, + ActionListener.wrap(bulkResponse -> { + if (bulkResponse.hasFailures()) { + int failureCount = 0; + for(BulkItemResponse item : bulkResponse.getItems()) { + if (item.isFailed()) { + failureCount++; + } + // TODO gather information on irrecoverable failures and update isIrrecoverableFailure + } + if (auditBulkFailures) { + auditor.warning(getJobId(), + "Experienced at least [" + + failureCount + + "] bulk index failures. See the logs of the node running the transform for details. " + + bulkResponse.buildFailureMessage()); + auditBulkFailures = false; + } + // This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure + // It increments the indexing failure, and then calls the `onFailure` logic + nextPhase.onFailure( + new BulkIndexingException("Bulk index experienced failures. " + + "See the logs of the node running the transform for details.")); + } else { + auditBulkFailures = true; + nextPhase.onResponse(bulkResponse); + } + }, nextPhase::onFailure)); + } + + @Override + protected void doSaveState(IndexerState indexerState, DataFrameIndexerPosition position, Runnable next) { + if (transformTask.getTaskState() == DataFrameTransformTaskState.FAILED) { + logger.debug("[{}] attempted to save state and stats while failed.", getJobId()); + // If we are failed, we should call next to allow failure handling to occur if necessary. + next.run(); + return; + } + if (indexerState.equals(IndexerState.ABORTING)) { + // If we're aborting, just invoke `next` (which is likely an onFailure handler) + next.run(); + return; + } + + // This means that the indexer was triggered to discover changes, found none, and exited early. + // If the state is `STOPPED` this means that DataFrameTransformTask#stop was called while we were checking for changes. + // Allow the stop call path to continue + if (hasSourceChanged == false && indexerState.equals(IndexerState.STOPPED) == false) { + next.run(); + return; + } + + DataFrameTransformTaskState taskState = transformTask.getTaskState(); + + if (indexerState.equals(IndexerState.STARTED) + && transformTask.getCheckpoint() == 1 + && this.isContinuous() == false) { + // set both to stopped so they are persisted as such + indexerState = IndexerState.STOPPED; + + auditor.info(transformConfig.getId(), "Data frame finished indexing all data, initiating stop"); + logger.info("[{}] data frame transform finished indexing all data, initiating stop.", transformConfig.getId()); + } + + // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING` + // OR we called `doSaveState` manually as the indexer was not actively running. + // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state + if (indexerState.equals(IndexerState.STOPPED)) { + // We don't want adjust the stored taskState because as soon as it is `STOPPED` a user could call + // .start again. + taskState = DataFrameTransformTaskState.STOPPED; + } + + final DataFrameTransformState state = new DataFrameTransformState( + taskState, + indexerState, + position, + transformTask.getCheckpoint(), + transformTask.getStateReason(), + getProgress()); + logger.debug("[{}] updating persistent state of transform to [{}].", transformConfig.getId(), state.toString()); + + // This could be `null` but the putOrUpdateTransformStoredDoc handles that case just fine + SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = transformTask.getSeqNoPrimaryTermAndIndex(); + + // Persist the current state and stats in the internal index. The interval of this method being + // called is controlled by AsyncTwoPhaseIndexer#onBulkResponse which calls doSaveState every so + // often when doing bulk indexing calls or at the end of one indexing run. + transformsConfigManager.putOrUpdateTransformStoredDoc( + new DataFrameTransformStoredDoc(getJobId(), state, getStats()), + seqNoPrimaryTermAndIndex, + ActionListener.wrap( + r -> { + transformTask.updateSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex, r); + // for auto stop shutdown the task + if (state.getTaskState().equals(DataFrameTransformTaskState.STOPPED)) { + transformTask.shutdown(); + } + // Only do this clean up once, if it succeeded, no reason to do the query again. + if (oldStatsCleanedUp.compareAndSet(false, true)) { + transformsConfigManager.deleteOldTransformStoredDocuments(getJobId(), ActionListener.wrap( + nil -> { + logger.trace("[{}] deleted old transform stats and state document", getJobId()); + next.run(); + }, + e -> { + String msg = LoggerMessageFormat.format("[{}] failed deleting old transform configurations.", + getJobId()); + logger.warn(msg, e); + // If we have failed, we should attempt the clean up again later + oldStatsCleanedUp.set(false); + next.run(); + } + )); + } else { + next.run(); + } + }, + statsExc -> { + logger.error(new ParameterizedMessage("[{}] updating stats of transform failed.", + transformConfig.getId()), + statsExc); + auditor.warning(getJobId(), + "Failure updating stats of transform: " + statsExc.getMessage()); + // for auto stop shutdown the task + if (state.getTaskState().equals(DataFrameTransformTaskState.STOPPED)) { + transformTask.shutdown(); + } + next.run(); + } + )); + } + + @Override + protected void onFailure(Exception exc) { + // the failure handler must not throw an exception due to internal problems + try { + handleFailure(exc); + } catch (Exception e) { + logger.error( + new ParameterizedMessage("[{}] data frame transform encountered an unexpected internal exception: ", getJobId()), + e); + } + } + + @Override + protected void onFinish(ActionListener listener) { + try { + // This indicates an early exit since no changes were found. + // So, don't treat this like a checkpoint being completed, as no work was done. + if (hasSourceChanged == false) { + listener.onResponse(null); + return; + } + // TODO: needs cleanup super is called with a listener, but listener.onResponse is called below + // super.onFinish() fortunately ignores the listener + super.onFinish(listener); + long checkpoint = transformTask.incrementCheckpoint(); + lastCheckpoint = getNextCheckpoint(); + nextCheckpoint = null; + // Reset our failure count as we have finished and may start again with a new checkpoint + failureCount.set(0); + transformTask.setStateReason(null); + + // With bucket_selector we could have read all the buckets and completed the transform + // but not "see" all the buckets since they were filtered out. Consequently, progress would + // show less than 100% even though we are done. + // NOTE: this method is called in the same thread as the processing thread. + // Theoretically, there should not be a race condition with updating progress here. + // NOTE 2: getPercentComplete should only NOT be null on the first (batch) checkpoint + if (progress != null && progress.getPercentComplete() != null && progress.getPercentComplete() < 100.0) { + progress.incrementDocsProcessed(progress.getTotalDocs() - progress.getDocumentsProcessed()); + } + // If the last checkpoint is now greater than 1, that means that we have just processed the first + // continuous checkpoint and should start recording the exponential averages + if (lastCheckpoint != null && lastCheckpoint.getCheckpoint() > 1) { + long docsIndexed = 0; + long docsProcessed = 0; + // This should not happen as we simply create a new one when we reach continuous checkpoints + // but this is a paranoid `null` check + if (progress != null) { + docsIndexed = progress.getDocumentsIndexed(); + docsProcessed = progress.getDocumentsProcessed(); + } + long durationMs = System.currentTimeMillis() - lastCheckpoint.getTimestamp(); + getStats().incrementCheckpointExponentialAverages(durationMs < 0 ? 0 : durationMs, docsIndexed, docsProcessed); + } + if (shouldAuditOnFinish(checkpoint)) { + auditor.info(getJobId(), + "Finished indexing for data frame transform checkpoint [" + checkpoint + "]."); + } + logger.debug( + "[{}] finished indexing for data frame transform checkpoint [{}].", getJobId(), checkpoint); + auditBulkFailures = true; + listener.onResponse(null); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Indicates if an audit message should be written when onFinish is called for the given checkpoint + * We audit the first checkpoint, and then every 10 checkpoints until completedCheckpoint == 99 + * Then we audit every 100, until completedCheckpoint == 999 + * + * Then we always audit every 1_000 checkpoints + * + * @param completedCheckpoint The checkpoint that was just completed + * @return {@code true} if an audit message should be written + */ + protected boolean shouldAuditOnFinish(long completedCheckpoint) { + if (++logCount % logEvery != 0) { + return false; + } + if (completedCheckpoint == 0) { + return true; + } + int log10Checkpoint = (int) Math.floor(Math.log10(completedCheckpoint)); + logEvery = log10Checkpoint >= 3 ? 1_000 : (int)Math.pow(10.0, log10Checkpoint); + logCount = 0; + return true; + } + + @Override + protected void onStop() { + auditor.info(transformConfig.getId(), "Data frame transform has stopped."); + logger.info("[{}] data frame transform has stopped.", transformConfig.getId()); + } + + @Override + protected void onAbort() { + auditor.info(transformConfig.getId(), "Received abort request, stopping data frame transform."); + logger.info("[{}] data frame transform received abort request. Stopping indexer.", transformConfig.getId()); + transformTask.shutdown(); + } + + @Override + protected void createCheckpoint(ActionListener listener) { + checkpointProvider.createNextCheckpoint(getLastCheckpoint(), ActionListener.wrap( + checkpoint -> transformsConfigManager.putTransformCheckpoint(checkpoint, + ActionListener.wrap( + putCheckPointResponse -> listener.onResponse(checkpoint), + createCheckpointException -> { + logger.warn(new ParameterizedMessage("[{}] failed to create checkpoint.", getJobId()), + createCheckpointException); + listener.onFailure( + new RuntimeException("Failed to create checkpoint due to " + createCheckpointException.getMessage(), + createCheckpointException)); + } + )), + getCheckPointException -> { + logger.warn(new ParameterizedMessage("[{}] failed to retrieve checkpoint.", getJobId()), + getCheckPointException); + listener.onFailure( + new RuntimeException("Failed to retrieve checkpoint due to " + getCheckPointException.getMessage(), + getCheckPointException)); + } + )); + } + + @Override + protected void sourceHasChanged(ActionListener hasChangedListener) { + checkpointProvider.sourceHasChanged(getLastCheckpoint(), + ActionListener.wrap( + hasChanged -> { + logger.trace("[{}] change detected [{}].", getJobId(), hasChanged); + hasChangedListener.onResponse(hasChanged); + }, + e -> { + logger.warn( + new ParameterizedMessage( + "[{}] failed to detect changes for data frame transform. Skipping update till next check.", + getJobId()), + e); + auditor.warning(getJobId(), + "Failed to detect changes for data frame transform, skipping update till next check. Exception: " + + e.getMessage()); + hasChangedListener.onResponse(false); + })); + } + + private boolean isIrrecoverableFailure(Exception e) { + return e instanceof IndexNotFoundException + || e instanceof AggregationResultUtils.AggregationExtractionException + || e instanceof TransformConfigReloadingException; + } + + synchronized void handleFailure(Exception e) { + logger.warn(new ParameterizedMessage("[{}] data frame transform encountered an exception: ", + getJobId()), + e); + if (handleCircuitBreakingException(e)) { + return; + } + + if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > transformTask.getNumFailureRetries()) { + String failureMessage = isIrrecoverableFailure(e) ? + "task encountered irrecoverable failure: " + e.getMessage() : + "task encountered more than " + transformTask.getNumFailureRetries() + " failures; latest failure: " + e.getMessage(); + failIndexer(failureMessage); + } else { + // Since our schedule fires again very quickly after failures it is possible to run into the same failure numerous + // times in a row, very quickly. We do not want to spam the audit log with repeated failures, so only record the first one + if (e.getMessage().equals(lastAuditedExceptionMessage) == false) { + auditor.warning(getJobId(), + "Data frame transform encountered an exception: " + e.getMessage() + + " Will attempt again at next scheduled trigger."); + lastAuditedExceptionMessage = e.getMessage(); + } + } + } + + @Override + protected void failIndexer(String failureMessage) { + logger.error("[{}] transform has failed; experienced: [{}].", getJobId(), failureMessage); + auditor.error(getJobId(), failureMessage); + transformTask.markAsFailed(failureMessage, ActionListener.wrap( + r -> { + // Successfully marked as failed, reset counter so that task can be restarted + failureCount.set(0); + }, e -> {})); + } + + // Considered a recoverable indexing failure + private static class BulkIndexingException extends ElasticsearchException { + BulkIndexingException(String msg, Object... args) { + super(msg, args); + } + } + + private static class TransformConfigReloadingException extends ElasticsearchException { + TransformConfigReloadingException(String msg, Throwable cause, Object... args) { + super(msg, cause, args); + } + } +} diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerBuilder.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerBuilder.java new file mode 100644 index 000000000000..672cdcc25eff --- /dev/null +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerBuilder.java @@ -0,0 +1,123 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.dataframe.transforms; + +import org.elasticsearch.client.Client; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPosition; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; +import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.dataframe.checkpoint.CheckpointProvider; +import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService; +import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; +import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +class ClientDataFrameIndexerBuilder { + private Client client; + private DataFrameTransformsConfigManager transformsConfigManager; + private DataFrameTransformsCheckpointService transformsCheckpointService; + private DataFrameAuditor auditor; + private Map fieldMappings; + private DataFrameTransformConfig transformConfig; + private DataFrameIndexerTransformStats initialStats; + private IndexerState indexerState = IndexerState.STOPPED; + private DataFrameIndexerPosition initialPosition; + private DataFrameTransformProgress progress; + private DataFrameTransformCheckpoint lastCheckpoint; + private DataFrameTransformCheckpoint nextCheckpoint; + + ClientDataFrameIndexerBuilder() { + this.initialStats = new DataFrameIndexerTransformStats(); + } + + ClientDataFrameIndexer build(DataFrameTransformTask parentTask) { + CheckpointProvider checkpointProvider = transformsCheckpointService.getCheckpointProvider(transformConfig); + + return new ClientDataFrameIndexer(this.transformsConfigManager, + checkpointProvider, + new AtomicReference<>(this.indexerState), + this.initialPosition, + this.client, + this.auditor, + this.initialStats, + this.transformConfig, + this.fieldMappings, + this.progress, + this.lastCheckpoint, + this.nextCheckpoint, + parentTask); + } + + ClientDataFrameIndexerBuilder setClient(Client client) { + this.client = client; + return this; + } + + ClientDataFrameIndexerBuilder setTransformsConfigManager(DataFrameTransformsConfigManager transformsConfigManager) { + this.transformsConfigManager = transformsConfigManager; + return this; + } + + ClientDataFrameIndexerBuilder setTransformsCheckpointService(DataFrameTransformsCheckpointService transformsCheckpointService) { + this.transformsCheckpointService = transformsCheckpointService; + return this; + } + + ClientDataFrameIndexerBuilder setAuditor(DataFrameAuditor auditor) { + this.auditor = auditor; + return this; + } + + ClientDataFrameIndexerBuilder setFieldMappings(Map fieldMappings) { + this.fieldMappings = fieldMappings; + return this; + } + + ClientDataFrameIndexerBuilder setTransformConfig(DataFrameTransformConfig transformConfig) { + this.transformConfig = transformConfig; + return this; + } + + DataFrameTransformConfig getTransformConfig() { + return this.transformConfig; + } + + ClientDataFrameIndexerBuilder setInitialStats(DataFrameIndexerTransformStats initialStats) { + this.initialStats = initialStats; + return this; + } + + ClientDataFrameIndexerBuilder setIndexerState(IndexerState indexerState) { + this.indexerState = indexerState; + return this; + } + + ClientDataFrameIndexerBuilder setInitialPosition(DataFrameIndexerPosition initialPosition) { + this.initialPosition = initialPosition; + return this; + } + + ClientDataFrameIndexerBuilder setProgress(DataFrameTransformProgress progress) { + this.progress = progress; + return this; + } + + ClientDataFrameIndexerBuilder setLastCheckpoint(DataFrameTransformCheckpoint lastCheckpoint) { + this.lastCheckpoint = lastCheckpoint; + return this; + } + + ClientDataFrameIndexerBuilder setNextCheckpoint(DataFrameTransformCheckpoint nextCheckpoint) { + this.nextCheckpoint = nextCheckpoint; + return this; + } +} \ No newline at end of file diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexer.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexer.java index 6076c6dd15b1..bb03df373cd5 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexer.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexer.java @@ -117,6 +117,11 @@ public abstract class DataFrameIndexer extends AsyncTwoPhaseIndexer listener) { buildTask.initializeIndexer(indexerBuilder); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java index a31c148a1ed4..a5d0bfd2ff9a 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformTask.java @@ -12,27 +12,15 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.bulk.BulkAction; -import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.search.SearchAction; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.Client; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction; @@ -40,29 +28,18 @@ import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTask import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPosition; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; -import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo; -import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; -import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; -import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; -import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine.Event; -import org.elasticsearch.xpack.dataframe.checkpoint.CheckpointProvider; import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService; import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; -import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; import org.elasticsearch.xpack.dataframe.persistence.SeqNoPrimaryTermAndIndex; -import org.elasticsearch.xpack.dataframe.transforms.pivot.AggregationResultUtils; -import java.time.Instant; import java.util.Arrays; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -94,7 +71,6 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S private final DataFrameAuditor auditor; private final DataFrameIndexerPosition initialPosition; private final IndexerState initialIndexerState; - private volatile Instant changesLastDetectedAt; private final SetOnce indexer = new SetOnce<>(); @@ -190,6 +166,10 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S return currentCheckpoint.get(); } + long incrementCheckpoint() { + return currentCheckpoint.getAndIncrement(); + } + public void getCheckpointingInfo(DataFrameTransformsCheckpointService transformsCheckpointService, ActionListener listener) { ClientDataFrameIndexer indexer = getIndexer(); @@ -209,10 +189,10 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S indexer.getProgress(), ActionListener.wrap( info -> { - if (changesLastDetectedAt == null) { + if (indexer.getChangesLastDetectedAt() == null) { listener.onResponse(info); } else { - listener.onResponse(info.setChangesLastDetectedAt(changesLastDetectedAt)); + listener.onResponse(info.setChangesLastDetectedAt(indexer.getChangesLastDetectedAt())); } }, listener::onFailure @@ -469,12 +449,12 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S } } - public DataFrameTransformTask setNumFailureRetries(int numFailureRetries) { + DataFrameTransformTask setNumFailureRetries(int numFailureRetries) { this.numFailureRetries = numFailureRetries; return this; } - public int getNumFailureRetries() { + int getNumFailureRetries() { return numFailureRetries; } @@ -517,656 +497,19 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S return seqNoPrimaryTermAndIndex.get(); } - static class ClientDataFrameIndexerBuilder { - private Client client; - private DataFrameTransformsConfigManager transformsConfigManager; - private DataFrameTransformsCheckpointService transformsCheckpointService; - private String transformId; - private DataFrameAuditor auditor; - private Map fieldMappings; - private DataFrameTransformConfig transformConfig; - private DataFrameIndexerTransformStats initialStats; - private IndexerState indexerState = IndexerState.STOPPED; - private DataFrameIndexerPosition initialPosition; - private DataFrameTransformProgress progress; - private DataFrameTransformCheckpoint lastCheckpoint; - private DataFrameTransformCheckpoint nextCheckpoint; - - ClientDataFrameIndexerBuilder(String transformId) { - this.transformId = transformId; - this.initialStats = new DataFrameIndexerTransformStats(); - } - - ClientDataFrameIndexer build(DataFrameTransformTask parentTask) { - CheckpointProvider checkpointProvider = transformsCheckpointService.getCheckpointProvider(transformConfig); - - return new ClientDataFrameIndexer(this.transformId, - this.transformsConfigManager, - checkpointProvider, - new AtomicReference<>(this.indexerState), - this.initialPosition, - this.client, - this.auditor, - this.initialStats, - this.transformConfig, - this.fieldMappings, - this.progress, - this.lastCheckpoint, - this.nextCheckpoint, - parentTask); - } - - ClientDataFrameIndexerBuilder setClient(Client client) { - this.client = client; - return this; - } - - ClientDataFrameIndexerBuilder setTransformsConfigManager(DataFrameTransformsConfigManager transformsConfigManager) { - this.transformsConfigManager = transformsConfigManager; - return this; - } - - ClientDataFrameIndexerBuilder setTransformsCheckpointService(DataFrameTransformsCheckpointService transformsCheckpointService) { - this.transformsCheckpointService = transformsCheckpointService; - return this; - } - - ClientDataFrameIndexerBuilder setTransformId(String transformId) { - this.transformId = transformId; - return this; - } - - ClientDataFrameIndexerBuilder setAuditor(DataFrameAuditor auditor) { - this.auditor = auditor; - return this; - } - - ClientDataFrameIndexerBuilder setFieldMappings(Map fieldMappings) { - this.fieldMappings = fieldMappings; - return this; - } - - ClientDataFrameIndexerBuilder setTransformConfig(DataFrameTransformConfig transformConfig) { - this.transformConfig = transformConfig; - return this; - } - - DataFrameTransformConfig getTransformConfig() { - return this.transformConfig; - } - - ClientDataFrameIndexerBuilder setInitialStats(DataFrameIndexerTransformStats initialStats) { - this.initialStats = initialStats; - return this; - } - - ClientDataFrameIndexerBuilder setIndexerState(IndexerState indexerState) { - this.indexerState = indexerState; - return this; - } - - ClientDataFrameIndexerBuilder setInitialPosition(DataFrameIndexerPosition initialPosition) { - this.initialPosition = initialPosition; - return this; - } - - ClientDataFrameIndexerBuilder setProgress(DataFrameTransformProgress progress) { - this.progress = progress; - return this; - } - - ClientDataFrameIndexerBuilder setLastCheckpoint(DataFrameTransformCheckpoint lastCheckpoint) { - this.lastCheckpoint = lastCheckpoint; - return this; - } - - ClientDataFrameIndexerBuilder setNextCheckpoint(DataFrameTransformCheckpoint nextCheckpoint) { - this.nextCheckpoint = nextCheckpoint; - return this; - } + ThreadPool getThreadPool() { + return threadPool; } - static class ClientDataFrameIndexer extends DataFrameIndexer { - - private long logEvery = 1; - private long logCount = 0; - private final Client client; - private final DataFrameTransformsConfigManager transformsConfigManager; - private final CheckpointProvider checkpointProvider; - private final String transformId; - private final DataFrameTransformTask transformTask; - private final AtomicInteger failureCount; - private volatile boolean auditBulkFailures = true; - // Indicates that the source has changed for the current run - private volatile boolean hasSourceChanged = true; - // Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index - private volatile String lastAuditedExceptionMessage = null; - private final AtomicBoolean oldStatsCleanedUp = new AtomicBoolean(false); - - ClientDataFrameIndexer(String transformId, - DataFrameTransformsConfigManager transformsConfigManager, - CheckpointProvider checkpointProvider, - AtomicReference initialState, - DataFrameIndexerPosition initialPosition, - Client client, - DataFrameAuditor auditor, - DataFrameIndexerTransformStats initialStats, - DataFrameTransformConfig transformConfig, - Map fieldMappings, - DataFrameTransformProgress transformProgress, - DataFrameTransformCheckpoint lastCheckpoint, - DataFrameTransformCheckpoint nextCheckpoint, - DataFrameTransformTask parentTask) { - super(ExceptionsHelper.requireNonNull(parentTask, "parentTask") - .threadPool - .executor(ThreadPool.Names.GENERIC), - ExceptionsHelper.requireNonNull(auditor, "auditor"), - transformConfig, - fieldMappings, - ExceptionsHelper.requireNonNull(initialState, "initialState"), - initialPosition, - initialStats == null ? new DataFrameIndexerTransformStats() : initialStats, - transformProgress, - lastCheckpoint, - nextCheckpoint); - this.transformId = ExceptionsHelper.requireNonNull(transformId, "transformId"); - this.transformsConfigManager = ExceptionsHelper.requireNonNull(transformsConfigManager, "transformsConfigManager"); - this.checkpointProvider = ExceptionsHelper.requireNonNull(checkpointProvider, "checkpointProvider"); - - this.client = ExceptionsHelper.requireNonNull(client, "client"); - this.transformTask = parentTask; - this.failureCount = new AtomicInteger(0); - } - - @Override - protected void onStart(long now, ActionListener listener) { - if (transformTask.taskState.get() == DataFrameTransformTaskState.FAILED) { - logger.debug("[{}] attempted to start while failed.", transformId); - listener.onFailure(new ElasticsearchException("Attempted to start a failed transform [{}].", transformId)); - return; - } - // On each run, we need to get the total number of docs and reset the count of processed docs - // Since multiple checkpoints can be executed in the task while it is running on the same node, we need to gather - // the progress here, and not in the executor. - ActionListener updateConfigListener = ActionListener.wrap( - updateConfigResponse -> { - if (initialRun()) { - createCheckpoint(ActionListener.wrap(cp -> { - nextCheckpoint = cp; - // If nextCheckpoint > 1, this means that we are now on the checkpoint AFTER the batch checkpoint - // Consequently, the idea of percent complete no longer makes sense. - if (nextCheckpoint.getCheckpoint() > 1) { - progress = new DataFrameTransformProgress(null, 0L, 0L); - super.onStart(now, listener); - return; - } - TransformProgressGatherer.getInitialProgress(this.client, buildFilterQuery(), getConfig(), ActionListener.wrap( - newProgress -> { - logger.trace("[{}] reset the progress from [{}] to [{}].", transformId, progress, newProgress); - progress = newProgress; - super.onStart(now, listener); - }, - failure -> { - progress = null; - logger.warn(new ParameterizedMessage("[{}] unable to load progress information for task.", - transformId), - failure); - super.onStart(now, listener); - } - )); - }, listener::onFailure)); - } else { - super.onStart(now, listener); - } - }, - listener::onFailure - ); - - // If we are continuous, we will want to verify we have the latest stored configuration - ActionListener changedSourceListener = ActionListener.wrap( - r -> { - if (isContinuous()) { - transformsConfigManager.getTransformConfiguration(getJobId(), ActionListener.wrap( - config -> { - transformConfig = config; - logger.debug("[{}] successfully refreshed data frame transform config from index.", transformId); - updateConfigListener.onResponse(null); - }, - failure -> { - String msg = DataFrameMessages.getMessage( - DataFrameMessages.FAILED_TO_RELOAD_TRANSFORM_CONFIGURATION, - getJobId()); - logger.error(msg, failure); - // If the transform config index or the transform config is gone, something serious occurred - // We are in an unknown state and should fail out - if (failure instanceof ResourceNotFoundException) { - updateConfigListener.onFailure(new TransformConfigReloadingException(msg, failure)); - } else { - auditor.warning(getJobId(), msg); - updateConfigListener.onResponse(null); - } - } - )); - } else { - updateConfigListener.onResponse(null); - } - }, - listener::onFailure - ); - - // If we are not on the initial batch checkpoint and its the first pass of whatever continuous checkpoint we are on, - // we should verify if there are local changes based on the sync config. If not, do not proceed further and exit. - if (transformTask.currentCheckpoint.get() > 0 && initialRun()) { - sourceHasChanged(ActionListener.wrap( - hasChanged -> { - hasSourceChanged = hasChanged; - if (hasChanged) { - transformTask.changesLastDetectedAt = Instant.now(); - logger.debug("[{}] source has changed, triggering new indexer run.", transformId); - changedSourceListener.onResponse(null); - } else { - logger.trace("[{}] source has not changed, finish indexer early.", transformId); - // No changes, stop executing - listener.onResponse(false); - } - }, - failure -> { - // If we failed determining if the source changed, it's safer to assume there were changes. - // We should allow the failure path to complete as normal - hasSourceChanged = true; - listener.onFailure(failure); - } - )); - } else { - hasSourceChanged = true; - changedSourceListener.onResponse(null); - } - } - - @Override - protected String getJobId() { - return transformId; - } - - public CheckpointProvider getCheckpointProvider() { - return checkpointProvider; - } - - @Override - public synchronized boolean maybeTriggerAsyncJob(long now) { - if (transformTask.taskState.get() == DataFrameTransformTaskState.FAILED) { - logger.debug("[{}] schedule was triggered for transform but task is failed. Ignoring trigger.", getJobId()); - return false; - } - - // ignore trigger if indexer is running, prevents log spam in A2P indexer - IndexerState indexerState = getState(); - if (IndexerState.INDEXING.equals(indexerState) || IndexerState.STOPPING.equals(indexerState)) { - logger.debug("[{}] indexer for transform has state [{}]. Ignoring trigger.", getJobId(), indexerState); - return false; - } - - return super.maybeTriggerAsyncJob(now); - } - - @Override - protected void doNextSearch(SearchRequest request, ActionListener nextPhase) { - if (transformTask.taskState.get() == DataFrameTransformTaskState.FAILED) { - logger.debug("[{}] attempted to search while failed.", transformId); - nextPhase.onFailure(new ElasticsearchException("Attempted to do a search request for failed transform [{}].", - transformId)); - return; - } - ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), ClientHelper.DATA_FRAME_ORIGIN, client, - SearchAction.INSTANCE, request, nextPhase); - } - - @Override - protected void doNextBulk(BulkRequest request, ActionListener nextPhase) { - if (transformTask.taskState.get() == DataFrameTransformTaskState.FAILED) { - logger.debug("[{}] attempted to bulk index while failed.", transformId); - nextPhase.onFailure(new ElasticsearchException("Attempted to do a bulk index request for failed transform [{}].", - transformId)); - return; - } - ClientHelper.executeWithHeadersAsync(transformConfig.getHeaders(), - ClientHelper.DATA_FRAME_ORIGIN, - client, - BulkAction.INSTANCE, - request, - ActionListener.wrap(bulkResponse -> { - if (bulkResponse.hasFailures()) { - int failureCount = 0; - for(BulkItemResponse item : bulkResponse.getItems()) { - if (item.isFailed()) { - failureCount++; - } - // TODO gather information on irrecoverable failures and update isIrrecoverableFailure - } - if (auditBulkFailures) { - auditor.warning(transformId, - "Experienced at least [" + - failureCount + - "] bulk index failures. See the logs of the node running the transform for details. " + - bulkResponse.buildFailureMessage()); - auditBulkFailures = false; - } - // This calls AsyncTwoPhaseIndexer#finishWithIndexingFailure - // It increments the indexing failure, and then calls the `onFailure` logic - nextPhase.onFailure( - new BulkIndexingException("Bulk index experienced failures. " + - "See the logs of the node running the transform for details.")); - } else { - auditBulkFailures = true; - nextPhase.onResponse(bulkResponse); - } - }, nextPhase::onFailure)); - } - - @Override - protected void doSaveState(IndexerState indexerState, DataFrameIndexerPosition position, Runnable next) { - if (transformTask.taskState.get() == DataFrameTransformTaskState.FAILED) { - logger.debug("[{}] attempted to save state and stats while failed.", transformId); - // If we are failed, we should call next to allow failure handling to occur if necessary. - next.run(); - return; - } - if (indexerState.equals(IndexerState.ABORTING)) { - // If we're aborting, just invoke `next` (which is likely an onFailure handler) - next.run(); - return; - } - - // This means that the indexer was triggered to discover changes, found none, and exited early. - // If the state is `STOPPED` this means that DataFrameTransformTask#stop was called while we were checking for changes. - // Allow the stop call path to continue - if (hasSourceChanged == false && indexerState.equals(IndexerState.STOPPED) == false) { - next.run(); - return; - } - - DataFrameTransformTaskState taskState = transformTask.taskState.get(); - - if (indexerState.equals(IndexerState.STARTED) - && transformTask.currentCheckpoint.get() == 1 - && this.isContinuous() == false) { - // set both to stopped so they are persisted as such - indexerState = IndexerState.STOPPED; - - auditor.info(transformConfig.getId(), "Data frame finished indexing all data, initiating stop"); - logger.info("[{}] data frame transform finished indexing all data, initiating stop.", transformConfig.getId()); - } - - // If we are `STOPPED` on a `doSaveState` call, that indicates we transitioned to `STOPPED` from `STOPPING` - // OR we called `doSaveState` manually as the indexer was not actively running. - // Since we save the state to an index, we should make sure that our task state is in parity with the indexer state - if (indexerState.equals(IndexerState.STOPPED)) { - // We don't want adjust the stored taskState because as soon as it is `STOPPED` a user could call - // .start again. - taskState = DataFrameTransformTaskState.STOPPED; - } - - final DataFrameTransformState state = new DataFrameTransformState( - taskState, - indexerState, - position, - transformTask.currentCheckpoint.get(), - transformTask.stateReason.get(), - getProgress()); - logger.debug("[{}] updating persistent state of transform to [{}].", transformConfig.getId(), state.toString()); - - // This could be `null` but the putOrUpdateTransformStoredDoc handles that case just fine - SeqNoPrimaryTermAndIndex seqNoPrimaryTermAndIndex = transformTask.getSeqNoPrimaryTermAndIndex(); - - // Persist the current state and stats in the internal index. The interval of this method being - // called is controlled by AsyncTwoPhaseIndexer#onBulkResponse which calls doSaveState every so - // often when doing bulk indexing calls or at the end of one indexing run. - transformsConfigManager.putOrUpdateTransformStoredDoc( - new DataFrameTransformStoredDoc(transformId, state, getStats()), - seqNoPrimaryTermAndIndex, - ActionListener.wrap( - r -> { - transformTask.updateSeqNoPrimaryTermAndIndex(seqNoPrimaryTermAndIndex, r); - // for auto stop shutdown the task - if (state.getTaskState().equals(DataFrameTransformTaskState.STOPPED)) { - transformTask.shutdown(); - } - // Only do this clean up once, if it succeeded, no reason to do the query again. - if (oldStatsCleanedUp.compareAndSet(false, true)) { - transformsConfigManager.deleteOldTransformStoredDocuments(transformId, ActionListener.wrap( - nil -> { - logger.trace("[{}] deleted old transform stats and state document", transformId); - next.run(); - }, - e -> { - String msg = LoggerMessageFormat.format("[{}] failed deleting old transform configurations.", - transformId); - logger.warn(msg, e); - // If we have failed, we should attempt the clean up again later - oldStatsCleanedUp.set(false); - next.run(); - } - )); - } else { - next.run(); - } - }, - statsExc -> { - logger.error(new ParameterizedMessage("[{}] updating stats of transform failed.", - transformConfig.getId()), - statsExc); - auditor.warning(getJobId(), - "Failure updating stats of transform: " + statsExc.getMessage()); - // for auto stop shutdown the task - if (state.getTaskState().equals(DataFrameTransformTaskState.STOPPED)) { - transformTask.shutdown(); - } - next.run(); - } - )); - } - - @Override - protected void onFailure(Exception exc) { - // the failure handler must not throw an exception due to internal problems - try { - handleFailure(exc); - } catch (Exception e) { - logger.error( - new ParameterizedMessage("[{}] data frame transform encountered an unexpected internal exception: ", transformId), - e); - } - } - - @Override - protected void onFinish(ActionListener listener) { - try { - // This indicates an early exit since no changes were found. - // So, don't treat this like a checkpoint being completed, as no work was done. - if (hasSourceChanged == false) { - listener.onResponse(null); - return; - } - // TODO: needs cleanup super is called with a listener, but listener.onResponse is called below - // super.onFinish() fortunately ignores the listener - super.onFinish(listener); - long checkpoint = transformTask.currentCheckpoint.getAndIncrement(); - lastCheckpoint = getNextCheckpoint(); - nextCheckpoint = null; - // Reset our failure count as we have finished and may start again with a new checkpoint - failureCount.set(0); - transformTask.stateReason.set(null); - - // With bucket_selector we could have read all the buckets and completed the transform - // but not "see" all the buckets since they were filtered out. Consequently, progress would - // show less than 100% even though we are done. - // NOTE: this method is called in the same thread as the processing thread. - // Theoretically, there should not be a race condition with updating progress here. - // NOTE 2: getPercentComplete should only NOT be null on the first (batch) checkpoint - if (progress != null && progress.getPercentComplete() != null && progress.getPercentComplete() < 100.0) { - progress.incrementDocsProcessed(progress.getTotalDocs() - progress.getDocumentsProcessed()); - } - // If the last checkpoint is now greater than 1, that means that we have just processed the first - // continuous checkpoint and should start recording the exponential averages - if (lastCheckpoint != null && lastCheckpoint.getCheckpoint() > 1) { - long docsIndexed = 0; - long docsProcessed = 0; - // This should not happen as we simply create a new one when we reach continuous checkpoints - // but this is a paranoid `null` check - if (progress != null) { - docsIndexed = progress.getDocumentsIndexed(); - docsProcessed = progress.getDocumentsProcessed(); - } - long durationMs = System.currentTimeMillis() - lastCheckpoint.getTimestamp(); - getStats().incrementCheckpointExponentialAverages(durationMs < 0 ? 0 : durationMs, docsIndexed, docsProcessed); - } - if (shouldAuditOnFinish(checkpoint)) { - auditor.info(transformTask.getTransformId(), - "Finished indexing for data frame transform checkpoint [" + checkpoint + "]."); - } - logger.debug( - "[{}] finished indexing for data frame transform checkpoint [{}].", getJobId(), checkpoint); - auditBulkFailures = true; - listener.onResponse(null); - } catch (Exception e) { - listener.onFailure(e); - } - } - - /** - * Indicates if an audit message should be written when onFinish is called for the given checkpoint - * We audit the first checkpoint, and then every 10 checkpoints until completedCheckpoint == 99 - * Then we audit every 100, until completedCheckpoint == 999 - * - * Then we always audit every 1_000 checkpoints - * - * @param completedCheckpoint The checkpoint that was just completed - * @return {@code true} if an audit message should be written - */ - protected boolean shouldAuditOnFinish(long completedCheckpoint) { - if (++logCount % logEvery != 0) { - return false; - } - if (completedCheckpoint == 0) { - return true; - } - int log10Checkpoint = (int) Math.floor(Math.log10(completedCheckpoint)); - logEvery = log10Checkpoint >= 3 ? 1_000 : (int)Math.pow(10.0, log10Checkpoint); - logCount = 0; - return true; - } - - @Override - protected void onStop() { - auditor.info(transformConfig.getId(), "Data frame transform has stopped."); - logger.info("[{}] data frame transform has stopped.", transformConfig.getId()); - } - - @Override - protected void onAbort() { - auditor.info(transformConfig.getId(), "Received abort request, stopping data frame transform."); - logger.info("[{}] data frame transform received abort request. Stopping indexer.", transformConfig.getId()); - transformTask.shutdown(); - } - - @Override - protected void createCheckpoint(ActionListener listener) { - checkpointProvider.createNextCheckpoint(getLastCheckpoint(), ActionListener.wrap( - checkpoint -> transformsConfigManager.putTransformCheckpoint(checkpoint, - ActionListener.wrap( - putCheckPointResponse -> listener.onResponse(checkpoint), - createCheckpointException -> { - logger.warn(new ParameterizedMessage("[{}] failed to create checkpoint.", transformId), - createCheckpointException); - listener.onFailure( - new RuntimeException("Failed to create checkpoint due to " + createCheckpointException.getMessage(), - createCheckpointException)); - } - )), - getCheckPointException -> { - logger.warn(new ParameterizedMessage("[{}] failed to retrieve checkpoint.", transformId), - getCheckPointException); - listener.onFailure( - new RuntimeException("Failed to retrieve checkpoint due to " + getCheckPointException.getMessage(), - getCheckPointException)); - } - )); - } - - @Override - protected void sourceHasChanged(ActionListener hasChangedListener) { - checkpointProvider.sourceHasChanged(getLastCheckpoint(), - ActionListener.wrap( - hasChanged -> { - logger.trace("[{}] change detected [{}].", transformId, hasChanged); - hasChangedListener.onResponse(hasChanged); - }, - e -> { - logger.warn( - new ParameterizedMessage( - "[{}] failed to detect changes for data frame transform. Skipping update till next check.", - transformId), - e); - auditor.warning(transformId, - "Failed to detect changes for data frame transform, skipping update till next check. Exception: " - + e.getMessage()); - hasChangedListener.onResponse(false); - })); - } - - private boolean isIrrecoverableFailure(Exception e) { - return e instanceof IndexNotFoundException - || e instanceof AggregationResultUtils.AggregationExtractionException - || e instanceof TransformConfigReloadingException; - } - - synchronized void handleFailure(Exception e) { - logger.warn(new ParameterizedMessage("[{}] data frame transform encountered an exception: ", - transformTask.getTransformId()), - e); - if (handleCircuitBreakingException(e)) { - return; - } - - if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > transformTask.getNumFailureRetries()) { - String failureMessage = isIrrecoverableFailure(e) ? - "task encountered irrecoverable failure: " + e.getMessage() : - "task encountered more than " + transformTask.getNumFailureRetries() + " failures; latest failure: " + e.getMessage(); - failIndexer(failureMessage); - } else { - // Since our schedule fires again very quickly after failures it is possible to run into the same failure numerous - // times in a row, very quickly. We do not want to spam the audit log with repeated failures, so only record the first one - if (e.getMessage().equals(lastAuditedExceptionMessage) == false) { - auditor.warning(transformTask.getTransformId(), - "Data frame transform encountered an exception: " + e.getMessage() + - " Will attempt again at next scheduled trigger."); - lastAuditedExceptionMessage = e.getMessage(); - } - } - } - - @Override - protected void failIndexer(String failureMessage) { - logger.error("[{}] transform has failed; experienced: [{}].", getJobId(), failureMessage); - auditor.error(transformTask.getTransformId(), failureMessage); - transformTask.markAsFailed(failureMessage, ActionListener.wrap( - r -> { - // Successfully marked as failed, reset counter so that task can be restarted - failureCount.set(0); - }, e -> {})); - } + DataFrameTransformTaskState getTaskState() { + return taskState.get(); } - // Considered a recoverable indexing failure - private static class BulkIndexingException extends ElasticsearchException { - BulkIndexingException(String msg, Object... args) { - super(msg, args); - } + void setStateReason(String reason) { + stateReason.set(reason); } - private static class TransformConfigReloadingException extends ElasticsearchException { - TransformConfigReloadingException(String msg, Throwable cause, Object... args) { - super(msg, cause, args); - } + String getStateReason() { + return stateReason.get(); } } diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerTests.java index 4a23a57efccc..e4c0085f9d98 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/ClientDataFrameIndexerTests.java @@ -46,7 +46,7 @@ public class ClientDataFrameIndexerTests extends ESTestCase { mock(DataFrameAuditor.class), threadPool, Collections.emptyMap()); - DataFrameTransformTask.ClientDataFrameIndexer indexer = new DataFrameTransformTask.ClientDataFrameIndexer(randomAlphaOfLength(10), + ClientDataFrameIndexer indexer = new ClientDataFrameIndexer( mock(DataFrameTransformsConfigManager.class), mock(CheckpointProvider.class), new AtomicReference<>(IndexerState.STOPPED), From 89857dd74f0798b18f338f7a98900b006a247e17 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Fri, 6 Sep 2019 14:01:22 +0100 Subject: [PATCH 090/103] [ML] Tolerate total_search_time_ms not mapped in get datafeed stats (#46432) ML users who upgrade from versions prior to 7.4 to 7.4 or later will have ML results indices that do not have mappings for the total_search_time_ms field. Therefore, when searching these indices we must tolerate this field not having a mapping. Fixes #46437 --- .../job/persistence/JobResultsProvider.java | 3 +- .../test/ml/get_datafeed_stats.yml | 59 +++++++++++++++++++ 2 files changed, 61 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java index ea0657a914f3..d9a2bc2e5c30 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java @@ -525,7 +525,8 @@ public class JobResultsProvider { .setSize(1) .setIndicesOptions(IndicesOptions.lenientExpandOpen()) .setQuery(QueryBuilders.idsQuery().addIds(DatafeedTimingStats.documentId(jobId))) - .addSort(SortBuilders.fieldSort(DatafeedTimingStats.TOTAL_SEARCH_TIME_MS.getPreferredName()).order(SortOrder.DESC)); + .addSort(SortBuilders.fieldSort(DatafeedTimingStats.TOTAL_SEARCH_TIME_MS.getPreferredName()) + .unmappedType("double").order(SortOrder.DESC)); } public void getAutodetectParams(Job job, Consumer consumer, Consumer errorHandler) { diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/get_datafeed_stats.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/get_datafeed_stats.yml index 37e424504975..eff3948801fa 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/get_datafeed_stats.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/get_datafeed_stats.yml @@ -255,3 +255,62 @@ setup: - match: { datafeeds.0.state: "stopped"} - match: { datafeeds.1.datafeed_id: "datafeed-2"} - match: { datafeeds.1.state: "stopped"} + +--- +"Test get datafeed stats when total_search_time_ms mapping is missing": + + - skip: + features: headers + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.delete: + index: ".ml-anomalies-shared" + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.create: + index: ".ml-special-index-to-avoid-picking-up-template" + body: + mappings: + properties: + timestamp: + type: date + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.put_alias: + index: ".ml-special-index-to-avoid-picking-up-template" + name: ".ml-anomalies-get-datafeed-stats-1" + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.put_alias: + index: ".ml-special-index-to-avoid-picking-up-template" + name: ".ml-anomalies-.write-get-datafeed-stats-1" + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.put_alias: + index: ".ml-special-index-to-avoid-picking-up-template" + name: ".ml-anomalies-get-datafeed-stats-2" + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.put_alias: + index: ".ml-special-index-to-avoid-picking-up-template" + name: ".ml-anomalies-.write-get-datafeed-stats-2" + + - do: + ml.get_datafeed_stats: {} + - match: { count: 2 } + - match: { datafeeds.0.datafeed_id: "datafeed-1"} + - match: { datafeeds.0.state: "stopped"} + - match: { datafeeds.1.datafeed_id: "datafeed-2"} + - match: { datafeeds.1.state: "stopped"} From e39cdd63c30f17d0760785317b73b59c4559993c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Istv=C3=A1n=20Zolt=C3=A1n=20Szab=C3=B3?= Date: Fri, 6 Sep 2019 15:17:18 +0200 Subject: [PATCH 091/103] [DOCS] Adds progress parameter description to the GET stats data frame analytics API doc. (#46434) --- .../apis/get-dfanalytics-stats.asciidoc | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc b/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc index 77051d786942..3afe309cbac2 100644 --- a/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc +++ b/docs/reference/ml/df-analytics/apis/get-dfanalytics-stats.asciidoc @@ -24,6 +24,7 @@ experimental[] `GET _ml/data_frame/analytics/*/_stats` + [[ml-get-dfanalytics-stats-prereq]] ==== {api-prereq-title} @@ -31,6 +32,7 @@ experimental[] information, see {stack-ov}/security-privileges.html[Security privileges] and {stack-ov}/built-in-roles.html[Built-in roles]. + [[ml-get-dfanalytics-stats-path-params]] ==== {api-path-parms-title} @@ -39,6 +41,7 @@ information, see {stack-ov}/security-privileges.html[Security privileges] and one of these options, the API returns information for the first hundred {dfanalytics-jobs}. + [[ml-get-dfanalytics-stats-query-params]] ==== {api-query-parms-title} @@ -64,6 +67,7 @@ when there are no matches or only partial matches. (Optional, integer) Specifies the maximum number of {dfanalytics-jobs} to obtain. The default value is `100`. + [[ml-get-dfanalytics-stats-response-body]] ==== {api-response-body-title} @@ -72,6 +76,24 @@ The API returns the following information: `data_frame_analytics`:: (array) An array of statistics objects for {dfanalytics-jobs}, which are sorted by the `id` value in ascending order. + + `id`:: + (string) The unique identifier of the {dfanalytics-job}. + + `state`:: + (string) Current state of the {dfanalytics-job}. + + `progress`:: + (array) The progress report of the {dfanalytics-job} by phase. + + `phase`:: + (string) Defines the phase of the {dfanalytics-job}. Possible phases: + `reindexing`, `loading_data`, `analyzing`, and `writing_results`. + + `progress_percent`:: + (integer) The progress that the {dfanalytics-job} has made expressed in + percentage. + [[ml-get-dfanalytics-stats-response-codes]] ==== {api-response-codes-title} @@ -80,6 +102,7 @@ The API returns the following information: If `allow_no_match` is `false`, this code indicates that there are no resources that match the request or only partial matches for the request. + [[ml-get-dfanalytics-stats-example]] ==== {api-examples-title} @@ -90,6 +113,7 @@ GET _ml/data_frame/analytics/loganalytics/_stats // CONSOLE // TEST[skip:TBD] + The API returns the following results: [source,console-result] From 73838499b0b28d13bcff0da0cbfedd72b844f21d Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 6 Sep 2019 09:22:36 -0400 Subject: [PATCH 092/103] [DOCS] Resort common-parms (#46419) --- docs/reference/rest-api/common-parms.asciidoc | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index 7eb73305b359..dbf7928d8514 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -178,25 +178,18 @@ tag::if_seq_no[] sequence number. See <>. end::if_seq_no[] -tag::include-defaults[] -`include_defaults`:: -(Optional, string) If `true`, return all default settings in the response. -Defaults to `false`. -end::include-defaults[] - -tag::include-type-name[] -`include_type_name`:: -deprecated:[7.0.0, Mapping types have been deprecated. See <>.] -(Optional, boolean) If `true`, a mapping type is expected in the body of -mappings. Defaults to `false`. -end::include-type-name[] - tag::index-ignore-unavailable[] `ignore_unavailable`:: (Optional, boolean) If `true`, missing or closed indices are not included in the response. Defaults to `false`. end::index-ignore-unavailable[] +tag::include-defaults[] +`include_defaults`:: +(Optional, string) If `true`, return all default settings in the response. +Defaults to `false`. +end::include-defaults[] + tag::include-segment-file-sizes[] `include_segment_file_sizes`:: (Optional, boolean) @@ -205,6 +198,13 @@ each one of the Lucene index files (only applies if segment stats are requested). Defaults to `false`. end::include-segment-file-sizes[] +tag::include-type-name[] +`include_type_name`:: +deprecated:[7.0.0, Mapping types have been deprecated. See <>.] +(Optional, boolean) If `true`, a mapping type is expected in the body of +mappings. Defaults to `false`. +end::include-type-name[] + tag::include-unloaded-segments[] `include_unloaded_segments`:: (Optional, boolean) If `true`, the response includes information from segments @@ -371,8 +371,8 @@ end::pipeline[] tag::preference[] `preference`:: - (Optional, string) Specifies the node or shard the operation should be - performed on. Random by default. +(Optional, string) Specifies the node or shard the operation should be +performed on. Random by default. end::preference[] tag::search-q[] @@ -430,15 +430,6 @@ tag::scroll_size[] Defaults to 100. end::scroll_size[] -tag::segment-search[] -If `true`, -the segment is searchable. -+ -If `false`, -the segment has most likely been written to disk -but needs a <> to be searchable. -end::segment-search[] - tag::search_timeout[] `search_timeout`:: (Optional, <> Explicit timeout for each search @@ -458,6 +449,15 @@ the segment generation and used internally to create file names in the directory of the shard. end::segment[] +tag::segment-search[] +If `true`, +the segment is searchable. ++ +If `false`, +the segment has most likely been written to disk +but needs a <> to be searchable. +end::segment-search[] + tag::settings[] `settings`:: (Optional, <>) Configuration From 97802d8aff9e77851d8a54a499e815f44fdebb58 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 6 Sep 2019 10:55:16 -0400 Subject: [PATCH 093/103] [DOCS] Change // CONSOLE comments to [source,console] (#46441) --- docs/plugins/repository-azure.asciidoc | 3 +- .../allocation/filtering.asciidoc | 9 +-- .../allocation/prioritization.asciidoc | 6 +- .../index-modules/index-sorting.asciidoc | 15 ++--- .../index-modules/similarity.asciidoc | 21 ++---- docs/reference/index-modules/slowlog.asciidoc | 6 +- docs/reference/index-modules/store.asciidoc | 6 +- docs/reference/indices/add-alias.asciidoc | 15 ++--- docs/reference/indices/alias-exists.asciidoc | 6 +- docs/reference/indices/aliases.asciidoc | 52 +++++---------- docs/reference/indices/analyze.asciidoc | 43 +++++------- docs/reference/indices/apis/freeze.asciidoc | 3 +- .../indices/apis/reload-analyzers.asciidoc | 6 +- docs/reference/indices/apis/unfreeze.asciidoc | 3 +- docs/reference/indices/clearcache.asciidoc | 14 ++-- docs/reference/indices/clone-index.asciidoc | 12 ++-- docs/reference/indices/close.asciidoc | 6 +- docs/reference/indices/create-index.asciidoc | 22 +++---- docs/reference/indices/delete-alias.asciidoc | 3 +- .../indices/delete-index-template.asciidoc | 6 +- docs/reference/indices/delete-index.asciidoc | 3 +- docs/reference/indices/flush.asciidoc | 15 ++--- docs/reference/indices/forcemerge.asciidoc | 12 ++-- docs/reference/indices/get-alias.asciidoc | 15 ++--- .../indices/get-field-mapping.asciidoc | 18 ++--- .../indices/get-index-template.asciidoc | 15 ++--- docs/reference/indices/get-index.asciidoc | 3 +- docs/reference/indices/get-mapping.asciidoc | 9 +-- docs/reference/indices/get-settings.asciidoc | 9 +-- .../reference/indices/indices-exists.asciidoc | 3 +- docs/reference/indices/open-close.asciidoc | 6 +- docs/reference/indices/put-mapping.asciidoc | 16 ++--- docs/reference/indices/recovery.asciidoc | 12 ++-- docs/reference/indices/refresh.asciidoc | 6 +- .../reference/indices/rollover-index.asciidoc | 29 ++++---- docs/reference/indices/shard-stores.asciidoc | 6 +- docs/reference/indices/shrink-index.asciidoc | 10 ++- docs/reference/indices/split-index.asciidoc | 12 ++-- .../indices/template-exists.asciidoc | 3 +- docs/reference/indices/templates.asciidoc | 15 ++--- docs/reference/indices/types-exists.asciidoc | 3 +- .../indices/update-settings.asciidoc | 18 ++--- docs/reference/ingest.asciidoc | 7 +- .../ingest/apis/delete-pipeline.asciidoc | 12 ++-- .../ingest/apis/get-pipeline.asciidoc | 15 ++--- .../ingest/apis/put-pipeline.asciidoc | 6 +- .../ingest/apis/simulate-pipeline.asciidoc | 6 +- docs/reference/ingest/ingest-node.asciidoc | 52 +++++---------- .../ingest/processors/circle.asciidoc | 9 +-- .../processors/date-index-name.asciidoc | 9 +-- .../ingest/processors/geoip.asciidoc | 15 ++--- .../reference/ingest/processors/grok.asciidoc | 9 +-- .../ingest/processors/pipeline.asciidoc | 9 +-- .../ingest/processors/script.asciidoc | 6 +- docs/reference/ingest/processors/set.asciidoc | 3 +- .../ingest/processors/user-agent.asciidoc | 3 +- .../licensing/delete-license.asciidoc | 3 +- .../licensing/get-basic-status.asciidoc | 3 +- docs/reference/licensing/get-license.asciidoc | 3 +- .../licensing/get-trial-status.asciidoc | 3 +- docs/reference/licensing/start-basic.asciidoc | 6 +- docs/reference/licensing/start-trial.asciidoc | 3 +- .../licensing/update-license.asciidoc | 6 +- docs/reference/mapping.asciidoc | 12 ++-- .../mapping/dynamic-mapping.asciidoc | 4 +- .../mapping/dynamic/field-mapping.asciidoc | 14 ++-- .../mapping/dynamic/templates.asciidoc | 31 ++++----- .../mapping/fields/field-names-field.asciidoc | 3 +- .../mapping/fields/id-field.asciidoc | 3 +- .../mapping/fields/ignored-field.asciidoc | 7 +- .../mapping/fields/index-field.asciidoc | 3 +- .../mapping/fields/meta-field.asciidoc | 7 +- .../mapping/fields/routing-field.asciidoc | 13 ++-- .../mapping/fields/source-field.asciidoc | 6 +- .../mapping/fields/type-field.asciidoc | 3 +- .../mapping/params/analyzer.asciidoc | 8 +-- docs/reference/mapping/params/boost.asciidoc | 9 +-- docs/reference/mapping/params/coerce.asciidoc | 8 +-- .../reference/mapping/params/copy-to.asciidoc | 4 +- .../mapping/params/doc-values.asciidoc | 4 +- .../reference/mapping/params/dynamic.asciidoc | 8 +-- .../params/eager-global-ordinals.asciidoc | 7 +- .../reference/mapping/params/enabled.asciidoc | 12 ++-- .../mapping/params/fielddata.asciidoc | 10 ++- docs/reference/mapping/params/format.asciidoc | 3 +- .../mapping/params/ignore-above.asciidoc | 4 +- .../mapping/params/ignore-malformed.asciidoc | 7 +- .../mapping/params/index-options.asciidoc | 4 +- .../mapping/params/index-prefixes.asciidoc | 6 +- .../mapping/params/multi-fields.asciidoc | 7 +- .../mapping/params/normalizer.asciidoc | 6 +- docs/reference/mapping/params/norms.asciidoc | 3 +- .../mapping/params/null-value.asciidoc | 4 +- .../params/position-increment-gap.asciidoc | 8 +-- .../mapping/params/properties.asciidoc | 7 +- .../mapping/params/search-analyzer.asciidoc | 3 +- .../mapping/params/similarity.asciidoc | 4 +- docs/reference/mapping/params/store.asciidoc | 4 +- .../mapping/params/term-vector.asciidoc | 4 +- .../mapping/removal_of_types.asciidoc | 29 ++++---- docs/reference/mapping/types/alias.asciidoc | 9 +-- docs/reference/mapping/types/array.asciidoc | 4 +- docs/reference/mapping/types/binary.asciidoc | 4 +- docs/reference/mapping/types/boolean.asciidoc | 7 +- docs/reference/mapping/types/date.asciidoc | 7 +- .../mapping/types/date_nanos.asciidoc | 3 +- .../mapping/types/dense-vector.asciidoc | 3 +- .../mapping/types/flattened.asciidoc | 11 ++-- .../mapping/types/geo-point.asciidoc | 4 +- .../mapping/types/geo-shape.asciidoc | 66 +++++++------------ docs/reference/mapping/types/ip.asciidoc | 12 ++-- docs/reference/mapping/types/keyword.asciidoc | 3 +- docs/reference/mapping/types/nested.asciidoc | 11 ++-- docs/reference/mapping/types/numeric.asciidoc | 3 +- docs/reference/mapping/types/object.asciidoc | 8 +-- .../mapping/types/parent-join.asciidoc | 33 ++++------ .../mapping/types/percolator.asciidoc | 45 +++++-------- docs/reference/mapping/types/range.asciidoc | 9 +-- .../mapping/types/rank-feature.asciidoc | 4 +- .../mapping/types/rank-features.asciidoc | 4 +- .../mapping/types/search-as-you-type.asciidoc | 12 ++-- docs/reference/mapping/types/shape.asciidoc | 60 ++++++----------- .../mapping/types/sparse-vector.asciidoc | 3 +- docs/reference/mapping/types/text.asciidoc | 3 +- .../mapping/types/token-count.asciidoc | 4 +- .../migration/apis/deprecation.asciidoc | 6 +- .../migration/migrate_8_0/snapshots.asciidoc | 3 +- .../anomaly-detection/aggregations.asciidoc | 6 +- .../anomaly-detection/apis/close-job.asciidoc | 3 +- .../apis/delete-calendar-event.asciidoc | 3 +- .../apis/delete-calendar-job.asciidoc | 3 +- .../apis/delete-calendar.asciidoc | 3 +- .../apis/delete-datafeed.asciidoc | 3 +- .../apis/delete-expired-data.asciidoc | 3 +- .../apis/delete-filter.asciidoc | 3 +- .../apis/delete-forecast.asciidoc | 3 +- .../apis/delete-job.asciidoc | 6 +- .../apis/delete-snapshot.asciidoc | 3 +- .../apis/find-file-structure.asciidoc | 4 +- .../anomaly-detection/apis/flush-job.asciidoc | 6 +- .../anomaly-detection/apis/forecast.asciidoc | 3 +- .../apis/get-bucket.asciidoc | 3 +- .../apis/get-calendar-event.asciidoc | 3 +- .../apis/get-calendar.asciidoc | 3 +- .../apis/get-category.asciidoc | 3 +- .../apis/get-datafeed-stats.asciidoc | 3 +- .../apis/get-datafeed.asciidoc | 3 +- .../apis/get-filter.asciidoc | 3 +- .../apis/get-influencer.asciidoc | 3 +- .../apis/get-job-stats.asciidoc | 3 +- .../anomaly-detection/apis/get-job.asciidoc | 3 +- .../apis/get-ml-info.asciidoc | 3 +- .../apis/get-overall-buckets.asciidoc | 6 +- .../apis/get-record.asciidoc | 3 +- .../apis/get-snapshot.asciidoc | 3 +- .../apis/jobresource.asciidoc | 3 +- .../anomaly-detection/apis/open-job.asciidoc | 3 +- .../apis/post-calendar-event.asciidoc | 3 +- .../apis/preview-datafeed.asciidoc | 3 +- .../apis/put-calendar-job.asciidoc | 3 +- .../apis/put-calendar.asciidoc | 3 +- .../apis/put-datafeed.asciidoc | 3 +- .../apis/put-filter.asciidoc | 3 +- .../anomaly-detection/apis/put-job.asciidoc | 3 +- .../apis/revert-snapshot.asciidoc | 3 +- .../apis/set-upgrade-mode.asciidoc | 12 +--- .../apis/start-datafeed.asciidoc | 3 +- 167 files changed, 518 insertions(+), 927 deletions(-) diff --git a/docs/plugins/repository-azure.asciidoc b/docs/plugins/repository-azure.asciidoc index cb0fa574f477..16a367cffa10 100644 --- a/docs/plugins/repository-azure.asciidoc +++ b/docs/plugins/repository-azure.asciidoc @@ -54,7 +54,7 @@ The settings have the form `azure.client.CLIENT_NAME.SETTING_NAME`. By default, the <> `client`. For example: -[source,js] +[source,console] ---- PUT _snapshot/my_backup { @@ -64,7 +64,6 @@ PUT _snapshot/my_backup } } ---- -// CONSOLE // TEST[skip:we don't have azure setup while testing this] Most client settings can be added to the `elasticsearch.yml` configuration file. diff --git a/docs/reference/index-modules/allocation/filtering.asciidoc b/docs/reference/index-modules/allocation/filtering.asciidoc index 0ae331d0e446..0ce8af2ebfa0 100644 --- a/docs/reference/index-modules/allocation/filtering.asciidoc +++ b/docs/reference/index-modules/allocation/filtering.asciidoc @@ -50,21 +50,20 @@ For example, to tell {es} to allocate shards from the `test` index to either `big` or `medium` nodes, use `index.routing.allocation.include`: + -- -[source,js] +[source,console] ------------------------ PUT test/_settings { "index.routing.allocation.include.size": "big,medium" } ------------------------ -// CONSOLE // TEST[s/^/PUT test\n/] If you specify multiple filters, all conditions must be satisfied for shards to be relocated. For example, to move the `test` index to `big` nodes in `rack1`, you could specify: -[source,js] +[source,console] ------------------------ PUT test/_settings { @@ -72,7 +71,6 @@ PUT test/_settings "index.routing.allocation.include.rack": "rack1" } ------------------------ -// CONSOLE // TEST[s/^/PUT test\n/] -- @@ -106,12 +104,11 @@ The index allocation settings support the following built-in attributes: You can use wildcards when specifying attribute values, for example: -[source,js] +[source,console] ------------------------ PUT test/_settings { "index.routing.allocation.include._ip": "192.168.2.*" } ------------------------ -// CONSOLE // TEST[skip:indexes don't assign] diff --git a/docs/reference/index-modules/allocation/prioritization.asciidoc b/docs/reference/index-modules/allocation/prioritization.asciidoc index 6693e6adb755..5a864b657bad 100644 --- a/docs/reference/index-modules/allocation/prioritization.asciidoc +++ b/docs/reference/index-modules/allocation/prioritization.asciidoc @@ -13,7 +13,7 @@ This means that, by default, newer indices will be recovered before older indice Use the per-index dynamically updatable `index.priority` setting to customise the index prioritization order. For instance: -[source,js] +[source,console] ------------------------------ PUT index_1 @@ -33,7 +33,6 @@ PUT index_4 } } ------------------------------ -// CONSOLE In the above example: @@ -45,12 +44,11 @@ In the above example: This setting accepts an integer, and can be updated on a live index with the <>: -[source,js] +[source,console] ------------------------------ PUT index_4/_settings { "index.priority": 1 } ------------------------------ -// CONSOLE // TEST[continued] diff --git a/docs/reference/index-modules/index-sorting.asciidoc b/docs/reference/index-modules/index-sorting.asciidoc index a387deaca6d1..40c4b08cb26f 100644 --- a/docs/reference/index-modules/index-sorting.asciidoc +++ b/docs/reference/index-modules/index-sorting.asciidoc @@ -12,7 +12,7 @@ An error will be thrown if index sorting is activated on an index that contains For instance the following example shows how to define a sort on a single field: -[source,js] +[source,console] -------------------------------------------------- PUT twitter { @@ -31,14 +31,13 @@ PUT twitter } } -------------------------------------------------- -// CONSOLE <1> This index is sorted by the `date` field <2> ... in descending order. It is also possible to sort the index by more than one field: -[source,js] +[source,console] -------------------------------------------------- PUT twitter { @@ -61,7 +60,6 @@ PUT twitter } } -------------------------------------------------- -// CONSOLE <1> This index is sorted by `username` first then by `date` <2> ... in ascending order for the `username` field and in descending order for the `date` field. @@ -112,7 +110,7 @@ Though when the index sort and the search sort are the same it is possible to li the number of documents that should be visited per segment to retrieve the N top ranked documents globally. For example, let's say we have an index that contains events sorted by a timestamp field: -[source,js] +[source,console] -------------------------------------------------- PUT events { @@ -131,13 +129,12 @@ PUT events } } -------------------------------------------------- -// CONSOLE <1> This index is sorted by timestamp in descending order (most recent first) You can search for the last 10 events with: -[source,js] +[source,console] -------------------------------------------------- GET /events/_search { @@ -147,7 +144,6 @@ GET /events/_search ] } -------------------------------------------------- -// CONSOLE // TEST[continued] Elasticsearch will detect that the top docs of each segment are already sorted in the index @@ -159,7 +155,7 @@ If you're only looking for the last 10 events and have no interest in the total number of documents that match the query you can set `track_total_hits` to false: -[source,js] +[source,console] -------------------------------------------------- GET /events/_search { @@ -170,7 +166,6 @@ GET /events/_search "track_total_hits": false } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The index sort will be used to rank the top documents and each segment will early terminate the collection after the first 10 matches. diff --git a/docs/reference/index-modules/similarity.asciidoc b/docs/reference/index-modules/similarity.asciidoc index ee6cf3958375..22b41901bde6 100644 --- a/docs/reference/index-modules/similarity.asciidoc +++ b/docs/reference/index-modules/similarity.asciidoc @@ -18,7 +18,7 @@ can be configured via the index settings as shown below. The index options can be provided when creating an index or updating index settings. -[source,js] +[source,console] -------------------------------------------------- PUT /index { @@ -37,12 +37,11 @@ PUT /index } } -------------------------------------------------- -// CONSOLE Here we configure the DFRSimilarity so it can be referenced as `my_similarity` in mappings as is illustrate in the below example: -[source,js] +[source,console] -------------------------------------------------- PUT /index/_mapping { @@ -51,7 +50,6 @@ PUT /index/_mapping } } -------------------------------------------------- -// CONSOLE // TEST[continued] [float] @@ -190,7 +188,7 @@ A similarity that allows you to use a script in order to specify how scores should be computed. For instance, the below example shows how to reimplement TF-IDF: -[source,js] +[source,console] -------------------------------------------------- PUT /index { @@ -237,7 +235,6 @@ GET /index/_search?explain=true } } -------------------------------------------------- -// CONSOLE Which yields: @@ -357,7 +354,7 @@ document-independent contribution to the score. The below configuration will give the same tf-idf scores but is slightly more efficient: -[source,js] +[source,console] -------------------------------------------------- PUT /index { @@ -385,11 +382,10 @@ PUT /index } } -------------------------------------------------- -// CONSOLE //////////////////// -[source,js] +[source,console] -------------------------------------------------- PUT /index/_doc/1 { @@ -413,7 +409,6 @@ GET /index/_search?explain=true } } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] @@ -523,7 +518,7 @@ By default, Elasticsearch will use whatever similarity is configured as You can change the default similarity for all fields in an index when it is <>: -[source,js] +[source,console] -------------------------------------------------- PUT /index { @@ -538,13 +533,12 @@ PUT /index } } -------------------------------------------------- -// CONSOLE If you want to change the default similarity after creating the index you must <> your index, send the following request and <> it again afterwards: -[source,js] +[source,console] -------------------------------------------------- POST /index/_close @@ -561,5 +555,4 @@ PUT /index/_settings POST /index/_open -------------------------------------------------- -// CONSOLE // TEST[continued] diff --git a/docs/reference/index-modules/slowlog.asciidoc b/docs/reference/index-modules/slowlog.asciidoc index 18dbab995eb7..34a6d89474c5 100644 --- a/docs/reference/index-modules/slowlog.asciidoc +++ b/docs/reference/index-modules/slowlog.asciidoc @@ -29,7 +29,7 @@ index.search.slowlog.level: info All of the above settings are _dynamic_ and can be set for each index using the <> API. For example: -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_settings { @@ -44,7 +44,6 @@ PUT /twitter/_settings "index.search.slowlog.level": "info" } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] By default, none are enabled (set to `-1`). Levels (`warn`, `info`, @@ -140,7 +139,7 @@ index.indexing.slowlog.source: 1000 All of the above settings are _dynamic_ and can be set for each index using the <> API. For example: -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_settings { @@ -152,7 +151,6 @@ PUT /twitter/_settings "index.indexing.slowlog.source": "1000" } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] By default Elasticsearch will log the first 1000 characters of the _source in diff --git a/docs/reference/index-modules/store.asciidoc b/docs/reference/index-modules/store.asciidoc index 8c1b99a42f2a..9bd25968e22f 100644 --- a/docs/reference/index-modules/store.asciidoc +++ b/docs/reference/index-modules/store.asciidoc @@ -22,7 +22,7 @@ index.store.type: niofs It is a _static_ setting that can be set on a per-index basis at index creation time: -[source,js] +[source,console] --------------------------------- PUT /my_index { @@ -31,7 +31,6 @@ PUT /my_index } } --------------------------------- -// CONSOLE WARNING: This is an expert-only setting and may be removed in the future. @@ -112,7 +111,7 @@ index.store.preload: ["nvd", "dvd"] or in the index settings at index creation time: -[source,js] +[source,console] --------------------------------- PUT /my_index { @@ -121,7 +120,6 @@ PUT /my_index } } --------------------------------- -// CONSOLE The default value is the empty array, which means that nothing will be loaded into the file-system cache eagerly. For indices that are actively searched, diff --git a/docs/reference/indices/add-alias.asciidoc b/docs/reference/indices/add-alias.asciidoc index a5c7d4c49e90..66460ce71353 100644 --- a/docs/reference/indices/add-alias.asciidoc +++ b/docs/reference/indices/add-alias.asciidoc @@ -8,11 +8,10 @@ Creates or updates an index alias. include::alias-exists.asciidoc[tag=index-alias-def] -[source,js] +[source,console] ---- PUT /twitter/_alias/alias1 ---- -// CONSOLE // TEST[setup:twitter] @@ -68,11 +67,10 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=index-routing] The following request creates an alias, `2030`, for the `logs_20302801` index. -[source,js] +[source,console] -------------------------------------------------- PUT /logs_20302801/_alias/2030 -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT logs_20302801\n/] [[add-alias-api-user-ex]] @@ -81,7 +79,7 @@ PUT /logs_20302801/_alias/2030 First, create an index, `users`, with a mapping for the `user_id` field: -[source,js] +[source,console] -------------------------------------------------- PUT /users { @@ -92,11 +90,10 @@ PUT /users } } -------------------------------------------------- -// CONSOLE Then add the index alias for a specific user, `user_12`: -[source,js] +[source,console] -------------------------------------------------- PUT /users/_alias/user_12 { @@ -108,7 +105,6 @@ PUT /users/_alias/user_12 } } -------------------------------------------------- -// CONSOLE // TEST[continued] [[alias-index-creation]] @@ -117,7 +113,7 @@ PUT /users/_alias/user_12 You can use the <> to add an index alias during index creation. -[source,js] +[source,console] -------------------------------------------------- PUT /logs_20302801 { @@ -136,4 +132,3 @@ PUT /logs_20302801 } } -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/indices/alias-exists.asciidoc b/docs/reference/indices/alias-exists.asciidoc index e1398f079952..2c56d5363d81 100644 --- a/docs/reference/indices/alias-exists.asciidoc +++ b/docs/reference/indices/alias-exists.asciidoc @@ -15,11 +15,10 @@ The returned HTTP status code indicates whether the index alias exists or not. A `404` means it does not exist, and `200` means it does. -[source,js] +[source,console] ---- HEAD /_alias/alias1 ---- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/PUT twitter\/_alias\/alias1\n/] @@ -68,11 +67,10 @@ Indicates one or more specified index aliases **do not** exist. [[alias-exists-api-example]] ==== {api-examples-title} -[source,js] +[source,console] ---- HEAD /_alias/2030 HEAD /_alias/20* HEAD /logs_20302801/_alias/* ---- -// CONSOLE // TEST[s/^/PUT logs_20302801\nPUT logs_20302801\/_alias\/2030\n/] diff --git a/docs/reference/indices/aliases.asciidoc b/docs/reference/indices/aliases.asciidoc index 9d3507f5ac8d..aff3e3b88fc1 100644 --- a/docs/reference/indices/aliases.asciidoc +++ b/docs/reference/indices/aliases.asciidoc @@ -8,7 +8,7 @@ Adds or removes index aliases. include::alias-exists.asciidoc[tag=index-alias-def] -[source,js] +[source,console] ---- POST /_aliases { @@ -17,7 +17,6 @@ POST /_aliases ] } ---- -// CONSOLE // TEST[setup:twitter] @@ -153,7 +152,7 @@ See <> for an example. The following request adds the `alias1` alias to the `test1` index. -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -162,7 +161,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test1\nPUT test2\n/] [[indices-aliases-api-remove-alias-ex]] @@ -170,7 +168,7 @@ POST /_aliases The following request removes the `alias1` alias. -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -179,7 +177,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[continued] [[indices-aliases-api-rename-alias-ex]] @@ -189,7 +186,7 @@ Renaming an alias is a simple `remove` then `add` operation within the same API. This operation is atomic, no need to worry about a short period of time where the alias does not point to an index: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -199,7 +196,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[continued] [[indices-aliases-api-add-multi-alias-ex]] @@ -208,7 +204,7 @@ POST /_aliases Associating an alias with more than one index is simply several `add` actions: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -218,12 +214,11 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test1\nPUT test2\n/] Multiple indices can be specified for an action with the `indices` array syntax: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -232,7 +227,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test1\nPUT test2\n/] To specify multiple aliases in one action, the corresponding `aliases` array @@ -241,7 +235,7 @@ syntax exists as well. For the example above, a glob pattern can also be used to associate an alias to more than one index that share a common name: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -250,7 +244,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test1\nPUT test2\n/] In this case, the alias is a point-in-time alias that will group all @@ -261,7 +254,7 @@ It is an error to index to an alias which points to more than one index. It is also possible to swap an index with an alias in one operation: -[source,js] +[source,console] -------------------------------------------------- PUT test <1> PUT test_2 <2> @@ -273,7 +266,7 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE + <1> An index we've added by mistake <2> The index we should have added <3> `remove_index` is just like <> @@ -289,7 +282,7 @@ this alias. To create a filtered alias, first we need to ensure that the fields already exist in the mapping: -[source,js] +[source,console] -------------------------------------------------- PUT /test1 { @@ -302,11 +295,10 @@ PUT /test1 } } -------------------------------------------------- -// CONSOLE Now we can create an alias that uses a filter on field `user`: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -321,7 +313,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[continued] [[aliases-routing]] @@ -335,7 +326,7 @@ The following command creates a new alias `alias1` that points to index `test`. After `alias1` is created, all operations with this alias are automatically modified to use value `1` for routing: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -350,13 +341,12 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test\n/] It's also possible to specify different routing values for searching and indexing operations: -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -372,7 +362,6 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test\n/] As shown in the example above, search routing may contain several values @@ -383,11 +372,10 @@ intersection of both search alias routing and routing specified in the parameter is used. For example the following command will use "2" as a routing value: -[source,js] +[source,console] -------------------------------------------------- GET /alias2/_search?q=user:kimchy&routing=2,3 -------------------------------------------------- -// CONSOLE // TEST[continued] [[aliases-write-index]] @@ -405,7 +393,7 @@ and index creation API. Setting an index to be the write index with an alias also affects how the alias is manipulated during Rollover (see <>). -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -426,36 +414,33 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test\nPUT test2\n/] In this example, we associate the alias `alias1` to both `test` and `test2`, where `test` will be the index chosen for writing to. -[source,js] +[source,console] -------------------------------------------------- PUT /alias1/_doc/1 { "foo": "bar" } -------------------------------------------------- -// CONSOLE // TEST[continued] The new document that was indexed to `/alias1/_doc/1` will be indexed as if it were `/test/_doc/1`. -[source,js] +[source,console] -------------------------------------------------- GET /test/_doc/1 -------------------------------------------------- -// CONSOLE // TEST[continued] To swap which index is the write index for an alias, the Aliases API can be leveraged to do an atomic swap. The swap is not dependent on the ordering of the actions. -[source,js] +[source,console] -------------------------------------------------- POST /_aliases { @@ -476,5 +461,4 @@ POST /_aliases ] } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test\nPUT test2\n/] diff --git a/docs/reference/indices/analyze.asciidoc b/docs/reference/indices/analyze.asciidoc index 75f0fb618ef0..8244a000c7e0 100644 --- a/docs/reference/indices/analyze.asciidoc +++ b/docs/reference/indices/analyze.asciidoc @@ -7,7 +7,7 @@ Performs <> on a text string and returns the resulting tokens. -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -15,7 +15,6 @@ GET /_analyze "text" : "Quick Brown Foxes!" } -------------------------------------------------- -// CONSOLE [[analyze-api-request]] @@ -137,7 +136,7 @@ See <> for a list of tokenizers. You can apply any of the built-in analyzers to the text string without specifying an index. -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -145,14 +144,13 @@ GET /_analyze "text" : "this is a test" } -------------------------------------------------- -// CONSOLE [[analyze-api-text-array-ex]] ===== Array of text strings If the `text` parameter is provided as array of strings, it is analyzed as a multi-value field. -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -160,7 +158,6 @@ GET /_analyze "text" : ["this is a test", "the second text"] } -------------------------------------------------- -// CONSOLE [[analyze-api-custom-analyzer-ex]] ===== Custom analyzer @@ -169,7 +166,7 @@ You can use the analyze API to test a custom transient analyzer built from tokenizers, token filters, and char filters. Token filters use the `filter` parameter: -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -178,9 +175,8 @@ GET /_analyze "text" : "this is a test" } -------------------------------------------------- -// CONSOLE -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -190,13 +186,12 @@ GET /_analyze "text" : "this is a test" } -------------------------------------------------- -// CONSOLE deprecated[5.0.0, Use `filter`/`char_filter` instead of `filters`/`char_filters` and `token_filters` has been removed] Custom tokenizers, token filters, and character filters can be specified in the request body as follows: -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -205,28 +200,26 @@ GET /_analyze "text" : "this is a test" } -------------------------------------------------- -// CONSOLE [[analyze-api-specific-index-ex]] ===== Specific index You can also run the analyze API against a specific index: -[source,js] +[source,console] -------------------------------------------------- GET /analyze_sample/_analyze { "text" : "this is a test" } -------------------------------------------------- -// CONSOLE // TEST[setup:analyze_sample] The above will run an analysis on the "this is a test" text, using the default index analyzer associated with the `analyze_sample` index. An `analyzer` can also be provided to use a different analyzer: -[source,js] +[source,console] -------------------------------------------------- GET /analyze_sample/_analyze { @@ -234,7 +227,6 @@ GET /analyze_sample/_analyze "text" : "this is a test" } -------------------------------------------------- -// CONSOLE // TEST[setup:analyze_sample] [[analyze-api-field-ex]] @@ -242,7 +234,7 @@ GET /analyze_sample/_analyze The analyzer can be derived based on a field mapping, for example: -[source,js] +[source,console] -------------------------------------------------- GET /analyze_sample/_analyze { @@ -250,7 +242,6 @@ GET /analyze_sample/_analyze "text" : "this is a test" } -------------------------------------------------- -// CONSOLE // TEST[setup:analyze_sample] Will cause the analysis to happen based on the analyzer configured in the @@ -261,7 +252,7 @@ mapping for `obj1.field1` (and if not, the default index analyzer). A `normalizer` can be provided for keyword field with normalizer associated with the `analyze_sample` index. -[source,js] +[source,console] -------------------------------------------------- GET /analyze_sample/_analyze { @@ -269,12 +260,11 @@ GET /analyze_sample/_analyze "text" : "BaR" } -------------------------------------------------- -// CONSOLE // TEST[setup:analyze_sample] Or by building a custom transient normalizer out of token filters and char filters. -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -282,7 +272,6 @@ GET /_analyze "text" : "BaR" } -------------------------------------------------- -// CONSOLE [[explain-analyze-api]] ===== Explain analyze @@ -292,7 +281,7 @@ You can filter token attributes you want to output by setting `attributes` optio NOTE: The format of the additional detail information is labelled as experimental in Lucene and it may change in the future. -[source,js] +[source,console] -------------------------------------------------- GET /_analyze { @@ -303,7 +292,7 @@ GET /_analyze "attributes" : ["keyword"] <1> } -------------------------------------------------- -// CONSOLE + <1> Set "keyword" to output "keyword" attribute only The request returns the following result: @@ -367,7 +356,7 @@ The following setting allows to limit the number of tokens that can be produced: the limit for a specific index: -[source,js] +[source,console] -------------------------------------------------- PUT /analyze_sample { @@ -376,15 +365,13 @@ PUT /analyze_sample } } -------------------------------------------------- -// CONSOLE -[source,js] +[source,console] -------------------------------------------------- GET /analyze_sample/_analyze { "text" : "this is a test" } -------------------------------------------------- -// CONSOLE // TEST[setup:analyze_sample] diff --git a/docs/reference/indices/apis/freeze.asciidoc b/docs/reference/indices/apis/freeze.asciidoc index b2f96725ba22..bb41de6763e3 100644 --- a/docs/reference/indices/apis/freeze.asciidoc +++ b/docs/reference/indices/apis/freeze.asciidoc @@ -39,11 +39,10 @@ limitation might be removed in the future. The following example freezes and unfreezes an index: -[source,js] +[source,console] -------------------------------------------------- POST /my_index/_freeze POST /my_index/_unfreeze -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_index\n/] diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index 657f6556df4b..1f5de4c3fb99 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -11,7 +11,7 @@ Synonym filters (both `synonym` and `synonym_graph`) can be declared as updateable if they are only used in <> with the `updateable` flag: -[source,js] +[source,console] -------------------------------------------------- PUT /my_index { @@ -45,7 +45,6 @@ PUT /my_index } } -------------------------------------------------- -// CONSOLE <1> Mark the synonym filter as updateable. <2> Synonym analyzer is usable as a search_analyzer. @@ -64,11 +63,10 @@ to update the synonym file contents on every data node (even the ones that don't hold shard copies; shards might be relocated there in the future) before calling reload to ensure the new state of the file is reflected everywhere in the cluster. -[source,js] +[source,console] -------------------------------------------------- POST /my_index/_reload_search_analyzers -------------------------------------------------- -// CONSOLE // TEST[continued] The reload request returns information about the nodes it was executed on and the diff --git a/docs/reference/indices/apis/unfreeze.asciidoc b/docs/reference/indices/apis/unfreeze.asciidoc index fa70259580c9..4f127ef54e07 100644 --- a/docs/reference/indices/apis/unfreeze.asciidoc +++ b/docs/reference/indices/apis/unfreeze.asciidoc @@ -38,10 +38,9 @@ limitation might be removed in the future. The following example freezes and unfreezes an index: -[source,js] +[source,console] -------------------------------------------------- POST /my_index/_freeze POST /my_index/_unfreeze -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_index\n/] diff --git a/docs/reference/indices/clearcache.asciidoc b/docs/reference/indices/clearcache.asciidoc index b09d993a1278..7d0f4e03ceaf 100644 --- a/docs/reference/indices/clearcache.asciidoc +++ b/docs/reference/indices/clearcache.asciidoc @@ -4,24 +4,23 @@ The clear cache API allows to clear either all caches or specific cached associated with one or more indices. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_cache/clear -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] The API, by default, will clear all caches. Specific caches can be cleaned explicitly by setting the `query`, `fielddata` or `request` url parameter to `true`. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_cache/clear?query=true <1> POST /twitter/_cache/clear?request=true <2> POST /twitter/_cache/clear?fielddata=true <3> -------------------------------------------------- -// CONSOLE // TEST[continued] + <1> Cleans only the query cache <2> Cleans only the request cache <3> Cleans only the fielddata cache @@ -31,12 +30,12 @@ cleared by specifying `fields` url parameter with a comma delimited list of the fields that should be cleared. Note that the provided names must refer to concrete fields -- objects and field aliases are not supported. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_cache/clear?fields=foo,bar <1> -------------------------------------------------- -// CONSOLE // TEST[continued] + <1> Clear the cache for the `foo` an `bar` field [float] @@ -45,11 +44,10 @@ POST /twitter/_cache/clear?fields=foo,bar <1> The clear cache API can be applied to more than one index with a single call, or even on `_all` the indices. -[source,js] +[source,console] -------------------------------------------------- POST /kimchy,elasticsearch/_cache/clear POST /_cache/clear -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT kimchy\nPUT elasticsearch\n/] diff --git a/docs/reference/indices/clone-index.asciidoc b/docs/reference/indices/clone-index.asciidoc index a5f195b2e459..d7eb2d209134 100644 --- a/docs/reference/indices/clone-index.asciidoc +++ b/docs/reference/indices/clone-index.asciidoc @@ -25,7 +25,7 @@ Cloning works as follows: Create a new index: -[source,js] +[source,console] -------------------------------------------------- PUT my_source_index { @@ -34,14 +34,13 @@ PUT my_source_index } } -------------------------------------------------- -// CONSOLE In order to clone an index, the index must be marked as read-only, and have <> `green`. This can be achieved with the following request: -[source,js] +[source,console] -------------------------------------------------- PUT /my_source_index/_settings { @@ -50,7 +49,6 @@ PUT /my_source_index/_settings } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> Prevents write operations to this index while still allowing metadata @@ -62,11 +60,10 @@ PUT /my_source_index/_settings To clone `my_source_index` into a new index called `my_target_index`, issue the following request: -[source,js] +[source,console] -------------------------------------------------- POST my_source_index/_clone/my_target_index -------------------------------------------------- -// CONSOLE // TEST[continued] The above request returns immediately once the target index has been added to @@ -89,7 +86,7 @@ Indices can only be cloned if they satisfy the following requirements: The `_clone` API is similar to the <> and accepts `settings` and `aliases` parameters for the target index: -[source,js] +[source,console] -------------------------------------------------- POST my_source_index/_clone/my_target_index { @@ -101,7 +98,6 @@ POST my_source_index/_clone/my_target_index } } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_source_index\n{"settings": {"index.blocks.write": true, "index.number_of_shards": "5"}}\n/] <1> The number of shards in the target index. This must be equal to the diff --git a/docs/reference/indices/close.asciidoc b/docs/reference/indices/close.asciidoc index d6997eca7450..b44d38d79326 100644 --- a/docs/reference/indices/close.asciidoc +++ b/docs/reference/indices/close.asciidoc @@ -6,11 +6,10 @@ Closes an index. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_close -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] @@ -61,11 +60,10 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=timeoutparms] The following example shows how to close an index: -[source,js] +[source,console] -------------------------------------------------- POST /my_index/_close -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_index\n/] The API returns following response: diff --git a/docs/reference/indices/create-index.asciidoc b/docs/reference/indices/create-index.asciidoc index 144b3b00c143..fef6ff96a523 100644 --- a/docs/reference/indices/create-index.asciidoc +++ b/docs/reference/indices/create-index.asciidoc @@ -6,11 +6,10 @@ Creates a new index. -[source,js] +[source,console] -------------------------------------------------- PUT /twitter -------------------------------------------------- -// CONSOLE [[indices-create-api-request]] @@ -77,7 +76,7 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=settings] Each index created can have specific settings associated with it, defined in the body: -[source,js] +[source,console] -------------------------------------------------- PUT /twitter { @@ -89,13 +88,13 @@ PUT /twitter } } -------------------------------------------------- -// CONSOLE + <1> Default for `number_of_shards` is 1 <2> Default for `number_of_replicas` is 1 (ie one replica for each primary shard) or more simplified -[source,js] +[source,console] -------------------------------------------------- PUT /twitter { @@ -105,7 +104,6 @@ PUT /twitter } } -------------------------------------------------- -// CONSOLE [NOTE] You do not have to explicitly specify `index` section inside the @@ -120,7 +118,7 @@ that can be set when creating an index, please check the The create index API allows for providing a mapping definition: -[source,js] +[source,console] -------------------------------------------------- PUT /test { @@ -134,7 +132,6 @@ PUT /test } } -------------------------------------------------- -// CONSOLE NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although specifying types in requests is now deprecated, a type can still be provided if the request parameter @@ -145,7 +142,7 @@ include_type_name is set. For more details, please see <>. The create index API allows also to provide a set of <>: -[source,js] +[source,console] -------------------------------------------------- PUT /test { @@ -160,7 +157,6 @@ PUT /test } } -------------------------------------------------- -// CONSOLE [[create-index-wait-for-active-shards]] ===== Wait For active shards @@ -193,7 +189,7 @@ We can change the default of only waiting for the primary shards to start throug setting `index.write.wait_for_active_shards` (note that changing this setting will also affect the `wait_for_active_shards` value on all subsequent write operations): -[source,js] +[source,console] -------------------------------------------------- PUT /test { @@ -202,16 +198,14 @@ PUT /test } } -------------------------------------------------- -// CONSOLE // TEST[skip:requires two nodes] or through the request parameter `wait_for_active_shards`: -[source,js] +[source,console] -------------------------------------------------- PUT /test?wait_for_active_shards=2 -------------------------------------------------- -// CONSOLE // TEST[skip:requires two nodes] A detailed explanation of `wait_for_active_shards` and its possible values can be found diff --git a/docs/reference/indices/delete-alias.asciidoc b/docs/reference/indices/delete-alias.asciidoc index e7ef772cbbd6..1df34a3753ec 100644 --- a/docs/reference/indices/delete-alias.asciidoc +++ b/docs/reference/indices/delete-alias.asciidoc @@ -8,11 +8,10 @@ Deletes an existing index alias. include::alias-exists.asciidoc[tag=index-alias-def] -[source,js] +[source,console] ---- DELETE /twitter/_alias/alias1 ---- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/PUT twitter\/_alias\/alias1\n/] diff --git a/docs/reference/indices/delete-index-template.asciidoc b/docs/reference/indices/delete-index-template.asciidoc index 9ee1158fa447..998ab580cbd1 100644 --- a/docs/reference/indices/delete-index-template.asciidoc +++ b/docs/reference/indices/delete-index-template.asciidoc @@ -7,7 +7,7 @@ Deletes an existing index. //// -[source,js] +[source,console] -------------------------------------------------- PUT _template/template_1 { @@ -17,15 +17,13 @@ PUT _template/template_1 } } -------------------------------------------------- -// CONSOLE // TESTSETUP //// -[source,js] +[source,console] -------------------------------------------------- DELETE /_template/template_1 -------------------------------------------------- -// CONSOLE [[delete-template-api-request]] diff --git a/docs/reference/indices/delete-index.asciidoc b/docs/reference/indices/delete-index.asciidoc index 7fe33149bc5b..8f0ad0868814 100644 --- a/docs/reference/indices/delete-index.asciidoc +++ b/docs/reference/indices/delete-index.asciidoc @@ -6,11 +6,10 @@ Deletes an existing index. -[source,js] +[source,console] -------------------------------------------------- DELETE /twitter -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 333cbc98b8d3..3c36c592bd70 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -22,11 +22,10 @@ call the flush API after indexing some documents then a successful response indicates that {es} has flushed all the documents that were indexed before the flush API was called. -[source,js] +[source,console] -------------------------------------------------- POST twitter/_flush -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] [float] @@ -53,13 +52,12 @@ uncommitted changes are present. This parameter should be considered internal. The flush API can be applied to more than one index with a single call, or even on `_all` the indices. -[source,js] +[source,console] -------------------------------------------------- POST kimchy,elasticsearch/_flush POST _flush -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT kimchy\nPUT elasticsearch\n/] [[synced-flush-api]] @@ -88,11 +86,10 @@ marker, recovery of this kind of cluster would be much slower. To check whether a shard has a `sync_id` marker or not, look for the `commit` section of the shard stats returned by the <> API: -[source,sh] +[source,console] -------------------------------------------------- GET twitter/_stats?filter_path=**.commit&level=shards <1> -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT twitter\nPOST twitter\/_flush\/synced\n/] <1> `filter_path` is used to reduce the verbosity of the response, but is entirely optional @@ -156,11 +153,10 @@ shards will fail to sync-flush. The successfully sync-flushed shards will have faster recovery times as long as the `sync_id` marker is not removed by a subsequent flush. -[source,sh] +[source,console] -------------------------------------------------- POST twitter/_flush/synced -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] The response contains details about how many shards were successfully @@ -256,10 +252,9 @@ be `409 Conflict`. The synced flush API can be applied to more than one index with a single call, or even on `_all` the indices. -[source,js] +[source,console] -------------------------------------------------- POST kimchy,elasticsearch/_flush/synced POST _flush/synced -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/indices/forcemerge.asciidoc b/docs/reference/indices/forcemerge.asciidoc index f478b5743e23..57c4796964f3 100644 --- a/docs/reference/indices/forcemerge.asciidoc +++ b/docs/reference/indices/forcemerge.asciidoc @@ -20,11 +20,10 @@ is lost before completion then the force merge process will continue in the background. Any new requests to force merge the same indices will also block until the ongoing force merge is complete. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_forcemerge -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] Force-merging can be useful with time-based indices and when using @@ -32,11 +31,10 @@ Force-merging can be useful with time-based indices and when using indexing traffic for a certain period of time, and once an index will receive no more writes its shards can be force-merged down to a single segment: -[source,js] +[source,console] -------------------------------------------------- POST /logs-000001/_forcemerge?max_num_segments=1 -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] // TEST[s/logs-000001/twitter/] @@ -64,11 +62,10 @@ 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] +[source,console] -------------------------------------------------- POST /kimchy/_forcemerge?only_expunge_deletes=false&max_num_segments=100&flush=true -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT kimchy\n/] [float] @@ -82,11 +79,10 @@ temporarily increase, up to double its size in case `max_num_segments` is set to `1`, as all segments need to be rewritten into a new one. -[source,js] +[source,console] -------------------------------------------------- POST /kimchy,elasticsearch/_forcemerge POST /_forcemerge -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT kimchy\nPUT elasticsearch\n/] diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index ebc4a5db64cb..14cb3a612ca1 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -8,11 +8,10 @@ Returns information about one or more index aliases. include::alias-exists.asciidoc[tag=index-alias-def] -[source,js] +[source,console] ---- GET /twitter/_alias/alias1 ---- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/PUT twitter\/_alias\/alias1\n/] @@ -71,7 +70,7 @@ with two aliases: in the `logs_20302801` index with a `year` field value of `2030` -[source,js] +[source,console] -------------------------------------------------- PUT /logs_20302801 { @@ -85,16 +84,14 @@ PUT /logs_20302801 } } -------------------------------------------------- -// CONSOLE The following get index alias API request returns all aliases for the index `logs_20302801`: -[source,js] +[source,console] -------------------------------------------------- GET /logs_20302801/_alias/* -------------------------------------------------- -// CONSOLE // TEST[continued] The API returns the following response: @@ -124,11 +121,10 @@ The API returns the following response: The following index alias API request returns the `2030` alias: -[source,js] +[source,console] -------------------------------------------------- GET /_alias/2030 -------------------------------------------------- -// CONSOLE // TEST[continued] The API returns the following response: @@ -156,11 +152,10 @@ The API returns the following response: The following index alias API request returns any alias that begin with `20`: -[source,js] +[source,console] -------------------------------------------------- GET /_alias/20* -------------------------------------------------- -// CONSOLE // TEST[continued] The API returns the following response: diff --git a/docs/reference/indices/get-field-mapping.asciidoc b/docs/reference/indices/get-field-mapping.asciidoc index 40ee958b22c5..c0e5e6d46a5c 100644 --- a/docs/reference/indices/get-field-mapping.asciidoc +++ b/docs/reference/indices/get-field-mapping.asciidoc @@ -8,11 +8,10 @@ Retrieves <> for one or more fields. This is useful if you don't need the <> of an index or your index contains a large number of fields. -[source,js] +[source,console] ---- GET /twitter/_mapping/field/user ---- -// CONSOLE // TEST[setup:twitter] @@ -62,7 +61,7 @@ You can provide field mappings when creating a new index. The following <> API request creates the `publications` index with several field mappings. -[source,js] +[source,console] -------------------------------------------------- PUT /publications { @@ -81,15 +80,13 @@ PUT /publications } } -------------------------------------------------- -// CONSOLE The following returns the mapping of the field `title` only: -[source,js] +[source,console] -------------------------------------------------- GET publications/_mapping/field/title -------------------------------------------------- -// CONSOLE // TEST[continued] The API returns the following response: @@ -119,11 +116,10 @@ The get mapping api allows you to specify a comma-separated list of fields. For instance to select the `id` of the `author` field, you must use its full name `author.id`. -[source,js] +[source,console] -------------------------------------------------- GET publications/_mapping/field/author.id,abstract,name -------------------------------------------------- -// CONSOLE // TEST[continued] returns: @@ -156,11 +152,10 @@ returns: The get field mapping API also supports wildcard notation. -[source,js] +[source,console] -------------------------------------------------- GET publications/_mapping/field/a* -------------------------------------------------- -// CONSOLE // TEST[continued] returns: @@ -209,7 +204,7 @@ following syntax: `host:port//_mapping/field/` where get mappings for all indices you can use `_all` for ``. The following are some examples: -[source,js] +[source,console] -------------------------------------------------- GET /twitter,kimchy/_mapping/field/message @@ -217,6 +212,5 @@ GET /_all/_mapping/field/message,user.id GET /_all/_mapping/field/*.id -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/PUT kimchy\nPUT book\n/] diff --git a/docs/reference/indices/get-index-template.asciidoc b/docs/reference/indices/get-index-template.asciidoc index 703c134abc9f..8c22ae54791d 100644 --- a/docs/reference/indices/get-index-template.asciidoc +++ b/docs/reference/indices/get-index-template.asciidoc @@ -7,7 +7,7 @@ Returns information about one or more index templates. //// -[source,js] +[source,console] -------------------------------------------------- PUT _template/template_1 { @@ -17,15 +17,13 @@ PUT _template/template_1 } } -------------------------------------------------- -// CONSOLE // TESTSETUP //// -[source,js] +[source,console] -------------------------------------------------- GET /_template/template_1 -------------------------------------------------- -// CONSOLE [[get-template-api-request]] @@ -62,28 +60,25 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=master-timeout] [[get-template-api-multiple-ex]] ===== Get multiple index templates -[source,js] +[source,console] -------------------------------------------------- GET /_template/template_1,template_2 -------------------------------------------------- -// CONSOLE [[get-template-api-wildcard-ex]] ===== Get index templates using a wildcard expression -[source,js] +[source,console] -------------------------------------------------- GET /_template/temp* -------------------------------------------------- -// CONSOLE [[get-template-api-all-ex]] ===== Get all index templates -[source,js] +[source,console] -------------------------------------------------- GET /_template -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/indices/get-index.asciidoc b/docs/reference/indices/get-index.asciidoc index fa6ebbf6681e..7d5a1c356193 100644 --- a/docs/reference/indices/get-index.asciidoc +++ b/docs/reference/indices/get-index.asciidoc @@ -6,11 +6,10 @@ Returns information about one or more indexes. -[source,js] +[source,console] -------------------------------------------------- GET /twitter -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although mappings diff --git a/docs/reference/indices/get-mapping.asciidoc b/docs/reference/indices/get-mapping.asciidoc index 45c608cd28dc..81b2cd826d90 100644 --- a/docs/reference/indices/get-mapping.asciidoc +++ b/docs/reference/indices/get-mapping.asciidoc @@ -6,11 +6,10 @@ Retrieves <> for indices in a cluster. -[source,js] +[source,console] -------------------------------------------------- GET /twitter/_mapping -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although mappings @@ -62,22 +61,20 @@ single call. General usage of the API follows the following syntax: list of names. To get mappings for all indices you can use `_all` for `{index}`. The following are some examples: -[source,js] +[source,console] -------------------------------------------------- GET /twitter,kimchy/_mapping -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/PUT kimchy\nPUT book\n/] If you want to get mappings of all indices and types then the following two examples are equivalent: -[source,js] +[source,console] -------------------------------------------------- GET /_all/_mapping GET /_mapping -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] diff --git a/docs/reference/indices/get-settings.asciidoc b/docs/reference/indices/get-settings.asciidoc index ce1ada57bd6d..e0a3df14992b 100644 --- a/docs/reference/indices/get-settings.asciidoc +++ b/docs/reference/indices/get-settings.asciidoc @@ -6,11 +6,10 @@ Returns setting information for an index. -[source,js] +[source,console] -------------------------------------------------- GET /twitter/_settings -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] @@ -63,7 +62,7 @@ The get settings API can be used to get settings for more than one index with a single call. To get settings for all indices you can use `_all` for ``. Wildcard expressions are also supported. The following are some examples: -[source,js] +[source,console] -------------------------------------------------- GET /twitter,kimchy/_settings @@ -71,7 +70,6 @@ GET /_all/_settings GET /log_2013_*/_settings -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/PUT kimchy\nPUT log_2013_01_01\n/] @@ -80,9 +78,8 @@ GET /log_2013_*/_settings The settings that are returned can be filtered with wildcard matching as follows: -[source,js] +[source,console] -------------------------------------------------- GET /log_2013_-*/_settings/index.number_* -------------------------------------------------- -// CONSOLE // TEST[continued] \ No newline at end of file diff --git a/docs/reference/indices/indices-exists.asciidoc b/docs/reference/indices/indices-exists.asciidoc index b85210af96d1..a506566f5c28 100644 --- a/docs/reference/indices/indices-exists.asciidoc +++ b/docs/reference/indices/indices-exists.asciidoc @@ -8,11 +8,10 @@ Checks if an index exists. The returned HTTP status code indicates if the index exists or not. A `404` means it does not exist, and `200` means it does. -[source,js] +[source,console] -------------------------------------------------- HEAD /twitter -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] diff --git a/docs/reference/indices/open-close.asciidoc b/docs/reference/indices/open-close.asciidoc index 2e43962c8fc8..3da1d3577e72 100644 --- a/docs/reference/indices/open-close.asciidoc +++ b/docs/reference/indices/open-close.asciidoc @@ -6,11 +6,10 @@ Opens a closed index. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_open -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] // TEST[s/^/POST \/twitter\/_close\n/] @@ -98,11 +97,10 @@ include::{docdir}/rest-api/common-parms.asciidoc[tag=timeoutparms] A closed index can be re-opened like this: -[source,js] +[source,console] -------------------------------------------------- POST /my_index/_open -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_index\nPOST my_index\/_close\n/] The API returns the following response: diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 46b33f042649..e9655edcc5df 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -7,7 +7,7 @@ Adds new fields to an existing index or changes the search settings of existing fields. -[source,js] +[source,console] ---- PUT /twitter/_mapping { @@ -18,7 +18,6 @@ PUT /twitter/_mapping } } ---- -// CONSOLE // TEST[setup:twitter] NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. @@ -87,16 +86,15 @@ The put mapping API requires an existing index. The following <> API request creates the `publications` index with no mapping. -[source,js] +[source,console] ---- PUT /publications ---- -// CONSOLE The following put mapping API request adds `title`, a new <> field, to the `publications` index. -[source,js] +[source,console] ---- PUT /publications/_mapping { @@ -105,7 +103,6 @@ PUT /publications/_mapping } } ---- -// CONSOLE // TEST[continued] [[put-mapping-api-multi-ex]] @@ -114,7 +111,7 @@ PUT /publications/_mapping The PUT mapping API can be applied to multiple indices with a single request. For example, we can update the `twitter-1` and `twitter-2` mappings at the same time: -[source,js] +[source,console] -------------------------------------------------- # Create the two indices PUT /twitter-1 @@ -130,7 +127,6 @@ PUT /twitter-1,twitter-2/_mapping <1> } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] <1> Note that the indices specified (`twitter-1,twitter-2`) follows <> and wildcard format. @@ -158,7 +154,7 @@ you only want to rename a field, consider adding an <> field. For example: -[source,js] +[source,console] ----------------------------------- PUT /my_index <1> { @@ -195,7 +191,7 @@ PUT /my_index/_mapping } } ----------------------------------- -// CONSOLE + <1> Create an index with a `first` field under the `name` <> field, and a `user_id` field. <2> Add a `last` field under the `name` object field. <3> Update the `ignore_above` setting from its default of 0. diff --git a/docs/reference/indices/recovery.asciidoc b/docs/reference/indices/recovery.asciidoc index 354ce0c6fc1e..54d25ac2c4da 100644 --- a/docs/reference/indices/recovery.asciidoc +++ b/docs/reference/indices/recovery.asciidoc @@ -6,11 +6,10 @@ Recovery status may be reported for specific indices, or cluster-wide. For example, the following command would show recovery information for the indices "index1" and "index2". -[source,js] +[source,console] -------------------------------------------------- GET index1,index2/_recovery?human -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT index1\nPUT index2\n/] To see cluster-wide recovery status simply leave out the index names. @@ -21,7 +20,7 @@ Here we create a repository and snapshot index1 in order to restore it right after and prints out the indices recovery result. -[source,js] +[source,console] -------------------------------------------------- # create the index PUT index1 @@ -41,7 +40,6 @@ DELETE index1 POST /_snapshot/my_repository/snap_1/_restore?wait_for_completion=true -------------------------------------------------- -// CONSOLE [source,console-result] -------------------------------------------------- @@ -62,11 +60,10 @@ POST /_snapshot/my_repository/snap_1/_restore?wait_for_completion=true ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- GET /_recovery?human -------------------------------------------------- -// CONSOLE // TEST[continued] Response: @@ -154,11 +151,10 @@ Additionally, the output shows the number and percent of files recovered, as wel In some cases a higher level of detail may be preferable. Setting "detailed=true" will present a list of physical files in recovery. -[source,js] +[source,console] -------------------------------------------------- GET _recovery?human&detailed=true -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT index1\n{"settings": {"index.number_of_shards": 1}}\n/] Response: diff --git a/docs/reference/indices/refresh.asciidoc b/docs/reference/indices/refresh.asciidoc index 743ea8c19e6b..bba4ec5c9a5d 100644 --- a/docs/reference/indices/refresh.asciidoc +++ b/docs/reference/indices/refresh.asciidoc @@ -7,11 +7,10 @@ The (near) real-time capabilities depend on the index engine used. For example, the internal one requires refresh to be called, but by default a refresh is scheduled periodically. -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_refresh -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] [float] @@ -20,11 +19,10 @@ POST /twitter/_refresh The refresh API can be applied to more than one index with a single call, or even on `_all` the indices. -[source,js] +[source,console] -------------------------------------------------- POST /kimchy,elasticsearch/_refresh POST /_refresh -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT kimchy\nPUT elasticsearch\n/] diff --git a/docs/reference/indices/rollover-index.asciidoc b/docs/reference/indices/rollover-index.asciidoc index 94295632c2fc..8372de550242 100644 --- a/docs/reference/indices/rollover-index.asciidoc +++ b/docs/reference/indices/rollover-index.asciidoc @@ -36,7 +36,7 @@ The available conditions are: | max_size | The maximum estimated size of the primary shard of the index |=== -[source,js] +[source,console] -------------------------------------------------- PUT /logs-000001 <1> { @@ -56,7 +56,6 @@ POST /logs_write/_rollover <2> } } -------------------------------------------------- -// CONSOLE // TEST[setup:huge_twitter] // TEST[s/# Add > 1000 documents to logs-000001/POST _reindex?refresh\n{"source":{"index":"twitter"},"dest":{"index":"logs-000001"}}/] <1> Creates an index called `logs-0000001` with the alias `logs_write`. @@ -98,7 +97,7 @@ of 6, regardless of the old index name. If the old name doesn't match this pattern then you must specify the name for the new index as follows: -[source,js] +[source,console] -------------------------------------------------- POST /my_alias/_rollover/my_new_index_name { @@ -109,7 +108,6 @@ POST /my_alias/_rollover/my_new_index_name } } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_old_index_name\nPUT my_old_index_name\/_alias\/my_alias\n/] [float] @@ -122,7 +120,7 @@ index name to end with a dash followed by a number, e.g. `logstash-2016.02.03-1` which is incremented every time the index is rolled over. For instance: -[source,js] +[source,console] -------------------------------------------------- # PUT / with URI encoding: PUT /%3Clogs-%7Bnow%2Fd%7D-1%3E <1> @@ -148,18 +146,17 @@ POST /logs_write/_rollover <2> } } -------------------------------------------------- -// CONSOLE // TEST[s/now/2016.10.31||/] + <1> Creates an index named with today's date (e.g.) `logs-2016.10.31-1` <2> Rolls over to a new index with today's date, e.g. `logs-2016.10.31-000002` if run immediately, or `logs-2016.11.01-000002` if run after 24 hours ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- GET _alias -------------------------------------------------- -// CONSOLE // TEST[continued] [source,console-result] @@ -182,12 +179,11 @@ These indices can then be referenced as described in the <>. For example, to search over indices created in the last three days, you could do the following: -[source,js] +[source,console] -------------------------------------------------- # GET /,,/_search GET /%3Clogs-%7Bnow%2Fd%7D-*%3E%2C%3Clogs-%7Bnow%2Fd-1d%7D-*%3E%2C%3Clogs-%7Bnow%2Fd-2d%7D-*%3E/_search -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[s/now/2016.10.31||/] @@ -201,7 +197,7 @@ matching <>. Additionally, you can specify override any values set in matching index templates. For example, the following `rollover` request overrides the `index.number_of_shards` setting: -[source,js] +[source,console] -------------------------------------------------- PUT /logs-000001 { @@ -222,7 +218,6 @@ POST /logs_write/_rollover } } -------------------------------------------------- -// CONSOLE [float] ==== Dry run @@ -230,7 +225,7 @@ POST /logs_write/_rollover The rollover API supports `dry_run` mode, where request conditions can be checked without performing the actual rollover: -[source,js] +[source,console] -------------------------------------------------- PUT /logs-000001 { @@ -248,7 +243,6 @@ POST /logs_write/_rollover?dry_run } } -------------------------------------------------- -// CONSOLE [float] ==== Wait For Active Shards @@ -272,7 +266,7 @@ indices that are being managed with Rollover. Look at the behavior of the aliases in the following example where `is_write_index` is set on the rolled over index. -[source,js] +[source,console] -------------------------------------------------- PUT my_logs_index-000001 { @@ -300,7 +294,7 @@ PUT logs/_doc/2 <2> "message": "a newer log" } -------------------------------------------------- -// CONSOLE + <1> configures `my_logs_index` as the write index for the `logs` alias <2> newly indexed documents against the `logs` alias will write to the new index @@ -323,11 +317,10 @@ PUT logs/_doc/2 <2> -------------------------------------------------- ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- GET _alias -------------------------------------------------- -// CONSOLE // TEST[continued] ////////////////////////// diff --git a/docs/reference/indices/shard-stores.asciidoc b/docs/reference/indices/shard-stores.asciidoc index 98697149adc1..8d4d8e3202cf 100644 --- a/docs/reference/indices/shard-stores.asciidoc +++ b/docs/reference/indices/shard-stores.asciidoc @@ -15,7 +15,7 @@ for shards, which has unassigned primaries. Endpoints include shard stores information for a specific index, several indices, or all: -[source,js] +[source,console] -------------------------------------------------- # return information of only index test GET /test/_shard_stores @@ -26,7 +26,6 @@ GET /test1,test2/_shard_stores # return information of all indices GET /_shard_stores -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT test\nPUT test1\nPUT test2\n/] The scope of shards to list store information can be changed through @@ -35,11 +34,10 @@ shards with at least one unassigned replica and 'red' for shards with unassigned primary shard. Use 'green' to list store information for shards with all assigned copies. -[source,js] +[source,console] -------------------------------------------------- GET /_shard_stores?status=green -------------------------------------------------- -// CONSOLE // TEST[setup:node] // TEST[s/^/PUT my-index\n{"settings":{"number_of_shards":1, "number_of_replicas": 0}}\nPOST my-index\/test\?refresh\n{"test": "test"}\n/] diff --git a/docs/reference/indices/shrink-index.asciidoc b/docs/reference/indices/shrink-index.asciidoc index 283bf59080c7..50874fd9eaef 100644 --- a/docs/reference/indices/shrink-index.asciidoc +++ b/docs/reference/indices/shrink-index.asciidoc @@ -34,7 +34,7 @@ same node and have <> `green`. These two conditions can be achieved with the following request: -[source,js] +[source,console] -------------------------------------------------- PUT /my_source_index/_settings { @@ -44,8 +44,8 @@ PUT /my_source_index/_settings } } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_source_index\n{"settings":{"index.number_of_shards":2}}\n/] + <1> Forces the relocation of a copy of each shard to the node with name `shrink_node_name`. See <> for more options. @@ -63,7 +63,7 @@ with the `wait_for_no_relocating_shards` parameter. To shrink `my_source_index` into a new index called `my_target_index`, issue the following request: -[source,js] +[source,console] -------------------------------------------------- POST my_source_index/_shrink/my_target_index { @@ -73,7 +73,6 @@ POST my_source_index/_shrink/my_target_index } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> Clear the allocation requirement copied from the source index. @@ -107,7 +106,7 @@ Indices can only be shrunk if they satisfy the following requirements: The `_shrink` API is similar to the <> and accepts `settings` and `aliases` parameters for the target index: -[source,js] +[source,console] -------------------------------------------------- POST my_source_index/_shrink/my_target_index { @@ -121,7 +120,6 @@ POST my_source_index/_shrink/my_target_index } } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_source_index\n{"settings": {"index.number_of_shards":5,"index.blocks.write": true}}\n/] <1> The number of shards in the target index. This must be a factor of the diff --git a/docs/reference/indices/split-index.asciidoc b/docs/reference/indices/split-index.asciidoc index dd7b1213cb39..1db5522cd479 100644 --- a/docs/reference/indices/split-index.asciidoc +++ b/docs/reference/indices/split-index.asciidoc @@ -85,7 +85,7 @@ compared to searching an index that would have +M+N+ shards. Create a new index: -[source,js] +[source,console] -------------------------------------------------- PUT my_source_index { @@ -94,14 +94,13 @@ PUT my_source_index } } -------------------------------------------------- -// CONSOLE In order to split an index, the index must be marked as read-only, and have <> `green`. This can be achieved with the following request: -[source,js] +[source,console] -------------------------------------------------- PUT /my_source_index/_settings { @@ -110,7 +109,6 @@ PUT /my_source_index/_settings } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> Prevents write operations to this index while still allowing metadata @@ -122,7 +120,7 @@ PUT /my_source_index/_settings To split `my_source_index` into a new index called `my_target_index`, issue the following request: -[source,js] +[source,console] -------------------------------------------------- POST my_source_index/_split/my_target_index { @@ -131,7 +129,6 @@ POST my_source_index/_split/my_target_index } } -------------------------------------------------- -// CONSOLE // TEST[continued] The above request returns immediately once the target index has been added to @@ -157,7 +154,7 @@ Indices can only be split if they satisfy the following requirements: The `_split` API is similar to the <> and accepts `settings` and `aliases` parameters for the target index: -[source,js] +[source,console] -------------------------------------------------- POST my_source_index/_split/my_target_index { @@ -169,7 +166,6 @@ POST my_source_index/_split/my_target_index } } -------------------------------------------------- -// CONSOLE // TEST[s/^/PUT my_source_index\n{"settings": {"index.blocks.write": true, "index.number_of_shards": "1"}}\n/] <1> The number of shards in the target index. This must be a factor of the diff --git a/docs/reference/indices/template-exists.asciidoc b/docs/reference/indices/template-exists.asciidoc index 550297a50598..43fa5131dadd 100644 --- a/docs/reference/indices/template-exists.asciidoc +++ b/docs/reference/indices/template-exists.asciidoc @@ -8,11 +8,10 @@ Checks if an index template exists. -[source,js] +[source,console] ----------------------------------------------- HEAD /_template/template_1 ----------------------------------------------- -// CONSOLE [[template-exists-api-request]] diff --git a/docs/reference/indices/templates.asciidoc b/docs/reference/indices/templates.asciidoc index 1c410a3d8fd9..b2871eed2cda 100644 --- a/docs/reference/indices/templates.asciidoc +++ b/docs/reference/indices/templates.asciidoc @@ -6,7 +6,7 @@ Creates or updates an index template. -[source,js] +[source,console] -------------------------------------------------- PUT _template/template_1 { @@ -30,7 +30,6 @@ PUT _template/template_1 } } -------------------------------------------------- -// CONSOLE // TESTSETUP @@ -128,7 +127,7 @@ This number is not automatically generated by {es}. You can include <> in an index template. -[source,js] +[source,console] -------------------------------------------------- PUT _template/template_1 { @@ -148,7 +147,6 @@ PUT _template/template_1 } } -------------------------------------------------- -// CONSOLE // TEST[s/^/DELETE _template\/template_1\n/] <1> the `{index}` placeholder in the alias name will be replaced with the @@ -164,7 +162,7 @@ of the index. The order of the merging can be controlled using the `order` parameter, with lower order being applied first, and higher orders overriding them. For example: -[source,js] +[source,console] -------------------------------------------------- PUT /_template/template_1 { @@ -190,7 +188,6 @@ PUT /_template/template_2 } } -------------------------------------------------- -// CONSOLE // TEST[s/^/DELETE _template\/template_1\n/] The above will disable storing the `_source`, but @@ -217,7 +214,7 @@ and not automatically generated by {es}. To unset a `version`, replace the template without specifying one. -[source,js] +[source,console] -------------------------------------------------- PUT /_template/template_1 { @@ -229,18 +226,16 @@ PUT /_template/template_1 "version": 123 } -------------------------------------------------- -// CONSOLE To check the `version`, you can use the <> API with the <> query parameter to return only the version number: -[source,js] +[source,console] -------------------------------------------------- GET /_template/template_1?filter_path=*.version -------------------------------------------------- -// CONSOLE // TEST[continued] The API returns the following response: diff --git a/docs/reference/indices/types-exists.asciidoc b/docs/reference/indices/types-exists.asciidoc index 7139b4cc7854..2a5fc3f4c1ef 100644 --- a/docs/reference/indices/types-exists.asciidoc +++ b/docs/reference/indices/types-exists.asciidoc @@ -5,11 +5,10 @@ deprecated[7.0.0, Types are deprecated and are in the process of being removed. Used to check if a type/types exists in an index/indices. -[source,js] +[source,console] -------------------------------------------------- HEAD twitter/_mapping/tweet -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] // TEST[warning:Type exists requests are deprecated, as types have been deprecated.] diff --git a/docs/reference/indices/update-settings.asciidoc b/docs/reference/indices/update-settings.asciidoc index d959af25b91e..dd00cce7a553 100644 --- a/docs/reference/indices/update-settings.asciidoc +++ b/docs/reference/indices/update-settings.asciidoc @@ -6,7 +6,7 @@ Changes an <> in real time. -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_settings { @@ -15,7 +15,6 @@ PUT /twitter/_settings } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] @@ -68,7 +67,7 @@ options for the index. See <>. ===== Reset an index setting To revert a setting to the default value, use `null`. For example: -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_settings { @@ -77,7 +76,6 @@ PUT /twitter/_settings } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] The list of per-index settings which can be updated dynamically on live @@ -93,7 +91,7 @@ the index from being more performant for bulk indexing, and then move it to more real time indexing state. Before the bulk indexing is started, use: -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_settings { @@ -102,7 +100,6 @@ PUT /twitter/_settings } } -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] (Another optimization option is to start the index without any replicas, @@ -111,7 +108,7 @@ and only later adding them, but that really depends on the use case). Then, once bulk indexing is done, the settings can be updated (back to the defaults for example): -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_settings { @@ -120,16 +117,14 @@ PUT /twitter/_settings } } -------------------------------------------------- -// CONSOLE // TEST[continued] And, a force merge should be called: -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_forcemerge?max_num_segments=5 -------------------------------------------------- -// CONSOLE // TEST[continued] [[update-settings-analysis]] @@ -144,7 +139,7 @@ and reopen the index. For example, the following commands add the `content` analyzer to `myindex`: -[source,js] +[source,console] -------------------------------------------------- POST /twitter/_close @@ -162,5 +157,4 @@ PUT /twitter/_settings POST /twitter/_open -------------------------------------------------- -// CONSOLE // TEST[setup:twitter] diff --git a/docs/reference/ingest.asciidoc b/docs/reference/ingest.asciidoc index a945a56c1942..135e0a571ec6 100644 --- a/docs/reference/ingest.asciidoc +++ b/docs/reference/ingest.asciidoc @@ -28,7 +28,7 @@ way, the ingest node knows which pipeline to use. For example: Create a pipeline -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/my_pipeline_id { @@ -43,19 +43,16 @@ PUT _ingest/pipeline/my_pipeline_id ] } -------------------------------------------------- -// CONSOLE -// TEST Index with defined pipeline -[source,js] +[source,console] -------------------------------------------------- PUT my-index/_doc/my-id?pipeline=my_pipeline_id { "foo": "bar" } -------------------------------------------------- -// CONSOLE // TEST[continued] Response: diff --git a/docs/reference/ingest/apis/delete-pipeline.asciidoc b/docs/reference/ingest/apis/delete-pipeline.asciidoc index bf7a667503ba..77e9a1c552f6 100644 --- a/docs/reference/ingest/apis/delete-pipeline.asciidoc +++ b/docs/reference/ingest/apis/delete-pipeline.asciidoc @@ -5,7 +5,7 @@ The delete pipeline API deletes pipelines by ID or wildcard match (`my-*`, `*`). ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/my-pipeline-id { @@ -21,15 +21,13 @@ PUT _ingest/pipeline/my-pipeline-id ] } -------------------------------------------------- -// CONSOLE ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- DELETE _ingest/pipeline/my-pipeline-id -------------------------------------------------- -// CONSOLE // TEST[continued] ////////////////////////// @@ -41,7 +39,7 @@ DELETE _ingest/pipeline/my-pipeline-id } -------------------------------------------------- -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/wild-one { @@ -55,15 +53,13 @@ PUT _ingest/pipeline/wild-two "processors" : [ ] } -------------------------------------------------- -// CONSOLE ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- DELETE _ingest/pipeline/* -------------------------------------------------- -// CONSOLE ////////////////////////// diff --git a/docs/reference/ingest/apis/get-pipeline.asciidoc b/docs/reference/ingest/apis/get-pipeline.asciidoc index 4d68315cb0d4..887ae870f525 100644 --- a/docs/reference/ingest/apis/get-pipeline.asciidoc +++ b/docs/reference/ingest/apis/get-pipeline.asciidoc @@ -5,7 +5,7 @@ The get pipeline API returns pipelines based on ID. This API always returns a lo ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/my-pipeline-id { @@ -20,15 +20,13 @@ PUT _ingest/pipeline/my-pipeline-id ] } -------------------------------------------------- -// CONSOLE ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- GET _ingest/pipeline/my-pipeline-id -------------------------------------------------- -// CONSOLE // TEST[continued] Example response: @@ -64,7 +62,7 @@ field is completely optional and it is meant solely for external management of pipelines. To unset a `version`, simply replace the pipeline without specifying one. -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/my-pipeline-id { @@ -80,17 +78,15 @@ PUT _ingest/pipeline/my-pipeline-id ] } -------------------------------------------------- -// CONSOLE To check for the `version`, you can <> using `filter_path` to limit the response to just the `version`: -[source,js] +[source,console] -------------------------------------------------- GET /_ingest/pipeline/my-pipeline-id?filter_path=*.version -------------------------------------------------- -// CONSOLE // TEST[continued] This should give a small response that makes it both easy and inexpensive to parse: @@ -106,11 +102,10 @@ This should give a small response that makes it both easy and inexpensive to par ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- DELETE /_ingest/pipeline/my-pipeline-id -------------------------------------------------- -// CONSOLE // TEST[continued] [source,console-result] diff --git a/docs/reference/ingest/apis/put-pipeline.asciidoc b/docs/reference/ingest/apis/put-pipeline.asciidoc index 9d2ba39dbaf5..fe7b388b9ab6 100644 --- a/docs/reference/ingest/apis/put-pipeline.asciidoc +++ b/docs/reference/ingest/apis/put-pipeline.asciidoc @@ -3,7 +3,7 @@ The put pipeline API adds pipelines and updates existing pipelines in the cluster. -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/my-pipeline-id { @@ -18,15 +18,13 @@ PUT _ingest/pipeline/my-pipeline-id ] } -------------------------------------------------- -// CONSOLE ////////////////////////// -[source,js] +[source,console] -------------------------------------------------- DELETE /_ingest/pipeline/my-pipeline-id -------------------------------------------------- -// CONSOLE // TEST[continued] [source,console-result] diff --git a/docs/reference/ingest/apis/simulate-pipeline.asciidoc b/docs/reference/ingest/apis/simulate-pipeline.asciidoc index deb464eac7a5..cab41ee54d16 100644 --- a/docs/reference/ingest/apis/simulate-pipeline.asciidoc +++ b/docs/reference/ingest/apis/simulate-pipeline.asciidoc @@ -46,7 +46,7 @@ POST _ingest/pipeline/my-pipeline-id/_simulate Here is an example of a simulate request with a pipeline defined in the request and its response: -[source,js] +[source,console] -------------------------------------------------- POST _ingest/pipeline/_simulate { @@ -80,7 +80,6 @@ POST _ingest/pipeline/_simulate ] } -------------------------------------------------- -// CONSOLE Response: @@ -131,7 +130,7 @@ to the request. Here is an example of a verbose request and its response: -[source,js] +[source,console] -------------------------------------------------- POST _ingest/pipeline/_simulate?verbose { @@ -171,7 +170,6 @@ POST _ingest/pipeline/_simulate?verbose ] } -------------------------------------------------- -// CONSOLE Response: diff --git a/docs/reference/ingest/ingest-node.asciidoc b/docs/reference/ingest/ingest-node.asciidoc index b9afd2727a9a..41463538606a 100644 --- a/docs/reference/ingest/ingest-node.asciidoc +++ b/docs/reference/ingest/ingest-node.asciidoc @@ -167,7 +167,7 @@ For example the following processor will <> the document (i.e. not index it) if the input document has a field named `network_name` and it is equal to `Guest`. -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/drop_guests_network { @@ -180,18 +180,16 @@ PUT _ingest/pipeline/drop_guests_network ] } -------------------------------------------------- -// CONSOLE Using that pipeline for an index request: -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/1?pipeline=drop_guests_network { "network_name" : "Guest" } -------------------------------------------------- -// CONSOLE // TEST[continued] Results in nothing indexed since the conditional evaluated to `true`. @@ -226,7 +224,7 @@ To help protect against NullPointerExceptions, null safe operations should be us Fortunately, Painless makes {painless}/painless-operators-reference.html#null-safe-operator[null safe] operations easy with the `?.` operator. -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/drop_guests_network { @@ -239,11 +237,10 @@ PUT _ingest/pipeline/drop_guests_network ] } -------------------------------------------------- -// CONSOLE The following document will get <> correctly: -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/1?pipeline=drop_guests_network { @@ -252,30 +249,27 @@ POST test/_doc/1?pipeline=drop_guests_network } } -------------------------------------------------- -// CONSOLE // TEST[continued] Thanks to the `?.` operator the following document will not throw an error. If the pipeline used a `.` the following document would throw a NullPointerException since the `network` object is not part of the source document. -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/2?pipeline=drop_guests_network { "foo" : "bar" } -------------------------------------------------- -// CONSOLE // TEST[continued] //// Hidden example assertion: -[source,js] +[source,console] -------------------------------------------------- GET test/_doc/2 -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] @@ -322,7 +316,7 @@ The source document may have the nested fields flattened as such: If this is the case, use the <> so that the nested fields may be used in a conditional. -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/drop_guests_network { @@ -340,18 +334,16 @@ PUT _ingest/pipeline/drop_guests_network ] } -------------------------------------------------- -// CONSOLE Now the following input document can be used with a conditional in the pipeline. -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/3?pipeline=drop_guests_network { "network.name": "Guest" } -------------------------------------------------- -// CONSOLE // TEST[continued] The `?.` operators works well for use in the `if` conditional @@ -392,7 +384,7 @@ A more complex `if` condition that drops the document (i.e. not index it) unless it has a multi-valued tag field with at least one value that contains the characters `prod` (case insensitive). -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/not_prod_dropper { @@ -405,7 +397,6 @@ PUT _ingest/pipeline/not_prod_dropper ] } -------------------------------------------------- -// CONSOLE The conditional needs to be all on one line since JSON does not support new line characters. However, Kibana's console supports @@ -438,14 +429,13 @@ PUT _ingest/pipeline/not_prod_dropper // NOTCONSOLE // TEST[continued] -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/1?pipeline=not_prod_dropper { "tags": ["application:myapp", "env:Stage"] } -------------------------------------------------- -// CONSOLE // TEST[continued] The document is <> since `prod` (case insensitive) @@ -454,23 +444,21 @@ is not found in the tags. The following document is indexed (i.e. not dropped) since `prod` (case insensitive) is found in the tags. -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/2?pipeline=not_prod_dropper { "tags": ["application:myapp", "env:Production"] } -------------------------------------------------- -// CONSOLE // TEST[continued] //// Hidden example assertion: -[source,js] +[source,console] -------------------------------------------------- GET test/_doc/2 -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] @@ -509,7 +497,7 @@ The combination of the `if` conditional and the <> can resul yet powerful means to process heterogeneous input. For example, you can define a single pipeline that delegates to other pipelines based on some criteria. -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/logs_pipeline { @@ -537,7 +525,6 @@ PUT _ingest/pipeline/logs_pipeline ] } -------------------------------------------------- -// CONSOLE The above example allows consumers to point to a single pipeline for all log based index requests. Based on the conditional, the correct pipeline will be called to process that type of data. @@ -556,7 +543,8 @@ expressions in the `if` condition. If regular expressions are enabled, operators such as `=~` can be used against a `/pattern/` for conditions. For example: -[source,js] + +[source,console] -------------------------------------------------- PUT _ingest/pipeline/check_url { @@ -571,9 +559,8 @@ PUT _ingest/pipeline/check_url ] } -------------------------------------------------- -// CONSOLE -[source,js] +[source,console] -------------------------------------------------- POST test/_doc/1?pipeline=check_url { @@ -582,18 +569,16 @@ POST test/_doc/1?pipeline=check_url } } -------------------------------------------------- -// CONSOLE // TEST[continued] Results in: //// Hidden example assertion: -[source,js] +[source,console] -------------------------------------------------- GET test/_doc/1 -------------------------------------------------- -// CONSOLE // TEST[continued] //// @@ -624,7 +609,7 @@ alternatives exist. For example in this case `startsWith` can be used to get the same result without using a regular expression: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/check_url { @@ -639,7 +624,6 @@ PUT _ingest/pipeline/check_url ] } -------------------------------------------------- -// CONSOLE [[handling-failure-in-pipelines]] == Handling Failures in Pipelines diff --git a/docs/reference/ingest/processors/circle.asciidoc b/docs/reference/ingest/processors/circle.asciidoc index 97120fe91815..7e577a59d50c 100644 --- a/docs/reference/ingest/processors/circle.asciidoc +++ b/docs/reference/ingest/processors/circle.asciidoc @@ -20,7 +20,7 @@ include::common-options.asciidoc[] image:images/spatial/error_distance.png[] -[source,js] +[source,console] -------------------------------------------------- PUT circles { @@ -47,7 +47,6 @@ PUT _ingest/pipeline/polygonize_circles ] } -------------------------------------------------- -// CONSOLE Using the above pipeline, we can attempt to index a document into the `circles` index. The circle can be represented as either a WKT circle or a GeoJSON circle. The resulting @@ -58,7 +57,7 @@ be translated to a WKT polygon, and GeoJSON circles will be translated to GeoJSO In this example a circle defined in WKT format is indexed -[source,js] +[source,console] -------------------------------------------------- PUT circles/_doc/1?pipeline=polygonize_circles { @@ -67,7 +66,6 @@ PUT circles/_doc/1?pipeline=polygonize_circles GET circles/_doc/1 -------------------------------------------------- -// CONSOLE // TEST[continued] The response from the above index request: @@ -93,7 +91,7 @@ The response from the above index request: In this example a circle defined in GeoJSON format is indexed -[source,js] +[source,console] -------------------------------------------------- PUT circles/_doc/2?pipeline=polygonize_circles { @@ -106,7 +104,6 @@ PUT circles/_doc/2?pipeline=polygonize_circles GET circles/_doc/2 -------------------------------------------------- -// CONSOLE // TEST[continued] The response from the above index request: diff --git a/docs/reference/ingest/processors/date-index-name.asciidoc b/docs/reference/ingest/processors/date-index-name.asciidoc index 783ecc9b2b1b..52e29dfb120e 100644 --- a/docs/reference/ingest/processors/date-index-name.asciidoc +++ b/docs/reference/ingest/processors/date-index-name.asciidoc @@ -16,7 +16,7 @@ expression. An example pipeline that points documents to a monthly index that starts with a `myindex-` prefix based on a date in the `date1` field: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/monthlyindex { @@ -32,19 +32,17 @@ PUT _ingest/pipeline/monthlyindex ] } -------------------------------------------------- -// CONSOLE Using that pipeline for an index request: -[source,js] +[source,console] -------------------------------------------------- PUT /myindex/_doc/1?pipeline=monthlyindex { "date1" : "2016-04-25T12:02:01.789Z" } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] @@ -74,7 +72,7 @@ To see the date-math value of the index supplied in the actual index request whi indexed into `myindex-2016-04-01` we can inspect the effects of the processor using a simulate request. -[source,js] +[source,console] -------------------------------------------------- POST _ingest/pipeline/_simulate { @@ -100,7 +98,6 @@ POST _ingest/pipeline/_simulate ] } -------------------------------------------------- -// CONSOLE and the result: diff --git a/docs/reference/ingest/processors/geoip.asciidoc b/docs/reference/ingest/processors/geoip.asciidoc index 7ce7b430ef61..3be65bb65be1 100644 --- a/docs/reference/ingest/processors/geoip.asciidoc +++ b/docs/reference/ingest/processors/geoip.asciidoc @@ -41,7 +41,7 @@ in `properties`. Here is an example that uses the default city database and adds the geographical information to the `geoip` field based on the `ip` field: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/geoip { @@ -60,7 +60,6 @@ PUT my_index/_doc/my_id?pipeline=geoip } GET my_index/_doc/my_id -------------------------------------------------- -// CONSOLE Which returns: @@ -90,7 +89,7 @@ Here is an example that uses the default country database and adds the geographical information to the `geo` field based on the `ip` field`. Note that this database is included in the module. So this: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/geoip { @@ -111,7 +110,6 @@ PUT my_index/_doc/my_id?pipeline=geoip } GET my_index/_doc/my_id -------------------------------------------------- -// CONSOLE returns this: @@ -143,7 +141,7 @@ occurs, no `target_field` is inserted into the document. Here is an example of what documents will be indexed as when information for "80.231.5.0" cannot be found: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/geoip { @@ -164,7 +162,6 @@ PUT my_index/_doc/my_id?pipeline=geoip GET my_index/_doc/my_id -------------------------------------------------- -// CONSOLE Which returns: @@ -194,7 +191,7 @@ as such in the mapping. You can use the following mapping for the example index above: -[source,js] +[source,console] -------------------------------------------------- PUT my_ip_locations { @@ -209,10 +206,9 @@ PUT my_ip_locations } } -------------------------------------------------- -// CONSOLE //// -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/geoip { @@ -251,7 +247,6 @@ GET /my_ip_locations/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] diff --git a/docs/reference/ingest/processors/grok.asciidoc b/docs/reference/ingest/processors/grok.asciidoc index f6f5fb3c9288..c58d447f4b7b 100644 --- a/docs/reference/ingest/processors/grok.asciidoc +++ b/docs/reference/ingest/processors/grok.asciidoc @@ -155,7 +155,7 @@ the same `or` behavior. Here is an example of such a configuration executed against the simulate API: -[source,js] +[source,console] -------------------------------------------------- POST _ingest/pipeline/_simulate { @@ -183,7 +183,6 @@ POST _ingest/pipeline/_simulate ] } -------------------------------------------------- -// CONSOLE response: @@ -216,7 +215,7 @@ that same pipeline, but with `"trace_match": true` configured: //// Hidden setup for example: -[source,js] +[source,console] -------------------------------------------------- POST _ingest/pipeline/_simulate { @@ -245,7 +244,6 @@ POST _ingest/pipeline/_simulate ] } -------------------------------------------------- -// CONSOLE //// [source,js] @@ -283,11 +281,10 @@ metadata and will not be indexed. The Grok Processor comes packaged with its own REST endpoint for retrieving which patterns the processor is packaged with. -[source,js] +[source,console] -------------------------------------------------- GET _ingest/processor/grok -------------------------------------------------- -// CONSOLE The above request will return a response body containing a key-value representation of the built-in patterns dictionary. diff --git a/docs/reference/ingest/processors/pipeline.asciidoc b/docs/reference/ingest/processors/pipeline.asciidoc index 1b138123a721..2eddf1631631 100644 --- a/docs/reference/ingest/processors/pipeline.asciidoc +++ b/docs/reference/ingest/processors/pipeline.asciidoc @@ -25,7 +25,7 @@ An example of using this processor for nesting pipelines would be: Define an inner pipeline: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/pipelineA { @@ -40,11 +40,10 @@ PUT _ingest/pipeline/pipelineA ] } -------------------------------------------------- -// CONSOLE Define another pipeline that uses the previously defined inner pipeline: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/pipelineB { @@ -64,20 +63,18 @@ PUT _ingest/pipeline/pipelineB ] } -------------------------------------------------- -// CONSOLE // TEST[continued] Now indexing a document while applying the outer pipeline will see the inner pipeline executed from the outer pipeline: -[source,js] +[source,console] -------------------------------------------------- PUT /myindex/_doc/1?pipeline=pipelineB { "field": "value" } -------------------------------------------------- -// CONSOLE // TEST[continued] Response from the index request: diff --git a/docs/reference/ingest/processors/script.asciidoc b/docs/reference/ingest/processors/script.asciidoc index 9e0dbe60b8d8..aab739d0419b 100644 --- a/docs/reference/ingest/processors/script.asciidoc +++ b/docs/reference/ingest/processors/script.asciidoc @@ -46,7 +46,7 @@ It is possible to use the Script Processor to manipulate document metadata like ingestion. Here is an example of an Ingest Pipeline that renames the index and type to `my_index` no matter what was provided in the original index request: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/my_index { @@ -63,18 +63,16 @@ PUT _ingest/pipeline/my_index ] } -------------------------------------------------- -// CONSOLE Using the above pipeline, we can attempt to index a document into the `any_index` index. -[source,js] +[source,console] -------------------------------------------------- PUT any_index/_doc/1?pipeline=my_index { "message": "text" } -------------------------------------------------- -// CONSOLE // TEST[continued] The response from the above index request: diff --git a/docs/reference/ingest/processors/set.asciidoc b/docs/reference/ingest/processors/set.asciidoc index 499e49c29b48..8ea0e7101e54 100644 --- a/docs/reference/ingest/processors/set.asciidoc +++ b/docs/reference/ingest/processors/set.asciidoc @@ -28,7 +28,7 @@ include::common-options.asciidoc[] This processor can also be used to copy data from one field to another. For example: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/set_os { @@ -54,7 +54,6 @@ POST _ingest/pipeline/set_os/_simulate ] } -------------------------------------------------- -// CONSOLE Result: [source,js] diff --git a/docs/reference/ingest/processors/user-agent.asciidoc b/docs/reference/ingest/processors/user-agent.asciidoc index 942ba9f14879..6993e449f47e 100644 --- a/docs/reference/ingest/processors/user-agent.asciidoc +++ b/docs/reference/ingest/processors/user-agent.asciidoc @@ -23,7 +23,7 @@ The ingest-user-agent module ships by default with the regexes.yaml made availab Here is an example that adds the user agent details to the `user_agent` field based on the `agent` field: -[source,js] +[source,console] -------------------------------------------------- PUT _ingest/pipeline/user_agent { @@ -42,7 +42,6 @@ PUT my_index/_doc/my_id?pipeline=user_agent } GET my_index/_doc/my_id -------------------------------------------------- -// CONSOLE Which returns diff --git a/docs/reference/licensing/delete-license.asciidoc b/docs/reference/licensing/delete-license.asciidoc index 425504f7cba1..b086fdac009a 100644 --- a/docs/reference/licensing/delete-license.asciidoc +++ b/docs/reference/licensing/delete-license.asciidoc @@ -31,11 +31,10 @@ For more information, see The following example queries the info API: -[source,js] +[source,console] ------------------------------------------------------------ DELETE /_license ------------------------------------------------------------ -// CONSOLE // TEST[skip:license testing issues] When the license is successfully deleted, the API returns the following response: diff --git a/docs/reference/licensing/get-basic-status.asciidoc b/docs/reference/licensing/get-basic-status.asciidoc index 673bb27e5962..38abc087d957 100644 --- a/docs/reference/licensing/get-basic-status.asciidoc +++ b/docs/reference/licensing/get-basic-status.asciidoc @@ -33,11 +33,10 @@ For more information, see The following example checks whether you are eligible to start a basic: -[source,js] +[source,console] ------------------------------------------------------------ GET /_license/basic_status ------------------------------------------------------------ -// CONSOLE Example response: [source,js] diff --git a/docs/reference/licensing/get-license.asciidoc b/docs/reference/licensing/get-license.asciidoc index 3be86bf3e734..c49e85384a04 100644 --- a/docs/reference/licensing/get-license.asciidoc +++ b/docs/reference/licensing/get-license.asciidoc @@ -44,11 +44,10 @@ For more information, see The following example provides information about a trial license: -[source,js] +[source,console] -------------------------------------------------- GET /_license -------------------------------------------------- -// CONSOLE [source,js] -------------------------------------------------- diff --git a/docs/reference/licensing/get-trial-status.asciidoc b/docs/reference/licensing/get-trial-status.asciidoc index b81dade1ad85..b6caaf0252ca 100644 --- a/docs/reference/licensing/get-trial-status.asciidoc +++ b/docs/reference/licensing/get-trial-status.asciidoc @@ -39,11 +39,10 @@ For more information, see The following example checks whether you are eligible to start a trial: -[source,js] +[source,console] ------------------------------------------------------------ GET /_license/trial_status ------------------------------------------------------------ -// CONSOLE Example response: [source,js] diff --git a/docs/reference/licensing/start-basic.asciidoc b/docs/reference/licensing/start-basic.asciidoc index d3c385059627..dd421bfda150 100644 --- a/docs/reference/licensing/start-basic.asciidoc +++ b/docs/reference/licensing/start-basic.asciidoc @@ -39,11 +39,10 @@ For more information, see The following example starts a basic license if you do not currently have a license: -[source,js] +[source,console] ------------------------------------------------------------ POST /_license/start_basic ------------------------------------------------------------ -// CONSOLE // TEST[skip:license testing issues] Example response: @@ -60,11 +59,10 @@ The following example starts a basic license if you currently have a license wit features than a basic license. As you are losing features, you must pass the acknowledge parameter: -[source,js] +[source,console] ------------------------------------------------------------ POST /_license/start_basic?acknowledge=true ------------------------------------------------------------ -// CONSOLE // TEST[skip:license testing issues] Example response: diff --git a/docs/reference/licensing/start-trial.asciidoc b/docs/reference/licensing/start-trial.asciidoc index 0afce9ec9d22..df5fd1dfae66 100644 --- a/docs/reference/licensing/start-trial.asciidoc +++ b/docs/reference/licensing/start-trial.asciidoc @@ -43,11 +43,10 @@ For more information, see The following example starts a 30-day trial license. The acknowledge parameter is required as you are initiating a license that will expire. -[source,js] +[source,console] ------------------------------------------------------------ POST /_license/start_trial?acknowledge=true ------------------------------------------------------------ -// CONSOLE // TEST[skip:license testing issues] Example response: diff --git a/docs/reference/licensing/update-license.asciidoc b/docs/reference/licensing/update-license.asciidoc index 04798c03947b..c4c8815b3c7f 100644 --- a/docs/reference/licensing/update-license.asciidoc +++ b/docs/reference/licensing/update-license.asciidoc @@ -55,7 +55,7 @@ install the license. See <>. The following example updates to a basic license: -[source,js] +[source,console] ------------------------------------------------------------ POST /_license { @@ -73,7 +73,6 @@ POST /_license ] } ------------------------------------------------------------ -// CONSOLE // TEST[skip:license testing issues] NOTE: These values are invalid; you must substitute the appropriate content @@ -132,7 +131,7 @@ receive the following response: To complete the update, you must re-submit the API request and set the `acknowledge` parameter to `true`. For example: -[source,js] +[source,console] ------------------------------------------------------------ POST /_license?acknowledge=true { @@ -150,7 +149,6 @@ POST /_license?acknowledge=true ] } ------------------------------------------------------------ -// CONSOLE // TEST[skip:license testing issues] Alternatively: diff --git a/docs/reference/mapping.asciidoc b/docs/reference/mapping.asciidoc index 2cac5be9ccf3..2c30bfacc808 100644 --- a/docs/reference/mapping.asciidoc +++ b/docs/reference/mapping.asciidoc @@ -129,7 +129,7 @@ You can create field mappings when you <> and You can use the <> API to create a new index with an explicit mapping. -[source,js] +[source,console] ---- PUT /my-index { @@ -142,7 +142,6 @@ PUT /my-index } } ---- -// CONSOLE <1> Creates `age`, an <> field <2> Creates `email`, a <> field @@ -159,7 +158,7 @@ The following example adds `employee-id`, a `keyword` field with an <> mapping parameter value of `false`. This means values for the `employee-id` field are stored but not indexed or available for search. -[source,js] +[source,console] ---- PUT /my-index/_mapping { @@ -171,7 +170,6 @@ PUT /my-index/_mapping } } ---- -// CONSOLE // TEST[continued] [float] @@ -187,11 +185,10 @@ include::{docdir}/indices/put-mapping.asciidoc[tag=put-field-mapping-exceptions] You can use the <> API to view the mapping of an existing index. -[source,js] +[source,console] ---- GET /my-index/_mapping ---- -// CONSOLE // TEST[continued] The API returns the following response: @@ -234,11 +231,10 @@ contains a large number of fields. The following request retrieves the mapping for the `employee-id` field. -[source,js] +[source,console] ---- GET /my-index/_mapping/field/employee-id ---- -// CONSOLE // TEST[continued] The API returns the following response: diff --git a/docs/reference/mapping/dynamic-mapping.asciidoc b/docs/reference/mapping/dynamic-mapping.asciidoc index 950d7b390510..8daae6f2ac03 100644 --- a/docs/reference/mapping/dynamic-mapping.asciidoc +++ b/docs/reference/mapping/dynamic-mapping.asciidoc @@ -7,12 +7,12 @@ To index a document, you don't have to first create an index, define a mapping type, and define your fields -- you can just index a document and the index, type, and fields will spring to life automatically: -[source,js] +[source,console] -------------------------------------------------- PUT data/_doc/1 <1> { "count": 5 } -------------------------------------------------- -// CONSOLE + <1> Creates the `data` index, the `_doc` mapping type, and a field called `count` with datatype `long`. diff --git a/docs/reference/mapping/dynamic/field-mapping.asciidoc b/docs/reference/mapping/dynamic/field-mapping.asciidoc index 735fddf3f84a..c27c81c44289 100644 --- a/docs/reference/mapping/dynamic/field-mapping.asciidoc +++ b/docs/reference/mapping/dynamic/field-mapping.asciidoc @@ -46,7 +46,7 @@ The default value for `dynamic_date_formats` is: For example: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1 { @@ -55,7 +55,7 @@ PUT my_index/_doc/1 GET my_index/_mapping <1> -------------------------------------------------- -// CONSOLE + <1> The `create_date` field has been added as a <> field with the <>: + `"yyyy/MM/dd HH:mm:ss Z||yyyy/MM/dd Z"`. @@ -64,7 +64,7 @@ GET my_index/_mapping <1> Dynamic date detection can be disabled by setting `date_detection` to `false`: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -78,7 +78,6 @@ PUT my_index/_doc/1 <1> "create": "2015/09/02" } -------------------------------------------------- -// CONSOLE <1> The `create_date` field has been added as a <> field. @@ -87,7 +86,7 @@ PUT my_index/_doc/1 <1> Alternatively, the `dynamic_date_formats` can be customised to support your own <>: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -101,7 +100,6 @@ PUT my_index/_doc/1 "create_date": "09/25/2015" } -------------------------------------------------- -// CONSOLE [[numeric-detection]] @@ -113,7 +111,7 @@ correct solution is to map these fields explicitly, but numeric detection (which is disabled by default) can be enabled to do this automatically: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -128,7 +126,7 @@ PUT my_index/_doc/1 "my_integer": "1" <2> } -------------------------------------------------- -// CONSOLE + <1> The `my_float` field is added as a <> field. <2> The `my_integer` field is added as a <> field. diff --git a/docs/reference/mapping/dynamic/templates.asciidoc b/docs/reference/mapping/dynamic/templates.asciidoc index c5109ab9c9fc..2dec3912d235 100644 --- a/docs/reference/mapping/dynamic/templates.asciidoc +++ b/docs/reference/mapping/dynamic/templates.asciidoc @@ -67,7 +67,7 @@ For example, if we wanted to map all integer fields as `integer` instead of `long`, and all `string` fields as both `text` and `keyword`, we could use the following template: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -105,7 +105,7 @@ PUT my_index/_doc/1 "my_string": "Some string" <2> } -------------------------------------------------- -// CONSOLE + <1> The `my_integer` field is mapped as an `integer`. <2> The `my_string` field is mapped as a `text`, with a `keyword` <>. @@ -121,7 +121,7 @@ The following example matches all `string` fields whose name starts with fields: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -147,7 +147,7 @@ PUT my_index/_doc/1 "long_text": "foo" <2> } -------------------------------------------------- -// CONSOLE + <1> The `long_num` field is mapped as a `long`. <2> The `long_text` field uses the default `string` mapping. @@ -175,7 +175,7 @@ final name, e.g. `some_object.*.some_field`. This example copies the values of any fields in the `name` object to the top-level `full_name` field, except for the `middle` field: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -204,14 +204,13 @@ PUT my_index/_doc/1 } } -------------------------------------------------- -// CONSOLE Note that the `path_match` and `path_unmatch` parameters match on object paths in addition to leaf fields. As an example, indexing the following document will result in an error because the `path_match` setting also matches the object field `name.title`, which can't be mapped as text: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/2 { @@ -225,7 +224,6 @@ PUT my_index/_doc/2 } } -------------------------------------------------- -// CONSOLE // TEST[continued] // TEST[catch:bad_request] @@ -237,7 +235,7 @@ with the field name and detected dynamic type. The following example sets all string fields to use an <> with the same name as the field, and disables <> for all non-string fields: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -272,7 +270,7 @@ PUT my_index/_doc/1 "count": 5 <2> } -------------------------------------------------- -// CONSOLE + <1> The `english` field is mapped as a `string` field with the `english` analyzer. <2> The `count` field is mapped as a `long` field with `doc_values` disabled. @@ -289,7 +287,7 @@ interested in full text search, you can make Elasticsearch map your fields only as `keyword`s. Note that this means that in order to search those fields, you will have to search on the exact same value that was indexed. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -307,7 +305,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE [[text-only-mappings-strings]] ===== `text`-only mappings for strings @@ -318,7 +315,7 @@ aggregations, sorting or exact search on your string fields, you could tell Elasticsearch to map it only as a text field (which was the default behaviour before 5.0): -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -336,7 +333,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE ===== Disabled norms @@ -344,7 +340,7 @@ Norms are index-time scoring factors. If you do not care about scoring, which would be the case for instance if you never sort documents by score, you could disable the storage of these scoring factors in the index and save some space. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -369,7 +365,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE The sub `keyword` field appears in this template to be consistent with the default rules of dynamic mappings. Of course if you do not need them because @@ -383,7 +378,7 @@ numeric fields that you will often aggregate on but never filter on. In such a case, you could disable indexing on those fields to save disk space and also maybe gain some indexing speed: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -411,7 +406,7 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE + <1> Like the default dynamic mapping rules, doubles are mapped as floats, which are usually accurate enough, yet require half the disk space. diff --git a/docs/reference/mapping/fields/field-names-field.asciidoc b/docs/reference/mapping/fields/field-names-field.asciidoc index 1ae4ab4c8fc8..1fe1760f1335 100644 --- a/docs/reference/mapping/fields/field-names-field.asciidoc +++ b/docs/reference/mapping/fields/field-names-field.asciidoc @@ -20,7 +20,7 @@ which have `doc_values` and `norms` disabled and you do not need to execute `exists` queries using those fields you might want to disable `_field_names` be adding the following to the mappings: -[source,js] +[source,console] -------------------------------------------------- PUT tweets { @@ -31,4 +31,3 @@ PUT tweets } } -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/mapping/fields/id-field.asciidoc b/docs/reference/mapping/fields/id-field.asciidoc index 5ccf1cc8ec75..9d93c60b3927 100644 --- a/docs/reference/mapping/fields/id-field.asciidoc +++ b/docs/reference/mapping/fields/id-field.asciidoc @@ -8,7 +8,7 @@ so that documents can be looked up either with the <> or the The value of the `_id` field is accessible in certain queries (`term`, `terms`, `match`, `query_string`, `simple_query_string`). -[source,js] +[source,console] -------------------------- # Example documents PUT my_index/_doc/1 @@ -30,7 +30,6 @@ GET my_index/_search } } -------------------------- -// CONSOLE <1> Querying on the `_id` field (also see the <>) diff --git a/docs/reference/mapping/fields/ignored-field.asciidoc b/docs/reference/mapping/fields/ignored-field.asciidoc index d2776ea86b26..c22e63988925 100644 --- a/docs/reference/mapping/fields/ignored-field.asciidoc +++ b/docs/reference/mapping/fields/ignored-field.asciidoc @@ -14,7 +14,7 @@ queries, and is returned as part of the search hits. For instance the below query matches all documents that have one or more fields that got ignored: -[source,js] +[source,console] -------------------------------------------------- GET _search { @@ -25,12 +25,11 @@ GET _search } } -------------------------------------------------- -// CONSOLE Similarly, the below query finds all documents whose `@timestamp` field was ignored at index time: -[source,js] +[source,console] -------------------------------------------------- GET _search { @@ -41,5 +40,3 @@ GET _search } } -------------------------------------------------- -// CONSOLE - diff --git a/docs/reference/mapping/fields/index-field.asciidoc b/docs/reference/mapping/fields/index-field.asciidoc index b11c1cee1440..47cb2414ed1b 100644 --- a/docs/reference/mapping/fields/index-field.asciidoc +++ b/docs/reference/mapping/fields/index-field.asciidoc @@ -13,7 +13,7 @@ in a `term` or `terms` query (or any query that is rewritten to a `term` query, such as the `match`, `query_string` or `simple_query_string` query), but it does not support `prefix`, `wildcard`, `regexp`, or `fuzzy` queries. -[source,js] +[source,console] -------------------------- # Example documents PUT index_1/_doc/1 @@ -58,7 +58,6 @@ GET index_1,index_2/_search } } -------------------------- -// CONSOLE <1> Querying on the `_index` field <2> Aggregating on the `_index` field diff --git a/docs/reference/mapping/fields/meta-field.asciidoc b/docs/reference/mapping/fields/meta-field.asciidoc index b2225dba4e81..1c698a3b223b 100644 --- a/docs/reference/mapping/fields/meta-field.asciidoc +++ b/docs/reference/mapping/fields/meta-field.asciidoc @@ -5,7 +5,7 @@ A mapping type can have custom meta data associated with it. These are not used at all by Elasticsearch, but can be used to store application-specific metadata, such as the class that a document belongs to: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -20,14 +20,14 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE + <1> This `_meta` info can be retrieved with the <> API. The `_meta` field can be updated on an existing type using the <> API: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_mapping { @@ -40,5 +40,4 @@ PUT my_index/_mapping } } -------------------------------------------------- -// CONSOLE // TEST[continued] diff --git a/docs/reference/mapping/fields/routing-field.asciidoc b/docs/reference/mapping/fields/routing-field.asciidoc index 6c75b91b522a..25c3571f08e8 100644 --- a/docs/reference/mapping/fields/routing-field.asciidoc +++ b/docs/reference/mapping/fields/routing-field.asciidoc @@ -11,7 +11,7 @@ The default value used for `_routing` is the document's < { @@ -20,7 +20,6 @@ PUT my_index/_doc/1?routing=user1&refresh=true <1> GET my_index/_doc/1?routing=user1 <2> ------------------------------ -// CONSOLE // TESTSETUP <1> This document uses `user1` as its routing value, instead of its ID. @@ -30,7 +29,7 @@ GET my_index/_doc/1?routing=user1 <2> The value of the `_routing` field is accessible in queries: -[source,js] +[source,console] -------------------------- GET my_index/_search { @@ -41,7 +40,6 @@ GET my_index/_search } } -------------------------- -// CONSOLE <1> Querying on the `_routing` field (also see the <>) @@ -51,7 +49,7 @@ Custom routing can reduce the impact of searches. Instead of having to fan out a search request to all the shards in an index, the request can be sent to just the shard that matches the specific routing value (or values): -[source,js] +[source,console] ------------------------------ GET my_index/_search?routing=user1,user2 <1> { @@ -62,7 +60,6 @@ GET my_index/_search?routing=user1,user2 <1> } } ------------------------------ -// CONSOLE <1> This search request will only be executed on the shards associated with the `user1` and `user2` routing values. @@ -77,7 +74,7 @@ Forgetting the routing value can lead to a document being indexed on more than one shard. As a safeguard, the `_routing` field can be configured to make a custom `routing` value required for all CRUD operations: -[source,js] +[source,console] ------------------------------ PUT my_index2 { @@ -93,8 +90,8 @@ PUT my_index2/_doc/1 <2> "text": "No routing value provided" } ------------------------------ -// CONSOLE // TEST[catch:bad_request] + <1> Routing is required for `_doc` documents. <2> This index request throws a `routing_missing_exception`. diff --git a/docs/reference/mapping/fields/source-field.asciidoc b/docs/reference/mapping/fields/source-field.asciidoc index 78cef20ea81b..a5c87453517e 100644 --- a/docs/reference/mapping/fields/source-field.asciidoc +++ b/docs/reference/mapping/fields/source-field.asciidoc @@ -12,7 +12,7 @@ _fetch_ requests, like <> or <>. Though very handy to have around, the source field does incur storage overhead within the index. For this reason, it can be disabled as follows: -[source,js] +[source,console] -------------------------------------------------- PUT tweets { @@ -23,7 +23,6 @@ PUT tweets } } -------------------------------------------------- -// CONSOLE [WARNING] .Think before disabling the `_source` field @@ -82,7 +81,7 @@ Elasticsearch index to another. Consider using The `includes`/`excludes` parameters (which also accept wildcards) can be used as follows: -[source,js] +[source,console] -------------------------------------------------- PUT logs { @@ -125,7 +124,6 @@ GET logs/_search } } -------------------------------------------------- -// CONSOLE <1> These fields will be removed from the stored `_source` field. <2> We can still search on this field, even though it is not in the stored `_source`. diff --git a/docs/reference/mapping/fields/type-field.asciidoc b/docs/reference/mapping/fields/type-field.asciidoc index c1cfa7971025..2c83596e42d3 100644 --- a/docs/reference/mapping/fields/type-field.asciidoc +++ b/docs/reference/mapping/fields/type-field.asciidoc @@ -10,7 +10,7 @@ indexed in order to make searching by type name fast. The value of the `_type` field is accessible in queries, aggregations, scripts, and when sorting: -[source,js] +[source,console] -------------------------- # Example documents @@ -52,7 +52,6 @@ GET my_index/_search } -------------------------- -// CONSOLE <1> Querying on the `_type` field <2> Aggregating on the `_type` field diff --git a/docs/reference/mapping/params/analyzer.asciidoc b/docs/reference/mapping/params/analyzer.asciidoc index b2652bf999a7..6d4ad699dc91 100644 --- a/docs/reference/mapping/params/analyzer.asciidoc +++ b/docs/reference/mapping/params/analyzer.asciidoc @@ -39,7 +39,7 @@ At query time, there are a few more layers: The easiest way to specify an analyzer for a particular field is to define it in the field mapping, as follows: -[source,js] +[source,console] -------------------------------------------------- PUT /my_index { @@ -70,7 +70,7 @@ GET my_index/_analyze <4> "text": "The quick Brown Foxes." } -------------------------------------------------- -// CONSOLE + <1> The `text` field uses the default `standard` analyzer`. <2> The `text.english` <> uses the `english` analyzer, which removes stop words and applies stemming. <3> This returns the tokens: [ `the`, `quick`, `brown`, `foxes` ]. @@ -89,7 +89,7 @@ To disable stop words for phrases a field utilising three analyzer settings will 2. A `search_analyzer` setting for non-phrase queries that will remove stop words 3. A `search_quote_analyzer` setting for phrase queries that will not remove stop words -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -151,7 +151,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> `my_analyzer` analyzer which tokens all terms including stop words <2> `my_stop_analyzer` analyzer which removes stop words <3> `analyzer` setting that points to the `my_analyzer` analyzer which will be used at index time diff --git a/docs/reference/mapping/params/boost.asciidoc b/docs/reference/mapping/params/boost.asciidoc index 6dfe564ed7c5..29b56e4692c3 100644 --- a/docs/reference/mapping/params/boost.asciidoc +++ b/docs/reference/mapping/params/boost.asciidoc @@ -4,7 +4,7 @@ Individual fields can be _boosted_ automatically -- count more towards the relevance score -- at query time, with the `boost` parameter as follows: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -21,7 +21,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE <1> Matches on the `title` field will have twice the weight as those on the `content` field, which has the default `boost` of `1.0`. @@ -30,7 +29,7 @@ NOTE: The boost is applied only for term queries (prefix, range and fuzzy querie You can achieve the same effect by using the boost parameter directly in the query, for instance the following query (with field time boost): -[source,js] +[source,console] -------------------------------------------------- POST _search { @@ -43,11 +42,10 @@ POST _search } } -------------------------------------------------- -// CONSOLE is equivalent to: -[source,js] +[source,console] -------------------------------------------------- POST _search { @@ -61,7 +59,6 @@ POST _search } } -------------------------------------------------- -// CONSOLE deprecated[5.0.0, "Index time boost is deprecated. Instead, the field mapping boost is applied at query time. For indices created before 5.0.0, the boost will still be applied at index time."] diff --git a/docs/reference/mapping/params/coerce.asciidoc b/docs/reference/mapping/params/coerce.asciidoc index be5b2a648c60..a8176336a2a0 100644 --- a/docs/reference/mapping/params/coerce.asciidoc +++ b/docs/reference/mapping/params/coerce.asciidoc @@ -15,7 +15,7 @@ For instance: For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -42,8 +42,8 @@ PUT my_index/_doc/2 "number_two": "10" <2> } -------------------------------------------------- -// CONSOLE // TEST[catch:bad_request] + <1> The `number_one` field will contain the integer `10`. <2> This document will be rejected because coercion is disabled. @@ -56,7 +56,7 @@ using the <>. The `index.mapping.coerce` setting can be set on the index level to disable coercion globally across all mapping types: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -82,7 +82,7 @@ PUT my_index/_doc/1 PUT my_index/_doc/2 { "number_two": "10" } <2> -------------------------------------------------- -// CONSOLE // TEST[catch:bad_request] + <1> The `number_one` field overrides the index level setting to enable coercion. <2> This document will be rejected because the `number_two` field inherits the index-level coercion setting. diff --git a/docs/reference/mapping/params/copy-to.asciidoc b/docs/reference/mapping/params/copy-to.asciidoc index 1796b31360ae..baad4d7cda45 100644 --- a/docs/reference/mapping/params/copy-to.asciidoc +++ b/docs/reference/mapping/params/copy-to.asciidoc @@ -6,7 +6,7 @@ fields into a group field, which can then be queried as a single field. For instance, the `first_name` and `last_name` fields can be copied to the `full_name` field as follows: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -46,7 +46,7 @@ GET my_index/_search } -------------------------------------------------- -// CONSOLE + <1> The values of the `first_name` and `last_name` fields are copied to the `full_name` field. diff --git a/docs/reference/mapping/params/doc-values.asciidoc b/docs/reference/mapping/params/doc-values.asciidoc index 5680cdabba88..53d6ac9ebdde 100644 --- a/docs/reference/mapping/params/doc-values.asciidoc +++ b/docs/reference/mapping/params/doc-values.asciidoc @@ -21,7 +21,7 @@ All fields which support doc values have them enabled by default. If you are sure that you don't need to sort or aggregate on a field, or access the field value from a script, you can disable doc values in order to save disk space: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -38,7 +38,7 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE + <1> The `status_code` field has `doc_values` enabled by default. <2> The `session_id` has `doc_values` disabled, but can still be queried. diff --git a/docs/reference/mapping/params/dynamic.asciidoc b/docs/reference/mapping/params/dynamic.asciidoc index 62d61f5f095a..6979b491bce8 100644 --- a/docs/reference/mapping/params/dynamic.asciidoc +++ b/docs/reference/mapping/params/dynamic.asciidoc @@ -5,7 +5,7 @@ By default, fields can be added _dynamically_ to a document, or to <> within a document, just by indexing a document containing the new field. For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1 <1> { @@ -31,7 +31,7 @@ PUT my_index/_doc/2 <3> GET my_index/_mapping <4> -------------------------------------------------- -// CONSOLE + <1> This document introduces the string field `username`, the object field `name`, and two string fields under the `name` object which can be referred to as `name.first` and `name.last`. @@ -56,7 +56,7 @@ The `dynamic` setting may be set at the mapping type level, and on each <>. Inner objects inherit the setting from their parent object or from the mapping type. For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -78,7 +78,7 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE + <1> Dynamic mapping is disabled at the type level, so no new top-level fields will be added dynamically. <2> The `user` object inherits the type-level setting. <3> The `user.social_networks` object enables dynamic mapping, so new fields may be added to this inner object. diff --git a/docs/reference/mapping/params/eager-global-ordinals.asciidoc b/docs/reference/mapping/params/eager-global-ordinals.asciidoc index 162049ec1323..22dd13c8d754 100644 --- a/docs/reference/mapping/params/eager-global-ordinals.asciidoc +++ b/docs/reference/mapping/params/eager-global-ordinals.asciidoc @@ -34,7 +34,7 @@ interested in search speed, it could be beneficial to set `eager_global_ordinals: true` on fields that you plan to use in terms aggregations: -[source,js] +[source,console] ------------ PUT my_index/_mapping { @@ -46,7 +46,6 @@ PUT my_index/_mapping } } ------------ -// CONSOLE // TEST[s/^/PUT my_index\n/] This will shift the cost of building the global ordinals from search-time to @@ -73,7 +72,7 @@ If you ever decide that you do not need to run `terms` aggregations on this field anymore, then you can disable eager loading of global ordinals at any time: -[source,js] +[source,console] ------------ PUT my_index/_mapping { @@ -85,6 +84,4 @@ PUT my_index/_mapping } } ------------ -// CONSOLE // TEST[continued] - diff --git a/docs/reference/mapping/params/enabled.asciidoc b/docs/reference/mapping/params/enabled.asciidoc index edbfb1f77d71..40d5ed1e800d 100644 --- a/docs/reference/mapping/params/enabled.asciidoc +++ b/docs/reference/mapping/params/enabled.asciidoc @@ -13,7 +13,7 @@ parsing of the contents of the field entirely. The JSON can still be retrieved from the <> field, but it is not searchable or stored in any other way: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -51,7 +51,7 @@ PUT my_index/_doc/session_2 "last_updated": "2015-12-06T18:22:13" } -------------------------------------------------- -// CONSOLE + <1> The `session_data` field is disabled. <2> Any arbitrary data can be passed to the `session_data` field as it will be entirely ignored. <3> The `session_data` will also ignore values that are not JSON objects. @@ -60,7 +60,7 @@ The entire mapping may be disabled as well, in which case the document is stored in the <> field, which means it can be retrieved, but none of its contents are indexed in any way: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -84,7 +84,7 @@ GET my_index/_doc/session_1 <2> GET my_index/_mapping <3> -------------------------------------------------- -// CONSOLE + <1> The entire mapping is disabled. <2> The document can be retrieved. <3> Checking the mapping reveals that no fields have been added. @@ -94,7 +94,8 @@ definition cannot be updated. Note that because Elasticsearch completely skips parsing the field contents, it is possible to add non-object data to a disabled field: -[source,js] + +[source,console] -------------------------------------------------- PUT my_index { @@ -113,6 +114,5 @@ PUT my_index/_doc/session_1 "session_data": "foo bar" <1> } -------------------------------------------------- -// CONSOLE <1> The document is added successfully, even though `session_data` contains non-object data. \ No newline at end of file diff --git a/docs/reference/mapping/params/fielddata.asciidoc b/docs/reference/mapping/params/fielddata.asciidoc index 8e46dd037024..df4e37566062 100644 --- a/docs/reference/mapping/params/fielddata.asciidoc +++ b/docs/reference/mapping/params/fielddata.asciidoc @@ -54,7 +54,7 @@ Instead, you should have a `text` field for full text searches, and an unanalyzed <> field with <> enabled for aggregations, as follows: -[source,js] +[source,console] --------------------------------- PUT my_index { @@ -72,7 +72,7 @@ PUT my_index } } --------------------------------- -// CONSOLE + <1> Use the `my_field` field for searches. <2> Use the `my_field.keyword` field for aggregations, sorting, or in scripts. @@ -82,7 +82,7 @@ PUT my_index You can enable fielddata on an existing `text` field using the <> as follows: -[source,js] +[source,console] ----------------------------------- PUT my_index/_mapping { @@ -94,7 +94,6 @@ PUT my_index/_mapping } } ----------------------------------- -// CONSOLE // TEST[continued] <1> The mapping that you specify for `my_field` should consist of the existing @@ -116,7 +115,7 @@ value for the field, as opposed to all docs in the segment. Small segments can be excluded completely by specifying the minimum number of docs that the segment should contain with `min_segment_size`: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -135,4 +134,3 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/mapping/params/format.asciidoc b/docs/reference/mapping/params/format.asciidoc index 8e79a217a1a4..3a7cf5409887 100644 --- a/docs/reference/mapping/params/format.asciidoc +++ b/docs/reference/mapping/params/format.asciidoc @@ -9,7 +9,7 @@ Besides the <>, your own <> can be specified using the familiar `yyyy/MM/dd` syntax: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -23,7 +23,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE Many APIs which support date values also support <> expressions, such as `now-1m/d` -- the current time, minus one month, rounded diff --git a/docs/reference/mapping/params/ignore-above.asciidoc b/docs/reference/mapping/params/ignore-above.asciidoc index 33c0eaf339f2..d4f534b8ab60 100644 --- a/docs/reference/mapping/params/ignore-above.asciidoc +++ b/docs/reference/mapping/params/ignore-above.asciidoc @@ -6,7 +6,7 @@ For arrays of strings, `ignore_above` will be applied for each array element sep NOTE: All strings/array elements will still be present in the `_source` field, if the latter is enabled which is the default in Elasticsearch. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -41,7 +41,7 @@ GET my_index/_search <4> } } -------------------------------------------------- -// CONSOLE + <1> This field will ignore any string longer than 20 characters. <2> This document is indexed successfully. <3> This document will be indexed, but without indexing the `message` field. diff --git a/docs/reference/mapping/params/ignore-malformed.asciidoc b/docs/reference/mapping/params/ignore-malformed.asciidoc index d84a7290eb7c..656b679ffe9e 100644 --- a/docs/reference/mapping/params/ignore-malformed.asciidoc +++ b/docs/reference/mapping/params/ignore-malformed.asciidoc @@ -12,7 +12,7 @@ indexed, but other fields in the document are processed normally. For example: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -41,8 +41,8 @@ PUT my_index/_doc/2 "number_two": "foo" <2> } -------------------------------------------------- -// CONSOLE // TEST[catch:bad_request] + <1> This document will have the `text` field indexed, but not the `number_one` field. <2> This document will be rejected because `number_two` does not allow malformed values. @@ -56,7 +56,7 @@ existing fields using the <>. The `index.mapping.ignore_malformed` setting can be set on the index level to allow to ignore malformed content globally across all mapping types. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -76,7 +76,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE <1> The `number_one` field inherits the index-level setting. <2> The `number_two` field overrides the index-level setting to turn off `ignore_malformed`. diff --git a/docs/reference/mapping/params/index-options.asciidoc b/docs/reference/mapping/params/index-options.asciidoc index 527050a87b29..26a59baa847e 100644 --- a/docs/reference/mapping/params/index-options.asciidoc +++ b/docs/reference/mapping/params/index-options.asciidoc @@ -33,7 +33,7 @@ NOTE: <> don't support the `index_options` parameter any <> string fields use `positions` as the default, and all other fields use `docs` as the default. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -66,5 +66,5 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The `text` field will use the postings for the highlighting by default because `offsets` are indexed. diff --git a/docs/reference/mapping/params/index-prefixes.asciidoc b/docs/reference/mapping/params/index-prefixes.asciidoc index 841ccdde0877..74a06798c49f 100644 --- a/docs/reference/mapping/params/index-prefixes.asciidoc +++ b/docs/reference/mapping/params/index-prefixes.asciidoc @@ -17,7 +17,7 @@ up prefix searches. It accepts the following optional settings: This example creates a text field using the default prefix length settings: -[source,js] +[source,console] -------------------------------- PUT my_index { @@ -31,14 +31,13 @@ PUT my_index } } -------------------------------- -// CONSOLE <1> An empty settings object will use the default `min_chars` and `max_chars` settings This example uses custom prefix length settings: -[source,js] +[source,console] -------------------------------- PUT my_index { @@ -55,4 +54,3 @@ PUT my_index } } -------------------------------- -// CONSOLE \ No newline at end of file diff --git a/docs/reference/mapping/params/multi-fields.asciidoc b/docs/reference/mapping/params/multi-fields.asciidoc index 448f7fd2e81e..7dd30932d60f 100644 --- a/docs/reference/mapping/params/multi-fields.asciidoc +++ b/docs/reference/mapping/params/multi-fields.asciidoc @@ -6,7 +6,7 @@ purposes. This is the purpose of _multi-fields_. For instance, a `string` field could be mapped as a `text` field for full-text search, and as a `keyword` field for sorting or aggregations: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -53,7 +53,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The `city.raw` field is a `keyword` version of the `city` field. <2> The `city` field can be used for full text search. <3> The `city.raw` field can be used for sorting and aggregations @@ -71,7 +71,7 @@ ways for better relevance. For instance we could index a field with the words, and again with the <> which stems words into their root form: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -110,7 +110,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE <1> The `text` field uses the `standard` analyzer. <2> The `text.english` field uses the `english` analyzer. diff --git a/docs/reference/mapping/params/normalizer.asciidoc b/docs/reference/mapping/params/normalizer.asciidoc index 0f8c09552f4c..288faa931b62 100644 --- a/docs/reference/mapping/params/normalizer.asciidoc +++ b/docs/reference/mapping/params/normalizer.asciidoc @@ -10,7 +10,7 @@ search-time when the `keyword` field is searched via a query parser such as the <> query or via a term-level query such as the <> query. -[source,js] +[source,console] -------------------------------- PUT index { @@ -70,7 +70,6 @@ GET index/_search } } -------------------------------- -// CONSOLE The above queries match documents 1 and 2 since `BÀR` is converted to `bar` at both index and query time. @@ -120,7 +119,7 @@ both index and query time. Also, the fact that keywords are converted prior to indexing also means that aggregations return normalized values: -[source,js] +[source,console] ---------------------------- GET index/_search { @@ -134,7 +133,6 @@ GET index/_search } } ---------------------------- -// CONSOLE // TEST[continued] returns diff --git a/docs/reference/mapping/params/norms.asciidoc b/docs/reference/mapping/params/norms.asciidoc index 6a250d296a2d..38f7e4d448c8 100644 --- a/docs/reference/mapping/params/norms.asciidoc +++ b/docs/reference/mapping/params/norms.asciidoc @@ -17,7 +17,7 @@ the <>. Norms can be disabled (but not reenabled after the fact), using the <> like so: -[source,js] +[source,console] ------------ PUT my_index/_mapping { @@ -29,7 +29,6 @@ PUT my_index/_mapping } } ------------ -// CONSOLE // TEST[s/^/PUT my_index\n/] NOTE: Norms will not be removed instantly, but will be removed as old segments diff --git a/docs/reference/mapping/params/null-value.asciidoc b/docs/reference/mapping/params/null-value.asciidoc index 0a618ddcac9b..46a13ec1ef2c 100644 --- a/docs/reference/mapping/params/null-value.asciidoc +++ b/docs/reference/mapping/params/null-value.asciidoc @@ -8,7 +8,7 @@ field has no values. The `null_value` parameter allows you to replace explicit `null` values with the specified value so that it can be indexed and searched. For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -41,7 +41,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> Replace explicit `null` values with the term `NULL`. <2> An empty array does not contain an explicit `null`, and so won't be replaced with the `null_value`. <3> A query for `NULL` returns document 1, but not document 2. diff --git a/docs/reference/mapping/params/position-increment-gap.asciidoc b/docs/reference/mapping/params/position-increment-gap.asciidoc index 853f98ade75b..57ecacd034a3 100644 --- a/docs/reference/mapping/params/position-increment-gap.asciidoc +++ b/docs/reference/mapping/params/position-increment-gap.asciidoc @@ -11,7 +11,7 @@ size of this gap is configured using `position_increment_gap` and defaults to For example: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1 { @@ -41,7 +41,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> This phrase query doesn't match our document which is totally expected. <2> This phrase query matches our document, even though `Abraham` and `Lincoln` are in separate strings, because `slop` > `position_increment_gap`. @@ -49,7 +49,7 @@ GET my_index/_search The `position_increment_gap` can be specified in the mapping. For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -77,7 +77,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The first term in the next array element will be 0 terms apart from the last term in the previous array element. <2> The phrase query matches our document which is weird, but its what we asked diff --git a/docs/reference/mapping/params/properties.asciidoc b/docs/reference/mapping/params/properties.asciidoc index 9837dd3d5d9f..44084839ddd0 100644 --- a/docs/reference/mapping/params/properties.asciidoc +++ b/docs/reference/mapping/params/properties.asciidoc @@ -13,7 +13,7 @@ be added: Below is an example of adding `properties` to a mapping type, an `object` field, and a `nested` field: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -55,7 +55,7 @@ PUT my_index/_doc/1 <4> ] } -------------------------------------------------- -// CONSOLE + <1> Properties in the top-level mappings definition. <2> Properties under the `manager` object field. <3> Properties under the `employees` nested field. @@ -70,7 +70,7 @@ fields using the <>. Inner fields can be referred to in queries, aggregations, etc., using _dot notation_: -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -96,7 +96,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] IMPORTANT: The full path to the inner field must be specified. diff --git a/docs/reference/mapping/params/search-analyzer.asciidoc b/docs/reference/mapping/params/search-analyzer.asciidoc index 9b142f58c596..887ce56534ae 100644 --- a/docs/reference/mapping/params/search-analyzer.asciidoc +++ b/docs/reference/mapping/params/search-analyzer.asciidoc @@ -12,7 +12,7 @@ tokenizer for autocomplete. By default, queries will use the `analyzer` defined in the field mapping, but this can be overridden with the `search_analyzer` setting: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -66,7 +66,6 @@ GET my_index/_search } -------------------------------------------------- -// CONSOLE <1> Analysis settings to define the custom `autocomplete` analyzer. <2> The `text` field uses the `autocomplete` analyzer at index time, but the `standard` analyzer at search time. diff --git a/docs/reference/mapping/params/similarity.asciidoc b/docs/reference/mapping/params/similarity.asciidoc index 8f1ce02fff7a..bd9e355dc2f6 100644 --- a/docs/reference/mapping/params/similarity.asciidoc +++ b/docs/reference/mapping/params/similarity.asciidoc @@ -29,7 +29,7 @@ configuration are: The `similarity` can be set on the field level when a field is first created, as follows: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -46,6 +46,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE + <1> The `default_field` uses the `BM25` similarity. <2> The `boolean_sim_field` uses the `boolean` similarity. diff --git a/docs/reference/mapping/params/store.asciidoc b/docs/reference/mapping/params/store.asciidoc index 7316499b6a42..5bde4530cac2 100644 --- a/docs/reference/mapping/params/store.asciidoc +++ b/docs/reference/mapping/params/store.asciidoc @@ -16,7 +16,7 @@ you have a document with a `title`, a `date`, and a very large `content` field, you may want to retrieve just the `title` and the `date` without having to extract those fields from a large `_source` field: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -49,7 +49,7 @@ GET my_index/_search "stored_fields": [ "title", "date" ] <2> } -------------------------------------------------- -// CONSOLE + <1> The `title` and `date` fields are stored. <2> This request will retrieve the values of the `title` and `date` fields. diff --git a/docs/reference/mapping/params/term-vector.asciidoc b/docs/reference/mapping/params/term-vector.asciidoc index 7a97955c1734..928d64ad1c34 100644 --- a/docs/reference/mapping/params/term-vector.asciidoc +++ b/docs/reference/mapping/params/term-vector.asciidoc @@ -31,7 +31,7 @@ The fast vector highlighter requires `with_positions_offsets`. WARNING: Setting `with_positions_offsets` will double the size of a field's index. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -64,7 +64,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The fast vector highlighter will be used by default for the `text` field because term vectors are enabled. diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index e6362ea76811..36287da44e98 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -454,7 +454,7 @@ warnings in 6.8, the parameter can be set to either `true` or `false`. In 7.0, s See some examples of interactions with Elasticsearch with this option set to `false`: -[source,js] +[source,console] -------------------------------------------------- PUT index?include_type_name=false { @@ -467,10 +467,10 @@ PUT index?include_type_name=false } } -------------------------------------------------- -// CONSOLE + <1> Mappings are included directly under the `mappings` key, without a type name. -[source,js] +[source,console] -------------------------------------------------- PUT index/_mappings?include_type_name=false { @@ -481,15 +481,14 @@ PUT index/_mappings?include_type_name=false } } -------------------------------------------------- -// CONSOLE // TEST[continued] + <1> Mappings are included directly under the `mappings` key, without a type name. -[source,js] +[source,console] -------------------------------------------------- GET index/_mappings?include_type_name=false -------------------------------------------------- -// CONSOLE // TEST[continued] The above call returns @@ -520,14 +519,13 @@ The above call returns In 7.0, index APIs must be called with the `{index}/_doc` path for automatic generation of the `_id` and `{index}/_doc/{id}` with explicit ids. -[source,js] +[source,console] -------------------------------------------------- PUT index/_doc/1 { "foo": "baz" } -------------------------------------------------- -// CONSOLE [source,console-result] -------------------------------------------------- @@ -549,11 +547,10 @@ PUT index/_doc/1 Similarly, the `get` and `delete` APIs use the path `{index}/_doc/{id}`: -[source,js] +[source,console] -------------------------------------------------- GET index/_doc/1 -------------------------------------------------- -// CONSOLE // TEST[continued] NOTE: In 7.0, `_doc` represents the endpoint name instead of the document type. @@ -563,7 +560,7 @@ The `_doc` component is a permanent part of the path for the document `index`, For API paths that contain both a type and endpoint name like `_update`, in 7.0 the endpoint will immediately follow the index name: -[source,js] +[source,console] -------------------------------------------------- POST index/_update/1 { @@ -574,14 +571,13 @@ POST index/_update/1 GET /index/_source/1 -------------------------------------------------- -// CONSOLE // TEST[continued] Types should also no longer appear in the body of requests. The following example of bulk indexing omits the type both in the URL, and in the individual bulk commands: -[source,js] +[source,console] -------------------------------------------------- POST _bulk { "index" : { "_index" : "index", "_id" : "3" } } @@ -589,7 +585,6 @@ POST _bulk { "index" : { "_index" : "index", "_id" : "4" } } { "foo" : "qux" } -------------------------------------------------- -// CONSOLE [float] ==== Search APIs @@ -612,7 +607,7 @@ in the response. For example, the following typeless `get` call will always return `_doc` as the type, even if the mapping has a custom type name like `my_type`: -[source,js] +[source,console] -------------------------------------------------- PUT index/my_type/1 { @@ -621,7 +616,6 @@ PUT index/my_type/1 GET index/_doc/1 -------------------------------------------------- -// CONSOLE [source,console-result] -------------------------------------------------- @@ -655,7 +649,7 @@ will be typeless in spite of the fact that it matches a template that defines a type. Both `index-1-01` and `index-2-01` will inherit the `foo` field from the template that they match. -[source,js] +[source,console] -------------------------------------------------- PUT _template/template1 { @@ -707,7 +701,6 @@ PUT index-2-01 } } -------------------------------------------------- -// CONSOLE In case of implicit index creation, because of documents that get indexed in an index that doesn't exist yet, the template is always honored. This is diff --git a/docs/reference/mapping/types/alias.asciidoc b/docs/reference/mapping/types/alias.asciidoc index 89229ce0bb9e..2e789bfa75a7 100644 --- a/docs/reference/mapping/types/alias.asciidoc +++ b/docs/reference/mapping/types/alias.asciidoc @@ -8,7 +8,7 @@ An `alias` mapping defines an alternate name for a field in the index. The alias can be used in place of the target field in <> requests, and selected other APIs like <>. -[source,js] +[source,console] -------------------------------- PUT trips { @@ -39,7 +39,6 @@ GET _search } } -------------------------------- -// CONSOLE <1> The path to the target field. Note that this must be the full path, including any parent objects (e.g. `object1.object2.field`). @@ -52,11 +51,10 @@ field values. Please see the section on <> f In some parts of the search request and when requesting field capabilities, field wildcard patterns can be provided. In these cases, the wildcard pattern will match field aliases in addition to concrete fields: -[source,js] +[source,console] -------------------------------- GET trips/_field_caps?fields=route_*,transit_mode -------------------------------- -// CONSOLE // TEST[continued] [[alias-targets]] @@ -84,7 +82,7 @@ will result in a failure. Likewise, aliases cannot be used as the target of `cop Because alias names are not present in the document source, aliases cannot be used when performing source filtering. For example, the following request will return an empty result for `_source`: -[source,js] +[source,console] -------------------------------- GET /_search { @@ -94,7 +92,6 @@ GET /_search "_source": "route_length_miles" } -------------------------------- -// CONSOLE // TEST[continued] Currently only the search and field capabilities APIs will accept and resolve field aliases. diff --git a/docs/reference/mapping/types/array.asciidoc b/docs/reference/mapping/types/array.asciidoc index c7e3e273619b..822076b05657 100644 --- a/docs/reference/mapping/types/array.asciidoc +++ b/docs/reference/mapping/types/array.asciidoc @@ -38,7 +38,7 @@ Nothing needs to be pre-configured in order to use arrays in documents, they are supported out of the box: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1 { @@ -75,7 +75,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The `tags` field is dynamically added as a `string` field. <2> The `lists` field is dynamically added as an `object` field. <3> The second document contains no arrays, but can be indexed into the same fields. diff --git a/docs/reference/mapping/types/binary.asciidoc b/docs/reference/mapping/types/binary.asciidoc index 41478d1965a6..535438d8bdd8 100644 --- a/docs/reference/mapping/types/binary.asciidoc +++ b/docs/reference/mapping/types/binary.asciidoc @@ -8,7 +8,7 @@ The `binary` type accepts a binary value as a https://en.wikipedia.org/wiki/Base64[Base64] encoded string. The field is not stored by default and is not searchable: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -30,7 +30,7 @@ PUT my_index/_doc/1 "blob": "U29tZSBiaW5hcnkgYmxvYg==" <1> } -------------------------------------------------- -// CONSOLE + <1> The Base64 encoded binary value must not have embedded newlines `\n`. [[binary-params]] diff --git a/docs/reference/mapping/types/boolean.asciidoc b/docs/reference/mapping/types/boolean.asciidoc index 790b5013d1af..116459d0660e 100644 --- a/docs/reference/mapping/types/boolean.asciidoc +++ b/docs/reference/mapping/types/boolean.asciidoc @@ -18,7 +18,7 @@ True values:: For example: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -45,7 +45,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> Indexing a document with `"true"`, which is interpreted as `true`. <2> Searching for documents with a JSON `true`. @@ -54,7 +54,7 @@ aggregation>> use `1` and `0` for the `key`, and the strings `"true"` and `"false"` for the `key_as_string`. Boolean fields when used in scripts, return `1` and `0`: -[source,js] +[source,console] -------------------------------------------------- POST my_index/_doc/1 { @@ -85,7 +85,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE [[boolean-params]] ==== Parameters for `boolean` fields diff --git a/docs/reference/mapping/types/date.asciidoc b/docs/reference/mapping/types/date.asciidoc index 3a5c48058461..43ede27831b6 100644 --- a/docs/reference/mapping/types/date.asciidoc +++ b/docs/reference/mapping/types/date.asciidoc @@ -31,7 +31,7 @@ or milliseconds-since-the-epoch. For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -58,7 +58,7 @@ GET my_index/_search "sort": { "date": "asc"} <5> } -------------------------------------------------- -// CONSOLE + <1> The `date` field uses the default `format`. <2> This document uses a plain date. <3> This document includes a time. @@ -73,7 +73,7 @@ Each format will be tried in turn until a matching format is found. The first format will be used to convert the _milliseconds-since-the-epoch_ value back into a string. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -87,7 +87,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE [[date-params]] ==== Parameters for `date` fields diff --git a/docs/reference/mapping/types/date_nanos.asciidoc b/docs/reference/mapping/types/date_nanos.asciidoc index 5fb84068bd23..cf7082571ec3 100644 --- a/docs/reference/mapping/types/date_nanos.asciidoc +++ b/docs/reference/mapping/types/date_nanos.asciidoc @@ -28,7 +28,7 @@ nano second part). For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index?include_type_name=true { @@ -79,7 +79,6 @@ GET my_index/_search ] } -------------------------------------------------- -// CONSOLE <1> The `date` field uses the default `format`. <2> This document uses a plain date. diff --git a/docs/reference/mapping/types/dense-vector.asciidoc b/docs/reference/mapping/types/dense-vector.asciidoc index a1799fae7188..98834d012346 100644 --- a/docs/reference/mapping/types/dense-vector.asciidoc +++ b/docs/reference/mapping/types/dense-vector.asciidoc @@ -18,7 +18,7 @@ a given query vector and the indexed document vector. You index a dense vector as an array of floats. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -48,7 +48,6 @@ PUT my_index/_doc/2 } -------------------------------------------------- -// CONSOLE <1> dims—the number of dimensions in the vector, required parameter. diff --git a/docs/reference/mapping/types/flattened.asciidoc b/docs/reference/mapping/types/flattened.asciidoc index 317fd54c1a09..7181d53c2ce0 100644 --- a/docs/reference/mapping/types/flattened.asciidoc +++ b/docs/reference/mapping/types/flattened.asciidoc @@ -33,7 +33,8 @@ search functionality. The default approach, where each subfield has its own entry in the mappings, works well in the majority of cases. An flattened object field can be created as follows: -[source,js] + +[source,console] -------------------------------- PUT bug_reports { @@ -62,7 +63,6 @@ POST bug_reports/_doc/1 } } -------------------------------- -// CONSOLE // TESTSETUP During indexing, tokens are created for each leaf value in the JSON object. The @@ -72,7 +72,7 @@ numbers or dates. Querying the top-level `flattened` field searches all leaf values in the object: -[source,js] +[source,console] -------------------------------- POST bug_reports/_search { @@ -81,10 +81,10 @@ POST bug_reports/_search } } -------------------------------- -// CONSOLE To query on a specific key in the flattened object, object dot notation is used: -[source,js] + +[source,console] -------------------------------- POST bug_reports/_search { @@ -93,7 +93,6 @@ POST bug_reports/_search } } -------------------------------- -// CONSOLE [[supported-operations]] ==== Supported operations diff --git a/docs/reference/mapping/types/geo-point.asciidoc b/docs/reference/mapping/types/geo-point.asciidoc index 4c21bebc8fe4..2dd90a6ae61a 100644 --- a/docs/reference/mapping/types/geo-point.asciidoc +++ b/docs/reference/mapping/types/geo-point.asciidoc @@ -16,7 +16,7 @@ Fields of type `geo_point` accept latitude-longitude pairs, which can be used: There are five ways that a geo-point may be specified, as demonstrated below: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -80,7 +80,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> Geo-point expressed as an object, with `lat` and `lon` keys. <2> Geo-point expressed as a string with the format: `"lat,lon"`. <3> Geo-point expressed as a geohash. diff --git a/docs/reference/mapping/types/geo-shape.asciidoc b/docs/reference/mapping/types/geo-shape.asciidoc index 814fa89fbab6..b39bf90609ae 100644 --- a/docs/reference/mapping/types/geo-shape.asciidoc +++ b/docs/reference/mapping/types/geo-shape.asciidoc @@ -215,7 +215,7 @@ the cell right next to it -- even though the shape is very close to the point. [float] ===== Example -[source,js] +[source,console] -------------------------------------------------- PUT /example { @@ -228,7 +228,6 @@ PUT /example } } -------------------------------------------------- -// CONSOLE // TESTSETUP This mapping definition maps the location field to the geo_shape @@ -304,7 +303,7 @@ A point is a single geographic coordinate, such as the location of a building or the current position given by a smartphone's Geolocation API. The following is an example of a point in GeoJSON. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -314,18 +313,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a point in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "POINT (-77.03653 38.897676)" } -------------------------------------------------- -// CONSOLE [float] [[geo-linestring]] @@ -336,7 +333,7 @@ specifying only two points, the `linestring` will represent a straight line. Specifying more than two points creates an arbitrary path. The following is an example of a LineString in GeoJSON. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -346,18 +343,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a LineString in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "LINESTRING (-77.03653 38.897676, -77.009051 38.889939)" } -------------------------------------------------- -// CONSOLE The above `linestring` would draw a straight line starting at the White House to the US Capitol Building. @@ -370,7 +365,7 @@ A polygon is defined by a list of a list of points. The first and last points in each (outer) list must be the same (the polygon must be closed). The following is an example of a Polygon in GeoJSON. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -382,24 +377,22 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a Polygon in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "POLYGON ((100.0 0.0, 101.0 0.0, 101.0 1.0, 100.0 1.0, 100.0 0.0))" } -------------------------------------------------- -// CONSOLE The first array represents the outer boundary of the polygon, the other arrays represent the interior shapes ("holes"). The following is a GeoJSON example of a polygon with a hole: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -412,18 +405,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a Polygon with a hole in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "POLYGON ((100.0 0.0, 101.0 0.0, 101.0 1.0, 100.0 1.0, 100.0 0.0), (100.2 0.2, 100.8 0.2, 100.8 0.8, 100.2 0.8, 100.2 0.2))" } -------------------------------------------------- -// CONSOLE *IMPORTANT NOTE:* WKT does not enforce a specific order for vertices thus ambiguous polygons around the dateline and poles are possible. @@ -444,7 +435,7 @@ The following provides an example of an ambiguous polygon. Elasticsearch will apply the GeoJSON standard to eliminate ambiguity resulting in a polygon that crosses the dateline. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -457,14 +448,13 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE // TEST[catch:/mapper_parsing_exception/] An `orientation` parameter can be defined when setting the geo_shape mapping (see <>). This will define vertex order for the coordinate list on the mapped geo_shape field. It can also be overridden on each document. The following is an example for overriding the orientation on a document: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -477,7 +467,6 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE [float] [[geo-multipoint]] @@ -485,7 +474,7 @@ POST /example/_doc The following is an example of a list of geojson points: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -497,18 +486,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a list of WKT points: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "MULTIPOINT (102.0 2.0, 103.0 2.0)" } -------------------------------------------------- -// CONSOLE [float] [[geo-multilinestring]] @@ -516,7 +503,7 @@ POST /example/_doc The following is an example of a list of geojson linestrings: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -530,18 +517,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a list of WKT linestrings: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "MULTILINESTRING ((102.0 2.0, 103.0 2.0, 103.0 3.0, 102.0 3.0), (100.0 0.0, 101.0 0.0, 101.0 1.0, 100.0 1.0), (100.2 0.2, 100.8 0.2, 100.8 0.8, 100.2 0.8))" } -------------------------------------------------- -// CONSOLE [float] [[geo-multipolygon]] @@ -549,7 +534,7 @@ POST /example/_doc The following is an example of a list of geojson polygons (second polygon contains a hole): -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -563,18 +548,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a list of WKT polygons (second polygon contains a hole): -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "MULTIPOLYGON (((102.0 2.0, 103.0 2.0, 103.0 3.0, 102.0 3.0, 102.0 2.0)), ((100.0 0.0, 101.0 0.0, 101.0 1.0, 100.0 1.0, 100.0 0.0), (100.2 0.2, 100.8 0.2, 100.8 0.8, 100.2 0.8, 100.2 0.2)))" } -------------------------------------------------- -// CONSOLE [float] [[geo-geometry_collection]] @@ -582,7 +565,7 @@ POST /example/_doc The following is an example of a collection of geojson geometry objects: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -601,18 +584,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a collection of WKT geometry objects: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "GEOMETRYCOLLECTION (POINT (100.0 0.0), LINESTRING (101.0 0.0, 102.0 1.0))" } -------------------------------------------------- -// CONSOLE [float] @@ -622,7 +603,7 @@ Elasticsearch supports an `envelope` type, which consists of coordinates for upper left and lower right points of the shape to represent a bounding rectangle in the format `[[minLon, maxLat], [maxLon, minLat]]`: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -632,20 +613,18 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of an envelope using the WKT BBOX format: *NOTE:* WKT specification expects the following order: minLon, maxLon, maxLat, minLat. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "BBOX (100.0, 102.0, 2.0, 0.0)" } -------------------------------------------------- -// CONSOLE [float] ===== Circle @@ -656,7 +635,7 @@ be indexed when using the `recursive` Prefix Tree strategy. For the default <> circles should be approximated using a `POLYGON`. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -667,7 +646,6 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE // TEST[skip:not supported in default] Note: The inner `radius` field is required. If not specified, then diff --git a/docs/reference/mapping/types/ip.asciidoc b/docs/reference/mapping/types/ip.asciidoc index a1a56cf69fd0..41a889915fb7 100644 --- a/docs/reference/mapping/types/ip.asciidoc +++ b/docs/reference/mapping/types/ip.asciidoc @@ -7,7 +7,7 @@ An `ip` field can index/store either https://en.wikipedia.org/wiki/IPv4[IPv4] or https://en.wikipedia.org/wiki/IPv6[IPv6] addresses. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -34,7 +34,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE // TESTSETUP NOTE: You can also store ip ranges in a single field using an <>. @@ -79,7 +78,7 @@ The most common way to query ip addresses is to use the https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing#CIDR_notation[CIDR] notation: `[ip_address]/[prefix_length]`. For instance: -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -90,11 +89,10 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE or -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -105,14 +103,13 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE Also beware that colons are special characters to the <> query, so ipv6 addresses will need to be escaped. The easiest way to do so is to put quotes around the searched value: -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -123,4 +120,3 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE diff --git a/docs/reference/mapping/types/keyword.asciidoc b/docs/reference/mapping/types/keyword.asciidoc index 61a603b4f2d1..e1051754c208 100644 --- a/docs/reference/mapping/types/keyword.asciidoc +++ b/docs/reference/mapping/types/keyword.asciidoc @@ -16,7 +16,7 @@ descriptions, it is likely that you should rather use a <> field. Below is an example of a mapping for a keyword field: -[source,js] +[source,console] -------------------------------- PUT my_index { @@ -29,7 +29,6 @@ PUT my_index } } -------------------------------- -// CONSOLE [[keyword-params]] ==== Parameters for keyword fields diff --git a/docs/reference/mapping/types/nested.asciidoc b/docs/reference/mapping/types/nested.asciidoc index 3113e86599c9..98f10bfe20dd 100644 --- a/docs/reference/mapping/types/nested.asciidoc +++ b/docs/reference/mapping/types/nested.asciidoc @@ -15,7 +15,7 @@ Lucene has no concept of inner objects, so Elasticsearch flattens object hierarchies into a simple list of field names and values. For instance, the following document: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1 { @@ -32,7 +32,7 @@ PUT my_index/_doc/1 ] } -------------------------------------------------- -// CONSOLE + <1> The `user` field is dynamically added as a field of type `object`. would be transformed internally into a document that looks more like this: @@ -51,7 +51,7 @@ The `user.first` and `user.last` fields are flattened into multi-value fields, and the association between `alice` and `white` is lost. This document would incorrectly match a query for `alice AND smith`: -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -65,7 +65,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] [[nested-fields-array-objects]] @@ -77,7 +76,7 @@ each object in the array, you should use the `nested` datatype instead of the the array as a separate hidden document, meaning that each nested object can be queried independently of the others, with the <>: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -146,7 +145,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The `user` field is mapped as type `nested` instead of type `object`. <2> This query doesn't match because `Alice` and `Smith` are not in the same nested object. <3> This query matches because `Alice` and `White` are in the same nested object. diff --git a/docs/reference/mapping/types/numeric.asciidoc b/docs/reference/mapping/types/numeric.asciidoc index 7298b54873b9..72bc672b8c67 100644 --- a/docs/reference/mapping/types/numeric.asciidoc +++ b/docs/reference/mapping/types/numeric.asciidoc @@ -18,7 +18,7 @@ The following numeric types are supported: Below is an example of configuring a mapping with numeric fields: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -38,7 +38,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE NOTE: The `double`, `float` and `half_float` types consider that `-0.0` and `+0.0` are different values. As a consequence, doing a `term` query on diff --git a/docs/reference/mapping/types/object.asciidoc b/docs/reference/mapping/types/object.asciidoc index e127415c6181..c865e8df6dd0 100644 --- a/docs/reference/mapping/types/object.asciidoc +++ b/docs/reference/mapping/types/object.asciidoc @@ -7,7 +7,7 @@ JSON documents are hierarchical in nature: the document may contain inner objects which, in turn, may contain inner objects themselves: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1 { <1> @@ -21,7 +21,7 @@ PUT my_index/_doc/1 } } -------------------------------------------------- -// CONSOLE + <1> The outer document is also a JSON object. <2> It contains an inner object called `manager`. <3> Which in turn contains an inner object called `name`. @@ -42,7 +42,7 @@ pairs, something like this: An explicit mapping for the above document could look like this: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -66,7 +66,7 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE + <1> Properties in the top-level mappings definition. <2> The `manager` field is an inner `object` field. <3> The `manager.name` field is an inner `object` field within the `manager` field. diff --git a/docs/reference/mapping/types/parent-join.asciidoc b/docs/reference/mapping/types/parent-join.asciidoc index 14c7b7b27589..94909417315d 100644 --- a/docs/reference/mapping/types/parent-join.asciidoc +++ b/docs/reference/mapping/types/parent-join.asciidoc @@ -10,7 +10,7 @@ The `relations` section defines a set of possible relations within the documents each relation being a parent name and a child name. A parent/child relation can be defined as follows: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -26,7 +26,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE <1> The name for the field <2> Defines a single relation where `question` is parent of `answer`. @@ -35,7 +34,7 @@ To index a document with a join, the name of the relation and the optional paren of the document must be provided in the `source`. For instance the following example creates two `parent` documents in the `question` context: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1?refresh { @@ -53,7 +52,6 @@ PUT my_index/_doc/2?refresh } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> This document is a `question` document. @@ -61,7 +59,7 @@ PUT my_index/_doc/2?refresh When indexing parent documents, you can choose to specify just the name of the relation as a shortcut instead of encapsulating it in the normal object notation: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1?refresh { @@ -75,7 +73,6 @@ PUT my_index/_doc/2?refresh "my_join_field": "question" } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> Simpler notation for a parent document just uses the relation name. @@ -88,7 +85,7 @@ always route child documents using their greater parent id. For instance the following example shows how to index two `child` documents: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/3?routing=1&refresh <1> { @@ -108,7 +105,6 @@ PUT my_index/_doc/4?routing=1&refresh } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The routing value is mandatory because parent and child documents must be indexed on the same shard @@ -156,7 +152,7 @@ document if it's a parent (`my_parent`). When searching an index that contains a `join` field, these two fields are always returned in the search response: -[source,js] +[source,console] -------------------------- GET my_index/_search { @@ -166,7 +162,6 @@ GET my_index/_search "sort": ["_id"] } -------------------------- -// CONSOLE // TEST[continued] Will return: @@ -264,7 +259,7 @@ The value of the `join` field is accessible in aggregations and scripts, and may be queried with the <>: -[source,js] +[source,console] -------------------------- GET my_index/_search { @@ -291,7 +286,6 @@ GET my_index/_search } } -------------------------- -// CONSOLE // TEST[continued] <1> Querying the `parent id` field (also see the <> and the <>) @@ -318,7 +312,7 @@ are occurring. When the `join` field is used infrequently and writes occur frequently it may make sense to disable eager loading: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -335,12 +329,11 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE The amount of heap used by global ordinals can be checked per parent relation as follows: -[source,sh] +[source,console] -------------------------------------------------- # Per-index GET _stats/fielddata?human&fields=my_join_field#question @@ -348,14 +341,13 @@ GET _stats/fielddata?human&fields=my_join_field#question # Per-node per-index GET _nodes/stats/indices/fielddata?human&fields=my_join_field#question -------------------------------------------------- -// CONSOLE // TEST[continued] ==== Multiple children per parent It is also possible to define multiple children for a single parent: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -371,7 +363,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE <1> `question` is parent of `answer` and `comment`. @@ -383,7 +374,7 @@ You should de-normalize your data if you care about performance. Multiple levels of parent/child: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -400,7 +391,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE <1> `question` is parent of `answer` and `comment` <2> `answer` is parent of `vote` @@ -419,7 +409,7 @@ Indexing a grandchild document requires a `routing` value equals to the grand-parent (the greater parent of the lineage): -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/3?routing=1&refresh <1> { @@ -430,7 +420,6 @@ PUT my_index/_doc/3?routing=1&refresh <1> } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> This child document must be on the same shard than its grand-parent and parent diff --git a/docs/reference/mapping/types/percolator.asciidoc b/docs/reference/mapping/types/percolator.asciidoc index 4c75f9c3068a..51498e34555d 100644 --- a/docs/reference/mapping/types/percolator.asciidoc +++ b/docs/reference/mapping/types/percolator.asciidoc @@ -16,7 +16,7 @@ query. If the following mapping configures the `percolator` field type for the `query` field: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -32,12 +32,11 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE // TESTSETUP Then you can index a query: -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/match_value { @@ -48,7 +47,6 @@ PUT my_index/_doc/match_value } } -------------------------------------------------- -// CONSOLE [IMPORTANT] ===================================== @@ -68,7 +66,7 @@ new releases. Reindexing percolator queries can be reindexed by using the <>. Lets take a look at the following index with a percolator field type: -[source,js] +[source,console] -------------------------------------------------- PUT index { @@ -105,7 +103,6 @@ PUT queries/_doc/1?refresh } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> It is always recommended to define an alias for your index, so that in case of a reindex systems / applications @@ -114,7 +111,7 @@ PUT queries/_doc/1?refresh Lets say you're going to upgrade to a new major version and in order for the new Elasticsearch version to still be able to read your queries you need to reindex your queries into a new index on the current Elasticsearch version: -[source,js] +[source,console] -------------------------------------------------- PUT new_index { @@ -158,14 +155,13 @@ POST _aliases ] } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> If you have an alias don't forget to point it to the new index. Executing the `percolate` query via the `queries` alias: -[source,js] +[source,console] -------------------------------------------------- GET /queries/_search { @@ -179,7 +175,6 @@ GET /queries/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] now returns matches from the new index: @@ -264,7 +259,7 @@ Lets say we want to index the following percolator query: with these settings and mapping: -[source,js] +[source,console] -------------------------------------------------- PUT /test_index { @@ -291,14 +286,13 @@ PUT /test_index } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> For the purpose of this example, this analyzer is considered expensive. First we need to use the analyze api to perform the text analysis prior to indexing: -[source,js] +[source,console] -------------------------------------------------- POST /test_index/_analyze { @@ -306,7 +300,6 @@ POST /test_index/_analyze "text" : "missing bicycles" } -------------------------------------------------- -// CONSOLE // TEST[continued] This results the following response: @@ -335,7 +328,7 @@ This results the following response: All the tokens in the returned order need to replace the query text in the percolator query: -[source,js] +[source,console] -------------------------------------------------- PUT /test_index/_doc/1?refresh { @@ -349,7 +342,6 @@ PUT /test_index/_doc/1?refresh } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> It is important to select a whitespace analyzer here, otherwise the analyzer defined in the mapping will be used, @@ -360,7 +352,7 @@ The analyze api prior to the indexing the percolator flow should be done for eac At percolate time nothing changes and the `percolate` query can be defined normally: -[source,js] +[source,console] -------------------------------------------------- GET /test_index/_search { @@ -374,7 +366,6 @@ GET /test_index/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] This results in a response like this: @@ -434,7 +425,7 @@ query on a field where the `edge_ngram` token filter is configured. Creating an index with custom analysis settings: -[source,js] +[source,console] -------------------------------------------------- PUT my_queries1 { @@ -478,7 +469,6 @@ PUT my_queries1 } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The analyzer that generates the prefix tokens to be used at index time only. @@ -503,7 +493,7 @@ Then instead of indexing the following query: this query below should be indexed: -[source,js] +[source,console] -------------------------------------------------- PUT /my_queries1/_doc/1?refresh { @@ -514,7 +504,6 @@ PUT /my_queries1/_doc/1?refresh } } -------------------------------------------------- -// CONSOLE // TEST[continued] This way can handle the second query more efficiently than the first query. @@ -522,7 +511,7 @@ This way can handle the second query more efficiently than the first query. The following search request will match with the previously indexed percolator query: -[source,js] +[source,console] -------------------------------------------------- GET /my_queries1/_search { @@ -536,7 +525,6 @@ GET /my_queries1/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] @@ -585,7 +573,7 @@ The same technique can also be used to speed up suffix wildcard searches. By using the `reverse` token filter before the `edge_ngram` token filter. -[source,js] +[source,console] -------------------------------------------------- PUT my_queries2 { @@ -638,7 +626,6 @@ PUT my_queries2 } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> A custom analyzer is needed at search time too, because otherwise @@ -661,7 +648,7 @@ Then instead of indexing the following query: the following query below should be indexed: -[source,js] +[source,console] -------------------------------------------------- PUT /my_queries2/_doc/2?refresh { @@ -672,7 +659,6 @@ PUT /my_queries2/_doc/2?refresh } } -------------------------------------------------- -// CONSOLE // TEST[continued] <1> The `match` query should be used instead of the `term` query, @@ -681,7 +667,7 @@ PUT /my_queries2/_doc/2?refresh The following search request will match with the previously indexed percolator query: -[source,js] +[source,console] -------------------------------------------------- GET /my_queries2/_search { @@ -695,7 +681,6 @@ GET /my_queries2/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] [float] diff --git a/docs/reference/mapping/types/range.asciidoc b/docs/reference/mapping/types/range.asciidoc index 91bbbd0d6d04..cd6bfac03fd6 100644 --- a/docs/reference/mapping/types/range.asciidoc +++ b/docs/reference/mapping/types/range.asciidoc @@ -59,7 +59,7 @@ PUT range_index/_doc/1?refresh The following is an example of a <> on the `integer_range` field named "expected_attendees". -[source,js] +[source,console] -------------------------------------------------- GET range_index/_search { @@ -72,7 +72,6 @@ GET range_index/_search } } -------------------------------------------------- -// CONSOLE The result produced by the above query. @@ -116,7 +115,7 @@ The result produced by the above query. The following is an example of a `date_range` query over the `date_range` field named "time_frame". -[source,js] +[source,console] -------------------------------------------------- GET range_index/_search { @@ -131,7 +130,6 @@ GET range_index/_search } } -------------------------------------------------- -// CONSOLE <1> Range queries work the same as described in <>. <2> Range queries over range <> support a `relation` parameter which can be one of `WITHIN`, `CONTAINS`, @@ -183,7 +181,7 @@ This query produces a similar result: In addition to the range format above, IP ranges can be provided in https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing#CIDR_notation[CIDR] notation: -[source,js] +[source,console] -------------------------------------------------- PUT range_index/_mapping { @@ -199,7 +197,6 @@ PUT range_index/_doc/2 "ip_whitelist" : "192.168.0.0/16" } -------------------------------------------------- -// CONSOLE [[range-params]] ==== Parameters for range fields diff --git a/docs/reference/mapping/types/rank-feature.asciidoc b/docs/reference/mapping/types/rank-feature.asciidoc index d066d0452d35..c7cc176e997a 100644 --- a/docs/reference/mapping/types/rank-feature.asciidoc +++ b/docs/reference/mapping/types/rank-feature.asciidoc @@ -7,7 +7,7 @@ A `rank_feature` field can index numbers so that they can later be used to boost documents in queries with a <> query. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -39,7 +39,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> Rank feature fields must use the `rank_feature` field type <2> Rank features that correlate negatively with the score need to declare it diff --git a/docs/reference/mapping/types/rank-features.asciidoc b/docs/reference/mapping/types/rank-features.asciidoc index b80db43651dd..38d7dc8297e6 100644 --- a/docs/reference/mapping/types/rank-features.asciidoc +++ b/docs/reference/mapping/types/rank-features.asciidoc @@ -12,7 +12,7 @@ It is analogous to the <> datatype but is better su when the list of features is sparse so that it wouldn't be reasonable to add one field to the mappings for each of them. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -50,7 +50,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> Rank features fields must use the `rank_features` field type <2> Rank features fields must be a hash with string keys and strictly positive numeric values diff --git a/docs/reference/mapping/types/search-as-you-type.asciidoc b/docs/reference/mapping/types/search-as-you-type.asciidoc index 21a1a46cf8c0..cc26f3666fda 100644 --- a/docs/reference/mapping/types/search-as-you-type.asciidoc +++ b/docs/reference/mapping/types/search-as-you-type.asciidoc @@ -14,7 +14,7 @@ position within the input) are supported. When adding a field of this type to a mapping -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -27,7 +27,6 @@ PUT my_index } } -------------------------------------------------- -// CONSOLE This creates the following fields @@ -68,14 +67,13 @@ The same input text is indexed into each of these fields automatically, with their differing analysis chains, when an indexed document has a value for the root field `my_field`. -[source,js] +[source,console] -------------------------------------------------- PUT my_index/_doc/1?refresh { "my_field": "quick brown fox jump lazy dog" } -------------------------------------------------- -// CONSOLE // TEST[continued] The most efficient way of querying to serve a search-as-you-type use case is @@ -85,7 +83,7 @@ usually a <> query of type terms in any order, but will score documents higher if they contain the terms in order in a shingle subfield. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -102,7 +100,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] [source,js] @@ -147,7 +144,7 @@ field. A <> can also be used if the last term should be matched exactly, and not as a prefix. Using phrase queries may be less efficient than using the `match_bool_prefix` query. -[source,js] +[source,console] -------------------------------------------------- GET my_index/_search { @@ -158,7 +155,6 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE // TEST[continued] [[specific-params]] diff --git a/docs/reference/mapping/types/shape.asciidoc b/docs/reference/mapping/types/shape.asciidoc index 7580f756bc16..9874eb6cba52 100644 --- a/docs/reference/mapping/types/shape.asciidoc +++ b/docs/reference/mapping/types/shape.asciidoc @@ -80,7 +80,7 @@ The following features are not yet supported: [float] ===== Example -[source,js] +[source,console] -------------------------------------------------- PUT /example { @@ -93,7 +93,6 @@ PUT /example } } -------------------------------------------------- -// CONSOLE // TESTSETUP This mapping definition maps the geometry field to the shape type. The indexer uses single @@ -145,7 +144,7 @@ A point is a single coordinate in cartesian `x, y` space. It may represent the location of an item of interest in a virtual world or projected space. The following is an example of a point in GeoJSON. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -155,18 +154,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a point in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "POINT (-377.03653 389.897676)" } -------------------------------------------------- -// CONSOLE [float] [[linestring]] @@ -177,7 +174,7 @@ specifying only two points, the `linestring` will represent a straight line. Specifying more than two points creates an arbitrary path. The following is an example of a LineString in GeoJSON. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -187,18 +184,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a LineString in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "LINESTRING (-377.03653 389.897676, -377.009051 389.889939)" } -------------------------------------------------- -// CONSOLE [float] [[polygon]] @@ -208,7 +203,7 @@ A polygon is defined by a list of a list of points. The first and last points in each (outer) list must be the same (the polygon must be closed). The following is an example of a Polygon in GeoJSON. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -220,24 +215,22 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a Polygon in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "POLYGON ((1000.0 -1001.0, 1001.0 -1001.0, 1001.0 -1000.0, 1000.0 -1000.0, 1000.0 -1001.0))" } -------------------------------------------------- -// CONSOLE The first array represents the outer boundary of the polygon, the other arrays represent the interior shapes ("holes"). The following is a GeoJSON example of a polygon with a hole: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -250,18 +243,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a Polygon with a hole in WKT: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "POLYGON ((1000.0 1000.0, 1001.0 1000.0, 1001.0 1001.0, 1000.0 1001.0, 1000.0 1000.0), (1000.2 1000.2, 1000.8 1000.2, 1000.8 1000.8, 1000.2 1000.8, 1000.2 1000.2))" } -------------------------------------------------- -// CONSOLE *IMPORTANT NOTE:* WKT does not enforce a specific order for vertices. https://tools.ietf.org/html/rfc7946#section-3.1.6[GeoJSON] mandates that the @@ -277,7 +268,7 @@ with the document. The following is an example of overriding the `orientation` parameters on a document: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -290,7 +281,6 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE [float] [[multipoint]] @@ -298,7 +288,7 @@ POST /example/_doc The following is an example of a list of geojson points: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -310,18 +300,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a list of WKT points: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "MULTIPOINT (1002.0 2000.0, 1003.0 2000.0)" } -------------------------------------------------- -// CONSOLE [float] [[multilinestring]] @@ -329,7 +317,7 @@ POST /example/_doc The following is an example of a list of geojson linestrings: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -343,18 +331,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a list of WKT linestrings: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "MULTILINESTRING ((1002.0 200.0, 1003.0 200.0, 1003.0 300.0, 1002.0 300.0), (1000.0 100.0, 1001.0 100.0, 1001.0 100.0, 1000.0 100.0), (1000.2 0.2, 1000.8 100.2, 1000.8 100.8, 1000.2 100.8))" } -------------------------------------------------- -// CONSOLE [float] [[multipolygon]] @@ -362,7 +348,7 @@ POST /example/_doc The following is an example of a list of geojson polygons (second polygon contains a hole): -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -376,18 +362,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a list of WKT polygons (second polygon contains a hole): -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "MULTIPOLYGON (((1002.0 200.0, 1003.0 200.0, 1003.0 300.0, 1002.0 300.0, 102.0 200.0)), ((1000.0 100.0, 1001.0 100.0, 1001.0 100.0, 1000.0 100.0, 1000.0 100.0), (1000.2 100.2, 1000.8 100.2, 1000.8 100.8, 1000.2 100.8, 1000.2 100.2)))" } -------------------------------------------------- -// CONSOLE [float] [[geometry_collection]] @@ -395,7 +379,7 @@ POST /example/_doc The following is an example of a collection of geojson geometry objects: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -414,18 +398,16 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of a collection of WKT geometry objects: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "GEOMETRYCOLLECTION (POINT (1000.0 100.0), LINESTRING (1001.0 100.0, 1002.0 100.0))" } -------------------------------------------------- -// CONSOLE [float] ===== Envelope @@ -434,7 +416,7 @@ Elasticsearch supports an `envelope` type, which consists of coordinates for upper left and lower right points of the shape to represent a bounding rectangle in the format `[[minX, maxY], [maxX, minY]]`: -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { @@ -444,20 +426,18 @@ POST /example/_doc } } -------------------------------------------------- -// CONSOLE The following is an example of an envelope using the WKT BBOX format: *NOTE:* WKT specification expects the following order: minLon, maxLon, maxLat, minLat. -[source,js] +[source,console] -------------------------------------------------- POST /example/_doc { "location" : "BBOX (1000.0, 1002.0, 2000.0, 1000.0)" } -------------------------------------------------- -// CONSOLE [float] ==== Sorting and Retrieving index Shapes diff --git a/docs/reference/mapping/types/sparse-vector.asciidoc b/docs/reference/mapping/types/sparse-vector.asciidoc index af3b6a510377..340786f3d986 100644 --- a/docs/reference/mapping/types/sparse-vector.asciidoc +++ b/docs/reference/mapping/types/sparse-vector.asciidoc @@ -23,7 +23,7 @@ are dimensions, and fields values are values for these dimensions. Dimensions are integer values from `0` to `65535` encoded as strings. Dimensions don't need to be in order. -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -52,7 +52,6 @@ PUT my_index/_doc/2 } -------------------------------------------------- -// CONSOLE Internally, each document's sparse vector is encoded as a binary doc value. Its size in bytes is equal to diff --git a/docs/reference/mapping/types/text.asciidoc b/docs/reference/mapping/types/text.asciidoc index 46477972d5cf..434d91fd4ec6 100644 --- a/docs/reference/mapping/types/text.asciidoc +++ b/docs/reference/mapping/types/text.asciidoc @@ -18,7 +18,7 @@ codes, or tags, it is likely that you should rather use a <> Below is an example of a mapping for a text field: -[source,js] +[source,console] -------------------------------- PUT my_index { @@ -31,7 +31,6 @@ PUT my_index } } -------------------------------- -// CONSOLE [[text-multi-fields]] ==== Use a field as both text and keyword diff --git a/docs/reference/mapping/types/token-count.asciidoc b/docs/reference/mapping/types/token-count.asciidoc index a435be1e54d5..69dfdbb31cd5 100644 --- a/docs/reference/mapping/types/token-count.asciidoc +++ b/docs/reference/mapping/types/token-count.asciidoc @@ -10,7 +10,7 @@ string. For instance: -[source,js] +[source,console] -------------------------------------------------- PUT my_index { @@ -44,7 +44,7 @@ GET my_index/_search } } -------------------------------------------------- -// CONSOLE + <1> The `name` field is an analyzed string field which uses the default `standard` analyzer. <2> The `name.length` field is a `token_count` <> which will index the number of tokens in the `name` field. <3> This query matches only the document containing `Rachel Alice Williams`, as it contains three tokens. diff --git a/docs/reference/migration/apis/deprecation.asciidoc b/docs/reference/migration/apis/deprecation.asciidoc index 5714d5b9a5f4..2673ce64d6a0 100644 --- a/docs/reference/migration/apis/deprecation.asciidoc +++ b/docs/reference/migration/apis/deprecation.asciidoc @@ -36,11 +36,10 @@ be removed or changed in the next major version. To see the list of offenders in your cluster, submit a GET request to the `_migration/deprecations` endpoint: -[source,js] +[source,console] -------------------------------------------------- GET /_migration/deprecations -------------------------------------------------- -// CONSOLE // TEST[skip:cannot assert tests have certain deprecations] Example response: @@ -115,9 +114,8 @@ warnings can be found under the `ml_settings` key. The following example request shows only index-level deprecations of all `logstash-*` indices: -[source,js] +[source,console] -------------------------------------------------- GET /logstash-*/_migration/deprecations -------------------------------------------------- -// CONSOLE // TEST[skip:cannot assert tests have certain deprecations] diff --git a/docs/reference/migration/migrate_8_0/snapshots.asciidoc b/docs/reference/migration/migrate_8_0/snapshots.asciidoc index 6bf8eba824d5..6a5b31961609 100644 --- a/docs/reference/migration/migrate_8_0/snapshots.asciidoc +++ b/docs/reference/migration/migrate_8_0/snapshots.asciidoc @@ -16,11 +16,10 @@ and now contains separate response for each repository. For example, requesting one snapshot from particular repository -[source,js] +[source,console] ----------------------------------- GET _snapshot/repo1/snap1 ----------------------------------- -// CONSOLE // TEST[skip:no repo and snapshots are created] produces the following response diff --git a/docs/reference/ml/anomaly-detection/aggregations.asciidoc b/docs/reference/ml/anomaly-detection/aggregations.asciidoc index fecf99df53e5..29e307fcbb80 100644 --- a/docs/reference/ml/anomaly-detection/aggregations.asciidoc +++ b/docs/reference/ml/anomaly-detection/aggregations.asciidoc @@ -22,7 +22,7 @@ you might want to just use the default search and scroll behavior. When you create or update an {anomaly-job}, you can include the names of aggregations, for example: -[source,js] +[source,console] ---------------------------------- PUT _ml/anomaly_detectors/farequote { @@ -40,7 +40,6 @@ PUT _ml/anomaly_detectors/farequote } } ---------------------------------- -// CONSOLE // TEST[skip:setup:farequote_data] In this example, the `airline`, `responsetime`, and `time` fields are @@ -53,7 +52,7 @@ aggregated. It applies to all detectors in the job. The aggregations are defined in the {dfeed} as follows: -[source,js] +[source,console] ---------------------------------- PUT _ml/datafeeds/datafeed-farequote { @@ -88,7 +87,6 @@ PUT _ml/datafeeds/datafeed-farequote } } ---------------------------------- -// CONSOLE // TEST[skip:setup:farequote_job] In this example, the aggregations have names that match the fields that they diff --git a/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc index 81afe7d81e75..48feb40f416e 100644 --- a/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/close-job.asciidoc @@ -98,11 +98,10 @@ when there are no matches or only partial matches. The following example closes the `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- POST _ml/anomaly_detectors/total-requests/_close -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_openjob] When the job is closed, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-calendar-event.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-calendar-event.asciidoc index 07e4906445b4..7f7978043a91 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-calendar-event.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-calendar-event.asciidoc @@ -43,11 +43,10 @@ events and delete the calendar, see the The following example deletes a scheduled event from the `planned-outages` calendar: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/calendars/planned-outages/events/LS8LJGEBMTCMA-qz49st -------------------------------------------------- -// CONSOLE // TEST[skip:catch:missing] When the event is removed, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc index 09df1f55a160..19e6c11fd483 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-calendar-job.asciidoc @@ -36,11 +36,10 @@ Deletes {anomaly-jobs} from a calendar. The following example removes the association between the `planned-outages` calendar and `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/calendars/planned-outages/jobs/total-requests -------------------------------------------------- -// CONSOLE // TEST[skip:setup:calendar_outages_addjob] When the job is removed from the calendar, you receive the following diff --git a/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc index b9b1e24181e3..48aac40b4320 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-calendar.asciidoc @@ -37,11 +37,10 @@ calendar. The following example deletes the `planned-outages` calendar: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/calendars/planned-outages -------------------------------------------------- -// CONSOLE // TEST[skip:setup:calendar_outages] When the calendar is deleted, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc index d08b763836f3..1509b98f6365 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-datafeed.asciidoc @@ -42,11 +42,10 @@ can delete it. The following example deletes the `datafeed-total-requests` {dfeed}: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/datafeeds/datafeed-total-requests -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_datafeed] When the {dfeed} is deleted, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc index 2a2008d6a690..c3375f968f4f 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-expired-data.asciidoc @@ -32,11 +32,10 @@ associated with any job are also deleted. The endpoint takes no arguments: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/_delete_expired_data -------------------------------------------------- -// CONSOLE // TEST When the expired data is deleted, you receive the following response: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc index 4943c96d45b2..0b9de2b698e1 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-filter.asciidoc @@ -38,11 +38,10 @@ update or delete the job before you can delete the filter. The following example deletes the `safe_domains` filter: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/filters/safe_domains -------------------------------------------------- -// CONSOLE // TEST[skip:setup:ml_filter_safe_domains] When the filter is deleted, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc index b8634591fca3..293fe0bf95db 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc @@ -67,11 +67,10 @@ For more information, see The following example deletes all forecasts from the `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/anomaly_detectors/total-requests/_forecast/_all -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_openjob] If the request does not encounter errors, you receive the following result: diff --git a/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc index 82d12ef83a3c..a258c50978fb 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc @@ -59,11 +59,10 @@ separated list. The following example deletes the `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/anomaly_detectors/total-requests -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_job] When the job is deleted, you receive the following results: @@ -77,11 +76,10 @@ When the job is deleted, you receive the following results: In the next example we delete the `total-requests` job asynchronously: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/anomaly_detectors/total-requests?wait_for_completion=false -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_job] When `wait_for_completion` is set to `false`, the response contains the id diff --git a/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc index 58beffdd9761..1fa01e789fac 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-snapshot.asciidoc @@ -41,11 +41,10 @@ the `model_snapshot_id` in the results from the get jobs API. The following example deletes the `1491948163` snapshot: -[source,js] +[source,console] -------------------------------------------------- DELETE _ml/anomaly_detectors/farequote/model_snapshots/1491948163 -------------------------------------------------- -// CONSOLE // TEST[skip:todo] When the snapshot is deleted, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc b/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc index 961927ed81e6..33dfd12e6fcc 100644 --- a/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc @@ -222,7 +222,7 @@ size, which defaults to 100 Mb. Suppose you have a newline-delimited JSON file that contains information about some books. You can send the contents to the `find_file_structure` endpoint: -[source,js] +[source,console] ---- POST _ml/find_file_structure {"name": "Leviathan Wakes", "author": "James S.A. Corey", "release_date": "2011-06-02", "page_count": 561} @@ -250,8 +250,6 @@ POST _ml/find_file_structure {"name": "The Left Hand of Darkness", "author": "Ursula K. Le Guin", "release_date": "1969-06-01", "page_count": 304} {"name": "The Moon is a Harsh Mistress", "author": "Robert A. Heinlein", "release_date": "1966-04-01", "page_count": 288} ---- -// CONSOLE -// TEST If the request does not encounter errors, you receive the following result: diff --git a/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc index 7d73cb96ad86..b681e165165e 100644 --- a/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc @@ -68,14 +68,13 @@ opened again before analyzing further data. The following example flushes the `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- POST _ml/anomaly_detectors/total-requests/_flush { "calc_interim": true } -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_openjob] When the operation succeeds, you receive the following results: @@ -95,14 +94,13 @@ If you want to flush the job to a specific timestamp, you can use the `advance_time` or `skip_time` parameters. For example, to advance to 11 AM GMT on January 1, 2018: -[source,js] +[source,console] -------------------------------------------------- POST _ml/anomaly_detectors/total-requests/_flush { "advance_time": "1514804400" } -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_openjob] When the operation succeeds, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/forecast.asciidoc b/docs/reference/ml/anomaly-detection/apis/forecast.asciidoc index e98a55305a53..ee53f3876d6a 100644 --- a/docs/reference/ml/anomaly-detection/apis/forecast.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/forecast.asciidoc @@ -58,14 +58,13 @@ forecast. For more information about this property, see <>. The following example requests a 10 day forecast for the `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- POST _ml/anomaly_detectors/total-requests/_forecast { "duration": "10d" } -------------------------------------------------- -// CONSOLE // TEST[skip:requires delay] When the forecast is created, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-bucket.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-bucket.asciidoc index b30e4e815598..d9db7484a6de 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-bucket.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-bucket.asciidoc @@ -89,7 +89,7 @@ The API returns the following information: The following example gets bucket information for the `it-ops-kpi` job: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/it-ops-kpi/results/buckets { @@ -97,7 +97,6 @@ GET _ml/anomaly_detectors/it-ops-kpi/results/buckets "start": "1454530200001" } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] In this example, the API returns a single result that matches the specified diff --git a/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc index 8ea7ad2af7ad..6059cde7a73a 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc @@ -86,11 +86,10 @@ The API returns the following information: The following example gets information about the scheduled events in the `planned-outages` calendar: -[source,js] +[source,console] -------------------------------------------------- GET _ml/calendars/planned-outages/events -------------------------------------------------- -// CONSOLE // TEST[skip:setup:calendar_outages_addevent] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc index 88b48752b68f..d30852bc8730 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-calendar.asciidoc @@ -68,11 +68,10 @@ The API returns the following information: The following example gets configuration information for the `planned-outages` calendar: -[source,js] +[source,console] -------------------------------------------------- GET _ml/calendars/planned-outages -------------------------------------------------- -// CONSOLE // TEST[skip:setup:calendar_outages_addjob] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-category.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-category.asciidoc index febc855abafc..6ce22a6f34c9 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-category.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-category.asciidoc @@ -66,7 +66,7 @@ The API returns the following information: The following example gets information about one category for the `esxi_log` job: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/esxi_log/results/categories { @@ -75,7 +75,6 @@ GET _ml/anomaly_detectors/esxi_log/results/categories } } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] In this example, the API returns the following information: diff --git a/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc index 42db449e377e..fad72220b6c7 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc @@ -89,11 +89,10 @@ The API returns the following information: The following example gets usage information for the `datafeed-total-requests` {dfeed}: -[source,js] +[source,console] -------------------------------------------------- GET _ml/datafeeds/datafeed-total-requests/_stats -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_startdf] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc index 917599c4b941..d5e632deffb9 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc @@ -85,11 +85,10 @@ The API returns the following information: The following example gets configuration information for the `datafeed-total-requests` {dfeed}: -[source,js] +[source,console] -------------------------------------------------- GET _ml/datafeeds/datafeed-total-requests -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_datafeed] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc index b45ed11eb00b..7afe33910e8e 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-filter.asciidoc @@ -66,11 +66,10 @@ The API returns the following information: The following example gets configuration information for the `safe_domains` filter: -[source,js] +[source,console] -------------------------------------------------- GET _ml/filters/safe_domains -------------------------------------------------- -// CONSOLE // TEST[skip:setup:ml_filter_safe_domains] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-influencer.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-influencer.asciidoc index 9a77f95fbeec..da8d5df00c19 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-influencer.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-influencer.asciidoc @@ -73,7 +73,7 @@ The API returns the following information: The following example gets influencer information for the `it_ops_new_kpi` job: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/it_ops_new_kpi/results/influencers { @@ -81,7 +81,6 @@ GET _ml/anomaly_detectors/it_ops_new_kpi/results/influencers "desc": true } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] In this example, the API returns the following information, sorted based on the diff --git a/docs/reference/ml/anomaly-detection/apis/get-job-stats.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-job-stats.asciidoc index aa0d5f9f0f58..0944c0451710 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-job-stats.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-job-stats.asciidoc @@ -82,11 +82,10 @@ The API returns the following information: The following example gets usage information for the `farequote` job: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/farequote/_stats -------------------------------------------------- -// CONSOLE // TEST[skip:todo] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc index d1c45f44b084..54d3d7cf2620 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc @@ -82,11 +82,10 @@ The API returns the following information: The following example gets configuration information for the `total-requests` job: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/total-requests -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_job] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc index 2c486741ffd4..4f0179629539 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc @@ -37,11 +37,10 @@ what those defaults are. The endpoint takes no arguments: -[source,js] +[source,console] -------------------------------------------------- GET _ml/info -------------------------------------------------- -// CONSOLE // TEST This is a possible response: diff --git a/docs/reference/ml/anomaly-detection/apis/get-overall-buckets.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-overall-buckets.asciidoc index e42d0137f072..db33ef64394d 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-overall-buckets.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-overall-buckets.asciidoc @@ -107,7 +107,7 @@ The API returns the following information: The following example gets overall buckets for {anomaly-jobs} with IDs matching `job-*`: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/job-*/results/overall_buckets { @@ -115,7 +115,6 @@ GET _ml/anomaly_detectors/job-*/results/overall_buckets "start": "1403532000000" } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] In this example, the API returns a single result that matches the specified @@ -153,7 +152,7 @@ score and time constraints. The `overall_score` is the max job score as The next example is similar but this time `top_n` is set to `2`: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/job-*/results/overall_buckets { @@ -162,7 +161,6 @@ GET _ml/anomaly_detectors/job-*/results/overall_buckets "start": "1403532000000" } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] Note how the `overall_score` is now the average of the top 2 job scores: diff --git a/docs/reference/ml/anomaly-detection/apis/get-record.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-record.asciidoc index 8872d5f8ebc3..8bbf53f28bc5 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-record.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-record.asciidoc @@ -73,7 +73,7 @@ The API returns the following information: The following example gets record information for the `it-ops-kpi` job: -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/it-ops-kpi/results/records { @@ -82,7 +82,6 @@ GET _ml/anomaly_detectors/it-ops-kpi/results/records "start": "1454944100000" } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] In this example, the API returns twelve results for the specified diff --git a/docs/reference/ml/anomaly-detection/apis/get-snapshot.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-snapshot.asciidoc index ca1b234c6045..733de6bece5b 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-snapshot.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-snapshot.asciidoc @@ -66,14 +66,13 @@ The API returns the following information: [[ml-get-snapshot-example]] ==== {api-examples-title} -[source,js] +[source,console] -------------------------------------------------- GET _ml/anomaly_detectors/farequote/model_snapshots { "start": "1491852977000" } -------------------------------------------------- -// CONSOLE // TEST[skip:todo] In this example, the API provides a single result: diff --git a/docs/reference/ml/anomaly-detection/apis/jobresource.asciidoc b/docs/reference/ml/anomaly-detection/apis/jobresource.asciidoc index e6e243120cf1..623e5a74de2f 100644 --- a/docs/reference/ml/anomaly-detection/apis/jobresource.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/jobresource.asciidoc @@ -357,7 +357,7 @@ If it is an object it has the following properties: If you omit the `categorization_analyzer`, the following default values are used: -[source,js] +[source,console] -------------------------------------------------- POST _ml/anomaly_detectors/_validate { @@ -384,7 +384,6 @@ POST _ml/anomaly_detectors/_validate } } -------------------------------------------------- -// CONSOLE If you specify any part of the `categorization_analyzer`, however, any omitted sub-properties are _not_ set to default values. diff --git a/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc index d73d8f93d392..d1425d258339 100644 --- a/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/open-job.asciidoc @@ -52,14 +52,13 @@ data is received. The following example opens the `total-requests` job and sets an optional property: -[source,js] +[source,console] -------------------------------------------------- POST _ml/anomaly_detectors/total-requests/_open { "timeout": "35m" } -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_job] When the job opens, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc b/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc index 39b91a00ba84..b4502bcb72a7 100644 --- a/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/post-calendar-event.asciidoc @@ -60,7 +60,7 @@ of which must have a start time, end time, and description. You can add scheduled events to the `planned-outages` calendar as follows: -[source,js] +[source,console] -------------------------------------------------- POST _ml/calendars/planned-outages/events { @@ -71,7 +71,6 @@ POST _ml/calendars/planned-outages/events ] } -------------------------------------------------- -// CONSOLE // TEST[skip:setup:calendar_outages_addjob] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc index 7fe46d65f4df..e2aa8699b323 100644 --- a/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/preview-datafeed.asciidoc @@ -48,11 +48,10 @@ it to ensure it is returning the expected data. The following example obtains a preview of the `datafeed-farequote` {dfeed}: -[source,js] +[source,console] -------------------------------------------------- GET _ml/datafeeds/datafeed-farequote/_preview -------------------------------------------------- -// CONSOLE // TEST[skip:setup:farequote_datafeed] The data that is returned for this example is as follows: diff --git a/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc index d2ad602fcd71..f6eb36db5fba 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-calendar-job.asciidoc @@ -33,11 +33,10 @@ Adds an {anomaly-job} to a calendar. [[ml-put-calendar-job-example]] ==== {api-examples-title} -[source,js] +[source,console] -------------------------------------------------- PUT _ml/calendars/planned-outages/jobs/total-requests -------------------------------------------------- -// CONSOLE // TEST[skip:setup:calendar_outages_openjob] The API returns the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc index 4e9e0fd194c2..764e3e61060f 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-calendar.asciidoc @@ -41,11 +41,10 @@ For more information, see [[ml-put-calendar-example]] ==== {api-examples-title} -[source,js] +[source,console] -------------------------------------------------- PUT _ml/calendars/planned-outages -------------------------------------------------- -// CONSOLE // TEST[skip:need-license] When the calendar is created, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc index 85fb6d23e324..248c8bba7ce7 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc @@ -114,7 +114,7 @@ see <>. The following example creates the `datafeed-total-requests` {dfeed}: -[source,js] +[source,console] -------------------------------------------------- PUT _ml/datafeeds/datafeed-total-requests { @@ -122,7 +122,6 @@ PUT _ml/datafeeds/datafeed-total-requests "indices": ["server-metrics"] } -------------------------------------------------- -// CONSOLE // TEST[skip:setup:server_metrics_job] When the {dfeed} is created, you receive the following results: diff --git a/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc index 38fa9424a4a7..41b276010265 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-filter.asciidoc @@ -49,7 +49,7 @@ the `custom_rules` property of < Date: Fri, 6 Sep 2019 11:35:27 -0400 Subject: [PATCH 094/103] [DOCS] Add index alias definition to glossary (#46339) --- docs/reference/glossary.asciidoc | 14 ++++++++++++++ docs/reference/indices/add-alias.asciidoc | 2 +- docs/reference/indices/alias-exists.asciidoc | 9 +-------- docs/reference/indices/aliases.asciidoc | 2 +- docs/reference/indices/delete-alias.asciidoc | 2 +- docs/reference/indices/get-alias.asciidoc | 2 +- 6 files changed, 19 insertions(+), 12 deletions(-) diff --git a/docs/reference/glossary.asciidoc b/docs/reference/glossary.asciidoc index e59e57f3c5a5..e13b5812f7b6 100644 --- a/docs/reference/glossary.asciidoc +++ b/docs/reference/glossary.asciidoc @@ -96,6 +96,20 @@ An index is a logical namespace which maps to one or more <> and can have zero or more <>. +[[glossary-index-alias]] index alias :: ++ +-- +// tag::index-alias-def[] +An index alias is a secondary name +used to refer to one or more existing indices. + +Most {es} APIs accept an index alias +in place of an index name. +// end::index-alias-def[] + +See <>. +-- + [[glossary-leader-index]] leader index :: Leader indices are the source indices for <>. They exist diff --git a/docs/reference/indices/add-alias.asciidoc b/docs/reference/indices/add-alias.asciidoc index 66460ce71353..427ddf631cec 100644 --- a/docs/reference/indices/add-alias.asciidoc +++ b/docs/reference/indices/add-alias.asciidoc @@ -6,7 +6,7 @@ Creates or updates an index alias. -include::alias-exists.asciidoc[tag=index-alias-def] +include::{docdir}/glossary.asciidoc[tag=index-alias-def] [source,console] ---- diff --git a/docs/reference/indices/alias-exists.asciidoc b/docs/reference/indices/alias-exists.asciidoc index 2c56d5363d81..8caa3080feeb 100644 --- a/docs/reference/indices/alias-exists.asciidoc +++ b/docs/reference/indices/alias-exists.asciidoc @@ -6,14 +6,7 @@ Checks if an index alias exists. -//tag::index-alias-def[] -An index alias is a secondary name -used to refer to one or more existing indices. -//end::index-alias-def[] - -The returned HTTP status code indicates whether the index alias exists or not. -A `404` means it does not exist, -and `200` means it does. +include::{docdir}/glossary.asciidoc[tag=index-alias-def] [source,console] ---- diff --git a/docs/reference/indices/aliases.asciidoc b/docs/reference/indices/aliases.asciidoc index aff3e3b88fc1..991b65b0793a 100644 --- a/docs/reference/indices/aliases.asciidoc +++ b/docs/reference/indices/aliases.asciidoc @@ -6,7 +6,7 @@ Adds or removes index aliases. -include::alias-exists.asciidoc[tag=index-alias-def] +include::{docdir}/glossary.asciidoc[tag=index-alias-def] [source,console] ---- diff --git a/docs/reference/indices/delete-alias.asciidoc b/docs/reference/indices/delete-alias.asciidoc index 1df34a3753ec..4ab144a2fd63 100644 --- a/docs/reference/indices/delete-alias.asciidoc +++ b/docs/reference/indices/delete-alias.asciidoc @@ -6,7 +6,7 @@ Deletes an existing index alias. -include::alias-exists.asciidoc[tag=index-alias-def] +include::{docdir}/glossary.asciidoc[tag=index-alias-def] [source,console] ---- diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index 14cb3a612ca1..8b25e3c21aca 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -6,7 +6,7 @@ Returns information about one or more index aliases. -include::alias-exists.asciidoc[tag=index-alias-def] +include::{docdir}/glossary.asciidoc[tag=index-alias-def] [source,console] ---- From 8167000951d35a2361d0fd288a06827cea46cd4a Mon Sep 17 00:00:00 2001 From: Anton <33043735+siffash@users.noreply.github.com> Date: Fri, 6 Sep 2019 19:01:34 +0300 Subject: [PATCH 095/103] [Docs] Fix typo in field-names-field.asciidoc (#46430) --- docs/reference/mapping/fields/field-names-field.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/mapping/fields/field-names-field.asciidoc b/docs/reference/mapping/fields/field-names-field.asciidoc index 1fe1760f1335..7f6689dc43ad 100644 --- a/docs/reference/mapping/fields/field-names-field.asciidoc +++ b/docs/reference/mapping/fields/field-names-field.asciidoc @@ -18,7 +18,7 @@ Disabling `_field_names` is often not necessary because it no longer carries the index overhead it once did. If you have a lot of fields which have `doc_values` and `norms` disabled and you do not need to execute `exists` queries using those fields you might want to disable -`_field_names` be adding the following to the mappings: +`_field_names` by adding the following to the mappings: [source,console] -------------------------------------------------- From e43be90e6c2e859b820a6949fab217a2338e90ae Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 6 Sep 2019 14:05:36 -0400 Subject: [PATCH 096/103] [DOCS] [5 of 5] Change // TESTRESPONSE comments to [source,console-results] (#46449) --- .../painless-guide/painless-debugging.asciidoc | 4 ++-- docs/plugins/ingest-attachment.asciidoc | 9 +++++---- .../administering/backup-cluster-config.asciidoc | 4 +--- .../bucket/adjacency-matrix-aggregation.asciidoc | 2 +- .../bucket/autodatehistogram-aggregation.asciidoc | 6 +++--- .../bucket/children-aggregation.asciidoc | 2 +- .../bucket/composite-aggregation.asciidoc | 4 ++-- .../bucket/datehistogram-aggregation.asciidoc | 12 ++++++------ .../bucket/daterange-aggregation.asciidoc | 6 +++--- .../diversified-sampler-aggregation.asciidoc | 4 ++-- .../bucket/filter-aggregation.asciidoc | 2 +- .../bucket/filters-aggregation.asciidoc | 6 +++--- .../bucket/geodistance-aggregation.asciidoc | 6 +++--- .../bucket/geohashgrid-aggregation.asciidoc | 4 ++-- .../bucket/geotilegrid-aggregation.asciidoc | 4 ++-- .../bucket/global-aggregation.asciidoc | 2 +- .../bucket/histogram-aggregation.asciidoc | 6 +++--- .../bucket/iprange-aggregation.asciidoc | 8 ++++---- .../bucket/missing-aggregation.asciidoc | 2 +- .../bucket/nested-aggregation.asciidoc | 2 +- .../bucket/parent-aggregation.asciidoc | 2 +- .../aggregations/bucket/range-aggregation.asciidoc | 8 ++++---- .../bucket/rare-terms-aggregation.asciidoc | 4 ++-- .../bucket/reverse-nested-aggregation.asciidoc | 2 +- .../bucket/sampler-aggregation.asciidoc | 4 ++-- .../bucket/significantterms-aggregation.asciidoc | 2 +- .../bucket/significanttext-aggregation.asciidoc | 2 +- .../aggregations/bucket/terms-aggregation.asciidoc | 7 ++++--- .../aggregations/matrix/stats-aggregation.asciidoc | 2 +- .../aggregations/metrics/avg-aggregation.asciidoc | 2 +- .../metrics/cardinality-aggregation.asciidoc | 2 +- .../metrics/extendedstats-aggregation.asciidoc | 2 +- .../metrics/geobounds-aggregation.asciidoc | 2 +- .../metrics/geocentroid-aggregation.asciidoc | 4 ++-- .../aggregations/metrics/max-aggregation.asciidoc | 2 +- .../median-absolute-deviation-aggregation.asciidoc | 2 +- .../aggregations/metrics/min-aggregation.asciidoc | 2 +- .../metrics/percentile-aggregation.asciidoc | 4 ++-- .../metrics/percentile-rank-aggregation.asciidoc | 4 ++-- .../metrics/scripted-metric-aggregation.asciidoc | 4 ++-- .../metrics/stats-aggregation.asciidoc | 2 +- .../aggregations/metrics/sum-aggregation.asciidoc | 2 +- .../metrics/tophits-aggregation.asciidoc | 5 +++-- .../metrics/valuecount-aggregation.asciidoc | 2 +- .../metrics/weighted-avg-aggregation.asciidoc | 4 ++-- docs/reference/aggregations/misc.asciidoc | 4 ++-- .../pipeline/avg-bucket-aggregation.asciidoc | 2 +- .../pipeline/bucket-script-aggregation.asciidoc | 2 +- .../pipeline/bucket-selector-aggregation.asciidoc | 3 ++- .../pipeline/bucket-sort-aggregation.asciidoc | 4 ++-- .../cumulative-cardinality-aggregation.asciidoc | 4 ++-- .../pipeline/cumulative-sum-aggregation.asciidoc | 2 +- .../pipeline/derivative-aggregation.asciidoc | 7 ++++--- .../extended-stats-bucket-aggregation.asciidoc | 2 +- .../pipeline/max-bucket-aggregation.asciidoc | 2 +- .../pipeline/min-bucket-aggregation.asciidoc | 2 +- .../pipeline/movfn-aggregation.asciidoc | 2 +- .../percentiles-bucket-aggregation.asciidoc | 2 +- .../pipeline/stats-bucket-aggregation.asciidoc | 2 +- .../pipeline/sum-bucket-aggregation.asciidoc | 2 +- .../pattern-replace-charfilter.asciidoc | 3 ++- .../tokenizers/edgengram-tokenizer.asciidoc | 2 +- docs/reference/api-conventions.asciidoc | 8 ++++---- .../ccr/apis/follow/get-follow-stats.asciidoc | 2 +- .../ccr/apis/follow/post-forget-follower.asciidoc | 4 ++-- docs/reference/ccr/apis/get-ccr-stats.asciidoc | 2 +- docs/reference/cluster/allocation-explain.asciidoc | 2 +- docs/reference/cluster/health.asciidoc | 2 +- docs/reference/cluster/nodes-info.asciidoc | 4 ++-- .../cluster/nodes-reload-secure-settings.asciidoc | 2 +- docs/reference/cluster/nodes-usage.asciidoc | 2 +- docs/reference/cluster/update-settings.asciidoc | 4 ++-- .../data-frames/apis/update-transform.asciidoc | 3 ++- docs/reference/docs/bulk.asciidoc | 2 +- docs/reference/docs/concurrency-control.asciidoc | 4 ++-- docs/reference/docs/delete-by-query.asciidoc | 4 ++-- docs/reference/docs/delete.asciidoc | 2 +- docs/reference/docs/index_.asciidoc | 4 ++-- docs/reference/docs/multi-get.asciidoc | 2 +- docs/reference/docs/reindex.asciidoc | 6 +++--- docs/reference/docs/termvectors.asciidoc | 4 ++-- docs/reference/docs/update-by-query.asciidoc | 4 ++-- docs/reference/getting-started.asciidoc | 8 ++++---- docs/reference/how-to/recipes/stemming.asciidoc | 6 +++--- docs/reference/ilm/apis/slm-api.asciidoc | 12 +++++++----- docs/reference/ilm/getting-started-slm.asciidoc | 3 ++- .../reference/index-modules/index-sorting.asciidoc | 2 +- docs/reference/index-modules/similarity.asciidoc | 4 ++-- .../indices/apis/reload-analyzers.asciidoc | 2 +- docs/reference/indices/flush.asciidoc | 4 ++-- docs/reference/indices/recovery.asciidoc | 5 +++-- docs/reference/indices/shard-stores.asciidoc | 2 +- docs/reference/indices/templates.asciidoc | 2 +- docs/reference/ingest.asciidoc | 3 ++- .../ingest/apis/simulate-pipeline.asciidoc | 4 ++-- docs/reference/ingest/ingest-node.asciidoc | 6 +++--- docs/reference/ingest/processors/circle.asciidoc | 4 ++-- .../ingest/processors/date-index-name.asciidoc | 4 ++-- docs/reference/ingest/processors/geoip.asciidoc | 8 ++++---- docs/reference/ingest/processors/grok.asciidoc | 4 ++-- docs/reference/ingest/processors/pipeline.asciidoc | 2 +- docs/reference/ingest/processors/script.asciidoc | 2 +- docs/reference/ingest/processors/set.asciidoc | 3 ++- .../ingest/processors/user-agent.asciidoc | 2 +- docs/reference/licensing/get-basic-status.asciidoc | 3 ++- docs/reference/licensing/get-license.asciidoc | 2 +- docs/reference/licensing/get-trial-status.asciidoc | 3 ++- docs/reference/mapping/params/normalizer.asciidoc | 4 ++-- docs/reference/mapping/types/parent-join.asciidoc | 2 +- docs/reference/mapping/types/percolator.asciidoc | 6 +++--- docs/reference/mapping/types/range.asciidoc | 7 +++---- .../mapping/types/search-as-you-type.asciidoc | 2 +- .../apis/delete-forecast.asciidoc | 1 + .../ml/anomaly-detection/apis/delete-job.asciidoc | 3 ++- .../apis/find-file-structure.asciidoc | 2 +- .../ml/anomaly-detection/apis/flush-job.asciidoc | 3 ++- .../apis/get-calendar-event.asciidoc | 2 +- .../apis/get-datafeed-stats.asciidoc | 3 ++- .../anomaly-detection/apis/get-datafeed.asciidoc | 3 ++- .../ml/anomaly-detection/apis/get-job.asciidoc | 3 ++- .../ml/anomaly-detection/apis/get-ml-info.asciidoc | 3 ++- .../anomaly-detection/apis/put-datafeed.asciidoc | 3 ++- .../ml/anomaly-detection/apis/put-job.asciidoc | 3 ++- .../apis/update-datafeed.asciidoc | 2 +- .../ml/anomaly-detection/apis/update-job.asciidoc | 2 +- .../ml/anomaly-detection/categories.asciidoc | 12 ++++++------ .../ml/anomaly-detection/customurl.asciidoc | 3 +-- .../ml/anomaly-detection/populations.asciidoc | 4 ++-- .../ml/df-analytics/apis/put-dfanalytics.asciidoc | 2 +- .../modules/cross-cluster-search.asciidoc | 4 ++-- docs/reference/query-dsl/percolate-query.asciidoc | 10 +++++----- .../query-dsl/span-containing-query.asciidoc | 3 +-- docs/reference/query-dsl/term-query.asciidoc | 2 +- docs/reference/query-dsl/terms-query.asciidoc | 2 +- docs/reference/rest-api/info.asciidoc | 3 ++- docs/reference/rollup/apis/rollup-search.asciidoc | 8 ++++---- .../rollup/rollup-getting-started.asciidoc | 4 ++-- .../rollup/rollup-search-limitations.asciidoc | 2 +- docs/reference/search/count.asciidoc | 3 +-- docs/reference/search/profile.asciidoc | 14 +++++++------- docs/reference/search/request-body.asciidoc | 4 ++-- .../reference/search/request/highlighting.asciidoc | 4 ++-- docs/reference/search/request/inner-hits.asciidoc | 8 ++++---- .../search/request/track-total-hits.asciidoc | 6 +++--- docs/reference/search/search-shards.asciidoc | 4 ++-- docs/reference/search/suggesters.asciidoc | 2 +- .../search/suggesters/completion-suggest.asciidoc | 4 ++-- docs/reference/search/suggesters/misc.asciidoc | 2 +- .../search/suggesters/phrase-suggest.asciidoc | 2 +- docs/reference/search/uri-request.asciidoc | 2 +- docs/reference/search/validate.asciidoc | 2 +- docs/reference/setup/logging-config.asciidoc | 1 + docs/reference/sql/endpoints/rest.asciidoc | 8 ++++---- x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc | 8 ++++---- .../en/rest-api/watcher/activate-watch.asciidoc | 4 ++-- .../en/rest-api/watcher/deactivate-watch.asciidoc | 4 ++-- .../en/rest-api/watcher/execute-watch.asciidoc | 2 +- x-pack/docs/en/rest-api/watcher/get-watch.asciidoc | 3 ++- 158 files changed, 304 insertions(+), 283 deletions(-) diff --git a/docs/painless/painless-guide/painless-debugging.asciidoc b/docs/painless/painless-guide/painless-debugging.asciidoc index e335bfe74ebd..fe56cb25f1e5 100644 --- a/docs/painless/painless-guide/painless-debugging.asciidoc +++ b/docs/painless/painless-guide/painless-debugging.asciidoc @@ -38,7 +38,7 @@ POST /hockey/_explain/1 Which shows that the class of `doc.first` is `org.elasticsearch.index.fielddata.ScriptDocValues.Longs` by responding with: -[source,js] +[source,console-result] --------------------------------------------------------- { "error": { @@ -68,7 +68,7 @@ POST /hockey/_update/1 The response looks like: -[source,js] +[source,console-result] --------------------------------------------------------- { "error" : { diff --git a/docs/plugins/ingest-attachment.asciidoc b/docs/plugins/ingest-attachment.asciidoc index a3d716ff2d9e..d0c2eba0f65e 100644 --- a/docs/plugins/ingest-attachment.asciidoc +++ b/docs/plugins/ingest-attachment.asciidoc @@ -55,7 +55,7 @@ GET my_index/_doc/my_id Returns this: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -140,7 +140,7 @@ GET my_index/_doc/my_id Returns this: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -190,7 +190,7 @@ GET my_index/_doc/my_id_2 Returns this: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -284,7 +284,8 @@ GET my_index/_doc/my_id // CONSOLE Returns this: -[source,js] + +[source,console-result] -------------------------------------------------- { "_index" : "my_index", diff --git a/docs/reference/administering/backup-cluster-config.asciidoc b/docs/reference/administering/backup-cluster-config.asciidoc index 373ff48618de..109227e81547 100644 --- a/docs/reference/administering/backup-cluster-config.asciidoc +++ b/docs/reference/administering/backup-cluster-config.asciidoc @@ -37,12 +37,10 @@ parameter for the snapshot API. Alternatively, you can extract these configuration values in text format by using the <>: -[source,js] +[source,console] -------------------------------------------------- GET _cluster/settings?pretty&flat_settings&filter_path=persistent -------------------------------------------------- -//CONSOLE -//TEST You can store the output of this as a file together with the rest of configuration files. diff --git a/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc b/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc index 8ed900a24fa0..cd5cc3d20eb1 100644 --- a/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/adjacency-matrix-aggregation.asciidoc @@ -62,7 +62,7 @@ of groups that have recorded interactions. Response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 9, diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index 575a3e75894c..8da1000dc2d4 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -57,7 +57,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -150,7 +150,7 @@ GET my_index/_search?size=0 UTC is used if no time zone is specified, three 1-hour buckets are returned starting at midnight UTC on 1 October 2015: -[source,js] +[source,console-result] --------------------------------- { ... @@ -205,7 +205,7 @@ Now three 1-hour buckets are still returned but the first bucket starts at 11:00pm on 30 September 2015 since that is the local time for the bucket in the specified time zone. -[source,js] +[source,console-result] --------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/children-aggregation.asciidoc b/docs/reference/aggregations/bucket/children-aggregation.asciidoc index 67d75c39448b..778e3e6e3a61 100644 --- a/docs/reference/aggregations/bucket/children-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/children-aggregation.asciidoc @@ -125,7 +125,7 @@ The above example returns the top question tags and per tag the top answer owner Possible response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 25, diff --git a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc index 2c50c844940f..337cdf8a1967 100644 --- a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc @@ -422,7 +422,7 @@ GET /_search \... returns: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -520,7 +520,7 @@ GET /_search \... returns: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc index b5633cecd026..70ff27b184b3 100644 --- a/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/datehistogram-aggregation.asciidoc @@ -307,7 +307,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -377,7 +377,7 @@ If you don't specify a timezone, UTC is used. This would result in both of these documents being placed into the same day bucket, which starts at midnight UTC on 1 October 2015: -[source,js] +[source,console-result] --------------------------------- { ... @@ -419,7 +419,7 @@ GET my_index/_search?size=0 Now the first document falls into the bucket for 30 September 2015, while the second document falls into the bucket for 1 October 2015: -[source,js] +[source,console-result] --------------------------------- { ... @@ -496,7 +496,7 @@ GET my_index/_search?size=0 Instead of a single bucket starting at midnight, the above request groups the documents into buckets starting at 6am: -[source,js] +[source,console-result] ----------------------------- { ... @@ -548,7 +548,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -650,7 +650,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc b/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc index c116bfcc785a..9a729c90f269 100644 --- a/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc @@ -41,7 +41,7 @@ documents dated since 10 months ago Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -317,7 +317,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -366,7 +366,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc b/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc index 7c825eebc646..4221601434bf 100644 --- a/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc @@ -57,7 +57,7 @@ POST /stackoverflow/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -126,7 +126,7 @@ POST /stackoverflow/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/filter-aggregation.asciidoc b/docs/reference/aggregations/bucket/filter-aggregation.asciidoc index d65b1851a947..e76b1390f630 100644 --- a/docs/reference/aggregations/bucket/filter-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/filter-aggregation.asciidoc @@ -25,7 +25,7 @@ In the above example, we calculate the average price of all the products that ar Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc index 0e7b11a010c4..90716624052b 100644 --- a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc @@ -39,7 +39,7 @@ and another for all those containing a warning. Response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 9, @@ -91,7 +91,7 @@ GET logs/_search The filtered buckets are returned in the same order as provided in the request. The response for this example would be: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 4, @@ -158,7 +158,7 @@ GET logs/_search The response would be something like the following: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 3, diff --git a/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc b/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc index 82838808a3e0..7dc38353147b 100644 --- a/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc @@ -50,7 +50,7 @@ POST /museums/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -166,7 +166,7 @@ POST /museums/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -220,7 +220,7 @@ POST /museums/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc b/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc index 3c84f6549299..0b0b34c1a33b 100644 --- a/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc @@ -59,7 +59,7 @@ POST /museums/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -148,7 +148,7 @@ POST /museums/_search?size=0 -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc b/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc index 43ca6b0176b1..2d8041eff2a7 100644 --- a/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geotilegrid-aggregation.asciidoc @@ -72,7 +72,7 @@ POST /museums/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -133,7 +133,7 @@ POST /museums/_search?size=0 -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/global-aggregation.asciidoc b/docs/reference/aggregations/bucket/global-aggregation.asciidoc index 9a17b7755d67..1fc3fe5e6f02 100644 --- a/docs/reference/aggregations/bucket/global-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/global-aggregation.asciidoc @@ -41,7 +41,7 @@ all products in our catalog, not just on the "shirts"). The response for the above aggregation: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc index 33dcecb450ad..f7e340af29bc 100644 --- a/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/histogram-aggregation.asciidoc @@ -37,7 +37,7 @@ POST /sales/_search?size=0 And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -96,7 +96,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -215,7 +215,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc b/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc index 67578af3694d..ad48efde2f4f 100644 --- a/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc @@ -27,7 +27,7 @@ GET /ip_addresses/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -76,7 +76,7 @@ GET /ip_addresses/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -130,7 +130,7 @@ GET /ip_addresses/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -178,7 +178,7 @@ GET /ip_addresses/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/missing-aggregation.asciidoc b/docs/reference/aggregations/bucket/missing-aggregation.asciidoc index e37b73ed97af..7bc1e0985b59 100644 --- a/docs/reference/aggregations/bucket/missing-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/missing-aggregation.asciidoc @@ -22,7 +22,7 @@ In the above example, we get the total number of products that do not have a pri Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/nested-aggregation.asciidoc b/docs/reference/aggregations/bucket/nested-aggregation.asciidoc index 329b937bbc35..caa735e138c3 100644 --- a/docs/reference/aggregations/bucket/nested-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/nested-aggregation.asciidoc @@ -55,7 +55,7 @@ Then one can define any type of aggregation over these nested documents. Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/parent-aggregation.asciidoc b/docs/reference/aggregations/bucket/parent-aggregation.asciidoc index 208c643b31a0..0bb00ccc3fb2 100644 --- a/docs/reference/aggregations/bucket/parent-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/parent-aggregation.asciidoc @@ -125,7 +125,7 @@ The above example returns the top answer owners and per owner the top question t Possible response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 9, diff --git a/docs/reference/aggregations/bucket/range-aggregation.asciidoc b/docs/reference/aggregations/bucket/range-aggregation.asciidoc index 525d8689efc8..b8de777e37ac 100644 --- a/docs/reference/aggregations/bucket/range-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/range-aggregation.asciidoc @@ -29,7 +29,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -87,7 +87,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -140,7 +140,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -329,7 +329,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc index 7757883a40fe..88500a4c887e 100644 --- a/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/rare-terms-aggregation.asciidoc @@ -102,7 +102,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -141,7 +141,7 @@ GET /_search This now shows the "jazz" term which has a `doc_count` of 2": -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc b/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc index 8e3b65bfe71e..b3ad66f39a2b 100644 --- a/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc @@ -98,7 +98,7 @@ object types have been defined in the mapping Possible response snippet: -[source,js] +[source,console-result] -------------------------------------------------- { "aggregations": { diff --git a/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc b/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc index 4a6c7bf8c7cc..1a1f8096b46e 100644 --- a/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc @@ -45,7 +45,7 @@ POST /stackoverflow/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -108,7 +108,7 @@ POST /stackoverflow/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc index 11e0c28323bb..3c9eb2ebe2f9 100644 --- a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc @@ -144,7 +144,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc b/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc index 95655f60ccbe..c6a9c97fa8e1 100644 --- a/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc @@ -58,7 +58,7 @@ GET news/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 9, diff --git a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc index 763a7074ee71..ae894f84ec3b 100644 --- a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc @@ -71,7 +71,7 @@ GET /_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -98,6 +98,7 @@ Response: } -------------------------------------------------- // TESTRESPONSE[s/\.\.\.//] + <1> an upper bound of the error on the document counts for each term, see <> <2> when there are lots of unique terms, Elasticsearch only returns the top terms; this number is the sum of the document counts for all buckets that are not part of the response <3> the list of the top buckets, the meaning of `top` being defined by the <> @@ -229,7 +230,7 @@ terms. This is calculated as the sum of the document count from the last term re given above the value would be 46 (2 + 15 + 29). This means that in the worst case scenario a term which was not returned could have the 4th highest document count. -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -285,7 +286,7 @@ The actual document count of Product C was 54 so the document count was only act it would be off by 15. Product A, however has an error of 0 for its document count, since every shard returned it we can be confident that the count returned is accurate. -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/matrix/stats-aggregation.asciidoc b/docs/reference/aggregations/matrix/stats-aggregation.asciidoc index ea1cd8063e6d..a7a3cb2953a2 100644 --- a/docs/reference/aggregations/matrix/stats-aggregation.asciidoc +++ b/docs/reference/aggregations/matrix/stats-aggregation.asciidoc @@ -53,7 +53,7 @@ GET /_search The aggregation type is `matrix_stats` and the `fields` setting defines the set of fields (as an array) for computing the statistics. The above request returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/avg-aggregation.asciidoc b/docs/reference/aggregations/metrics/avg-aggregation.asciidoc index ca7328859c97..5b7b6f618021 100644 --- a/docs/reference/aggregations/metrics/avg-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/avg-aggregation.asciidoc @@ -19,7 +19,7 @@ POST /exams/_search?size=0 The above aggregation computes the average grade over all documents. The aggregation type is `avg` and the `field` setting defines the numeric field of the documents the average will be computed on. The above will return the following: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc index 534c69e704d9..6c33e93957e2 100644 --- a/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/cardinality-aggregation.asciidoc @@ -24,7 +24,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc b/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc index 17a313202cb0..b420d27014e5 100644 --- a/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/extendedstats-aggregation.asciidoc @@ -22,7 +22,7 @@ GET /exams/_search The above aggregation computes the grades statistics over all documents. The aggregation type is `extended_stats` and the `field` setting defines the numeric field of the documents the stats will be computed on. The above will return the following: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc b/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc index e012ae78159f..f705bbe0bede 100644 --- a/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc @@ -56,7 +56,7 @@ The above aggregation demonstrates how one would compute the bounding box of the The response for the above aggregation: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc index cd9b4acfe967..d26897c9de7e 100644 --- a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc @@ -50,7 +50,7 @@ The above aggregation demonstrates how one would compute the centroid of the loc The response for the above aggregation: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -95,7 +95,7 @@ for finding the central location for museums in each city. The response for the above aggregation: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/max-aggregation.asciidoc b/docs/reference/aggregations/metrics/max-aggregation.asciidoc index 70731dbb5157..d79997d58ac9 100644 --- a/docs/reference/aggregations/metrics/max-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/max-aggregation.asciidoc @@ -25,7 +25,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc b/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc index 336074fd2f48..5944e78cc4fe 100644 --- a/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/median-absolute-deviation-aggregation.asciidoc @@ -50,7 +50,7 @@ The resulting median absolute deviation of `2` tells us that there is a fair amount of variability in the ratings. Reviewers must have diverse opinions about this product. -[source,js] +[source,console-result] --------------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/min-aggregation.asciidoc b/docs/reference/aggregations/metrics/min-aggregation.asciidoc index bc904e7403b0..daf3da4ba103 100644 --- a/docs/reference/aggregations/metrics/min-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/min-aggregation.asciidoc @@ -25,7 +25,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc b/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc index aafc99a6a51b..4e5f330a35c1 100644 --- a/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/percentile-aggregation.asciidoc @@ -44,7 +44,7 @@ GET latency/_search By default, the `percentile` metric will generate a range of percentiles: `[ 1, 5, 25, 50, 75, 95, 99 ]`. The response will look like this: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -116,7 +116,7 @@ GET latency/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc b/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc index cb6624d25b2b..1c2938eba268 100644 --- a/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/percentile-rank-aggregation.asciidoc @@ -43,7 +43,7 @@ GET latency/_search The response will look like this: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -87,7 +87,7 @@ GET latency/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc b/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc index cfded78bbdf5..f73dec1eb8be 100644 --- a/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/scripted-metric-aggregation.asciidoc @@ -32,7 +32,7 @@ The above aggregation demonstrates how one would use the script aggregation comp The response for the above aggregation: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 218, @@ -84,7 +84,7 @@ in a global `params` object so that it can be shared between the scripts. //// Verify this response as well but in a hidden block. -[source,js] +[source,console-result] -------------------------------------------------- { "took": 218, diff --git a/docs/reference/aggregations/metrics/stats-aggregation.asciidoc b/docs/reference/aggregations/metrics/stats-aggregation.asciidoc index 233d8139ff44..c166dc425891 100644 --- a/docs/reference/aggregations/metrics/stats-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/stats-aggregation.asciidoc @@ -21,7 +21,7 @@ POST /exams/_search?size=0 The above aggregation computes the grades statistics over all documents. The aggregation type is `stats` and the `field` setting defines the numeric field of the documents the stats will be computed on. The above will return the following: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/sum-aggregation.asciidoc b/docs/reference/aggregations/metrics/sum-aggregation.asciidoc index eeff864e0ada..0f843f94c40d 100644 --- a/docs/reference/aggregations/metrics/sum-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/sum-aggregation.asciidoc @@ -26,7 +26,7 @@ POST /sales/_search?size=0 Resulting in: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc b/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc index 1b8f224e5f01..1f43a038e7c1 100644 --- a/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc @@ -67,7 +67,7 @@ POST /sales/_search?size=0 Possible response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -310,7 +310,7 @@ POST /sales/_search Top hits response snippet with a nested hit, which resides in the first slot of array field `comments`: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -355,6 +355,7 @@ Top hits response snippet with a nested hit, which resides in the first slot of } -------------------------------------------------- // TESTRESPONSE[s/\.\.\.//] + <1> Name of the array field containing the nested hit <2> Position if the nested hit in the containing array <3> Source of the nested hit diff --git a/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc b/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc index 1a617c36ca47..74cc0ae76447 100644 --- a/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/valuecount-aggregation.asciidoc @@ -19,7 +19,7 @@ POST /sales/_search?size=0 Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc b/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc index 16c14b920527..01be89f47947 100644 --- a/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc @@ -74,7 +74,7 @@ POST /exams/_search Which yields a response like: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -126,7 +126,7 @@ POST /exams/_search The three values (`1`, `2`, and `3`) will be included as independent values, all with the weight of `2`: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/aggregations/misc.asciidoc b/docs/reference/aggregations/misc.asciidoc index 36bb2a9afd0c..99a19247acfc 100644 --- a/docs/reference/aggregations/misc.asciidoc +++ b/docs/reference/aggregations/misc.asciidoc @@ -62,7 +62,7 @@ GET /twitter/_search Then that piece of metadata will be returned in place for our `titles` terms aggregation -[source,js] +[source,console-result] -------------------------------------------------- { "aggregations": { @@ -118,7 +118,7 @@ GET /twitter/_search?typed_keys In the response, the aggregations names will be changed to respectively `date_histogram#tweets_over_time` and `top_hits#top_users`, reflecting the internal types of each aggregation: -[source,js] +[source,console-result] -------------------------------------------------- { "aggregations": { diff --git a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc index 12fced9c6dd2..44fbc6fe01f9 100644 --- a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc @@ -68,7 +68,7 @@ POST /_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc index e3e6be8718d5..99617c7f788a 100644 --- a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc @@ -90,7 +90,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc index a710901b943d..2e16afd4a0a9 100644 --- a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc @@ -78,7 +78,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, @@ -112,4 +112,5 @@ And the following may be the response: // TESTRESPONSE[s/"took": 11/"took": $body.took/] // TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] // TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] + <1> Bucket for `2015/02/01 00:00:00` has been removed as its total sales was less than 200 diff --git a/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc index 25d4ccef4b01..45a8498b0879 100644 --- a/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc @@ -84,7 +84,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 82, @@ -162,7 +162,7 @@ POST /sales/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc b/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc index 99dd32f85c42..01419f4b9a34 100644 --- a/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/cumulative-cardinality-aggregation.asciidoc @@ -71,7 +71,7 @@ GET /user_hits/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, @@ -174,7 +174,7 @@ GET /user_hits/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc index 6cc06d65bd53..7570063f5d7e 100644 --- a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc @@ -64,7 +64,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc index b0f2eb6cabb3..36e51c05e156 100644 --- a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc @@ -67,7 +67,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, @@ -165,7 +165,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 50, @@ -259,7 +259,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 50, @@ -309,5 +309,6 @@ And the following may be the response: // TESTRESPONSE[s/"took": 50/"took": $body.took/] // TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] // TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] + <1> `value` is reported in the original units of 'per month' <2> `normalized_value` is reported in the desired units of 'per day' diff --git a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc index 6ad9f61f3a16..0ec03c99e481 100644 --- a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc @@ -69,7 +69,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc index 07f72d655792..82c072d0eafa 100644 --- a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc @@ -67,7 +67,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc index 217c3dc0ad74..b02b351fa665 100644 --- a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc @@ -67,7 +67,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc index 7f221d235900..32b740cc9c96 100644 --- a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc @@ -78,7 +78,7 @@ The `buckets_path` parameter is then used to "point" at one of the sibling metri An example response from the above aggregation may look like: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc index dd9017807d93..c04aebdd89b7 100644 --- a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc @@ -70,7 +70,7 @@ the `sales` aggregation in the `sales_per_month` date histogram. And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc index 827dc1b5559a..596bd95dbc11 100644 --- a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc @@ -66,7 +66,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc index 0c88ca406675..1bae8bcb5251 100644 --- a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc @@ -66,7 +66,7 @@ POST /sales/_search And the following may be the response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 11, diff --git a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc index fe76d4edb06b..a6fc0dcb39e1 100644 --- a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc @@ -226,7 +226,7 @@ GET my_index/_search The output from the above is: -[source,js] +[source,console-result] ---------------------------- { "timed_out": false, @@ -263,4 +263,5 @@ The output from the above is: } ---------------------------- // TESTRESPONSE[s/"took".*/"took": "$body.took",/] + <1> Note the incorrect highlight. diff --git a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc index 9d9b7f83e5e7..d0bd39c47397 100644 --- a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc @@ -284,7 +284,7 @@ GET my_index/_search ///////////////////// -[source,js] +[source,console-result] ---------------------------- { "took": $body.took, diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index 518d71fc5bcd..f1460d623133 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -222,7 +222,7 @@ GET /_search?q=elasticsearch&filter_path=took,hits.hits._id,hits.hits._score Responds: -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 3, @@ -383,7 +383,7 @@ GET twitter/_settings?flat_settings=true Returns: -[source,js] +[source,console-result] -------------------------------------------------- { "twitter" : { @@ -414,7 +414,7 @@ GET twitter/_settings?flat_settings=false Returns: -[source,js] +[source,console-result] -------------------------------------------------- { "twitter" : { @@ -612,7 +612,7 @@ POST /twitter/_search?size=surprise_me&error_trace=true The response looks like: -[source,js] +[source,console-result] ---------------------------------------------------------------------- { "error": { diff --git a/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc b/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc index 87496b24d68a..f9683e615a68 100644 --- a/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc +++ b/docs/reference/ccr/apis/follow/get-follow-stats.asciidoc @@ -213,7 +213,7 @@ GET /follower_index/_ccr/stats The API returns the following results: -[source,js] +[source,console-result] -------------------------------------------------- { "indices" : [ diff --git a/docs/reference/ccr/apis/follow/post-forget-follower.asciidoc b/docs/reference/ccr/apis/follow/post-forget-follower.asciidoc index 96c8349848ef..287c9045eb14 100644 --- a/docs/reference/ccr/apis/follow/post-forget-follower.asciidoc +++ b/docs/reference/ccr/apis/follow/post-forget-follower.asciidoc @@ -52,7 +52,7 @@ POST //_ccr/forget_follower // TEST[s//leader_remote_cluster/] // TEST[skip_shard_failures] -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { @@ -142,7 +142,7 @@ POST /leader_index/_ccr/forget_follower The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { diff --git a/docs/reference/ccr/apis/get-ccr-stats.asciidoc b/docs/reference/ccr/apis/get-ccr-stats.asciidoc index 37625ae5b1c3..b4cf77c00f6f 100644 --- a/docs/reference/ccr/apis/get-ccr-stats.asciidoc +++ b/docs/reference/ccr/apis/get-ccr-stats.asciidoc @@ -101,7 +101,7 @@ GET /_ccr/stats The API returns the following results: -[source,js] +[source,console-result] -------------------------------------------------- { "auto_follow_stats" : { diff --git a/docs/reference/cluster/allocation-explain.asciidoc b/docs/reference/cluster/allocation-explain.asciidoc index 752074cde397..4101ea96af88 100644 --- a/docs/reference/cluster/allocation-explain.asciidoc +++ b/docs/reference/cluster/allocation-explain.asciidoc @@ -119,7 +119,7 @@ GET /_cluster/allocation/explain The API returns the following response for an unassigned primary shard: -[source,js] +[source,console-result] -------------------------------------------------- { "index" : "idx", diff --git a/docs/reference/cluster/health.asciidoc b/docs/reference/cluster/health.asciidoc index d43bc51ee5e4..97d946d25ed8 100644 --- a/docs/reference/cluster/health.asciidoc +++ b/docs/reference/cluster/health.asciidoc @@ -147,7 +147,7 @@ GET _cluster/health The API returns the following response in case of a quiet single node cluster with a single index with one shard and one replica: -[source,js] +[source,console-result] -------------------------------------------------- { "cluster_name" : "testcluster", diff --git a/docs/reference/cluster/nodes-info.asciidoc b/docs/reference/cluster/nodes-info.asciidoc index 2c7a74719e04..a5a7654930d7 100644 --- a/docs/reference/cluster/nodes-info.asciidoc +++ b/docs/reference/cluster/nodes-info.asciidoc @@ -195,7 +195,7 @@ GET /_nodes/plugins The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "_nodes": ... @@ -267,7 +267,7 @@ GET /_nodes/ingest The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "_nodes": ... diff --git a/docs/reference/cluster/nodes-reload-secure-settings.asciidoc b/docs/reference/cluster/nodes-reload-secure-settings.asciidoc index 68bca72be248..b99f93dfc531 100644 --- a/docs/reference/cluster/nodes-reload-secure-settings.asciidoc +++ b/docs/reference/cluster/nodes-reload-secure-settings.asciidoc @@ -35,7 +35,7 @@ node id. Each value has the node `name` and an optional `reload_exception` field. The `reload_exception` field is a serialization of the exception that was thrown during the reload process, if any. -[source,js] +[source,console-result] -------------------------------------------------- { "_nodes": { diff --git a/docs/reference/cluster/nodes-usage.asciidoc b/docs/reference/cluster/nodes-usage.asciidoc index 2a0b9f7b6136..5b1559790c3b 100644 --- a/docs/reference/cluster/nodes-usage.asciidoc +++ b/docs/reference/cluster/nodes-usage.asciidoc @@ -63,7 +63,7 @@ GET _nodes/usage The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "_nodes": { diff --git a/docs/reference/cluster/update-settings.asciidoc b/docs/reference/cluster/update-settings.asciidoc index e2da6c516176..223a9c554448 100644 --- a/docs/reference/cluster/update-settings.asciidoc +++ b/docs/reference/cluster/update-settings.asciidoc @@ -87,7 +87,7 @@ PUT /_cluster/settings?flat_settings=true The response to an update returns the changed setting, as in this response to the transient example: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -116,7 +116,7 @@ PUT /_cluster/settings The response does not include settings that have been reset: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/data-frames/apis/update-transform.asciidoc b/docs/reference/data-frames/apis/update-transform.asciidoc index dfa6f853ad81..499c25e6df23 100644 --- a/docs/reference/data-frames/apis/update-transform.asciidoc +++ b/docs/reference/data-frames/apis/update-transform.asciidoc @@ -153,7 +153,8 @@ POST _data_frame/transforms/simple-kibana-ecomm-pivot/_update // TEST[setup:simple_kibana_continuous_pivot] When the transform is updated, you receive the updated configuration: -[source,js] + +[source,console-result] ---- { "id": "simple-kibana-ecomm-pivot", diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index 440d2dd70931..bb83464e55a0 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -84,7 +84,7 @@ POST _bulk The result of this bulk operation is: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 30, diff --git a/docs/reference/docs/concurrency-control.asciidoc b/docs/reference/docs/concurrency-control.asciidoc index 4b02239ee6ce..a7c38f6c222e 100644 --- a/docs/reference/docs/concurrency-control.asciidoc +++ b/docs/reference/docs/concurrency-control.asciidoc @@ -33,7 +33,7 @@ PUT products/_doc/1567 You can see the assigned sequence number and primary term in the `_seq_no` and `_primary_term` fields of the response: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { @@ -67,7 +67,7 @@ GET products/_doc/1567 returns: -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "products", diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index c8f18f2c4796..bded6cc1812d 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -27,7 +27,7 @@ POST /twitter/_delete_by_query //// -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 147, @@ -261,7 +261,7 @@ POST /twitter/_delete_by_query The JSON response looks like this: -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 147, diff --git a/docs/reference/docs/delete.asciidoc b/docs/reference/docs/delete.asciidoc index 66f2daf03354..4d4a81e58253 100644 --- a/docs/reference/docs/delete.asciidoc +++ b/docs/reference/docs/delete.asciidoc @@ -176,7 +176,7 @@ DELETE /twitter/_doc/1 The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 3e70cf45dbc5..e95f2ba4b81e 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -203,7 +203,7 @@ POST twitter/_doc/ The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { @@ -466,7 +466,7 @@ PUT twitter/_doc/1 The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { diff --git a/docs/reference/docs/multi-get.asciidoc b/docs/reference/docs/multi-get.asciidoc index 3360f2c06ffa..28b76dbe42c7 100644 --- a/docs/reference/docs/multi-get.asciidoc +++ b/docs/reference/docs/multi-get.asciidoc @@ -54,7 +54,7 @@ GET /test/_mget And type: -[source,js] +[source,console] -------------------------------------------------- GET /test/_doc/_mget { diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index 5a2544260149..f9f33beb2102 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -29,7 +29,7 @@ POST _reindex That will return something like this: -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 147, @@ -689,7 +689,7 @@ POST /_reindex?wait_for_completion The JSON response looks like this: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 639, @@ -956,7 +956,7 @@ GET test2/_doc/1 which will return: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, diff --git a/docs/reference/docs/termvectors.asciidoc b/docs/reference/docs/termvectors.asciidoc index 515d75ff3baf..a24f3ea95488 100644 --- a/docs/reference/docs/termvectors.asciidoc +++ b/docs/reference/docs/termvectors.asciidoc @@ -202,7 +202,7 @@ GET /twitter/_termvectors/1 Response: -[source,js] +[source,console-result] -------------------------------------------------- { "_id": "1", @@ -339,7 +339,7 @@ GET /twitter/_termvectors Response: -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "twitter", diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index c51964345150..c4972fa89bb2 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -15,7 +15,7 @@ POST twitter/_update_by_query?conflicts=proceed That will return something like this: -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 147, @@ -260,7 +260,7 @@ POST /twitter/_update_by_query?conflicts=proceed The JSON response looks like this: -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 147, diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index 4c5b3535787b..34dfd7dc5c0d 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -227,7 +227,7 @@ indexes the `name` field. Since this is a new document, the response shows that the result of the operation was that version 1 of the document was created: -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "customer", @@ -262,7 +262,7 @@ GET /customer/_doc/1 The response indicates that a document with the specified ID was found and shows the original source fields that were indexed. -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "customer", @@ -373,7 +373,7 @@ GET /bank/_search By default, the `hits` section of the response includes the first 10 documents that match the search criteria: -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 63, @@ -573,7 +573,7 @@ The `buckets` in the response are the values of the `state` field. The can see that there are 27 accounts in `ID` (Idaho). Because the request set `size=0`, the response only contains the aggregation results. -[source,js] +[source,console-result] -------------------------------------------------- { "took": 29, diff --git a/docs/reference/how-to/recipes/stemming.asciidoc b/docs/reference/how-to/recipes/stemming.asciidoc index d435ce1c94d4..f68d8c7ff87c 100644 --- a/docs/reference/how-to/recipes/stemming.asciidoc +++ b/docs/reference/how-to/recipes/stemming.asciidoc @@ -70,7 +70,7 @@ GET index/_search // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took": 2, @@ -131,7 +131,7 @@ GET index/_search // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took": 1, @@ -191,7 +191,7 @@ GET index/_search // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took": 2, diff --git a/docs/reference/ilm/apis/slm-api.asciidoc b/docs/reference/ilm/apis/slm-api.asciidoc index 7b32a3fc5090..0466924e3dee 100644 --- a/docs/reference/ilm/apis/slm-api.asciidoc +++ b/docs/reference/ilm/apis/slm-api.asciidoc @@ -121,7 +121,7 @@ GET /_slm/policy/daily-snapshots?human The output looks similar to the following: -[source,js] +[source,console-result] -------------------------------------------------- { "daily-snapshots" : { @@ -183,7 +183,7 @@ PUT /_slm/policy/daily-snapshots/_execute This API will immediately return with the generated snapshot name -[source,js] +[source,console-result] -------------------------------------------------- { "snapshot_name": "daily-snap-2019.04.24-gwrqoo2xtea3q57vvg0uea" @@ -206,7 +206,7 @@ GET /_slm/policy/daily-snapshots?human Which, in this case shows an error because the index did not exist: -[source,js] +[source,console-result] -------------------------------------------------- { "daily-snapshots" : { @@ -235,6 +235,7 @@ Which, in this case shows an error because the index did not exist: } -------------------------------------------------- // TESTRESPONSE[skip:the presence of last_failure is asynchronous and will be present for users, but is untestable] + <1> The last unsuccessfully initiated snapshot by this policy, along with the details of its failure In this case, it failed due to the "important" index not existing and @@ -269,7 +270,7 @@ PUT /_slm/policy/daily-snapshots/_execute // CONSOLE // TEST[skip:we can't handle snapshots in docs tests] -[source,js] +[source,console-result] -------------------------------------------------- { "snapshot_name": "daily-snap-2019.04.24-tmtnyjtrsxkhbrrdcgg18a" @@ -289,7 +290,7 @@ GET /_slm/policy/daily-snapshots?human Which now includes the successful snapshot information: -[source,js] +[source,console-result] -------------------------------------------------- { "daily-snapshots" : { @@ -323,6 +324,7 @@ Which now includes the successful snapshot information: } -------------------------------------------------- // TESTRESPONSE[skip:the presence of last_failure and last_success is asynchronous and will be present for users, but is untestable] + <1> The policy's version has been incremented because it was updated <2> The last successfully initiated snapshot information diff --git a/docs/reference/ilm/getting-started-slm.asciidoc b/docs/reference/ilm/getting-started-slm.asciidoc index 5849101ffe6c..baef2021f779 100644 --- a/docs/reference/ilm/getting-started-slm.asciidoc +++ b/docs/reference/ilm/getting-started-slm.asciidoc @@ -158,7 +158,7 @@ This request will return a response that includes the policy, as well as information about the last time the policy succeeded and failed, as well as the next time the policy will be executed. -[source,js] +[source,console-result] -------------------------------------------------- { "nightly-snapshots" : { @@ -190,6 +190,7 @@ next time the policy will be executed. } -------------------------------------------------- // TESTRESPONSE[skip:the presence of last_failure and last_success is asynchronous and will be present for users, but is untestable] + <1> information about the last time the policy successfully initated a snapshot <2> the name of the snapshot that was successfully initiated <3> information about the last time the policy failed to initiate a snapshot diff --git a/docs/reference/index-modules/index-sorting.asciidoc b/docs/reference/index-modules/index-sorting.asciidoc index 40c4b08cb26f..340ffa8491cf 100644 --- a/docs/reference/index-modules/index-sorting.asciidoc +++ b/docs/reference/index-modules/index-sorting.asciidoc @@ -173,7 +173,7 @@ GET /events/_search This time, Elasticsearch will not try to count the number of documents and will be able to terminate the query as soon as N documents have been collected per segment. -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... diff --git a/docs/reference/index-modules/similarity.asciidoc b/docs/reference/index-modules/similarity.asciidoc index 22b41901bde6..3b03bcc94f13 100644 --- a/docs/reference/index-modules/similarity.asciidoc +++ b/docs/reference/index-modules/similarity.asciidoc @@ -238,7 +238,7 @@ GET /index/_search?explain=true Which yields: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 12, @@ -411,7 +411,7 @@ GET /index/_search?explain=true -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took": 1, diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index 1f5de4c3fb99..6897af337a8f 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -72,7 +72,7 @@ POST /my_index/_reload_search_analyzers The reload request returns information about the nodes it was executed on and the analyzers that were reloaded: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 3c36c592bd70..b8a9f43f95f0 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -96,7 +96,7 @@ GET twitter/_stats?filter_path=**.commit&level=shards <1> which returns something similar to: -[source,js] +[source,console-result] -------------------------------------------------- { "indices": { @@ -165,7 +165,7 @@ sync-flushed and information about any failure. Here is what it looks like when all shards of a two shards and one replica index successfully sync-flushed: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": { diff --git a/docs/reference/indices/recovery.asciidoc b/docs/reference/indices/recovery.asciidoc index 54d25ac2c4da..9bebc2388f12 100644 --- a/docs/reference/indices/recovery.asciidoc +++ b/docs/reference/indices/recovery.asciidoc @@ -67,7 +67,8 @@ GET /_recovery?human // TEST[continued] Response: -[source,js] + +[source,console-result] -------------------------------------------------- { "index1" : { @@ -159,7 +160,7 @@ GET _recovery?human&detailed=true Response: -[source,js] +[source,console-result] -------------------------------------------------- { "index1" : { diff --git a/docs/reference/indices/shard-stores.asciidoc b/docs/reference/indices/shard-stores.asciidoc index 8d4d8e3202cf..355b23cacf7d 100644 --- a/docs/reference/indices/shard-stores.asciidoc +++ b/docs/reference/indices/shard-stores.asciidoc @@ -45,7 +45,7 @@ Response: The shard stores information is grouped by indices and shard ids. -[source,js] +[source,console-result] -------------------------------------------------- { "indices": { diff --git a/docs/reference/indices/templates.asciidoc b/docs/reference/indices/templates.asciidoc index b2871eed2cda..fc0cba1ce363 100644 --- a/docs/reference/indices/templates.asciidoc +++ b/docs/reference/indices/templates.asciidoc @@ -240,7 +240,7 @@ GET /_template/template_1?filter_path=*.version The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "template_1" : { diff --git a/docs/reference/ingest.asciidoc b/docs/reference/ingest.asciidoc index 135e0a571ec6..8b1cadd4d7e3 100644 --- a/docs/reference/ingest.asciidoc +++ b/docs/reference/ingest.asciidoc @@ -56,7 +56,8 @@ PUT my-index/_doc/my-id?pipeline=my_pipeline_id // TEST[continued] Response: -[source,js] + +[source,console-result] -------------------------------------------------- { "_index" : "my-index", diff --git a/docs/reference/ingest/apis/simulate-pipeline.asciidoc b/docs/reference/ingest/apis/simulate-pipeline.asciidoc index cab41ee54d16..278b4607121f 100644 --- a/docs/reference/ingest/apis/simulate-pipeline.asciidoc +++ b/docs/reference/ingest/apis/simulate-pipeline.asciidoc @@ -83,7 +83,7 @@ POST _ingest/pipeline/_simulate Response: -[source,js] +[source,console-result] -------------------------------------------------- { "docs": [ @@ -173,7 +173,7 @@ POST _ingest/pipeline/_simulate?verbose Response: -[source,js] +[source,console-result] -------------------------------------------------- { "docs": [ diff --git a/docs/reference/ingest/ingest-node.asciidoc b/docs/reference/ingest/ingest-node.asciidoc index 41463538606a..e9813f44f538 100644 --- a/docs/reference/ingest/ingest-node.asciidoc +++ b/docs/reference/ingest/ingest-node.asciidoc @@ -272,7 +272,7 @@ GET test/_doc/2 -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "test", @@ -461,7 +461,7 @@ GET test/_doc/2 -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "test", @@ -582,7 +582,7 @@ GET test/_doc/1 // TEST[continued] //// -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "test", diff --git a/docs/reference/ingest/processors/circle.asciidoc b/docs/reference/ingest/processors/circle.asciidoc index 7e577a59d50c..5669c6941478 100644 --- a/docs/reference/ingest/processors/circle.asciidoc +++ b/docs/reference/ingest/processors/circle.asciidoc @@ -70,7 +70,7 @@ GET circles/_doc/1 The response from the above index request: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -108,7 +108,7 @@ GET circles/_doc/2 The response from the above index request: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, diff --git a/docs/reference/ingest/processors/date-index-name.asciidoc b/docs/reference/ingest/processors/date-index-name.asciidoc index 52e29dfb120e..7ffae4909a34 100644 --- a/docs/reference/ingest/processors/date-index-name.asciidoc +++ b/docs/reference/ingest/processors/date-index-name.asciidoc @@ -45,7 +45,7 @@ PUT /myindex/_doc/1?pipeline=monthlyindex -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "myindex-2016-04-01", @@ -101,7 +101,7 @@ POST _ingest/pipeline/_simulate and the result: -[source,js] +[source,console-result] -------------------------------------------------- { "docs" : [ diff --git a/docs/reference/ingest/processors/geoip.asciidoc b/docs/reference/ingest/processors/geoip.asciidoc index 3be65bb65be1..322719e4995c 100644 --- a/docs/reference/ingest/processors/geoip.asciidoc +++ b/docs/reference/ingest/processors/geoip.asciidoc @@ -63,7 +63,7 @@ GET my_index/_doc/my_id Which returns: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -113,7 +113,7 @@ GET my_index/_doc/my_id returns this: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -165,7 +165,7 @@ GET my_index/_doc/my_id Which returns: -[source,js] +[source,console-result] -------------------------------------------------- { "_index" : "my_index", @@ -249,7 +249,7 @@ GET /my_ip_locations/_search -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 3, diff --git a/docs/reference/ingest/processors/grok.asciidoc b/docs/reference/ingest/processors/grok.asciidoc index c58d447f4b7b..1ea259e7cb9a 100644 --- a/docs/reference/ingest/processors/grok.asciidoc +++ b/docs/reference/ingest/processors/grok.asciidoc @@ -186,7 +186,7 @@ POST _ingest/pipeline/_simulate response: -[source,js] +[source,console-result] -------------------------------------------------- { "docs": [ @@ -246,7 +246,7 @@ POST _ingest/pipeline/_simulate -------------------------------------------------- //// -[source,js] +[source,console-result] -------------------------------------------------- { "docs": [ diff --git a/docs/reference/ingest/processors/pipeline.asciidoc b/docs/reference/ingest/processors/pipeline.asciidoc index 2eddf1631631..ff0744b33722 100644 --- a/docs/reference/ingest/processors/pipeline.asciidoc +++ b/docs/reference/ingest/processors/pipeline.asciidoc @@ -79,7 +79,7 @@ PUT /myindex/_doc/1?pipeline=pipelineB Response from the index request: -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "myindex", diff --git a/docs/reference/ingest/processors/script.asciidoc b/docs/reference/ingest/processors/script.asciidoc index aab739d0419b..d9aed04f57de 100644 --- a/docs/reference/ingest/processors/script.asciidoc +++ b/docs/reference/ingest/processors/script.asciidoc @@ -77,7 +77,7 @@ PUT any_index/_doc/1?pipeline=my_index The response from the above index request: -[source,js] +[source,console-result] -------------------------------------------------- { "_index": "my_index", diff --git a/docs/reference/ingest/processors/set.asciidoc b/docs/reference/ingest/processors/set.asciidoc index 8ea0e7101e54..9aba07b73f65 100644 --- a/docs/reference/ingest/processors/set.asciidoc +++ b/docs/reference/ingest/processors/set.asciidoc @@ -56,7 +56,8 @@ POST _ingest/pipeline/set_os/_simulate -------------------------------------------------- Result: -[source,js] + +[source,console-result] -------------------------------------------------- { "docs" : [ diff --git a/docs/reference/ingest/processors/user-agent.asciidoc b/docs/reference/ingest/processors/user-agent.asciidoc index 6993e449f47e..67dc76c478ea 100644 --- a/docs/reference/ingest/processors/user-agent.asciidoc +++ b/docs/reference/ingest/processors/user-agent.asciidoc @@ -45,7 +45,7 @@ GET my_index/_doc/my_id Which returns -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, diff --git a/docs/reference/licensing/get-basic-status.asciidoc b/docs/reference/licensing/get-basic-status.asciidoc index 38abc087d957..59b91d1c6c99 100644 --- a/docs/reference/licensing/get-basic-status.asciidoc +++ b/docs/reference/licensing/get-basic-status.asciidoc @@ -39,7 +39,8 @@ GET /_license/basic_status ------------------------------------------------------------ Example response: -[source,js] + +[source,console-result] ------------------------------------------------------------ { "eligible_to_start_basic": true diff --git a/docs/reference/licensing/get-license.asciidoc b/docs/reference/licensing/get-license.asciidoc index c49e85384a04..f38e8c0fffdb 100644 --- a/docs/reference/licensing/get-license.asciidoc +++ b/docs/reference/licensing/get-license.asciidoc @@ -49,7 +49,7 @@ The following example provides information about a trial license: GET /_license -------------------------------------------------- -[source,js] +[source,console-result] -------------------------------------------------- { "license" : { diff --git a/docs/reference/licensing/get-trial-status.asciidoc b/docs/reference/licensing/get-trial-status.asciidoc index b6caaf0252ca..cb1a3772ca06 100644 --- a/docs/reference/licensing/get-trial-status.asciidoc +++ b/docs/reference/licensing/get-trial-status.asciidoc @@ -45,7 +45,8 @@ GET /_license/trial_status ------------------------------------------------------------ Example response: -[source,js] + +[source,console-result] ------------------------------------------------------------ { "eligible_to_start_trial": true diff --git a/docs/reference/mapping/params/normalizer.asciidoc b/docs/reference/mapping/params/normalizer.asciidoc index 288faa931b62..b85c45fd807e 100644 --- a/docs/reference/mapping/params/normalizer.asciidoc +++ b/docs/reference/mapping/params/normalizer.asciidoc @@ -74,7 +74,7 @@ GET index/_search The above queries match documents 1 and 2 since `BÀR` is converted to `bar` at both index and query time. -[source,js] +[source,console-result] ---------------------------- { "took": $body.took, @@ -137,7 +137,7 @@ GET index/_search returns -[source,js] +[source,console-result] ---------------------------- { "took": 43, diff --git a/docs/reference/mapping/types/parent-join.asciidoc b/docs/reference/mapping/types/parent-join.asciidoc index 94909417315d..292e5b7b468e 100644 --- a/docs/reference/mapping/types/parent-join.asciidoc +++ b/docs/reference/mapping/types/parent-join.asciidoc @@ -166,7 +166,7 @@ GET my_index/_search Will return: -[source,js] +[source,console-result] -------------------------------------------------- { ..., diff --git a/docs/reference/mapping/types/percolator.asciidoc b/docs/reference/mapping/types/percolator.asciidoc index 51498e34555d..cd958c6fe234 100644 --- a/docs/reference/mapping/types/percolator.asciidoc +++ b/docs/reference/mapping/types/percolator.asciidoc @@ -179,7 +179,7 @@ GET /queries/_search now returns matches from the new index: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 3, @@ -370,7 +370,7 @@ GET /test_index/_search This results in a response like this: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 6, @@ -527,7 +527,7 @@ GET /my_queries1/_search -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took": 6, diff --git a/docs/reference/mapping/types/range.asciidoc b/docs/reference/mapping/types/range.asciidoc index cd6bfac03fd6..186096c0e9db 100644 --- a/docs/reference/mapping/types/range.asciidoc +++ b/docs/reference/mapping/types/range.asciidoc @@ -17,7 +17,7 @@ The following range types are supported: Below is an example of configuring a mapping with various range fields followed by an example that indexes several range types. -[source,js] +[source,console] -------------------------------------------------- PUT range_index { @@ -49,7 +49,6 @@ PUT range_index/_doc/1?refresh } } -------------------------------------------------- -//CONSOLE // TESTSETUP <1> `date_range` types accept the same field parameters defined by the <> type. @@ -75,7 +74,7 @@ GET range_index/_search The result produced by the above query. -[source,js] +[source,console-result] -------------------------------------------------- { "took": 13, @@ -137,7 +136,7 @@ GET range_index/_search This query produces a similar result: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 13, diff --git a/docs/reference/mapping/types/search-as-you-type.asciidoc b/docs/reference/mapping/types/search-as-you-type.asciidoc index cc26f3666fda..12e608fed878 100644 --- a/docs/reference/mapping/types/search-as-you-type.asciidoc +++ b/docs/reference/mapping/types/search-as-you-type.asciidoc @@ -102,7 +102,7 @@ GET my_index/_search -------------------------------------------------- // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "took" : 44, diff --git a/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc index 293fe0bf95db..dd6577f2262b 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-forecast.asciidoc @@ -74,6 +74,7 @@ DELETE _ml/anomaly_detectors/total-requests/_forecast/_all // TEST[skip:setup:server_metrics_openjob] If the request does not encounter errors, you receive the following result: + [source,js] ---- { diff --git a/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc index a258c50978fb..202f7cfcd333 100644 --- a/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/delete-job.asciidoc @@ -84,7 +84,8 @@ DELETE _ml/anomaly_detectors/total-requests?wait_for_completion=false When `wait_for_completion` is set to `false`, the response contains the id of the job deletion task: -[source,js] + +[source,console-result] ---- { "task": "oTUltX4IQMOUUVeiohTt8A:39" diff --git a/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc b/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc index 33dfd12e6fcc..5fa56eacb820 100644 --- a/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/find-file-structure.asciidoc @@ -253,7 +253,7 @@ POST _ml/find_file_structure If the request does not encounter errors, you receive the following result: -[source,js] +[source,console-result] ---- { "num_lines_analyzed" : 24, <1> diff --git a/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc index b681e165165e..a306817e3c04 100644 --- a/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/flush-job.asciidoc @@ -78,7 +78,8 @@ POST _ml/anomaly_detectors/total-requests/_flush // TEST[skip:setup:server_metrics_openjob] When the operation succeeds, you receive the following results: -[source,js] + +[source,console-result] ---- { "flushed": true, diff --git a/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc index 6059cde7a73a..6d6dbbcdadd6 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-calendar-event.asciidoc @@ -94,7 +94,7 @@ GET _ml/calendars/planned-outages/events The API returns the following results: -[source,js] +[source,console-result] ---- { "count": 3, diff --git a/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc index fad72220b6c7..ae142720e273 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-datafeed-stats.asciidoc @@ -96,7 +96,8 @@ GET _ml/datafeeds/datafeed-total-requests/_stats // TEST[skip:setup:server_metrics_startdf] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count": 1, diff --git a/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc index d5e632deffb9..1903be47e661 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-datafeed.asciidoc @@ -92,7 +92,8 @@ GET _ml/datafeeds/datafeed-total-requests // TEST[skip:setup:server_metrics_datafeed] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count": 1, diff --git a/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc index 54d3d7cf2620..c1d6be4eb1c1 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-job.asciidoc @@ -89,7 +89,8 @@ GET _ml/anomaly_detectors/total-requests // TEST[skip:setup:server_metrics_job] The API returns the following results: -[source,js] + +[source,console-result] ---- { "count": 1, diff --git a/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc b/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc index 4f0179629539..a90ed90ebc67 100644 --- a/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/get-ml-info.asciidoc @@ -44,7 +44,8 @@ GET _ml/info // TEST This is a possible response: -[source,js] + +[source,console-result] ---- { "defaults" : { diff --git a/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc index 248c8bba7ce7..e6c64c58e60e 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-datafeed.asciidoc @@ -125,7 +125,8 @@ PUT _ml/datafeeds/datafeed-total-requests // TEST[skip:setup:server_metrics_job] When the {dfeed} is created, you receive the following results: -[source,js] + +[source,console-result] ---- { "datafeed_id": "datafeed-total-requests", diff --git a/docs/reference/ml/anomaly-detection/apis/put-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/put-job.asciidoc index 8bd11cac8927..6b331bbf5569 100644 --- a/docs/reference/ml/anomaly-detection/apis/put-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/put-job.asciidoc @@ -119,7 +119,8 @@ PUT _ml/anomaly_detectors/total-requests -------------------------------------------------- When the job is created, you receive the following results: -[source,js] + +[source,console-result] ---- { "job_id" : "total-requests", diff --git a/docs/reference/ml/anomaly-detection/apis/update-datafeed.asciidoc b/docs/reference/ml/anomaly-detection/apis/update-datafeed.asciidoc index 58a6ae56700f..eb9e0d7dceee 100644 --- a/docs/reference/ml/anomaly-detection/apis/update-datafeed.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/update-datafeed.asciidoc @@ -119,7 +119,7 @@ POST _ml/datafeeds/datafeed-total-requests/_update When the {dfeed} is updated, you receive the full {dfeed} configuration with with the updated values: -[source,js] +[source,console-result] ---- { "datafeed_id": "datafeed-total-requests", diff --git a/docs/reference/ml/anomaly-detection/apis/update-job.asciidoc b/docs/reference/ml/anomaly-detection/apis/update-job.asciidoc index 7098ba1ebbce..e35ac4386382 100644 --- a/docs/reference/ml/anomaly-detection/apis/update-job.asciidoc +++ b/docs/reference/ml/anomaly-detection/apis/update-job.asciidoc @@ -131,7 +131,7 @@ POST _ml/anomaly_detectors/total-requests/_update When the {anomaly-job} is updated, you receive a summary of the job configuration information, including the updated property values. For example: -[source,js] +[source,console-result] ---- { "job_id": "total-requests", diff --git a/docs/reference/ml/anomaly-detection/categories.asciidoc b/docs/reference/ml/anomaly-detection/categories.asciidoc index aa7fd8a45023..a5e4d0554161 100644 --- a/docs/reference/ml/anomaly-detection/categories.asciidoc +++ b/docs/reference/ml/anomaly-detection/categories.asciidoc @@ -19,7 +19,7 @@ time. You can then detect anomalies and surface rare events or unusual types of messages by using count or rare functions. For example: //Obtained from it_ops_new_app_logs.sh -[source,js] +[source,console] ---------------------------------- PUT _ml/anomaly_detectors/it_ops_new_logs { @@ -43,8 +43,8 @@ PUT _ml/anomaly_detectors/it_ops_new_logs } } ---------------------------------- -//CONSOLE // TEST[skip:needs-licence] + <1> The `categorization_field_name` property indicates which field will be categorized. <2> The resulting categories are used in a detector by setting `by_field_name`, @@ -89,7 +89,7 @@ supported, as described in the <>. You can, however, change the tokenization rules by customizing the way the categorization field values are interpreted. For example: -[source,js] +[source,console] ---------------------------------- PUT _ml/anomaly_detectors/it_ops_new_logs2 { @@ -127,8 +127,8 @@ PUT _ml/anomaly_detectors/it_ops_new_logs2 } } ---------------------------------- -//CONSOLE // TEST[skip:needs-licence] + <1> The {ref}/analysis-pattern-replace-charfilter.html[`pattern_replace` character filter] here achieves exactly the same as the `categorization_filters` in the first @@ -151,7 +151,7 @@ equivalent to the following analyzer, which is defined using only built-in {es} {ref}/analysis-tokenizers.html[tokenizers] and {ref}/analysis-tokenfilters.html[token filters]: -[source,js] +[source,console] ---------------------------------- PUT _ml/anomaly_detectors/it_ops_new_logs3 { @@ -194,8 +194,8 @@ PUT _ml/anomaly_detectors/it_ops_new_logs3 } } ---------------------------------- -//CONSOLE // TEST[skip:needs-licence] + <1> Tokens basically consist of hyphens, digits, letters, underscores and dots. <2> By default, categorization ignores tokens that begin with a digit. <3> By default, categorization also ignores tokens that are hexadecimal numbers. diff --git a/docs/reference/ml/anomaly-detection/customurl.asciidoc b/docs/reference/ml/anomaly-detection/customurl.asciidoc index 89a5f5d8ce89..1c0c463e9e8f 100644 --- a/docs/reference/ml/anomaly-detection/customurl.asciidoc +++ b/docs/reference/ml/anomaly-detection/customurl.asciidoc @@ -91,7 +91,7 @@ dashboard query seeks a match for all of the terms of the category. For example, the following API updates a job to add a custom URL that uses `$earliest$`, `$latest$`, and `$mlcategoryterms$` tokens: -[source,js] +[source,console] ---------------------------------- POST _ml/anomaly_detectors/sample_job/_update { @@ -106,7 +106,6 @@ POST _ml/anomaly_detectors/sample_job/_update } } ---------------------------------- -//CONSOLE //TEST[skip:setup:sample_job] When you click this custom URL in the anomalies table in {kib}, it opens up the diff --git a/docs/reference/ml/anomaly-detection/populations.asciidoc b/docs/reference/ml/anomaly-detection/populations.asciidoc index febbbb29e5ee..decbe6a3b35c 100644 --- a/docs/reference/ml/anomaly-detection/populations.asciidoc +++ b/docs/reference/ml/anomaly-detection/populations.asciidoc @@ -26,7 +26,7 @@ of hundreds of thousands or millions of entities. To specify the population, use the `over_field_name` property. For example: -[source,js] +[source,console] ---------------------------------- PUT _ml/anomaly_detectors/population { @@ -50,8 +50,8 @@ PUT _ml/anomaly_detectors/population } } ---------------------------------- -//CONSOLE // TEST[skip:needs-licence] + <1> This `over_field_name` property indicates that the metrics for each client ( as identified by their IP address) are analyzed relative to other clients in each bucket. diff --git a/docs/reference/ml/df-analytics/apis/put-dfanalytics.asciidoc b/docs/reference/ml/df-analytics/apis/put-dfanalytics.asciidoc index b46de02ca0bf..9a8c6ecc5cbe 100644 --- a/docs/reference/ml/df-analytics/apis/put-dfanalytics.asciidoc +++ b/docs/reference/ml/df-analytics/apis/put-dfanalytics.asciidoc @@ -147,7 +147,7 @@ PUT _ml/data_frame/analytics/loganalytics The API returns the following result: -[source,js] +[source,console-result] ---- { "id": "loganalytics", diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index 3a5f17a2bf1d..f2f7fdfc6666 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -76,7 +76,7 @@ GET /cluster_one:twitter/_search The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 150, @@ -148,7 +148,7 @@ GET /twitter,cluster_one:twitter,cluster_two:twitter/_search The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 150, diff --git a/docs/reference/query-dsl/percolate-query.asciidoc b/docs/reference/query-dsl/percolate-query.asciidoc index b32a7328acba..192b80b47355 100644 --- a/docs/reference/query-dsl/percolate-query.asciidoc +++ b/docs/reference/query-dsl/percolate-query.asciidoc @@ -78,7 +78,7 @@ GET /my-index/_search The above request will yield the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 13, @@ -229,7 +229,7 @@ GET /my-index/_search <1> The documents array contains 4 documents that are going to be percolated at the same time. -[source,js] +[source,console-result] -------------------------------------------------- { "took": 13, @@ -410,7 +410,7 @@ GET /my-index/_search This will yield the following response. -[source,js] +[source,console-result] -------------------------------------------------- { "took": 7, @@ -518,7 +518,7 @@ GET /my-index/_search The slightly different response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 13, @@ -617,7 +617,7 @@ If that isn't specified then the `field` parameter will be used, which in this c The above search request returns a response similar to this: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 13, diff --git a/docs/reference/query-dsl/span-containing-query.asciidoc b/docs/reference/query-dsl/span-containing-query.asciidoc index 7b5fb0ba7aea..86d01a95914d 100644 --- a/docs/reference/query-dsl/span-containing-query.asciidoc +++ b/docs/reference/query-dsl/span-containing-query.asciidoc @@ -7,7 +7,7 @@ Returns matches which enclose another span query. The span containing query maps to Lucene `SpanContainingQuery`. Here is an example: -[source,js] +[source,console] -------------------------------------------------- GET /_search { @@ -30,7 +30,6 @@ GET /_search } } -------------------------------------------------- -// CONSOLE The `big` and `little` clauses can be any span type query. Matching spans from `big` that contain matches from `little` are returned. diff --git a/docs/reference/query-dsl/term-query.asciidoc b/docs/reference/query-dsl/term-query.asciidoc index a80f065dcd78..aa49a051dff6 100644 --- a/docs/reference/query-dsl/term-query.asciidoc +++ b/docs/reference/query-dsl/term-query.asciidoc @@ -188,7 +188,7 @@ any documents containing the `quick`, `brown`, or `fox` tokens in the Here's the response for the `match` query search containing the indexed document in the results. -[source,js] +[source,console-result] ---- { "took" : 1, diff --git a/docs/reference/query-dsl/terms-query.asciidoc b/docs/reference/query-dsl/terms-query.asciidoc index 71aa11eb1257..fa7f7c1c7d3f 100644 --- a/docs/reference/query-dsl/terms-query.asciidoc +++ b/docs/reference/query-dsl/terms-query.asciidoc @@ -208,7 +208,7 @@ GET my_index/_search?pretty Because document 2 and document 1 both contain `blue` as a value in the `color` field, {es} returns both documents. -[source,js] +[source,console-result] ---- { "took" : 17, diff --git a/docs/reference/rest-api/info.asciidoc b/docs/reference/rest-api/info.asciidoc index 224e77bd1e0c..ee66ce8935d2 100644 --- a/docs/reference/rest-api/info.asciidoc +++ b/docs/reference/rest-api/info.asciidoc @@ -48,7 +48,8 @@ GET /_xpack // CONSOLE Example response: -[source,js] + +[source,console-result] ------------------------------------------------------------ { "build" : { diff --git a/docs/reference/rollup/apis/rollup-search.asciidoc b/docs/reference/rollup/apis/rollup-search.asciidoc index ec2a554d09ff..127d06f6df9c 100644 --- a/docs/reference/rollup/apis/rollup-search.asciidoc +++ b/docs/reference/rollup/apis/rollup-search.asciidoc @@ -109,7 +109,7 @@ GET /sensor_rollup/_rollup_search The query is targeting the `sensor_rollup` data, since this contains the rollup data as configured in the job. A `max` aggregation has been used on the `temperature` field, yielding the following response: -[source,js] +[source,console-result] ---- { "took" : 102, @@ -159,7 +159,7 @@ GET sensor_rollup/_rollup_search // TEST[continued] // TEST[catch:/illegal_argument_exception/] -[source,js] +[source,console-result] ---- { "error" : { @@ -215,7 +215,7 @@ index will be used. The response to the above query will look as expected, despite spanning rollup and non-rollup indices: -[source,js] +[source,console-result] ---- { "took" : 102, @@ -238,4 +238,4 @@ The response to the above query will look as expected, despite spanning rollup a } ---- // TESTRESPONSE[s/"took" : 102/"took" : $body.$_path/] -// TESTRESPONSE[s/"_shards" : \.\.\. /"_shards" : $body.$_path/] \ No newline at end of file +// TESTRESPONSE[s/"_shards" : \.\.\. /"_shards" : $body.$_path/] diff --git a/docs/reference/rollup/rollup-getting-started.asciidoc b/docs/reference/rollup/rollup-getting-started.asciidoc index 7dcc36d29a9d..06bca3c9d486 100644 --- a/docs/reference/rollup/rollup-getting-started.asciidoc +++ b/docs/reference/rollup/rollup-getting-started.asciidoc @@ -149,7 +149,7 @@ is exactly as you'd expect. If you were to execute that query, you'd receive a result that looks like a normal aggregation response: -[source,js] +[source,console-result] ---- { "took" : 102, @@ -223,7 +223,7 @@ GET /sensor_rollup/_rollup_search Which returns a corresponding response: -[source,js] +[source,console-result] ---- { "took" : 93, diff --git a/docs/reference/rollup/rollup-search-limitations.asciidoc b/docs/reference/rollup/rollup-search-limitations.asciidoc index 81cb162bd3c1..5b53cf615e11 100644 --- a/docs/reference/rollup/rollup-search-limitations.asciidoc +++ b/docs/reference/rollup/rollup-search-limitations.asciidoc @@ -61,7 +61,7 @@ GET sensor_rollup/_rollup_search The response will tell you that the field and aggregation were not possible, because no rollup jobs were found which contained them: -[source,js] +[source,console-result] ---- { "error" : { diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index fa0c4d64d2ea..331cf70685ee 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -7,7 +7,7 @@ The query can either be provided using a simple query string as a parameter, or using the <> defined within the request body. Here is an example: -[source,js] +[source,console] -------------------------------------------------- PUT /twitter/_doc/1?refresh { @@ -23,7 +23,6 @@ GET /twitter/_count } } -------------------------------------------------- -//CONSOLE NOTE: The query being sent in the body must be nested in a `query` key, same as the <> works diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 44628133f6bf..2d5ceb674540 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -36,7 +36,7 @@ for the search This will yield the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 25, @@ -175,7 +175,7 @@ to more complex examples. First, the overall structure of the profile response is as follows: -[source,js] +[source,console-result] -------------------------------------------------- { "profile": { @@ -251,7 +251,7 @@ The overall structure of this query tree will resemble your original Elasticsear (or sometimes very) different. It will also use similar but not always identical naming. Using our previous `match` query example, let's analyze the `query` section: -[source,js] +[source,console-result] -------------------------------------------------- "query": [ { @@ -300,7 +300,7 @@ breakdown, etc). Children are allowed to have their own children. The `breakdown` component lists detailed timing statistics about low-level Lucene execution: -[source,js] +[source,console-result] -------------------------------------------------- "breakdown": { "score": 51306, @@ -410,7 +410,7 @@ filters, etc. Looking at the previous example: -[source,js] +[source,console-result] -------------------------------------------------- "collector": [ { @@ -554,7 +554,7 @@ This example has: And the response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -746,7 +746,7 @@ GET /twitter/_search This yields the following aggregation profile output: -[source,js] +[source,console-result] -------------------------------------------------- { "profile" : { diff --git a/docs/reference/search/request-body.asciidoc b/docs/reference/search/request-body.asciidoc index b84d695e9f10..9d34145232de 100644 --- a/docs/reference/search/request-body.asciidoc +++ b/docs/reference/search/request-body.asciidoc @@ -104,7 +104,7 @@ GET /twitter/_search The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 1, @@ -170,7 +170,7 @@ as many documents matching the query when it was early terminated. Also if the query was terminated early, the `terminated_early` flag will be set to `true` in the response. -[source,js] +[source,console-result] -------------------------------------------------- { "took": 3, diff --git a/docs/reference/search/request/highlighting.asciidoc b/docs/reference/search/request/highlighting.asciidoc index 58448f246f61..a800bebf7d44 100644 --- a/docs/reference/search/request/highlighting.asciidoc +++ b/docs/reference/search/request/highlighting.asciidoc @@ -851,7 +851,7 @@ GET twitter/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... @@ -910,7 +910,7 @@ GET twitter/_search Response: -[source,js] +[source,console-result] -------------------------------------------------- { ... diff --git a/docs/reference/search/request/inner-hits.asciidoc b/docs/reference/search/request/inner-hits.asciidoc index 8aef784e69d5..8e3641f95250 100644 --- a/docs/reference/search/request/inner-hits.asciidoc +++ b/docs/reference/search/request/inner-hits.asciidoc @@ -130,7 +130,7 @@ POST test/_search An example of a response snippet that could be generated from the above search request: -[source,js] +[source,console-result] -------------------------------------------------- { ..., @@ -259,7 +259,7 @@ POST test/_search Response not included in text but tested for completeness sake. -[source,js] +[source,console-result] -------------------------------------------------- { ..., @@ -378,7 +378,7 @@ POST test/_search Which would look like: -[source,js] +[source,console-result] -------------------------------------------------- { ..., @@ -493,7 +493,7 @@ POST test/_search An example of a response snippet that could be generated from the above search request: -[source,js] +[source,console-result] -------------------------------------------------- { ..., diff --git a/docs/reference/search/request/track-total-hits.asciidoc b/docs/reference/search/request/track-total-hits.asciidoc index 24b1f1f39acf..9635d8f4a46f 100644 --- a/docs/reference/search/request/track-total-hits.asciidoc +++ b/docs/reference/search/request/track-total-hits.asciidoc @@ -36,7 +36,7 @@ GET twitter/_search \... returns: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... @@ -86,7 +86,7 @@ bound of the total (`"gte"`). For instance the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... @@ -157,7 +157,7 @@ GET twitter/_search \... returns: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... diff --git a/docs/reference/search/search-shards.asciidoc b/docs/reference/search/search-shards.asciidoc index 288f8b7545ec..5d9001055257 100644 --- a/docs/reference/search/search-shards.asciidoc +++ b/docs/reference/search/search-shards.asciidoc @@ -64,7 +64,7 @@ GET /twitter/_search_shards The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes": ..., @@ -149,7 +149,7 @@ GET /twitter/_search_shards?routing=foo,bar The API returns the following result: -[source,js] +[source,console-result] -------------------------------------------------- { "nodes": ..., diff --git a/docs/reference/search/suggesters.asciidoc b/docs/reference/search/suggesters.asciidoc index 78aa5d362856..e9b7d8c92586 100644 --- a/docs/reference/search/suggesters.asciidoc +++ b/docs/reference/search/suggesters.asciidoc @@ -68,7 +68,7 @@ entries. Each entry is effectively a token from the suggest text and contains the suggestion entry text, the original start offset and length in the suggest text and if found an arbitrary number of options. -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... diff --git a/docs/reference/search/suggesters/completion-suggest.asciidoc b/docs/reference/search/suggesters/completion-suggest.asciidoc index 1090eb6652dc..86a1701719df 100644 --- a/docs/reference/search/suggesters/completion-suggest.asciidoc +++ b/docs/reference/search/suggesters/completion-suggest.asciidoc @@ -171,7 +171,7 @@ POST music/_search?pretty returns this response: -[source,js] +[source,console-result] -------------------------------------------------- { "_shards" : { @@ -243,7 +243,7 @@ POST music/_search Which should look like: -[source,js] +[source,console-result] -------------------------------------------------- { "took": 6, diff --git a/docs/reference/search/suggesters/misc.asciidoc b/docs/reference/search/suggesters/misc.asciidoc index 075b2a653233..104cd51c3fd5 100644 --- a/docs/reference/search/suggesters/misc.asciidoc +++ b/docs/reference/search/suggesters/misc.asciidoc @@ -30,7 +30,7 @@ POST _search?typed_keys In the response, the suggester names will be changed to respectively `term#my-first-suggester` and `phrase#my-second-suggester`, reflecting the types of each suggestion: -[source,js] +[source,console-result] -------------------------------------------------- { "suggest": { diff --git a/docs/reference/search/suggesters/phrase-suggest.asciidoc b/docs/reference/search/suggesters/phrase-suggest.asciidoc index 9d49fc16bd00..dd93f084c0e5 100644 --- a/docs/reference/search/suggesters/phrase-suggest.asciidoc +++ b/docs/reference/search/suggesters/phrase-suggest.asciidoc @@ -106,7 +106,7 @@ POST test/_search The response contains suggestions scored by the most likely spell correction first. In this case we received the expected correction "nobel prize". -[source,js] +[source,console-result] -------------------------------------------------- { "_shards": ... diff --git a/docs/reference/search/uri-request.asciidoc b/docs/reference/search/uri-request.asciidoc index 5e0a1fc8b7ba..1c523fae2a91 100644 --- a/docs/reference/search/uri-request.asciidoc +++ b/docs/reference/search/uri-request.asciidoc @@ -118,7 +118,7 @@ GET twitter/_search?q=user:kimchy The API returns the following response: -[source,js] +[source,console-result] -------------------------------------------------- { "timed_out": false, diff --git a/docs/reference/search/validate.asciidoc b/docs/reference/search/validate.asciidoc index 4f6d948fd3ea..1b77fde008cc 100644 --- a/docs/reference/search/validate.asciidoc +++ b/docs/reference/search/validate.asciidoc @@ -118,7 +118,7 @@ GET twitter/_validate/query?explain=true responds with: -[source,js] +[source,console-result] -------------------------------------------------- { "valid" : false, diff --git a/docs/reference/setup/logging-config.asciidoc b/docs/reference/setup/logging-config.asciidoc index 29a125ec4722..4cc2615d214e 100644 --- a/docs/reference/setup/logging-config.asciidoc +++ b/docs/reference/setup/logging-config.asciidoc @@ -215,6 +215,7 @@ log level to `error`. You can identify what is triggering deprecated functionality if `X-Opaque-Id` was used as an HTTP header. The user ID is included in the `X-Opaque-ID` field in deprecation JSON logs. + [source,js] --------------------------- { diff --git a/docs/reference/sql/endpoints/rest.asciidoc b/docs/reference/sql/endpoints/rest.asciidoc index 650b834c4620..0bc98e34c0d7 100644 --- a/docs/reference/sql/endpoints/rest.asciidoc +++ b/docs/reference/sql/endpoints/rest.asciidoc @@ -143,7 +143,7 @@ POST /_sql?format=json Which returns: -[source,js] +[source,console-result] -------------------------------------------------- { "columns": [ @@ -285,7 +285,7 @@ POST /_sql?format=json Which looks like: -[source,js] +[source,console-result] -------------------------------------------------- { "rows" : [ @@ -388,7 +388,7 @@ POST /_sql?format=json Which returns: -[source,js] +[source,console-result] -------------------------------------------------- { "columns": [ @@ -424,7 +424,7 @@ POST /_sql?format=json Which looks like: -[source,js] +[source,console-result] -------------------------------------------------- { "values": [ diff --git a/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc index 742b17c0c998..2a60e09f65ce 100644 --- a/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc @@ -103,7 +103,7 @@ GET _watcher/watch/my_watch The action state of a newly-created watch is `awaits_successful_execution`: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -148,7 +148,7 @@ GET _watcher/watch/my_watch and the action is now in `ackable` state: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -199,7 +199,7 @@ GET _watcher/watch/my_watch // CONSOLE // TEST[continued] -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -266,7 +266,7 @@ POST _watcher/watch/my_watch/_ack The response looks like a get watch response, but only contains the status: -[source,js] +[source,console-result] -------------------------------------------------- { "status": { diff --git a/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc index aa78af4439b1..a2cd3fb17d6b 100644 --- a/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/activate-watch.asciidoc @@ -54,7 +54,7 @@ GET _watcher/watch/my_watch // CONSOLE // TEST[setup:my_inactive_watch] -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -89,7 +89,7 @@ PUT _watcher/watch/my_watch/_activate The new state of the watch is returned as part of its overall status: -[source,js] +[source,console-result] -------------------------------------------------- { "status": { diff --git a/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc index 6023d020a4a6..763020d00124 100644 --- a/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/deactivate-watch.asciidoc @@ -54,7 +54,7 @@ GET _watcher/watch/my_watch // CONSOLE // TEST[setup:my_active_watch] -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -89,7 +89,7 @@ PUT _watcher/watch/my_watch/_deactivate The new state of the watch is returned as part of its overall status: -[source,js] +[source,console-result] -------------------------------------------------- { "status": { diff --git a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc index 42df82ddc886..e02e839b5501 100644 --- a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc @@ -186,7 +186,7 @@ POST _watcher/watch/my_watch/_execute This is an example of the output: -[source,js] +[source,console-result] -------------------------------------------------- { "_id": "my_watch_0-2015-06-02T23:17:55.124Z", <1> diff --git a/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc index b24d6be70e70..5b5f43043ec7 100644 --- a/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc @@ -54,7 +54,7 @@ GET _watcher/watch/my_watch Response: -[source,js] +[source,console-result] -------------------------------------------------- { "found": true, @@ -106,4 +106,5 @@ Response: } -------------------------------------------------- // TESTRESPONSE[s/"timestamp": "2015-05-26T18:21:08.630Z"/"timestamp": "$body.status.state.timestamp"/] + <1> The current status of the watch From 8d1ad42625630168fec5e7ba2e4f712e87941ca8 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 6 Sep 2019 14:10:36 -0400 Subject: [PATCH 097/103] [DOCS] Correct definition for `allow_no_indices` parameter (#46450) --- docs/reference/rest-api/common-parms.asciidoc | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index dbf7928d8514..3e860e58557c 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -12,10 +12,13 @@ end::aliases[] tag::allow-no-indices[] `allow_no_indices`:: -(Optional, boolean) If `true`, the request returns an error if a wildcard -expression or `_all` value retrieves only missing or closed indices. This -parameter also applies to <> that point to a -missing or closed index. +(Optional, boolean) If `true`, +the request does *not* return an error +if a wildcard expression +or `_all` value retrieves only missing or closed indices. ++ +This parameter also applies to <> +that point to a missing or closed index. end::allow-no-indices[] tag::analyzer[] From 05f5d1155d25dadc5eeb9fa12f2c1f74d72aa133 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 6 Sep 2019 20:22:26 +0200 Subject: [PATCH 098/103] Increase REST-Test Client Timeout to 60s (#46455) We are seeing requests take more than the default 30s which leads to requests being retried and returning unexpected failures like e.g. "index already exists" because the initial requests that timed out, worked out functionally anyway. => double the timeout to reduce the likelihood of the failures described in #46091 => As suggested in the issue, we should in a follow-up turn off retrying all-together probably --- .../java/org/elasticsearch/test/rest/ESRestTestCase.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 90a0a656142a..9e2bdbab1e56 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -78,6 +78,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.TimeUnit; @@ -837,11 +838,9 @@ public abstract class ESRestTestCase extends ESTestCase { } builder.setDefaultHeaders(defaultHeaders); } - final String socketTimeoutString = settings.get(CLIENT_SOCKET_TIMEOUT); - if (socketTimeoutString != null) { - final TimeValue socketTimeout = TimeValue.parseTimeValue(socketTimeoutString, CLIENT_SOCKET_TIMEOUT); - builder.setRequestConfigCallback(conf -> conf.setSocketTimeout(Math.toIntExact(socketTimeout.getMillis()))); - } + final String socketTimeoutString = Objects.requireNonNullElse(settings.get(CLIENT_SOCKET_TIMEOUT), "60s"); + final TimeValue socketTimeout = TimeValue.parseTimeValue(socketTimeoutString, CLIENT_SOCKET_TIMEOUT); + builder.setRequestConfigCallback(conf -> conf.setSocketTimeout(Math.toIntExact(socketTimeout.getMillis()))); if (settings.hasValue(CLIENT_PATH_PREFIX)) { builder.setPathPrefix(settings.get(CLIENT_PATH_PREFIX)); } From 5a82389c123bd555e5a22319a29adc7ec36bc71f Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 6 Sep 2019 16:46:41 -0400 Subject: [PATCH 099/103] [DOCS] Remove cat request from Index Segments API requests (#46463) --- docs/reference/indices/segments.asciidoc | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/reference/indices/segments.asciidoc b/docs/reference/indices/segments.asciidoc index 5500fba2d9f4..3bdeaa8b648f 100644 --- a/docs/reference/indices/segments.asciidoc +++ b/docs/reference/indices/segments.asciidoc @@ -21,8 +21,6 @@ GET /twitter/_segments `GET /_segments` -`GET /_cat/segments/` - [[index-segments-api-path-params]] ==== {api-path-parms-title} From cb92828e8ec9d9d241bd6189e5835fd99f8b9a44 Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Sat, 7 Sep 2019 09:13:42 +0300 Subject: [PATCH 100/103] SQL: fix scripting for grouped by datetime functions (#46421) * Fix issue with painless scripting not being correctly generated when datetime functions are used for GROUPing of an INTERVAL operation. --- .../main/resources/datetime-interval.csv-spec | 94 +++++++++++++++++++ .../scalar/datetime/DateTimeFunction.java | 1 - .../datetime/NamedDateTimeFunction.java | 20 ++-- .../datetime/NonIsoDateTimeFunction.java | 20 ++-- .../function/scalar/datetime/Quarter.java | 18 ++-- .../sql/planner/QueryTranslatorTests.java | 35 +++++++ 6 files changed, 156 insertions(+), 32 deletions(-) diff --git a/x-pack/plugin/sql/qa/src/main/resources/datetime-interval.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/datetime-interval.csv-spec index bfb28775bc3b..b5ca63b6781c 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/datetime-interval.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/datetime-interval.csv-spec @@ -309,3 +309,97 @@ SELECT birth_date, MAX(hire_date) - INTERVAL 1 YEAR AS f FROM test_emp GROUP BY 1952-06-13T00:00:00Z|1953 1952-07-08T00:00:00Z|1953 ; + +monthOfDatePlusInterval_And_GroupBy +SELECT WEEK_OF_YEAR(birth_date + INTERVAL 25 YEAR) x, COUNT(*) c FROM test_emp GROUP BY x HAVING c >= 3 ORDER BY c DESC; + + x:i | c:l +---------------+--------------- +null |10 +22 |6 +4 |4 +16 |4 +30 |4 +40 |4 +45 |4 +1 |3 +8 |3 +21 |3 +28 |3 +32 |3 +37 |3 +; + +dayOfWeekPlusInterval_And_GroupBy +SELECT DOW(birth_date + INTERVAL 5 YEAR) x, COUNT(*) c FROM test_emp GROUP BY x HAVING c >= 3 ORDER BY c DESC; + + x:i | c:l +---------------+--------------- +2 |18 +3 |15 +5 |15 +4 |12 +6 |12 +null |10 +7 |10 +1 |8 +; + +dayNamePlusInterval_And_GroupBy +SELECT DAY_NAME(birth_date + INTERVAL 5 YEAR) x, COUNT(*) c FROM test_emp GROUP BY x HAVING c >= 10 ORDER BY c DESC; + + x:s | c:l +---------------+--------------- +Monday |18 +Thursday |15 +Tuesday |15 +Friday |12 +Wednesday |12 +null |10 +Saturday |10 +; + +monthNamePlusInterval_And_GroupBy +SELECT MONTH_NAME(birth_date + INTERVAL 5 YEAR) x, COUNT(*) c FROM test_emp GROUP BY x HAVING c >= 5 ORDER BY c DESC; + + x:s | c:l +---------------+--------------- +null |10 +May |10 +September |10 +July |9 +October |9 +April |8 +February |8 +November |8 +December |7 +June |7 +August |6 +January |6 +; + +quarterPlusInterval_And_GroupBy +SELECT QUARTER(birth_date + INTERVAL 5 YEAR) x, COUNT(*) c FROM test_emp GROUP BY x HAVING c >= 5 ORDER BY x DESC; + + x:i | c:l +---------------+--------------- +4 |24 +3 |25 +2 |25 +1 |16 +null |10 +; + +dayOfMonthPlusInterval_And_GroupBy +SELECT DOM(birth_date + INTERVAL 5 YEAR) x, COUNT(*) c FROM test_emp GROUP BY x HAVING c >= 5 ORDER BY x DESC; + + x:i | c:l +---------------+--------------- +25 |5 +23 |6 +21 |5 +19 |7 +7 |5 +1 |5 +null |10 +; \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DateTimeFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DateTimeFunction.java index d314056ea64e..f4cccb9e7fd5 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DateTimeFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DateTimeFunction.java @@ -49,7 +49,6 @@ public abstract class DateTimeFunction extends BaseDateTimeFunction { .variable(extractor.chronoField().name()); return new ScriptTemplate(template, params.build(), dataType()); - } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NamedDateTimeFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NamedDateTimeFunction.java index 35397df5ef4a..0dada3f3c0e1 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NamedDateTimeFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NamedDateTimeFunction.java @@ -6,18 +6,16 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.FieldAttribute; import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.NamedDateTimeProcessor.NameExtractor; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; +import org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.util.StringUtils; import java.time.ZoneId; -import java.util.Locale; -import static java.lang.String.format; import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder; /* @@ -33,14 +31,14 @@ abstract class NamedDateTimeFunction extends BaseDateTimeFunction { } @Override - public ScriptTemplate scriptWithField(FieldAttribute field) { - return new ScriptTemplate( - formatTemplate(format(Locale.ROOT, "{sql}.%s(doc[{}].value, {})", - StringUtils.underscoreToLowerCamelCase(nameExtractor.name()))), - paramsBuilder() - .variable(field.name()) - .variable(zoneId().getId()).build(), - dataType()); + public ScriptTemplate asScript() { + ScriptTemplate script = super.asScript(); + String template = formatTemplate("{sql}." + StringUtils.underscoreToLowerCamelCase(nameExtractor.name()) + + "(" + script.template() + ", {})"); + + ParamsBuilder params = paramsBuilder().script(script.params()).variable(zoneId().getId()); + + return new ScriptTemplate(template, params.build(), dataType()); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NonIsoDateTimeFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NonIsoDateTimeFunction.java index 1aee57ae80bc..576ed6bc9ab9 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NonIsoDateTimeFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/NonIsoDateTimeFunction.java @@ -6,18 +6,16 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.FieldAttribute; import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.NonIsoDateTimeProcessor.NonIsoDateTimeExtractor; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; +import org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.util.StringUtils; import java.time.ZoneId; -import java.util.Locale; -import static java.lang.String.format; import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder; /* @@ -33,14 +31,14 @@ abstract class NonIsoDateTimeFunction extends BaseDateTimeFunction { } @Override - public ScriptTemplate scriptWithField(FieldAttribute field) { - return new ScriptTemplate( - formatTemplate(format(Locale.ROOT, "{sql}.%s(doc[{}].value, {})", - StringUtils.underscoreToLowerCamelCase(extractor.name()))), - paramsBuilder() - .variable(field.name()) - .variable(zoneId().getId()).build(), - dataType()); + public ScriptTemplate asScript() { + ScriptTemplate script = super.asScript(); + String template = formatTemplate("{sql}." + StringUtils.underscoreToLowerCamelCase(extractor.name()) + + "(" + script.template() + ", {})"); + + ParamsBuilder params = paramsBuilder().script(script.params()).variable(zoneId().getId()); + + return new ScriptTemplate(template, params.build(), dataType()); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/Quarter.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/Quarter.java index 275e7181bc31..f23082c7e035 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/Quarter.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/Quarter.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.FieldAttribute; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; +import org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; import org.elasticsearch.xpack.sql.tree.NodeInfo.NodeCtor2; import org.elasticsearch.xpack.sql.tree.Source; @@ -23,15 +23,15 @@ public class Quarter extends BaseDateTimeFunction { public Quarter(Source source, Expression field, ZoneId zoneId) { super(source, field, zoneId); } - + @Override - public ScriptTemplate scriptWithField(FieldAttribute field) { - return new ScriptTemplate(formatTemplate("{sql}.quarter(doc[{}].value, {})"), - paramsBuilder() - .variable(field.name()) - .variable(zoneId().getId()) - .build(), - dataType()); + public ScriptTemplate asScript() { + ScriptTemplate script = super.asScript(); + String template = formatTemplate("{sql}.quarter(" + script.template() + ", {})"); + + ParamsBuilder params = paramsBuilder().script(script.params()).variable(zoneId().getId()); + + return new ScriptTemplate(template, params.build(), dataType()); } @Override diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java index adc8fd60af28..80d9202d5bfe 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry; import org.elasticsearch.xpack.sql.expression.function.grouping.Histogram; import org.elasticsearch.xpack.sql.expression.function.scalar.Cast; +import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeProcessor.DateTimeExtractor; import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation; import org.elasticsearch.xpack.sql.expression.function.scalar.math.Round; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; @@ -1160,4 +1161,38 @@ public class QueryTranslatorTests extends ESTestCase { + "\"lang\":\"painless\"," + "\"params\":{\"v0\":\"date\",\"v1\":\"P1Y\",\"v2\":\"INTERVAL_YEAR\",\"v3\":\"2019-03-11T12:34:56.000Z\"}},")); } + + public void testChronoFieldBasedDateTimeFunctionsWithMathIntervalAndGroupBy() { + DateTimeExtractor randomFunction = randomValueOtherThan(DateTimeExtractor.YEAR, () -> randomFrom(DateTimeExtractor.values())); + PhysicalPlan p = optimizeAndPlan( + "SELECT " + + randomFunction.name() + + "(date + INTERVAL 1 YEAR) FROM test GROUP BY " + randomFunction.name() + "(date + INTERVAL 1 YEAR)"); + assertEquals(EsQueryExec.class, p.getClass()); + EsQueryExec eqe = (EsQueryExec) p; + assertThat(eqe.queryContainer().toString().replaceAll("\\s+", ""), containsString( + "{\"terms\":{\"script\":{\"source\":\"InternalSqlScriptUtils.dateTimeChrono(" + + "InternalSqlScriptUtils.add(InternalSqlScriptUtils.docValue(doc,params.v0)," + + "InternalSqlScriptUtils.intervalYearMonth(params.v1,params.v2)),params.v3,params.v4)\"," + + "\"lang\":\"painless\",\"params\":{\"v0\":\"date\",\"v1\":\"P1Y\",\"v2\":\"INTERVAL_YEAR\"," + + "\"v3\":\"Z\",\"v4\":\"" + randomFunction.chronoField().name() + "\"}},\"missing_bucket\":true," + + "\"value_type\":\"long\",\"order\":\"asc\"}}}]}}}}")); + } + + public void testDateTimeFunctionsWithMathIntervalAndGroupBy() { + String[] functions = new String[] {"DAY_NAME", "MONTH_NAME", "DAY_OF_WEEK", "WEEK_OF_YEAR", "QUARTER"}; + String[] scriptMethods = new String[] {"dayName", "monthName", "dayOfWeek", "weekOfYear", "quarter"}; + int pos = randomIntBetween(0, functions.length - 1); + PhysicalPlan p = optimizeAndPlan( + "SELECT " + + functions[pos] + + "(date + INTERVAL 1 YEAR) FROM test GROUP BY " + functions[pos] + "(date + INTERVAL 1 YEAR)"); + assertEquals(EsQueryExec.class, p.getClass()); + EsQueryExec eqe = (EsQueryExec) p; + assertThat(eqe.queryContainer().toString().replaceAll("\\s+", ""), containsString( + "{\"terms\":{\"script\":{\"source\":\"InternalSqlScriptUtils." + scriptMethods[pos] + + "(InternalSqlScriptUtils.add(InternalSqlScriptUtils.docValue(doc,params.v0)," + + "InternalSqlScriptUtils.intervalYearMonth(params.v1,params.v2)),params.v3)\",\"lang\":\"painless\"," + + "\"params\":{\"v0\":\"date\",\"v1\":\"P1Y\",\"v2\":\"INTERVAL_YEAR\",\"v3\":\"Z\"}},\"missing_bucket\":true,")); + } } From a6152f42a47a1ccd668e5892778c8bd2d3a78c4c Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Sat, 7 Sep 2019 09:14:02 +0300 Subject: [PATCH 101/103] Use `null` schema response for `SYS TABLES` command. (#46386) --- .../qa/src/main/resources/setup_mock_metadata_get_tables.sql | 4 ++-- .../main/resources/setup_mock_metadata_get_types_of_table.sql | 4 ++-- .../xpack/sql/plan/logical/command/sys/SysTables.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_tables.sql b/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_tables.sql index db40c6b90865..1f11c085ee06 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_tables.sql +++ b/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_tables.sql @@ -9,7 +9,7 @@ CREATE TABLE mock ( SELF_REFERENCING_COL_NAME VARCHAR, REF_GENERATION VARCHAR ) AS -SELECT '', 'test1', 'BASE TABLE', '', null, null, null, null, null FROM DUAL +SELECT null, 'test1', 'BASE TABLE', '', null, null, null, null, null FROM DUAL UNION ALL -SELECT '', 'test2', 'BASE TABLE', '', null, null, null, null, null FROM DUAL +SELECT null, 'test2', 'BASE TABLE', '', null, null, null, null, null FROM DUAL ; diff --git a/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_types_of_table.sql b/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_types_of_table.sql index db40c6b90865..1f11c085ee06 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_types_of_table.sql +++ b/x-pack/plugin/sql/qa/src/main/resources/setup_mock_metadata_get_types_of_table.sql @@ -9,7 +9,7 @@ CREATE TABLE mock ( SELF_REFERENCING_COL_NAME VARCHAR, REF_GENERATION VARCHAR ) AS -SELECT '', 'test1', 'BASE TABLE', '', null, null, null, null, null FROM DUAL +SELECT null, 'test1', 'BASE TABLE', '', null, null, null, null, null FROM DUAL UNION ALL -SELECT '', 'test2', 'BASE TABLE', '', null, null, null, null, null FROM DUAL +SELECT null, 'test2', 'BASE TABLE', '', null, null, null, null, null FROM DUAL ; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTables.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTables.java index a3b8f1817415..190224c583c0 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTables.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTables.java @@ -146,7 +146,7 @@ public class SysTables extends Command { .sorted(Comparator. comparing(i -> legacyName(i.type())) .thenComparing(Comparator.comparing(i -> i.name()))) .map(t -> asList(cluster, - EMPTY, + null, t.name(), legacyName(t.type()), EMPTY, From 191f3e1b315618ee6f9913f8e51d11e953a5b9f4 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 7 Sep 2019 10:44:25 -0400 Subject: [PATCH 102/103] Ignore replication for noop updates (#46458) Previously, we ignore replication for noop updates because they do not have sequence numbers. Since #44603, we started assigning sequence numbers to noop updates leading them to be replicated to replicas. This bug occurs only on 8.0 for it requires #41065 and #44603. Closes #46366 --- .../action/bulk/TransportShardBulkAction.java | 30 +++++++++---------- .../action/bulk/BulkWithUpdatesIT.java | 29 ++++++++++++++++++ 2 files changed, 43 insertions(+), 16 deletions(-) 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 e761c10ead14..a2fae5f5bbca 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -393,22 +393,20 @@ public class TransportShardBulkAction extends TransportWriteAction docWriteRequest = item.request(); + final BulkItemRequest item = request.items()[i]; final BulkItemResponse response = item.getPrimaryResponse(); - final BulkItemResponse.Failure failure = response.getFailure(); - final DocWriteResponse writeResponse = response.getResponse(); - final long seqNum = failure == null ? writeResponse.getSeqNo() : failure.getSeqNo(); - if (seqNum == SequenceNumbers.UNASSIGNED_SEQ_NO) { - assert failure != null || writeResponse.getResult() == DocWriteResponse.Result.NOOP - || writeResponse.getResult() == DocWriteResponse.Result.NOT_FOUND; - continue; - } - if (failure == null) { - operationResult = performOpOnReplica(writeResponse, docWriteRequest, replica); + final Engine.Result operationResult; + if (item.getPrimaryResponse().isFailed()) { + if (response.getFailure().getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO) { + continue; // ignore replication as we didn't generate a sequence number for this request. + } + operationResult = replica.markSeqNoAsNoop(response.getFailure().getSeqNo(), response.getFailure().getMessage()); } else { - operationResult = replica.markSeqNoAsNoop(seqNum, failure.getMessage()); + if (response.getResponse().getResult() == DocWriteResponse.Result.NOOP) { + continue; // ignore replication as it's a noop + } + assert response.getResponse().getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO; + operationResult = performOpOnReplica(response.getResponse(), item.request(), replica); } assert operationResult != null : "operation result must never be null when primary response has no failure"; location = syncOperationResultOrThrow(operationResult, location); @@ -435,8 +433,8 @@ public class TransportShardBulkAction extends TransportWriteAction Date: Tue, 3 Sep 2019 12:31:24 -0400 Subject: [PATCH 103/103] Strengthen testUpdate in rolling upgrade We hit a bug where we can't partially update documents created in a mixed cluster between 5.x and 6.x. Although this bug does not affect 7.0 or later, we should have a good test that catches this issue. Relates #46198 --- .../elasticsearch/upgrades/RecoveryIT.java | 30 +++++++++++++++---- 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java index df38fab7787d..858a2d606706 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java @@ -41,6 +41,7 @@ import org.hamcrest.Matchers; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -653,13 +654,30 @@ public class RecoveryIT extends AbstractRollingTestCase { .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 2); createIndex(index, settings.build()); + indexDocs(index, 0, 100); } - ensureGreen(index); - indexDocs(index, 0, 10); - for (int i = 0; i < 10; i++) { - Request update = new Request("POST", index + "/_update/" + i); - update.setJsonEntity("{\"doc\": {\"f\": " + randomNonNegativeLong() + "}}"); - client().performRequest(update); + if (randomBoolean()) { + ensureGreen(index); + } + Map updates = new HashMap<>(); + for (int docId = 0; docId < 100; docId++) { + final int times = randomIntBetween(0, 2); + for (int i = 0; i < times; i++) { + long value = randomNonNegativeLong(); + Request update = new Request("POST", index + "/_update/" + docId); + update.setJsonEntity("{\"doc\": {\"updated_field\": " + value + "}}"); + client().performRequest(update); + updates.put(docId, value); + } + } + client().performRequest(new Request("POST", index + "/_refresh")); + for (int docId : updates.keySet()) { + Request get = new Request("GET", index + "/_doc/" + docId); + Map doc = entityAsMap(client().performRequest(get)); + assertThat(XContentMapValues.extractValue("_source.updated_field", doc), equalTo(updates.get(docId))); + } + if (randomBoolean()) { + syncedFlush(index); } }