From 77f1ac655e1da9b37d4c44f33774c32872e5475f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 11 Apr 2019 18:03:50 +0200 Subject: [PATCH 001/260] Simplify Snapshot Resiliency Test (#40930) * Thanks to #39793 dynamic mapping updates don't contain blocking operations anymore so we don't have to manually put the mapping in this test and can keep it a little simpler --- .../snapshots/SnapshotResiliencyTests.java | 42 ++++++++----------- 1 file changed, 18 insertions(+), 24 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 4f179968a16ec..208dabff531b2 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -47,7 +47,6 @@ import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.admin.indices.delete.TransportDeleteIndexAction; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingAction; -import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.admin.indices.mapping.put.TransportPutMappingAction; import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresAction; import org.elasticsearch.action.admin.indices.shards.TransportIndicesShardStoresAction; @@ -260,29 +259,24 @@ public void testSuccessfulSnapshotAndRestore() { ))); })); final AtomicInteger countdown = new AtomicInteger(documents); - masterNode.client.admin().indices().putMapping( - new PutMappingRequest(index).type("_doc").source("foo", "type=text"), - assertNoFailureListener(r -> { - for (int i = 0; i < documents; ++i) { - masterNode.client.bulk( - new BulkRequest().add(new IndexRequest(index).source( - Collections.singletonMap("foo", "bar" + i))) - .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), - assertNoFailureListener( - bulkResponse -> { - assertFalse( - "Failures in bulkresponse: " + bulkResponse.buildFailureMessage(), - bulkResponse.hasFailures()); - if (countdown.decrementAndGet() == 0) { - afterIndexing.run(); - } - })); - } - if (documents == 0) { - afterIndexing.run(); - } - } - )); + for (int i = 0; i < documents; ++i) { + masterNode.client.bulk( + new BulkRequest().add(new IndexRequest(index).source( + Collections.singletonMap("foo", "bar" + i))) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), + assertNoFailureListener( + bulkResponse -> { + assertFalse( + "Failures in bulkresponse: " + bulkResponse.buildFailureMessage(), + bulkResponse.hasFailures()); + if (countdown.decrementAndGet() == 0) { + afterIndexing.run(); + } + })); + } + if (documents == 0) { + afterIndexing.run(); + } })))); runUntil(documentCountVerified::get, TimeUnit.MINUTES.toMillis(5L)); assertTrue(createdSnapshot.get()); From 36c29d3b13d0eca984ffbc965ccf6bdec275f4e1 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 11 Apr 2019 13:54:22 -0500 Subject: [PATCH 002/260] [ML] checking if p-tasks metadata is null before updating state (#41091) (#41124) * [ML] checking if p-tasks metadata is null before updating state * Adding test that validates fix * removing debug println --- .../org/elasticsearch/xpack/ml/MlConfigMigrator.java | 12 ++++++++---- .../xpack/ml/integration/MlConfigMigratorIT.java | 12 +++++++++++- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java index a59769f97ce78..d1673dd3c914c 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java @@ -246,10 +246,14 @@ public ClusterState execute(ClusterState currentState) { currentState.metaData().custom(PersistentTasksCustomMetaData.TYPE), currentState.nodes()); ClusterState.Builder newState = ClusterState.builder(currentState); - newState.metaData(MetaData.builder(currentState.getMetaData()) - .putCustom(MlMetadata.TYPE, removed.mlMetadata) - .putCustom(PersistentTasksCustomMetaData.TYPE, updatedTasks) - .build()); + MetaData.Builder metaDataBuilder = MetaData.builder(currentState.getMetaData()) + .putCustom(MlMetadata.TYPE, removed.mlMetadata); + + // If there are no tasks in the cluster state metadata to begin with, this could be null. + if (updatedTasks != null) { + metaDataBuilder = metaDataBuilder.putCustom(PersistentTasksCustomMetaData.TYPE, updatedTasks); + } + newState.metaData(metaDataBuilder.build()); return newState.build(); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 0eda4d4dcad58..69b82ef984671 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.ml.integration; +import com.carrotsearch.hppc.cursors.ObjectCursor; import org.elasticsearch.Version; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; @@ -47,10 +48,12 @@ import java.io.IOException; import java.io.InputStream; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -148,9 +151,13 @@ public void testMigrateConfigs() throws InterruptedException, IOException { .routingTable(routingTable.build()) .build(); when(clusterService.state()).thenReturn(clusterState); - + List customs = new ArrayList<>(); doAnswer(invocation -> { ClusterStateUpdateTask listener = (ClusterStateUpdateTask) invocation.getArguments()[1]; + ClusterState result = listener.execute(clusterState); + for (ObjectCursor value : result.metaData().customs().values()){ + customs.add(value.value); + } listener.clusterStateProcessed("source", mock(ClusterState.class), mock(ClusterState.class)); return null; }).when(clusterService).submitStateUpdateTask(eq("remove-migrated-ml-configs"), any()); @@ -164,6 +171,9 @@ public void testMigrateConfigs() throws InterruptedException, IOException { blockingCall(actionListener -> mlConfigMigrator.migrateConfigs(clusterState, actionListener), responseHolder, exceptionHolder); + // Verify that we have custom values in the new cluster state and that none of them is null + assertThat(customs.size(), greaterThan(0)); + assertThat(customs.stream().anyMatch(Objects::isNull), is(false)); assertNull(exceptionHolder.get()); assertTrue(responseHolder.get()); assertSnapshot(mlMetadata.build()); From 13454376a4729d9029db871205a493b782d8a6bb Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Thu, 11 Apr 2019 12:04:04 -0700 Subject: [PATCH 003/260] [DOCS] Fixes callout for Asciidoctor migration (#41127) --- .../aggregations/bucket/daterange-aggregation.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc b/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc index b49466b81807f..fdb6e07b09837 100644 --- a/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/daterange-aggregation.asciidoc @@ -278,7 +278,7 @@ POST /sales/_search?size=0 "time_zone": "CET", "ranges": [ { "to": "2016/02/01" }, <1> - { "from": "2016/02/01", "to" : "now/d" <2>}, + { "from": "2016/02/01", "to" : "now/d" }, <2> { "from": "now/d" } ] } From 3ec0cc5f8ed0567593bf1bc2352ff164c2de662e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Apr 2019 15:15:00 -0400 Subject: [PATCH 004/260] Do not trim unsafe commits when open readonly engine (#41041) Today we always trim unsafe commits (whose max_seq_no >= global checkpoint) before starting a read-write or read-only engine. This is mandatory for read-write engines because they must start with the safe commit. This is also fine for read-only engines since most of the cases we should have exactly one commit after closing an index (trimming is a noop). However, this is dangerous for following indices which might have more than one commits when they are being closed. With this change, we move the trimming logic to the ctor of InternalEngine so we won't trim anything if we are going to open a read-only engine. --- .../index/engine/InternalEngine.java | 13 ++++- .../index/engine/ReadOnlyEngine.java | 47 ++++++++++--------- .../elasticsearch/index/shard/IndexShard.java | 17 +------ .../index/engine/InternalEngineTests.java | 33 ------------- .../index/engine/ReadOnlyEngineTests.java | 21 ++------- .../index/shard/IndexShardTests.java | 38 +++++++++++++++ 6 files changed, 80 insertions(+), 89 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 8d20c80d09f64..654d31d22671a 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -78,6 +78,7 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogCorruptedException; @@ -87,6 +88,7 @@ import java.io.Closeable; import java.io.IOException; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; @@ -182,10 +184,10 @@ public InternalEngine(EngineConfig engineConfig) { boolean success = false; try { this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().relativeTimeInMillis(); - mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings()); throttle = new IndexThrottle(); try { + trimUnsafeCommits(engineConfig); translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier()); assert translog.getGeneration() != null; this.translog = translog; @@ -2745,4 +2747,13 @@ public void reinitializeMaxSeqNoOfUpdatesOrDeletes() { final long maxSeqNo = SequenceNumbers.max(localCheckpointTracker.getMaxSeqNo(), translog.getMaxSeqNo()); advanceMaxSeqNoOfUpdatesOrDeletes(maxSeqNo); } + + private static void trimUnsafeCommits(EngineConfig engineConfig) throws IOException { + final Store store = engineConfig.getStore(); + final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); + final Path translogPath = engineConfig.getTranslogConfig().getTranslogPath(); + final long globalCheckpoint = Translog.readGlobalCheckpoint(translogPath, translogUUID); + final long minRetainedTranslogGen = Translog.readMinTranslogGeneration(translogPath, translogUUID); + store.trimUnsafeCommits(globalCheckpoint, minRetainedTranslogGen, engineConfig.getIndexSettings().getIndexVersionCreated()); + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index 5e0b1e60036a8..b5618d5b9cbc4 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -30,7 +30,6 @@ import org.apache.lucene.search.SearcherManager; import org.apache.lucene.store.Directory; import org.apache.lucene.store.Lock; -import org.elasticsearch.Assertions; import org.elasticsearch.Version; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; @@ -102,23 +101,8 @@ public ReadOnlyEngine(EngineConfig config, SeqNoStats seqNoStats, TranslogStats this.lastCommittedSegmentInfos = Lucene.readSegmentInfos(directory); this.translogStats = translogStats == null ? new TranslogStats(0, 0, 0, 0, 0) : translogStats; if (seqNoStats == null) { - seqNoStats = buildSeqNoStats(lastCommittedSegmentInfos); - // Before 8.0 the global checkpoint is not known and up to date when the engine is created after - // peer recovery, so we only check the max seq no / global checkpoint coherency when the global - // checkpoint is different from the unassigned sequence number value. - // In addition to that we only execute the check if the index the engine belongs to has been - // created after the refactoring of the Close Index API and its TransportVerifyShardBeforeCloseAction - // that guarantee that all operations have been flushed to Lucene. - final long globalCheckpoint = engineConfig.getGlobalCheckpointSupplier().getAsLong(); - final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated(); - if (indexVersionCreated.onOrAfter(Version.V_8_0_0) || - (globalCheckpoint != SequenceNumbers.UNASSIGNED_SEQ_NO && indexVersionCreated.onOrAfter(Version.V_6_7_0))) { - if (seqNoStats.getMaxSeqNo() != globalCheckpoint) { - assertMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats.getMaxSeqNo(), globalCheckpoint); - throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo() - + "] from last commit does not match global checkpoint [" + globalCheckpoint + "]"); - } - } + seqNoStats = buildSeqNoStats(config, lastCommittedSegmentInfos); + ensureMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats); } this.seqNoStats = seqNoStats; this.indexCommit = Lucene.getIndexCommit(lastCommittedSegmentInfos, directory); @@ -138,10 +122,27 @@ public ReadOnlyEngine(EngineConfig config, SeqNoStats seqNoStats, TranslogStats } } - protected void assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) { - if (Assertions.ENABLED) { - assert false : "max seq. no. [" + maxSeqNo + "] does not match [" + globalCheckpoint + "]"; + protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(final SeqNoStats seqNoStats) { + // Before 8.0 the global checkpoint is not known and up to date when the engine is created after + // peer recovery, so we only check the max seq no / global checkpoint coherency when the global + // checkpoint is different from the unassigned sequence number value. + // In addition to that we only execute the check if the index the engine belongs to has been + // created after the refactoring of the Close Index API and its TransportVerifyShardBeforeCloseAction + // that guarantee that all operations have been flushed to Lucene. + final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated(); + if (indexVersionCreated.onOrAfter(Version.V_8_0_0) || + (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO && indexVersionCreated.onOrAfter(Version.V_6_7_0))) { + if (seqNoStats.getMaxSeqNo() != seqNoStats.getGlobalCheckpoint()) { + throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo() + + "] from last commit does not match global checkpoint [" + seqNoStats.getGlobalCheckpoint() + "]"); + } } + assert assertMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats.getMaxSeqNo(), seqNoStats.getMaxSeqNo()); + } + + protected boolean assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) { + assert maxSeqNo == globalCheckpoint : "max seq. no. [" + maxSeqNo + "] does not match [" + globalCheckpoint + "]"; + return true; } @Override @@ -198,12 +199,12 @@ protected void closeNoLock(String reason, CountDownLatch closedLatch) { } } - public static SeqNoStats buildSeqNoStats(SegmentInfos infos) { + private static SeqNoStats buildSeqNoStats(EngineConfig config, SegmentInfos infos) { final SequenceNumbers.CommitInfo seqNoStats = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(infos.userData.entrySet()); long maxSeqNo = seqNoStats.maxSeqNo; long localCheckpoint = seqNoStats.localCheckpoint; - return new SeqNoStats(maxSeqNo, localCheckpoint, localCheckpoint); + return new SeqNoStats(maxSeqNo, localCheckpoint, config.getGlobalCheckpointSupplier().getAsLong()); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 90f7c662b91c5..f2a5a96003a0e 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1439,7 +1439,6 @@ private void innerOpenEngineAndTranslog() throws IOException { assert recoveryState.getRecoverySource().expectEmptyRetentionLeases() == false || getRetentionLeases().leases().isEmpty() : "expected empty set of retention leases with recovery source [" + recoveryState.getRecoverySource() + "] but got " + getRetentionLeases(); - trimUnsafeCommits(); synchronized (mutex) { verifyNotClosed(); assert currentEngineReference.get() == null : "engine is running"; @@ -1458,15 +1457,6 @@ private void innerOpenEngineAndTranslog() throws IOException { assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage(); } - private void trimUnsafeCommits() throws IOException { - assert currentEngineReference.get() == null || currentEngineReference.get() instanceof ReadOnlyEngine : "a write engine is running"; - final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); - final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); - final long minRetainedTranslogGen = Translog.readMinTranslogGeneration(translogConfig.getTranslogPath(), translogUUID); - assertMaxUnsafeAutoIdInCommit(); - store.trimUnsafeCommits(globalCheckpoint, minRetainedTranslogGen, indexSettings.getIndexVersionCreated()); - } - private boolean assertSequenceNumbersInCommit() throws IOException { final Map userData = SegmentInfos.readLatestCommit(store.directory()).getUserData(); assert userData.containsKey(SequenceNumbers.LOCAL_CHECKPOINT_KEY) : "commit point doesn't contains a local checkpoint"; @@ -1474,11 +1464,6 @@ private boolean assertSequenceNumbersInCommit() throws IOException { assert userData.containsKey(Engine.HISTORY_UUID_KEY) : "commit point doesn't contains a history uuid"; assert userData.get(Engine.HISTORY_UUID_KEY).equals(getHistoryUUID()) : "commit point history uuid [" + userData.get(Engine.HISTORY_UUID_KEY) + "] is different than engine [" + getHistoryUUID() + "]"; - return true; - } - - private boolean assertMaxUnsafeAutoIdInCommit() throws IOException { - final Map userData = SegmentInfos.readLatestCommit(store.directory()).getUserData(); assert userData.containsKey(Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID) : "opening index which was created post 5.5.0 but " + Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID + " is not found in commit"; @@ -3100,8 +3085,8 @@ void resetEngineToGlobalCheckpoint() throws IOException { Engine newEngine = null; try { final long globalCheckpoint = getGlobalCheckpoint(); - trimUnsafeCommits(); synchronized (mutex) { + assert currentEngineReference.get() instanceof ReadOnlyEngine : "another write engine is running"; verifyNotClosed(); // we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata). newEngine = engineFactory.newReadWriteEngine(newEngineConfig()); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 3636967e66104..53bef02a6ebb2 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -674,7 +674,6 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { InternalEngine engine = createEngine(store, translog); engine.close(); - trimUnsafeCommits(engine.config()); engine = new InternalEngine(engine.config()); assertTrue(engine.isRecovering()); engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); @@ -691,7 +690,6 @@ public void testFlushIsDisabledDuringTranslogRecovery() throws IOException { engine.index(indexForDoc(doc)); engine.close(); - trimUnsafeCommits(engine.config()); engine = new InternalEngine(engine.config()); expectThrows(IllegalStateException.class, () -> engine.flush(true, true)); assertTrue(engine.isRecovering()); @@ -726,7 +724,6 @@ public void testTranslogMultipleOperationsSameDocument() throws IOException { } finally { IOUtils.close(engine); } - trimUnsafeCommits(engine.config()); try (Engine recoveringEngine = new InternalEngine(engine.config())) { recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -755,7 +752,6 @@ public void testTranslogRecoveryDoesNotReplayIntoTranslog() throws IOException { Engine recoveringEngine = null; try { final AtomicBoolean committed = new AtomicBoolean(); - trimUnsafeCommits(initialEngine.config()); recoveringEngine = new InternalEngine(initialEngine.config()) { @Override @@ -798,7 +794,6 @@ public void testTranslogRecoveryWithMultipleGenerations() throws IOException { } } initialEngine.close(); - trimUnsafeCommits(initialEngine.config()); recoveringEngine = new InternalEngine(initialEngine.config()); recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -834,14 +829,12 @@ public void testRecoveryFromTranslogUpToSeqNo() throws IOException { globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint())); engine.syncTranslog(); } - trimUnsafeCommits(config); try (InternalEngine engine = new InternalEngine(config)) { engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertThat(engine.getLocalCheckpoint(), equalTo(maxSeqNo)); assertThat(engine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(maxSeqNo)); } - trimUnsafeCommits(config); try (InternalEngine engine = new InternalEngine(config)) { long upToSeqNo = randomLongBetween(globalCheckpoint.get(), maxSeqNo); engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); @@ -1259,7 +1252,6 @@ public void testSyncedFlushSurvivesEngineRestart() throws IOException { UNASSIGNED_SEQ_NO, shardId, primaryTerm.get()); store.associateIndexWithNewTranslog(translogUUID); } - trimUnsafeCommits(config); engine = new InternalEngine(config); engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -1280,7 +1272,6 @@ public void testSyncedFlushVanishesOnReplay() throws IOException { engine.index(indexForDoc(doc)); EngineConfig config = engine.config(); engine.close(); - trimUnsafeCommits(config); engine = new InternalEngine(config); engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -2379,7 +2370,6 @@ public void testSeqNoAndCheckpoints() throws IOException { IOUtils.close(initialEngine); } - trimUnsafeCommits(initialEngine.engineConfig); try (InternalEngine recoveringEngine = new InternalEngine(initialEngine.config())) { recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -2724,7 +2714,6 @@ public void testCurrentTranslogIDisCommitted() throws IOException { // open and recover tlog { for (int i = 0; i < 2; i++) { - trimUnsafeCommits(config); try (InternalEngine engine = new InternalEngine(config)) { assertTrue(engine.isRecovering()); Map userData = engine.getLastCommittedSegmentInfos().getUserData(); @@ -2751,7 +2740,6 @@ public void testCurrentTranslogIDisCommitted() throws IOException { Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); store.associateIndexWithNewTranslog(translogUUID); - trimUnsafeCommits(config); try (InternalEngine engine = new InternalEngine(config)) { Map userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); @@ -2766,7 +2754,6 @@ public void testCurrentTranslogIDisCommitted() throws IOException { // open and recover tlog with empty tlog { for (int i = 0; i < 2; i++) { - trimUnsafeCommits(config); try (InternalEngine engine = new InternalEngine(config)) { Map userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); @@ -2831,7 +2818,6 @@ public void testTranslogReplayWithFailure() throws IOException { boolean started = false; InternalEngine engine = null; try { - trimUnsafeCommits(config(defaultSettings, store, translogPath, NoMergePolicy.INSTANCE, null)); engine = createEngine(store, translogPath); started = true; } catch (EngineException | IOException e) { @@ -2917,7 +2903,6 @@ public void testSkipTranslogReplay() throws IOException { EngineConfig config = engine.config(); assertVisibleCount(engine, numDocs); engine.close(); - trimUnsafeCommits(config); try (InternalEngine engine = new InternalEngine(config)) { engine.skipTranslogRecovery(); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -2960,7 +2945,6 @@ public void testTranslogReplay() throws IOException { translogHandler.mappingUpdate = dynamicUpdate(); engine.close(); - trimUnsafeCommits(copy(engine.config(), inSyncGlobalCheckpointSupplier)); // we need to reuse the engine config unless the parser.mappingModified won't work engine = new InternalEngine(copy(engine.config(), inSyncGlobalCheckpointSupplier)); engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); @@ -4093,7 +4077,6 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro } finally { IOUtils.close(initialEngine); } - trimUnsafeCommits(initialEngine.config()); try (Engine recoveringEngine = new InternalEngine(initialEngine.config())) { recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -4199,7 +4182,6 @@ public void testNoOps() throws IOException { final BiFunction supplier = (ms, lcp) -> new LocalCheckpointTracker( maxSeqNo, localCheckpoint); - trimUnsafeCommits(engine.config()); EngineConfig noopEngineConfig = copy(engine.config(), new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy())); noOpEngine = new InternalEngine(noopEngineConfig, supplier) { @@ -4443,7 +4425,6 @@ public void testRestoreLocalHistoryFromTranslog() throws IOException { prevDocs = getDocIds(engine, true); totalTranslogOps = engine.getTranslog().totalOperations(); } - trimUnsafeCommits(engineConfig); try (InternalEngine engine = new InternalEngine(engineConfig)) { engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, globalCheckpoint.get()); @@ -4490,7 +4471,6 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { assertEquals(docs - 1, engine.getLocalCheckpoint()); assertEquals(maxSeqIDOnReplica, replicaEngine.getSeqNoStats(-1).getMaxSeqNo()); assertEquals(checkpointOnReplica, replicaEngine.getLocalCheckpoint()); - trimUnsafeCommits(copy(replicaEngine.config(), globalCheckpoint::get)); recoveringEngine = new InternalEngine(copy(replicaEngine.config(), globalCheckpoint::get)); assertEquals(numDocsOnReplica, getTranslog(recoveringEngine).stats().getUncommittedOperations()); recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); @@ -4525,7 +4505,6 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { // now do it again to make sure we preserve values etc. try { - trimUnsafeCommits(replicaEngine.config()); recoveringEngine = new InternalEngine(copy(replicaEngine.config(), globalCheckpoint::get)); if (flushed) { assertThat(recoveringEngine.getTranslogStats().getUncommittedOperations(), equalTo(0)); @@ -5401,7 +5380,6 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { } else { engine.flushAndClose(); } - trimUnsafeCommits(engine.config()); try (InternalEngine recoveringEngine = new InternalEngine(engine.config())) { assertThat(recoveringEngine.getMinRetainedSeqNo(), equalTo(lastMinRetainedSeqNo)); } @@ -5556,7 +5534,6 @@ public void testRebuildLocalCheckpointTracker() throws Exception { engine.syncTranslog(); docs = getDocIds(engine, true); } - trimUnsafeCommits(config); Set seqNosInSafeCommit = null; for (int i = commits.size() - 1; i >= 0; i--) { if (commits.get(i).stream().allMatch(op -> op.seqNo() <= globalCheckpoint.get())) { @@ -5600,7 +5577,6 @@ public void testOpenSoftDeletesIndexWithSoftDeletesDisabled() throws Exception { IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(Settings.builder() .put(defaultSettings.getSettings()).put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false)).build()); EngineConfig config = config(softDeletesDisabled, store, translogPath, newMergePolicy(), null, null, globalCheckpoint::get); - trimUnsafeCommits(config); try (InternalEngine engine = createEngine(config)) { assertThat(getDocIds(engine, true), equalTo(docs)); } @@ -5620,15 +5596,6 @@ public void testRequireSoftDeletesWhenAccessingChangesSnapshot() throws Exceptio } } - static void trimUnsafeCommits(EngineConfig config) throws IOException { - final Store store = config.getStore(); - final TranslogConfig translogConfig = config.getTranslogConfig(); - final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); - final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); - final long minRetainedTranslogGen = Translog.readMinTranslogGeneration(translogConfig.getTranslogPath(), translogUUID); - store.trimUnsafeCommits(globalCheckpoint, minRetainedTranslogGen, config.getIndexSettings().getIndexVersionCreated()); - } - void assertLuceneOperations(InternalEngine engine, long expectedAppends, long expectedUpdates, long expectedDeletes) { String message = "Lucene operations mismatched;" + " appends [actual:" + engine.getNumDocAppends() + ", expected:" + expectedAppends + "]," + diff --git a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java index e19bdc42b0156..e66094d7321a7 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -48,9 +48,6 @@ public void testReadOnlyEngine() throws Exception { try (InternalEngine engine = createEngine(config)) { Engine.Get get = null; for (int i = 0; i < numDocs; i++) { - if (rarely()) { - continue; // gap in sequence number - } ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); @@ -94,7 +91,6 @@ public void testReadOnlyEngine() throws Exception { } // Close and reopen the main engine - InternalEngineTests.trimUnsafeCommits(config); try (InternalEngine recoveringEngine = new InternalEngine(config)) { recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); @@ -117,17 +113,15 @@ public void testFlushes() throws IOException { int numDocs = scaledRandomIntBetween(10, 1000); try (InternalEngine engine = createEngine(config)) { for (int i = 0; i < numDocs; i++) { - if (rarely()) { - continue; // gap in sequence number - } ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); if (rarely()) { engine.flush(); } - globalCheckpoint.set(i); + globalCheckpoint.set(engine.getLocalCheckpoint()); } + globalCheckpoint.set(engine.getLocalCheckpoint()); engine.syncTranslog(); engine.flushAndClose(); readOnlyEngine = new ReadOnlyEngine(engine.engineConfig, null , null, true, Function.identity()); @@ -141,7 +135,6 @@ public void testFlushes() throws IOException { public void testEnsureMaxSeqNoIsEqualToGlobalCheckpoint() throws IOException { IOUtils.close(engine, store); - Engine readOnlyEngine = null; final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); try (Store store = createStore()) { EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); @@ -159,16 +152,15 @@ public void testEnsureMaxSeqNoIsEqualToGlobalCheckpoint() throws IOException { engine.flushAndClose(); IllegalStateException exception = expectThrows(IllegalStateException.class, - () -> new ReadOnlyEngine(engine.engineConfig, null, null, true, Function.identity()) { + () -> new ReadOnlyEngine(config, null, null, true, Function.identity()) { @Override - protected void assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) { + protected boolean assertMaxSeqNoEqualsToGlobalCheckpoint(final long maxSeqNo, final long globalCheckpoint) { // we don't want the assertion to trip in this test + return true; } }); assertThat(exception.getMessage(), equalTo("Maximum sequence number [" + maxSeqNo + "] from last commit does not match global checkpoint [" + globalCheckpoint.get() + "]")); - } finally { - IOUtils.close(readOnlyEngine); } } } @@ -219,9 +211,6 @@ public void testRecoverFromTranslogAppliesNoOperations() throws IOException { int numDocs = scaledRandomIntBetween(10, 1000); try (InternalEngine engine = createEngine(config)) { for (int i = 0; i < numDocs; i++) { - if (rarely()) { - continue; // gap in sequence number - } ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index bf2499c6d1e2a..d79a2b652b4f7 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -86,6 +86,7 @@ import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.fielddata.FieldDataStats; @@ -154,6 +155,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.LongFunction; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -3818,4 +3820,40 @@ private void randomReplicaOperationPermitAcquisition(final IndexShard indexShard indexShard.acquireAllReplicaOperationsPermits(opPrimaryTerm, globalCheckpoint, maxSeqNoOfUpdatesOrDeletes, listener, timeout); } } + + public void testDoNotTrimCommitsWhenOpenReadOnlyEngine() throws Exception { + final IndexShard shard = newStartedShard(false, Settings.EMPTY, new InternalEngineFactory()); + long numDocs = randomLongBetween(1, 20); + long seqNo = 0; + for (long i = 0; i < numDocs; i++) { + if (rarely()) { + seqNo++; // create gaps in sequence numbers + } + shard.applyIndexOperationOnReplica(seqNo, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + new SourceToParse(shard.shardId.getIndexName(), "_doc", Long.toString(i), new BytesArray("{}"), XContentType.JSON)); + shard.updateGlobalCheckpointOnReplica(shard.getLocalCheckpoint(), "test"); + if (randomInt(100) < 10) { + shard.flush(new FlushRequest()); + } + seqNo++; + } + shard.flush(new FlushRequest()); + assertThat(shard.docStats().getCount(), equalTo(numDocs)); + final ShardRouting replicaRouting = shard.routingEntry(); + ShardRouting readonlyShardRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), true, + ShardRoutingState.INITIALIZING, RecoverySource.ExistingStoreRecoverySource.INSTANCE); + final IndexShard readonlyShard = reinitShard(shard, readonlyShardRouting, + engineConfig -> new ReadOnlyEngine(engineConfig, null, null, false, Function.identity()) { + @Override + protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(SeqNoStats seqNoStats) { + // just like a following shard, we need to skip this check for now. + } + } + ); + DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); + readonlyShard.markAsRecovering("store", new RecoveryState(readonlyShard.routingEntry(), localNode, null)); + assertTrue(readonlyShard.recoverFromStore()); + assertThat(readonlyShard.docStats().getCount(), equalTo(numDocs)); + closeShards(readonlyShard); + } } From b41e3fc09f2584626c5fe8c9889ba70d9cd4cbb2 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 11 Apr 2019 20:26:18 +0100 Subject: [PATCH 005/260] Move primary term from replicas proxy to repl op (#41119) A small refactoring that removes the primaryTerm field from ReplicasProxy and instead passes it directly in to the methods that need it. Relates #40706. --- ...TransportVerifyShardBeforeCloseAction.java | 12 +-- .../TransportResyncReplicationAction.java | 16 ++- .../replication/ReplicationOperation.java | 102 ++++++++++-------- .../TransportReplicationAction.java | 18 ++-- .../replication/TransportWriteAction.java | 13 +-- ...portVerifyShardBeforeCloseActionTests.java | 99 +++++++++-------- .../ReplicationOperationTests.java | 48 ++++----- .../TransportReplicationActionTests.java | 19 ++-- .../TransportWriteActionTests.java | 9 +- .../ESIndexLevelReplicationTestCase.java | 11 +- 10 files changed, 173 insertions(+), 174 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index f29bf6987a085..7d691717de1f2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -115,8 +115,8 @@ private void executeShardOperation(final ShardRequest request, final IndexShard } @Override - protected ReplicationOperation.Replicas newReplicasProxy(final long primaryTerm) { - return new VerifyShardBeforeCloseActionReplicasProxy(primaryTerm); + protected ReplicationOperation.Replicas newReplicasProxy() { + return new VerifyShardBeforeCloseActionReplicasProxy(); } /** @@ -125,13 +125,9 @@ protected ReplicationOperation.Replicas newReplicasProxy(final lon * or reopened in an unverified state with potential non flushed translog operations. */ class VerifyShardBeforeCloseActionReplicasProxy extends ReplicasProxy { - - VerifyShardBeforeCloseActionReplicasProxy(final long primaryTerm) { - super(primaryTerm); - } - @Override - public void markShardCopyAsStaleIfNeeded(final ShardId shardId, final String allocationId, final ActionListener listener) { + public void markShardCopyAsStaleIfNeeded(final ShardId shardId, final String allocationId, final long primaryTerm, + final ActionListener listener) { shardStateAction.remoteShardFailed(shardId, allocationId, primaryTerm, true, "mark copy as stale", null, listener); } } diff --git a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index bfe3274996160..464cd3168bf7e 100644 --- a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -68,8 +68,8 @@ protected ResyncReplicationResponse newResponseInstance() { } @Override - protected ReplicationOperation.Replicas newReplicasProxy(long primaryTerm) { - return new ResyncActionReplicasProxy(primaryTerm); + protected ReplicationOperation.Replicas newReplicasProxy() { + return new ResyncActionReplicasProxy(); } @Override @@ -96,9 +96,10 @@ public static ResyncReplicationRequest performOnPrimary(ResyncReplicationRequest } @Override - protected WriteReplicaResult shardOperationOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception { + protected WriteReplicaResult shardOperationOnReplica(ResyncReplicationRequest request, + IndexShard replica) throws Exception { Translog.Location location = performOnReplica(request, replica); - return new WriteReplicaResult(request, location, null, replica, logger); + return new WriteReplicaResult<>(request, location, null, replica, logger); } public static Translog.Location performOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception { @@ -174,12 +175,9 @@ public void handleException(TransportException exp) { */ class ResyncActionReplicasProxy extends ReplicasProxy { - ResyncActionReplicasProxy(long primaryTerm) { - super(primaryTerm); - } - @Override - public void failShardIfNeeded(ShardRouting replica, String message, Exception exception, ActionListener listener) { + public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, + ActionListener listener) { shardStateAction.remoteShardFailed( replica.shardId(), replica.allocationId().getId(), primaryTerm, false, message, exception, listener); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index c8c102dfd85e7..e5c2136aae56d 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -71,7 +71,10 @@ public class ReplicationOperation< private final Primary primary; private final Replicas replicasProxy; private final AtomicBoolean finished = new AtomicBoolean(); - protected final ActionListener resultListener; + private final long primaryTerm; + + // exposed for tests + final ActionListener resultListener; private volatile PrimaryResultT primaryResult = null; @@ -80,13 +83,14 @@ public class ReplicationOperation< public ReplicationOperation(Request request, Primary primary, ActionListener listener, Replicas replicas, - Logger logger, String opType) { + Logger logger, String opType, long primaryTerm) { this.replicasProxy = replicas; this.primary = primary; this.resultListener = listener; this.logger = logger; this.request = request; this.opType = opType; + this.primaryTerm = primaryTerm; } public void execute() throws Exception { @@ -137,7 +141,7 @@ private void markUnavailableShardsAsStale(ReplicaRequest replicaRequest, Replica // if inSyncAllocationIds contains allocation ids of shards that don't exist in RoutingTable, mark copies as stale for (String allocationId : replicationGroup.getUnavailableInSyncShards()) { pendingActions.incrementAndGet(); - replicasProxy.markShardCopyAsStaleIfNeeded(replicaRequest.shardId(), allocationId, + replicasProxy.markShardCopyAsStaleIfNeeded(replicaRequest.shardId(), allocationId, primaryTerm, ActionListener.wrap(r -> decPendingAndFinishIfNeeded(), ReplicationOperation.this::onNoLongerPrimary)); } } @@ -165,44 +169,45 @@ private void performOnReplica(final ShardRouting shard, final ReplicaRequest rep totalShards.incrementAndGet(); pendingActions.incrementAndGet(); - replicasProxy.performOn(shard, replicaRequest, globalCheckpoint, maxSeqNoOfUpdatesOrDeletes, new ActionListener() { - @Override - public void onResponse(ReplicaResponse response) { - successfulShards.incrementAndGet(); - try { - primary.updateLocalCheckpointForShard(shard.allocationId().getId(), response.localCheckpoint()); - primary.updateGlobalCheckpointForShard(shard.allocationId().getId(), response.globalCheckpoint()); - } catch (final AlreadyClosedException e) { - // okay, the index was deleted or this shard was never activated after a relocation; fall through and finish normally - } catch (final Exception e) { - // fail the primary but fall through and let the rest of operation processing complete - final String message = String.format(Locale.ROOT, "primary failed updating local checkpoint for replica %s", shard); - primary.failShard(message, e); + replicasProxy.performOn(shard, replicaRequest, primaryTerm, globalCheckpoint, maxSeqNoOfUpdatesOrDeletes, + new ActionListener<>() { + @Override + public void onResponse(ReplicaResponse response) { + successfulShards.incrementAndGet(); + try { + primary.updateLocalCheckpointForShard(shard.allocationId().getId(), response.localCheckpoint()); + primary.updateGlobalCheckpointForShard(shard.allocationId().getId(), response.globalCheckpoint()); + } catch (final AlreadyClosedException e) { + // the index was deleted or this shard was never activated after a relocation; fall through and finish normally + } catch (final Exception e) { + // fail the primary but fall through and let the rest of operation processing complete + final String message = String.format(Locale.ROOT, "primary failed updating local checkpoint for replica %s", shard); + primary.failShard(message, e); + } + decPendingAndFinishIfNeeded(); } - decPendingAndFinishIfNeeded(); - } - @Override - public void onFailure(Exception replicaException) { - logger.trace(() -> new ParameterizedMessage( - "[{}] failure while performing [{}] on replica {}, request [{}]", - shard.shardId(), opType, shard, replicaRequest), replicaException); - // Only report "critical" exceptions - TODO: Reach out to the master node to get the latest shard state then report. - if (TransportActions.isShardNotAvailableException(replicaException) == false) { - RestStatus restStatus = ExceptionsHelper.status(replicaException); - shardReplicaFailures.add(new ReplicationResponse.ShardInfo.Failure( - shard.shardId(), shard.currentNodeId(), replicaException, restStatus, false)); + @Override + public void onFailure(Exception replicaException) { + logger.trace(() -> new ParameterizedMessage( + "[{}] failure while performing [{}] on replica {}, request [{}]", + shard.shardId(), opType, shard, replicaRequest), replicaException); + // Only report "critical" exceptions - TODO: Reach out to the master node to get the latest shard state then report. + if (TransportActions.isShardNotAvailableException(replicaException) == false) { + RestStatus restStatus = ExceptionsHelper.status(replicaException); + shardReplicaFailures.add(new ReplicationResponse.ShardInfo.Failure( + shard.shardId(), shard.currentNodeId(), replicaException, restStatus, false)); + } + String message = String.format(Locale.ROOT, "failed to perform %s on replica %s", opType, shard); + replicasProxy.failShardIfNeeded(shard, primaryTerm, message, replicaException, + ActionListener.wrap(r -> decPendingAndFinishIfNeeded(), ReplicationOperation.this::onNoLongerPrimary)); } - String message = String.format(Locale.ROOT, "failed to perform %s on replica %s", opType, shard); - replicasProxy.failShardIfNeeded(shard, message, replicaException, - ActionListener.wrap(r -> decPendingAndFinishIfNeeded(), ReplicationOperation.this::onNoLongerPrimary)); - } - @Override - public String toString() { - return "[" + replicaRequest + "][" + shard + "]"; - } - }); + @Override + public String toString() { + return "[" + replicaRequest + "][" + shard + "]"; + } + }); } private void onNoLongerPrimary(Exception failure) { @@ -373,25 +378,27 @@ public interface Replicas> { * * @param replica the shard this request should be executed on * @param replicaRequest the operation to perform + * @param primaryTerm the primary term * @param globalCheckpoint the global checkpoint on the primary * @param maxSeqNoOfUpdatesOrDeletes the max seq_no of updates (index operations overwriting Lucene) or deletes on primary * after this replication was executed on it. * @param listener callback for handling the response or failure */ - void performOn(ShardRouting replica, RequestT replicaRequest, long globalCheckpoint, - long maxSeqNoOfUpdatesOrDeletes, ActionListener listener); + void performOn(ShardRouting replica, RequestT replicaRequest, + long primaryTerm, long globalCheckpoint, long maxSeqNoOfUpdatesOrDeletes, ActionListener listener); /** * Fail the specified shard if needed, removing it from the current set * of active shards. Whether a failure is needed is left up to the * implementation. * - * @param replica shard to fail - * @param message a (short) description of the reason - * @param exception the original exception which caused the ReplicationOperation to request the shard to be failed - * @param listener a listener that will be notified when the failing shard has been removed from the in-sync set + * @param replica shard to fail + * @param primaryTerm the primary term + * @param message a (short) description of the reason + * @param exception the original exception which caused the ReplicationOperation to request the shard to be failed + * @param listener a listener that will be notified when the failing shard has been removed from the in-sync set */ - void failShardIfNeeded(ShardRouting replica, String message, Exception exception, ActionListener listener); + void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, ActionListener listener); /** * Marks shard copy as stale if needed, removing its allocation id from @@ -400,9 +407,10 @@ void performOn(ShardRouting replica, RequestT replicaRequest, long globalCheckpo * * @param shardId shard id * @param allocationId allocation id to remove from the set of in-sync allocation ids + * @param primaryTerm the primary term * @param listener a listener that will be notified when the failing shard has been removed from the in-sync set */ - void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, ActionListener listener); + void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, ActionListener listener); } /** @@ -427,11 +435,11 @@ public interface ReplicaResponse { } public static class RetryOnPrimaryException extends ElasticsearchException { - public RetryOnPrimaryException(ShardId shardId, String msg) { + RetryOnPrimaryException(ShardId shardId, String msg) { this(shardId, msg, null); } - public RetryOnPrimaryException(ShardId shardId, String msg, Throwable cause) { + RetryOnPrimaryException(ShardId shardId, String msg, Throwable cause) { super(msg, cause); setShard(shardId); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 96c6cc3afa04c..e9c071e5a0ed1 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -163,8 +163,8 @@ protected void doExecute(Task task, Request request, ActionListener li new ReroutePhase((ReplicationTask) task, request, listener).run(); } - protected ReplicationOperation.Replicas newReplicasProxy(long primaryTerm) { - return new ReplicasProxy(primaryTerm); + protected ReplicationOperation.Replicas newReplicasProxy() { + return new ReplicasProxy(); } protected abstract Response newResponseInstance(); @@ -409,7 +409,7 @@ protected ReplicationOperation> listener, PrimaryShardReference primaryShardReference) { return new ReplicationOperation<>(request, primaryShardReference, listener, - newReplicasProxy(primaryRequest.getPrimaryTerm()), logger, actionName); + newReplicasProxy(), logger, actionName, primaryRequest.getPrimaryTerm()); } } @@ -1021,16 +1021,11 @@ public int hashCode() { */ protected class ReplicasProxy implements ReplicationOperation.Replicas { - protected final long primaryTerm; - - public ReplicasProxy(long primaryTerm) { - this.primaryTerm = primaryTerm; - } - @Override public void performOn( final ShardRouting replica, final ReplicaRequest request, + final long primaryTerm, final long globalCheckpoint, final long maxSeqNoOfUpdatesOrDeletes, final ActionListener listener) { @@ -1051,7 +1046,8 @@ public void performOn( } @Override - public void failShardIfNeeded(ShardRouting replica, String message, Exception exception, ActionListener listener) { + public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, + ActionListener listener) { // This does not need to fail the shard. The idea is that this // is a non-write operation (something like a refresh or a global // checkpoint sync) and therefore the replica should still be @@ -1060,7 +1056,7 @@ public void failShardIfNeeded(ShardRouting replica, String message, Exception ex } @Override - public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, ActionListener listener) { + public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, ActionListener listener) { // This does not need to make the shard stale. The idea is that this // is a non-write operation (something like a refresh or a global // checkpoint sync) and therefore the replica should still be diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java index 15c49d1030374..86e2760c9012d 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportWriteAction.java @@ -96,8 +96,8 @@ public static Location locationToSync(Location current, Location next) { } @Override - protected ReplicationOperation.Replicas newReplicasProxy(long primaryTerm) { - return new WriteActionReplicasProxy(primaryTerm); + protected ReplicationOperation.Replicas newReplicasProxy() { + return new WriteActionReplicasProxy(); } /** @@ -371,12 +371,9 @@ void run() { */ class WriteActionReplicasProxy extends ReplicasProxy { - WriteActionReplicasProxy(long primaryTerm) { - super(primaryTerm); - } - @Override - public void failShardIfNeeded(ShardRouting replica, String message, Exception exception, ActionListener listener) { + public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, + ActionListener listener) { if (TransportActions.isShardNotAvailableException(exception) == false) { logger.warn(new ParameterizedMessage("[{}] {}", replica.shardId(), message), exception); } @@ -385,7 +382,7 @@ public void failShardIfNeeded(ShardRouting replica, String message, Exception ex } @Override - public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, ActionListener listener) { + public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, ActionListener listener) { shardStateAction.remoteShardFailed(shardId, allocationId, primaryTerm, true, "mark copy as stale", null, listener); } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index f90f40311c1b8..75f70ad02df64 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -139,7 +139,7 @@ private void executeOnPrimaryOrReplica() throws Throwable { final TaskId taskId = new TaskId("_node_id", randomNonNegativeLong()); final TransportVerifyShardBeforeCloseAction.ShardRequest request = new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId(), clusterBlock, taskId); - final PlainActionFuture res = PlainActionFuture.newFuture(); + final PlainActionFuture res = PlainActionFuture.newFuture(); action.shardOperationOnPrimary(request, indexShard, ActionListener.wrap( r -> { assertNotNull(r); @@ -228,10 +228,10 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { TaskId taskId = new TaskId(clusterService.localNode().getId(), 0L); TransportVerifyShardBeforeCloseAction.ShardRequest request = new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, clusterBlock, taskId); - ReplicationOperation.Replicas proxy = action.newReplicasProxy(primaryTerm); + ReplicationOperation.Replicas proxy = action.newReplicasProxy(); ReplicationOperation operation = - new ReplicationOperation<>(request, createPrimary(primaryRouting, replicationGroup), listener, proxy, logger, "test"); + TransportVerifyShardBeforeCloseAction.ShardRequest, PrimaryResult> operation = new ReplicationOperation<>( + request, createPrimary(primaryRouting, replicationGroup), listener, proxy, logger, "test", primaryTerm); operation.execute(); final CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear(); @@ -268,53 +268,50 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { TransportVerifyShardBeforeCloseAction.ShardRequest, PrimaryResult> createPrimary(final ShardRouting primary, final ReplicationGroup replicationGroup) { - return new ReplicationOperation.Primary< - TransportVerifyShardBeforeCloseAction.ShardRequest, - TransportVerifyShardBeforeCloseAction.ShardRequest, - PrimaryResult>() { - @Override - public ShardRouting routingEntry() { - return primary; - } - - @Override - public ReplicationGroup getReplicationGroup() { - return replicationGroup; - } - - @Override - public void perform( - TransportVerifyShardBeforeCloseAction.ShardRequest request, ActionListener listener) { - listener.onResponse(new PrimaryResult(request)); - } - - @Override - public void failShard(String message, Exception exception) { - - } - - @Override - public void updateLocalCheckpointForShard(String allocationId, long checkpoint) { - } - - @Override - public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) { - } - - @Override - public long localCheckpoint() { - return 0; - } - - @Override - public long globalCheckpoint() { - return 0; - } - - @Override - public long maxSeqNoOfUpdatesOrDeletes() { - return 0; - } + return new ReplicationOperation.Primary<>() { + @Override + public ShardRouting routingEntry() { + return primary; + } + + @Override + public ReplicationGroup getReplicationGroup() { + return replicationGroup; + } + + @Override + public void perform( + TransportVerifyShardBeforeCloseAction.ShardRequest request, ActionListener listener) { + listener.onResponse(new PrimaryResult(request)); + } + + @Override + public void failShard(String message, Exception exception) { + + } + + @Override + public void updateLocalCheckpointForShard(String allocationId, long checkpoint) { + } + + @Override + public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) { + } + + @Override + public long localCheckpoint() { + return 0; + } + + @Override + public long globalCheckpoint() { + return 0; + } + + @Override + public long maxSeqNoOfUpdatesOrDeletes() { + return 0; + } }; } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index d493b70337208..c959e3ed45d1a 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -116,10 +116,10 @@ public void testReplication() throws Exception { Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); - final TestReplicaProxy replicasProxy = new TestReplicaProxy(primaryTerm, simulatedFailures); + final TestReplicaProxy replicasProxy = new TestReplicaProxy(simulatedFailures); final TestPrimary primary = new TestPrimary(primaryShard, () -> replicationGroup); - final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy); + final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy, primaryTerm); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); assertThat(request.processedOnReplicas, equalTo(expectedReplicas)); @@ -213,12 +213,12 @@ public void testNoLongerPrimary() throws Exception { } else { shardActionFailure = new ShardStateAction.NoLongerPrimaryShardException(failedReplica.shardId(), "the king is dead"); } - final TestReplicaProxy replicasProxy = new TestReplicaProxy(primaryTerm, expectedFailures) { + final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures) { @Override - public void failShardIfNeeded(ShardRouting replica, String message, Exception exception, + public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, ActionListener shardActionListener) { if (testPrimaryDemotedOnStaleShardCopies) { - super.failShardIfNeeded(replica, message, exception, shardActionListener); + super.failShardIfNeeded(replica, primaryTerm, message, exception, shardActionListener); } else { assertThat(replica, equalTo(failedReplica)); shardActionListener.onFailure(shardActionFailure); @@ -226,11 +226,12 @@ public void failShardIfNeeded(ShardRouting replica, String message, Exception ex } @Override - public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, ActionListener shardActionListener) { + public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, + ActionListener shardActionListener) { if (testPrimaryDemotedOnStaleShardCopies) { shardActionListener.onFailure(shardActionFailure); } else { - super.markShardCopyAsStaleIfNeeded(shardId, allocationId, shardActionListener); + super.markShardCopyAsStaleIfNeeded(shardId, allocationId, primaryTerm, shardActionListener); } } }; @@ -242,7 +243,7 @@ public void failShard(String message, Exception exception) { assertTrue(primaryFailed.compareAndSet(false, true)); } }; - final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy); + final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy, primaryTerm); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); @@ -299,7 +300,7 @@ public void perform(Request request, ActionListener listener) { Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, - new TestReplicaProxy(primaryTerm)); + new TestReplicaProxy(), primaryTerm); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); @@ -343,7 +344,7 @@ public void testWaitForActiveShards() throws Exception { final ShardRouting primaryShard = shardRoutingTable.primaryShard(); final TestReplicationOperation op = new TestReplicationOperation(request, new TestPrimary(primaryShard, () -> initialReplicationGroup), - listener, new TestReplicaProxy(primaryTerm), logger, "test"); + listener, new TestReplicaProxy(), logger, "test", primaryTerm); if (passesActiveShardCheck) { assertThat(op.checkActiveShardCount(), nullValue()); @@ -401,8 +402,8 @@ public void updateLocalCheckpointForShard(String allocationId, long checkpoint) }; final PlainActionFuture listener = new PlainActionFuture<>(); - final ReplicationOperation.Replicas replicas = new TestReplicaProxy(primaryTerm, Collections.emptyMap()); - TestReplicationOperation operation = new TestReplicationOperation(request, primary, listener, replicas); + final ReplicationOperation.Replicas replicas = new TestReplicaProxy(Collections.emptyMap()); + TestReplicationOperation operation = new TestReplicationOperation(request, primary, listener, replicas, primaryTerm); operation.execute(); assertThat(primaryFailed.get(), equalTo(fatal)); @@ -577,14 +578,11 @@ static class TestReplicaProxy implements ReplicationOperation.Replicas final Set markedAsStaleCopies = ConcurrentCollections.newConcurrentSet(); - final long primaryTerm; - - TestReplicaProxy(long primaryTerm) { - this(primaryTerm, Collections.emptyMap()); + TestReplicaProxy() { + this(Collections.emptyMap()); } - TestReplicaProxy(long primaryTerm, Map opFailures) { - this.primaryTerm = primaryTerm; + TestReplicaProxy(Map opFailures) { this.opFailures = opFailures; } @@ -592,6 +590,7 @@ static class TestReplicaProxy implements ReplicationOperation.Replicas public void performOn( final ShardRouting replica, final Request request, + final long primaryTerm, final long globalCheckpoint, final long maxSeqNoOfUpdatesOrDeletes, final ActionListener listener) { @@ -609,7 +608,8 @@ public void performOn( } @Override - public void failShardIfNeeded(ShardRouting replica, String message, Exception exception, ActionListener listener) { + public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, + ActionListener listener) { if (failedReplicas.add(replica) == false) { fail("replica [" + replica + "] was failed twice"); } @@ -621,7 +621,7 @@ public void failShardIfNeeded(ShardRouting replica, String message, Exception ex } @Override - public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, ActionListener listener) { + public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, ActionListener listener) { if (markedAsStaleCopies.add(allocationId) == false) { fail("replica [" + allocationId + "] was marked as stale twice"); } @@ -631,14 +631,14 @@ public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, A class TestReplicationOperation extends ReplicationOperation { TestReplicationOperation(Request request, Primary primary, - ActionListener listener, Replicas replicas) { - this(request, primary, listener, replicas, ReplicationOperationTests.this.logger, "test"); + ActionListener listener, Replicas replicas, long primaryTerm) { + this(request, primary, listener, replicas, ReplicationOperationTests.this.logger, "test", primaryTerm); } TestReplicationOperation(Request request, Primary primary, ActionListener listener, - Replicas replicas, Logger logger, String opType) { - super(request, primary, listener, replicas, logger, opType); + Replicas replicas, Logger logger, String opType, long primaryTerm) { + super(request, primary, listener, replicas, logger, opType, primaryTerm); } } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index e256ec4e92679..2cdd3ad2fe480 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -603,7 +603,7 @@ action.new AsyncPrimaryAction(primaryRequest, listener, task) { Request request, ActionListener> actionListener, TransportReplicationAction.PrimaryShardReference primaryShardReference) { - return new NoopReplicationOperation(request, actionListener) { + return new NoopReplicationOperation(request, actionListener, primaryTerm) { @Override public void execute() throws Exception { assertPhase(task, "primary"); @@ -661,7 +661,7 @@ action.new AsyncPrimaryAction(primaryRequest, listener, task) { Request request, ActionListener> actionListener, TransportReplicationAction.PrimaryShardReference primaryShardReference) { - return new NoopReplicationOperation(request, actionListener) { + return new NoopReplicationOperation(request, actionListener, primaryTerm) { @Override public void execute() throws Exception { assertPhase(task, "primary"); @@ -710,7 +710,8 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { ClusterState state = stateWithActivePrimary(index, true, 1 + randomInt(3), randomInt(2)); logger.info("using state: {}", state); setState(clusterService, state); - ReplicationOperation.Replicas proxy = action.newReplicasProxy(state.metaData().index(index).primaryTerm(0)); + final long primaryTerm = state.metaData().index(index).primaryTerm(0); + ReplicationOperation.Replicas proxy = action.newReplicasProxy(); // check that at unknown node fails PlainActionFuture listener = new PlainActionFuture<>(); @@ -720,6 +721,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { TestShardRouting.newShardRouting(shardId, "NOT THERE", routingState == ShardRoutingState.RELOCATING ? state.nodes().iterator().next().getId() : null, false, routingState), new Request(NO_SHARD_ID), + primaryTerm, randomNonNegativeLong(), randomNonNegativeLong(), listener); @@ -730,7 +732,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { final ShardRouting replica = randomFrom(shardRoutings.replicaShards().stream() .filter(ShardRouting::assignedToNode).collect(Collectors.toList())); listener = new PlainActionFuture<>(); - proxy.performOn(replica, new Request(NO_SHARD_ID), randomNonNegativeLong(), randomNonNegativeLong(), listener); + proxy.performOn(replica, new Request(NO_SHARD_ID), primaryTerm, randomNonNegativeLong(), randomNonNegativeLong(), listener); assertFalse(listener.isDone()); CapturingTransport.CapturedRequest[] captures = transport.getCapturedRequestsAndClear(); @@ -753,7 +755,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { AtomicReference failure = new AtomicReference<>(); AtomicBoolean success = new AtomicBoolean(); - proxy.failShardIfNeeded(replica, "test", new ElasticsearchException("simulated"), + proxy.failShardIfNeeded(replica, primaryTerm, "test", new ElasticsearchException("simulated"), ActionListener.wrap(r -> success.set(true), failure::set)); CapturingTransport.CapturedRequest[] shardFailedRequests = transport.getCapturedRequestsAndClear(); // A replication action doesn't not fail the request @@ -836,7 +838,7 @@ action.new AsyncPrimaryAction(primaryRequest, listener, task) { if (throwExceptionOnCreation) { throw new ElasticsearchException("simulated exception, during createReplicatedOperation"); } - return new NoopReplicationOperation(request, actionListener) { + return new NoopReplicationOperation(request, actionListener, primaryTerm) { @Override public void execute() throws Exception { assertIndexShardCounter(1); @@ -1322,8 +1324,9 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService class NoopReplicationOperation extends ReplicationOperation> { - NoopReplicationOperation(Request request, ActionListener> listener) { - super(request, null, listener, null, TransportReplicationActionTests.this.logger, "noop"); + NoopReplicationOperation(Request request, ActionListener> listener, + long primaryTerm) { + super(request, null, listener, null, TransportReplicationActionTests.this.logger, "noop", primaryTerm); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 98c4f215fca8a..1a7e5a73e7523 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -278,7 +278,8 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { ClusterState state = ClusterStateCreationUtils.stateWithActivePrimary(index, true, 1 + randomInt(3), randomInt(2)); logger.info("using state: {}", state); ClusterServiceUtils.setState(clusterService, state); - ReplicationOperation.Replicas proxy = action.newReplicasProxy(state.metaData().index(index).primaryTerm(0)); + final long primaryTerm = state.metaData().index(index).primaryTerm(0); + ReplicationOperation.Replicas proxy = action.newReplicasProxy(); // check that at unknown node fails PlainActionFuture listener = new PlainActionFuture<>(); @@ -288,7 +289,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { TestShardRouting.newShardRouting(shardId, "NOT THERE", routingState == ShardRoutingState.RELOCATING ? state.nodes().iterator().next().getId() : null, false, routingState), new TestRequest(), - randomNonNegativeLong(), randomNonNegativeLong(), listener); + primaryTerm, randomNonNegativeLong(), randomNonNegativeLong(), listener); assertTrue(listener.isDone()); assertListenerThrows("non existent node should throw a NoNodeAvailableException", listener, NoNodeAvailableException.class); @@ -296,7 +297,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { final ShardRouting replica = randomFrom(shardRoutings.replicaShards().stream() .filter(ShardRouting::assignedToNode).collect(Collectors.toList())); listener = new PlainActionFuture<>(); - proxy.performOn(replica, new TestRequest(), randomNonNegativeLong(), randomNonNegativeLong(), listener); + proxy.performOn(replica, new TestRequest(), primaryTerm, randomNonNegativeLong(), randomNonNegativeLong(), listener); assertFalse(listener.isDone()); CapturingTransport.CapturedRequest[] captures = transport.getCapturedRequestsAndClear(); @@ -319,7 +320,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { AtomicReference failure = new AtomicReference<>(); AtomicBoolean success = new AtomicBoolean(); - proxy.failShardIfNeeded(replica, "test", new ElasticsearchException("simulated"), + proxy.failShardIfNeeded(replica, primaryTerm, "test", new ElasticsearchException("simulated"), ActionListener.wrap(r -> success.set(true), failure::set)); CapturingTransport.CapturedRequest[] shardFailedRequests = transport.getCapturedRequestsAndClear(); // A write replication action proxy should fail the shard diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 64e31059c5fab..9691677f643b9 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -608,8 +608,8 @@ protected ReplicationAction(Request request, ActionListener listener, public void execute() { try { new ReplicationOperation<>(request, new PrimaryRef(), - ActionListener.wrap(result -> result.respond(listener), listener::onFailure), new ReplicasRef(), logger, opType - ).execute(); + ActionListener.wrap(result -> result.respond(listener), listener::onFailure), new ReplicasRef(), logger, opType, + primaryTerm).execute(); } catch (Exception e) { listener.onFailure(e); } @@ -678,6 +678,7 @@ class ReplicasRef implements ReplicationOperation.Replicas { public void performOn( final ShardRouting replicaRouting, final ReplicaRequest request, + final long primaryTerm, final long globalCheckpoint, final long maxSeqNoOfUpdatesOrDeletes, final ActionListener listener) { @@ -700,12 +701,14 @@ public void performOn( } @Override - public void failShardIfNeeded(ShardRouting replica, String message, Exception exception, ActionListener listener) { + public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception, + ActionListener listener) { throw new UnsupportedOperationException("failing shard " + replica + " isn't supported. failure: " + message, exception); } @Override - public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, ActionListener listener) { + public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, + ActionListener listener) { throw new UnsupportedOperationException("can't mark " + shardId + ", aid [" + allocationId + "] as stale"); } } From 391ac5a709548a893508f822ba0df7d728c860b0 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 11 Apr 2019 15:46:13 -0400 Subject: [PATCH 006/260] [DOCS] Fix callout for Asciidoctor migration (#41129) --- docs/reference/query-dsl/script-score-query.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/query-dsl/script-score-query.asciidoc b/docs/reference/query-dsl/script-score-query.asciidoc index f3f0bc8af48c9..edab8a5fc45f5 100644 --- a/docs/reference/query-dsl/script-score-query.asciidoc +++ b/docs/reference/query-dsl/script-score-query.asciidoc @@ -96,7 +96,7 @@ cosine similarity between a given query vector and document vectors. "match_all": {} }, "script": { - "source": "cosineSimilarity(params.query_vector, doc['my_dense_vector']) + 1.0" <1>, + "source": "cosineSimilarity(params.query_vector, doc['my_dense_vector']) + 1.0", <1> "params": { "query_vector": [4, 3.4, -0.2] <2> } From 891844f0ef89ad0a6935b64c3edc88755823133c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Przemys=C5=82aw=20Witek?= Date: Thu, 11 Apr 2019 22:23:28 +0200 Subject: [PATCH 007/260] Increase maximum forecast interval to 10 years. (#41082) Increase the maximum duration to ~10 years (3650 days). --- .../xpack/core/ml/action/ForecastJobAction.java | 4 ++-- .../src/test/resources/rest-api-spec/test/ml/forecast.yml | 8 ++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ForecastJobAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ForecastJobAction.java index 0ea66ad0937c2..fb107579c6e66 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ForecastJobAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/ForecastJobAction.java @@ -48,8 +48,8 @@ public static class Request extends JobTaskRequest implements ToXConten public static final ParseField DURATION = new ParseField("duration"); public static final ParseField EXPIRES_IN = new ParseField("expires_in"); - // Max allowed duration: 8 weeks - private static final TimeValue MAX_DURATION = TimeValue.parseTimeValue("56d", ""); + // Max allowed duration: 10 years + private static final TimeValue MAX_DURATION = TimeValue.parseTimeValue("3650d", ""); private static final ObjectParser PARSER = new ObjectParser<>(NAME, Request::new); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/forecast.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/forecast.yml index 998f70de9d35d..a81b6dba08e48 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/forecast.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/forecast.yml @@ -47,6 +47,14 @@ setup: job_id: "forecast-job" duration: "-1s" +--- +"Test forecast given duration is too large": + - do: + catch: /\[duration\] must be 3650d or less[:] \[3651d\]/ + ml.forecast: + job_id: "forecast-job" + duration: "3651d" + --- "Test forecast given expires_in is negative": - do: From 1c062a752b453cecbfa5bfac0bdf76aac2df53b2 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Thu, 11 Apr 2019 13:58:47 -0700 Subject: [PATCH 008/260] Remove settings members from Node (#40811) This commit removes the settings member variable from Node. This member made it confusing which settings should actually be looked at. Now all settings are accessed through the final environment. --- .../java/org/elasticsearch/node/Node.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 45c47b9d49a05..1963d0c8b40c6 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -240,7 +240,6 @@ public class Node implements Closeable { */ private final Logger logger; private final Injector injector; - private final Settings settings; private final Environment environment; private final NodeEnvironment nodeEnvironment; private final PluginsService pluginsService; @@ -306,12 +305,12 @@ protected Node( this.pluginsService = new PluginsService(tmpSettings, environment.configFile(), environment.modulesFile(), environment.pluginsFile(), classpathPlugins); - this.settings = pluginsService.updatedSettings(); + final Settings settings = pluginsService.updatedSettings(); localNodeFactory = new LocalNodeFactory(settings, nodeEnvironment.nodeId()); // create the environment based on the finalized (processed) view of the settings // this is just to makes sure that people get the same settings, no matter where they ask them from - this.environment = new Environment(this.settings, environment.configFile()); + this.environment = new Environment(settings, environment.configFile()); Environment.assertEquivalent(environment, this.environment); final List> executorBuilders = pluginsService.getExecutorBuilders(settings); @@ -341,7 +340,7 @@ protected Node( .collect(Collectors.toSet()); final SettingsModule settingsModule = - new SettingsModule(this.settings, additionalSettings, additionalSettingsFilter, settingsUpgraders); + new SettingsModule(settings, additionalSettings, additionalSettingsFilter, settingsUpgraders); scriptModule.registerClusterSettingsListeners(settingsModule.getClusterSettings()); resourcesToClose.add(resourceWatcherService); final NetworkService networkService = new NetworkService( @@ -483,7 +482,7 @@ protected Node( modules.add(new RepositoriesModule(this.environment, pluginsService.filterPlugins(RepositoryPlugin.class), transportService, clusterService, threadPool, xContentRegistry)); - final DiscoveryModule discoveryModule = new DiscoveryModule(this.settings, threadPool, transportService, namedWriteableRegistry, + final DiscoveryModule discoveryModule = new DiscoveryModule(settings, threadPool, transportService, namedWriteableRegistry, networkService, clusterService.getMasterService(), clusterService.getClusterApplierService(), clusterService.getClusterSettings(), pluginsService.filterPlugins(DiscoveryPlugin.class), clusterModule.getAllocationService(), environment.configFile(), gatewayMetaState); @@ -602,7 +601,7 @@ protected void processRecoverySettings(ClusterSettings clusterSettings, Recovery * The settings that are used by this node. Contains original settings as well as additional settings provided by plugins. */ public Settings settings() { - return this.settings; + return this.environment.settings(); } /** @@ -668,7 +667,7 @@ public Node start() throws NodeValidationException { 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. - if (DiscoveryNode.isMasterNode(settings) || DiscoveryNode.isDataNode(settings)) { + if (DiscoveryNode.isMasterNode(settings()) || DiscoveryNode.isDataNode(settings())) { onDiskMetadata = injector.getInstance(GatewayMetaState.class).getMetaData(); } else { onDiskMetadata = MetaData.EMPTY_META_DATA; @@ -686,7 +685,7 @@ public Node start() throws NodeValidationException { : "clusterService has a different local node than the factory provided"; transportService.acceptIncomingRequests(); discovery.startInitialJoin(); - final TimeValue initialStateTimeout = INITIAL_STATE_TIMEOUT_SETTING.get(settings); + final TimeValue initialStateTimeout = INITIAL_STATE_TIMEOUT_SETTING.get(settings()); configureNodeAndClusterIdStateListener(clusterService); if (initialStateTimeout.millis() > 0) { @@ -725,7 +724,7 @@ public void onTimeout(TimeValue timeout) { injector.getInstance(HttpServerTransport.class).start(); - if (WRITE_PORTS_FILE_SETTING.get(settings)) { + if (WRITE_PORTS_FILE_SETTING.get(settings())) { TransportService transport = injector.getInstance(TransportService.class); writePortsFile("transport", transport.boundAddress()); HttpServerTransport http = injector.getInstance(HttpServerTransport.class); @@ -963,7 +962,7 @@ protected SearchService newSearchService(ClusterService clusterService, IndicesS private List getCustomNameResolvers(List discoveryPlugins) { List customNameResolvers = new ArrayList<>(); for (DiscoveryPlugin discoveryPlugin : discoveryPlugins) { - NetworkService.CustomNameResolver customNameResolver = discoveryPlugin.getCustomNameResolver(settings); + NetworkService.CustomNameResolver customNameResolver = discoveryPlugin.getCustomNameResolver(settings()); if (customNameResolver != null) { customNameResolvers.add(customNameResolver); } From badb8559fb16081e50fd262b7bac4b35711ae85f Mon Sep 17 00:00:00 2001 From: Antonio Matarrese Date: Thu, 11 Apr 2019 17:38:25 -0500 Subject: [PATCH 009/260] Use the breadth first collection mode for significant terms aggs. This helps avoid memory issues when computing deep sub-aggregations. Because it should be rare to use sub-aggregations with significant terms, we opted to always choose breadth first as opposed to exposing a `collect_mode` option. Closes #28652. --- .../significantterms-aggregation.asciidoc | 6 ++++ .../bucket/terms-aggregation.asciidoc | 1 + ...balOrdinalsSignificantTermsAggregator.java | 11 +++++-- .../SignificantLongTermsAggregator.java | 14 ++++++-- .../SignificantStringTermsAggregator.java | 14 ++++++-- .../SignificantTermsSignificanceScoreIT.java | 33 +++++++++++++++++++ 6 files changed, 71 insertions(+), 8 deletions(-) diff --git a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc index e29fbac0c5649..5766cb1a73e58 100644 --- a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc @@ -542,6 +542,12 @@ It is possible (although rarely required) to filter the values for which buckets `exclude` parameters which are based on a regular expression string or arrays of exact terms. This functionality mirrors the features described in the <> documentation. +==== Collect mode + +To avoid memory issues, the `significant_terms` aggregation always computes child aggregations in `breadth_first` mode. +A description of the different collection modes can be found in the +<> documentation. + ==== Execution hint There are different mechanisms by which terms aggregations can be executed: diff --git a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc index 8db3c6b59ee1a..34d50992e8068 100644 --- a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc @@ -775,6 +775,7 @@ fields, then use `copy_to` in your mapping to create a new dedicated field at index time which contains the values from both fields. You can aggregate on this single field, which will benefit from the global ordinals optimization. +[[search-aggregations-bucket-terms-aggregation-collect]] ==== Collect mode Deferring calculation of child aggregations diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java index 25f83caa3eb92..d641a2773e615 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java @@ -65,7 +65,7 @@ public GlobalOrdinalsSignificantTermsAggregator(String name, List pipelineAggregators, Map metaData) throws IOException { super(name, factories, valuesSource, null, format, bucketCountThresholds, includeExclude, context, parent, - forceRemapGlobalOrds, SubAggCollectionMode.DEPTH_FIRST, false, pipelineAggregators, metaData); + forceRemapGlobalOrds, SubAggCollectionMode.BREADTH_FIRST, false, pipelineAggregators, metaData); this.significanceHeuristic = significanceHeuristic; this.termsAggFactory = termsAggFactory; this.numCollectedDocs = 0; @@ -146,12 +146,19 @@ public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws } final SignificantStringTerms.Bucket[] list = new SignificantStringTerms.Bucket[ordered.size()]; + final long[] survivingBucketOrds = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { final SignificantStringTerms.Bucket bucket = ordered.pop(); + survivingBucketOrds[i] = bucket.bucketOrd; + list[i] = bucket; + } + + runDeferredCollections(survivingBucketOrds); + + for (SignificantStringTerms.Bucket bucket : list) { // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be recycled at some point bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); bucket.aggregations = bucketAggregations(bucket.bucketOrd); - list[i] = bucket; } return new SignificantStringTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java index 235b3f41c08f8..2fcba9f09bf7e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java @@ -50,7 +50,7 @@ public SignificantLongTermsAggregator(String name, AggregatorFactories factories List pipelineAggregators, Map metaData) throws IOException { super(name, factories, valuesSource, format, null, bucketCountThresholds, context, parent, - SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, pipelineAggregators, metaData); + SubAggCollectionMode.BREADTH_FIRST, false, includeExclude, pipelineAggregators, metaData); this.significanceHeuristic = significanceHeuristic; this.termsAggFactory = termsAggFactory; } @@ -106,12 +106,20 @@ public SignificantLongTerms buildAggregation(long owningBucketOrdinal) throws IO } } - final SignificantLongTerms.Bucket[] list = new SignificantLongTerms.Bucket[ordered.size()]; + SignificantLongTerms.Bucket[] list = new SignificantLongTerms.Bucket[ordered.size()]; + final long[] survivingBucketOrds = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { final SignificantLongTerms.Bucket bucket = ordered.pop(); - bucket.aggregations = bucketAggregations(bucket.bucketOrd); + survivingBucketOrds[i] = bucket.bucketOrd; list[i] = bucket; } + + runDeferredCollections(survivingBucketOrds); + + for (SignificantLongTerms.Bucket bucket : list) { + bucket.aggregations = bucketAggregations(bucket.bucketOrd); + } + return new SignificantLongTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), pipelineAggregators(), metaData(), format, subsetSize, supersetSize, significanceHeuristic, Arrays.asList(list)); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java index 56258758907c1..91ade2e42f740 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java @@ -57,7 +57,7 @@ public SignificantStringTermsAggregator(String name, AggregatorFactories factori List pipelineAggregators, Map metaData) throws IOException { super(name, factories, valuesSource, null, format, bucketCountThresholds, includeExclude, aggregationContext, parent, - SubAggCollectionMode.DEPTH_FIRST, false, pipelineAggregators, metaData); + SubAggCollectionMode.BREADTH_FIRST, false, pipelineAggregators, metaData); this.significanceHeuristic = significanceHeuristic; this.termsAggFactory = termsAggFactory; } @@ -113,12 +113,20 @@ public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws } final SignificantStringTerms.Bucket[] list = new SignificantStringTerms.Bucket[ordered.size()]; + final long[] survivingBucketOrds = new long[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { final SignificantStringTerms.Bucket bucket = ordered.pop(); - // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be recycled at some point + survivingBucketOrds[i] = bucket.bucketOrd; + list[i] = bucket; + } + + runDeferredCollections(survivingBucketOrds); + + for (SignificantStringTerms.Bucket bucket : list) { + // the terms are owned by the BytesRefHash, we need to pull a copy since the BytesRef hash data may be + // recycled at some point bucket.termBytes = BytesRef.deepCopyOf(bucket.termBytes); bucket.aggregations = bucketAggregations(bucket.bucketOrd); - list[i] = bucket; } return new SignificantStringTerms( name, bucketCountThresholds.getRequiredSize(), diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java index 658f830857180..d21519fa96754 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.plugins.Plugin; @@ -38,6 +39,7 @@ import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptType; import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter; import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms; @@ -543,6 +545,37 @@ public void testScoresEqualForPositiveAndNegative(SignificanceHeuristic heuristi } } + /** + * A simple test that adds a sub-aggregation to a significant terms aggregation, + * to help check that sub-aggregation collection is handled correctly. + */ + public void testSubAggregations() throws Exception { + indexEqualTestData(); + + QueryBuilder query = QueryBuilders.termsQuery(TEXT_FIELD, "a", "b"); + AggregationBuilder subAgg = terms("class").field(CLASS_FIELD); + AggregationBuilder agg = significantTerms("significant_terms") + .field(TEXT_FIELD) + .executionHint(randomExecutionHint()) + .significanceHeuristic(new ChiSquare(true, true)) + .minDocCount(1).shardSize(1000).size(1000) + .subAggregation(subAgg); + + SearchResponse response = client().prepareSearch("test") + .setQuery(query) + .addAggregation(agg) + .get(); + assertSearchResponse(response); + + SignificantTerms sigTerms = response.getAggregations().get("significant_terms"); + assertThat(sigTerms.getBuckets().size(), equalTo(2)); + + for (SignificantTerms.Bucket bucket : sigTerms) { + StringTerms terms = bucket.getAggregations().get("class"); + assertThat(terms.getBuckets().size(), equalTo(2)); + } + } + private void indexEqualTestData() throws ExecutionException, InterruptedException { assertAcked(prepareCreate("test") .setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)) From 0bac74c852cfbe963280f18c0865697639c3a4ca Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Apr 2019 15:53:02 -0400 Subject: [PATCH 010/260] Adjust bwc version for flush parameter validation Relates to #40213 --- .../resources/rest-api-spec/test/indices.flush/10_basic.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.flush/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.flush/10_basic.yml index a4a61b0e598d1..6bea9f0bf2319 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.flush/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.flush/10_basic.yml @@ -54,8 +54,8 @@ --- "Flush parameters validation": - skip: - version: " - 7.9.99" - reason: flush parameters validation is introduced in 8.0 + version: " - 7.0.99" + reason: flush parameters validation is introduced in 7.1.0 - do: indices.create: index: test From 3cdb2acaf22c8ed8fa8ec1adeaac3540dbafd12f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Apr 2019 18:41:21 -0400 Subject: [PATCH 011/260] Fix msu assertion in restore shard history test Since #40249, we always reinitialize max_seq_no_of_updates to max_seq_no when a promoting primary restores history regardless of whether it did rollback previously or not. Closes #40929 --- .../org/elasticsearch/index/shard/IndexShardTests.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index d79a2b652b4f7..05a81c6de3c18 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1092,7 +1092,6 @@ public void testGlobalCheckpointSync() throws IOException { closeShards(replicaShard, primaryShard); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/40929") public void testRestoreLocalHistoryFromTranslogOnPromotion() throws IOException, InterruptedException { final IndexShard indexShard = newStartedShard(false); final int operations = 1024 - scaledRandomIntBetween(0, 1024); @@ -1147,13 +1146,8 @@ public void onFailure(Exception e) { assertThat(indexShard.getLocalCheckpoint(), equalTo(maxSeqNo)); assertThat(indexShard.seqNoStats().getMaxSeqNo(), equalTo(maxSeqNo)); assertThat(getShardDocUIDs(indexShard), equalTo(docsBeforeRollback)); - if (shouldRollback) { - // we conservatively roll MSU forward to maxSeqNo during restoreLocalHistory, ideally it should become just - // currentMaxSeqNoOfUpdates - assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(maxSeqNo)); - } else { - assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(Math.max(currentMaxSeqNoOfUpdates, maxSeqNoOfUpdatesOrDeletes))); - } + // we conservatively roll MSU forward to maxSeqNo during restoreLocalHistory, ideally it should become just currentMaxSeqNoOfUpdates + assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(maxSeqNo)); closeShard(indexShard, false); } From 4188945b83e98b8052a01de2f1d1d4e6c1032874 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 11 Apr 2019 20:53:28 -0400 Subject: [PATCH 012/260] Adjust BWC for #40823 Relates #40823 --- .../java/org/elasticsearch/index/engine/ReadOnlyEngine.java | 2 +- .../indices/recovery/RecoveryCleanFilesRequest.java | 4 ++-- .../org/elasticsearch/indices/recovery/RecoveryTarget.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index b5618d5b9cbc4..777aff88e9dbc 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -130,7 +130,7 @@ protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(final SeqNoStats seqNoStat // created after the refactoring of the Close Index API and its TransportVerifyShardBeforeCloseAction // that guarantee that all operations have been flushed to Lucene. final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated(); - if (indexVersionCreated.onOrAfter(Version.V_8_0_0) || + if (indexVersionCreated.onOrAfter(Version.V_7_1_0) || (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO && indexVersionCreated.onOrAfter(Version.V_6_7_0))) { if (seqNoStats.getMaxSeqNo() != seqNoStats.getGlobalCheckpoint()) { throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo() diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java index bec31642d1fc6..d23f89a769c79 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java @@ -52,7 +52,7 @@ public class RecoveryCleanFilesRequest extends TransportRequest { shardId = ShardId.readShardId(in); snapshotFiles = new Store.MetadataSnapshot(in); totalTranslogOps = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { globalCheckpoint = in.readZLong(); } else { globalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; @@ -66,7 +66,7 @@ public void writeTo(StreamOutput out) throws IOException { shardId.writeTo(out); snapshotFiles.writeTo(out); out.writeVInt(totalTranslogOps); - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { out.writeZLong(globalCheckpoint); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 128e3bbc20818..fbc0dbde51098 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -289,7 +289,7 @@ public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTra state().getTranslog().totalOperations(totalTranslogOps); indexShard().openEngineAndSkipTranslogRecovery(); assert indexShard.getGlobalCheckpoint() >= indexShard.seqNoStats().getMaxSeqNo() || - indexShard.indexSettings().getIndexVersionCreated().before(Version.V_8_0_0) + indexShard.indexSettings().getIndexVersionCreated().before(Version.V_7_1_0) : "global checkpoint is not initialized [" + indexShard.seqNoStats() + "]"; return null; }); @@ -399,7 +399,7 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada store.ensureIndexHasHistoryUUID(); } assert globalCheckpoint >= Long.parseLong(sourceMetaData.getCommitUserData().get(SequenceNumbers.MAX_SEQ_NO)) - || indexShard.indexSettings().getIndexVersionCreated().before(Version.V_8_0_0) : + || indexShard.indexSettings().getIndexVersionCreated().before(Version.V_7_1_0) : "invalid global checkpoint[" + globalCheckpoint + "] source_meta_data [" + sourceMetaData.getCommitUserData() + "]"; final String translogUUID = Translog.createEmptyTranslog( indexShard.shardPath().resolveTranslog(), globalCheckpoint, shardId, indexShard.getPendingPrimaryTerm()); From 4137ad624e93ea2033c6c2253c9ff5b64c0091ba Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Fri, 12 Apr 2019 07:14:30 +0200 Subject: [PATCH 013/260] Improve error message when polygons contains twice the same point in no-consecutive position (#41051) When a polygon contains a self-intersection due to have twice the same point in no-consecutive position, the polygon builder tries to split the polygon. During the split one of the polygons become invalid as it is not closed and an error is thrown which is not related to the real issue. We detect this situation now and throw a more meaningful error. --- .../common/geo/builders/PolygonBuilder.java | 27 ++++++++++++++----- .../common/geo/ShapeBuilderTests.java | 18 +++++++++++++ 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java b/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java index e4751de04bfe3..97503efc033b0 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java @@ -406,7 +406,7 @@ protected static org.elasticsearch.geo.geometry.MultiPolygon multipolygon(Coordi * @param edges a list of edges to which all edges of the component will be added (could be null) * @return number of edges that belong to this component */ - private static int component(final Edge edge, final int id, final ArrayList edges) { + private static int component(final Edge edge, final int id, final ArrayList edges, double[] partitionPoint) { // find a coordinate that is not part of the dateline Edge any = edge; while(any.coordinate.x == +DATELINE || any.coordinate.x == -DATELINE) { @@ -438,6 +438,9 @@ private static int component(final Edge edge, final int id, final ArrayList 0 && current.next != edge) { throw new InvalidShapeException("Shape contains more than one shared point"); } @@ -479,10 +482,20 @@ private static int component(final Edge edge, final int id, final ArrayList> for (int i = 0; i < edges.length; i++) { if (edges[i].component >= 0) { - int length = component(edges[i], -(components.size()+numHoles+1), mainEdges); + double[] partitionPoint = new double[3]; + int length = component(edges[i], -(components.size()+numHoles+1), mainEdges, partitionPoint); List component = new ArrayList<>(); - component.add(coordinates(edges[i], new Coordinate[length+1])); + component.add(coordinates(edges[i], new Coordinate[length+1], partitionPoint)); components.add(component); } } diff --git a/server/src/test/java/org/elasticsearch/common/geo/ShapeBuilderTests.java b/server/src/test/java/org/elasticsearch/common/geo/ShapeBuilderTests.java index 32f1b333c4ead..3c653db2d1537 100644 --- a/server/src/test/java/org/elasticsearch/common/geo/ShapeBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/common/geo/ShapeBuilderTests.java @@ -759,4 +759,22 @@ public void testPolygon3D() { assertEquals(expected, pb.toString()); } + + public void testInvalidSelfCrossingPolygon() { + PolygonBuilder builder = new PolygonBuilder(new CoordinatesBuilder() + .coordinate(0, 0) + .coordinate(0, 2) + .coordinate(1, 1.9) + .coordinate(0.5, 1.8) + .coordinate(1.5, 1.8) + .coordinate(1, 1.9) + .coordinate(2, 2) + .coordinate(2, 0) + .coordinate(0, 0) + ); + Exception e = expectThrows(InvalidShapeException.class, () -> builder.close().buildS4J()); + assertThat(e.getMessage(), containsString("Self-intersection at or near point [")); + e = expectThrows(InvalidShapeException.class, () -> builder.close().buildGeometry()); + assertThat(e.getMessage(), containsString("Self-intersection at or near point [")); + } } From 36ea60517735ef14baf3954e6d53bdf314794ec9 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Fri, 12 Apr 2019 07:29:55 +0200 Subject: [PATCH 014/260] Rollup/DataFrame: disallow partial results (#41114) disallow partial results in rollup and data frame, after this change the client throws an error directly replacing the previous runtime exception thrown, allowing better error handling in implementations. --- .../core/indexing/AsyncTwoPhaseIndexer.java | 24 ++++++++++--------- .../indexing/AsyncTwoPhaseIndexerTests.java | 4 ++-- .../rollup/job/RollupIndexerStateTests.java | 16 +++++++++---- 3 files changed, 26 insertions(+), 18 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 e859e0db754ba..636a3978443e0 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 @@ -14,7 +14,6 @@ import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import java.util.Arrays; import java.util.List; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; @@ -150,8 +149,7 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { // fire off the search. Note this is async, the method will return from here executor.execute(() -> { onStart(now, ActionListener.wrap(r -> { - stats.markStartSearch(); - doNextSearch(buildSearchRequest(), ActionListener.wrap(this::onSearchResponse, this::finishWithSearchFailure)); + nextSearch(ActionListener.wrap(this::onSearchResponse, this::finishWithSearchFailure)); }, e -> { finishAndSetState(); onFailure(e); @@ -305,10 +303,9 @@ private void onSearchResponse(SearchResponse searchResponse) { if (checkState(getState()) == false) { return; } - if (searchResponse.getShardFailures().length != 0) { - throw new RuntimeException("Shard failures encountered while running indexer for job [" + getJobId() + "]: " - + Arrays.toString(searchResponse.getShardFailures())); - } + + // allowPartialSearchResults is set to false, so we should never see shard failures here + assert (searchResponse.getShardFailures().length == 0); stats.incrementNumPages(1); IterationResult iterationResult = doProcess(searchResponse); @@ -362,18 +359,23 @@ private void onBulkResponse(BulkResponse response, JobPosition position) { // TODO probably something more intelligent than every-50 is needed if (stats.getNumPages() > 0 && stats.getNumPages() % 50 == 0) { doSaveState(IndexerState.INDEXING, position, () -> { - stats.markStartSearch(); - doNextSearch(buildSearchRequest(), listener); + nextSearch(listener); }); } else { - stats.markStartSearch(); - doNextSearch(buildSearchRequest(), listener); + nextSearch(listener); } } catch (Exception e) { finishWithIndexingFailure(e); } } + private void nextSearch(ActionListener listener) { + stats.markStartSearch(); + // ensure that partial results are not accepted and cause a search failure + SearchRequest searchRequest = buildSearchRequest().allowPartialSearchResults(false); + doNextSearch(searchRequest, listener); + } + /** * Checks the {@link IndexerState} and returns false if the execution should be * stopped. diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java index f7f97288b2381..b39c4f1a25a76 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexerTests.java @@ -72,7 +72,7 @@ private void awaitForLatch() { protected SearchRequest buildSearchRequest() { assertThat(step, equalTo(1)); ++step; - return null; + return new SearchRequest(); } @Override @@ -151,7 +151,7 @@ protected IterationResult doProcess(SearchResponse searchResponse) { protected SearchRequest buildSearchRequest() { assertThat(step, equalTo(1)); ++step; - return null; + return new SearchRequest(); } @Override diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java index 49baeb4fade04..e6264b02bca16 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponseSections; @@ -197,7 +198,13 @@ protected void doNextSearch(SearchRequest request, ActionListener state = new AtomicReference<>(IndexerState.STOPPED); Function searchFunction = searchRequest -> { - ShardSearchFailure[] failures = new ShardSearchFailure[]{new ShardSearchFailure(new RuntimeException("failed"))}; - return new SearchResponse(null, null, 1, 1, 0, 0, - failures, null); + throw new SearchPhaseExecutionException("query", "Partial shards failure", + new ShardSearchFailure[] { new ShardSearchFailure(new RuntimeException("failed")) }); }; Function bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100); Consumer failureConsumer = e -> { - assertThat(e.getMessage(), startsWith("Shard failures encountered while running indexer for job")); + assertThat(e.getMessage(), startsWith("Partial shards failure")); isFinished.set(true); }; From 03afb814c1e3b8acda987bbdf3cc1baf02016b11 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 12 Apr 2019 12:31:30 +0300 Subject: [PATCH 015/260] Use the built image in docker tests (#40314) Instead of allowing docker-compose to rebuild it. With this change we tag the image with a test label, and use that in the testing as this is simpler that dealing with a dynamically generated docker-compose file. --- distribution/docker/build.gradle | 13 +++++++------ distribution/docker/docker-compose.yml | 8 ++------ 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/distribution/docker/build.gradle b/distribution/docker/build.gradle index 03556edb7c5d7..07e8926b2a367 100644 --- a/distribution/docker/build.gradle +++ b/distribution/docker/build.gradle @@ -90,10 +90,7 @@ void addCopyDockerfileTask(final boolean oss) { } preProcessFixture { - dependsOn taskName("copy", true, "DockerContext") - dependsOn taskName("copy", true, "Dockerfile") - dependsOn taskName("copy", false, "DockerContext") - dependsOn taskName("copy", false, "Dockerfile") + dependsOn assemble } postProcessFixture.doLast { @@ -110,12 +107,16 @@ void addBuildDockerImage(final boolean oss) { dependsOn taskName("copy", oss, "Dockerfile") List tags if (oss) { - tags = [ "docker.elastic.co/elasticsearch/elasticsearch-oss:${VersionProperties.elasticsearch}" ] + tags = [ + "docker.elastic.co/elasticsearch/elasticsearch-oss:${VersionProperties.elasticsearch}", + "elasticsearch-oss:test" + ] } else { tags = [ "elasticsearch:${VersionProperties.elasticsearch}", "docker.elastic.co/elasticsearch/elasticsearch:${VersionProperties.elasticsearch}", - "docker.elastic.co/elasticsearch/elasticsearch-full:${VersionProperties.elasticsearch}" + "docker.elastic.co/elasticsearch/elasticsearch-full:${VersionProperties.elasticsearch}", + "elasticsearch:test", ] } executable 'docker' diff --git a/distribution/docker/docker-compose.yml b/distribution/docker/docker-compose.yml index 3f220aa9e91c0..3207afd501aaf 100644 --- a/distribution/docker/docker-compose.yml +++ b/distribution/docker/docker-compose.yml @@ -2,18 +2,14 @@ version: '3' services: elasticsearch-default: - build: - context: ./build/docker - dockerfile: Dockerfile + image: elasticsearch:test environment: - cluster.name=elasticsearch-default - discovery.type=single-node ports: - "9200" elasticsearch-oss: - build: - context: ./build/oss-docker - dockerfile: Dockerfile + image: elasticsearch-oss:test environment: - cluster.name=elasticsearch-oss - discovery.type=single-node From 97a584e566dcec88721638c0e115ace8accdf21e Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 12 Apr 2019 12:38:29 +0300 Subject: [PATCH 016/260] Simplify testclusters, don't allow cross project clusters (#40972) * Simplify testclusters, don't allow cross project clusters --- .../testclusters/TestClustersPlugin.java | 188 +++++++++--------- 1 file changed, 89 insertions(+), 99 deletions(-) diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java index efbf061fefaf7..e9586f4c4bad5 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClustersPlugin.java @@ -42,11 +42,11 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -56,19 +56,18 @@ public class TestClustersPlugin implements Plugin { private static final String LIST_TASK_NAME = "listTestClusters"; private static final String NODE_EXTENSION_NAME = "testClusters"; - static final String HELPER_CONFIGURATION_NAME = "testclusters"; + private static final String HELPER_CONFIGURATION_NAME = "testclusters"; private static final String SYNC_ARTIFACTS_TASK_NAME = "syncTestClustersArtifacts"; private static final int EXECUTOR_SHUTDOWN_TIMEOUT = 1; private static final TimeUnit EXECUTOR_SHUTDOWN_TIMEOUT_UNIT = TimeUnit.MINUTES; private static final Logger logger = Logging.getLogger(TestClustersPlugin.class); - // this is static because we need a single mapping across multi project builds, as some of the listeners we use, - // like task graph are singletons across multi project builds. - private static final Map> usedClusters = new ConcurrentHashMap<>(); - private static final Map claimsInventory = new ConcurrentHashMap<>(); - private static final Set runningClusters = Collections.synchronizedSet(new HashSet<>()); - private static volatile ExecutorService executorService; + private final Map> usedClusters = new HashMap<>(); + private final Map claimsInventory = new HashMap<>(); + private final Set runningClusters =new HashSet<>(); + private final Thread shutdownHook = new Thread(this::shutDownAllClusters); + private ExecutorService executorService = Executors.newSingleThreadExecutor(); @Override public void apply(Project project) { @@ -81,10 +80,8 @@ public void apply(Project project) { createListClustersTask(project, container); // create DSL for tasks to mark clusters these use - createUseClusterTaskExtension(project); + createUseClusterTaskExtension(project, container); - // There's a single Gradle instance for multi project builds, this means that some configuration needs to be - // done only once even if the plugin is applied multiple times as a part of multi project build if (rootProject.getConfigurations().findByName(HELPER_CONFIGURATION_NAME) == null) { // We use a single configuration on the root project to resolve all testcluster dependencies ( like distros ) // at once, only once without the need to repeat it for each project. This pays off assuming that most @@ -95,18 +92,14 @@ public void apply(Project project) { "ES distributions and plugins." ); - // When running in the Daemon it's possible for this to hold references to past - usedClusters.clear(); - claimsInventory.clear(); - runningClusters.clear(); - // We have a single task to sync the helper configuration to "artifacts dir" // the clusters will look for artifacts there based on the naming conventions. // Tasks that use a cluster will add this as a dependency automatically so it's guaranteed to run early in // the build. rootProject.getTasks().create(SYNC_ARTIFACTS_TASK_NAME, sync -> { sync.getInputs().files((Callable) helperConfiguration::getAsFileTree); - sync.getOutputs().dir(getTestClustersConfigurationExtractDir(project)); + sync.getOutputs().dir(new File(project.getRootProject().getBuildDir(), "testclusters/extract")); + // NOTE: Gradle doesn't allow a lambda here ( fails at runtime ) sync.doLast(new Action() { @Override public void execute(Task task) { @@ -121,33 +114,33 @@ public void execute(Task task) { } else { throw new IllegalArgumentException("Can't extract " + file + " unknown file extension"); } - spec.from(files).into(getTestClustersConfigurationExtractDir(project) + "/" + + spec.from(files).into(new File(project.getRootProject().getBuildDir(), "testclusters/extract") + "/" + resolvedArtifact.getModuleVersion().getId().getGroup() ); })); } }); }); + } - // When we know what tasks will run, we claim the clusters of those task to differentiate between clusters - // that are defined in the build script and the ones that will actually be used in this invocation of gradle - // we use this information to determine when the last task that required the cluster executed so that we can - // terminate the cluster right away and free up resources. - configureClaimClustersHook(project); + // When we know what tasks will run, we claim the clusters of those task to differentiate between clusters + // that are defined in the build script and the ones that will actually be used in this invocation of gradle + // we use this information to determine when the last task that required the cluster executed so that we can + // terminate the cluster right away and free up resources. + configureClaimClustersHook(project); - // Before each task, we determine if a cluster needs to be started for that task. - configureStartClustersHook(project); + // Before each task, we determine if a cluster needs to be started for that task. + configureStartClustersHook(project); - // After each task we determine if there are clusters that are no longer needed. - configureStopClustersHook(project); + // After each task we determine if there are clusters that are no longer needed. + configureStopClustersHook(project); - // configure hooks to make sure no test cluster processes survive the build - configureCleanupHooks(project); + // configure hooks to make sure no test cluster processes survive the build + configureCleanupHooks(project); - // Since we have everything modeled in the DSL, add all the required dependencies e.x. the distribution to the - // configuration so the user doesn't have to repeat this. - autoConfigureClusterDependencies(project, rootProject, container); - } + // Since we have everything modeled in the DSL, add all the required dependencies e.x. the distribution to the + // configuration so the user doesn't have to repeat this. + autoConfigureClusterDependencies(project, rootProject, container); } private NamedDomainObjectContainer createTestClustersContainerExtension(Project project) { @@ -158,7 +151,7 @@ private NamedDomainObjectContainer createTestClustersConta project.getPath(), name, project, - getTestClustersConfigurationExtractDir(project), + new File(project.getRootProject().getBuildDir(), "testclusters/extract"), new File(project.getBuildDir(), "testclusters") ) ); @@ -178,7 +171,7 @@ private void createListClustersTask(Project project, NamedDomainObjectContainer< ); } - private static void createUseClusterTaskExtension(Project project) { + private void createUseClusterTaskExtension(Project project, NamedDomainObjectContainer container) { // register an extension for all current and future tasks, so that any task can declare that it wants to use a // specific cluster. project.getTasks().all((Task task) -> @@ -187,6 +180,12 @@ private static void createUseClusterTaskExtension(Project project) { "useCluster", new Closure(project, task) { public void doCall(ElasticsearchCluster cluster) { + if (container.contains(cluster) == false) { + throw new TestClustersException( + "Task " + task.getPath() + " can't use test cluster from" + + " another project " + cluster + ); + } Object thisObject = this.getThisObject(); if (thisObject instanceof Task == false) { throw new AssertionError("Expected " + thisObject + " to be an instance of " + @@ -201,35 +200,38 @@ public void doCall(ElasticsearchCluster cluster) { ); } - private static void configureClaimClustersHook(Project project) { - project.getGradle().getTaskGraph().whenReady(taskExecutionGraph -> - taskExecutionGraph.getAllTasks() - .forEach(task -> - usedClusters.getOrDefault(task, Collections.emptyList()).forEach(each -> { - synchronized (claimsInventory) { - claimsInventory.put(each, claimsInventory.getOrDefault(each, 0) + 1); - } - each.freeze(); - }) - ) - ); + private void configureClaimClustersHook(Project project) { + // Once we know all the tasks that need to execute, we claim all the clusters that belong to those and count the + // claims so we'll know when it's safe to stop them. + project.getGradle().getTaskGraph().whenReady(taskExecutionGraph -> { + Set forExecution = taskExecutionGraph.getAllTasks().stream() + .map(Task::getPath) + .collect(Collectors.toSet()); + + usedClusters.forEach((task, listOfClusters) -> + listOfClusters.forEach(elasticsearchCluster -> { + if (forExecution.contains(task.getPath())) { + elasticsearchCluster.freeze(); + claimsInventory.put(elasticsearchCluster, claimsInventory.getOrDefault(elasticsearchCluster, 0) + 1); + } + })); + + logger.info("Claims inventory: {}", claimsInventory); + }); } - private static void configureStartClustersHook(Project project) { + private void configureStartClustersHook(Project project) { project.getGradle().addListener( new TaskActionListener() { @Override public void beforeActions(Task task) { // we only start the cluster before the actions, so we'll not start it if the task is up-to-date - final List clustersToStart; - synchronized (runningClusters) { - clustersToStart = usedClusters.getOrDefault(task,Collections.emptyList()).stream() - .filter(each -> runningClusters.contains(each) == false) - .collect(Collectors.toList()); - runningClusters.addAll(clustersToStart); - } - clustersToStart.forEach(ElasticsearchCluster::start); - + usedClusters.getOrDefault(task, Collections.emptyList()).stream() + .filter(each -> runningClusters.contains(each) == false) + .forEach(elasticsearchCluster -> { + elasticsearchCluster.start(); + runningClusters.add(elasticsearchCluster); + }); } @Override public void afterActions(Task task) {} @@ -237,7 +239,7 @@ public void afterActions(Task task) {} ); } - private static void configureStopClustersHook(Project project) { + private void configureStopClustersHook(Project project) { project.getGradle().addListener( new TaskExecutionListener() { @Override @@ -251,25 +253,19 @@ public void afterExecute(Task task, TaskState state) { if (state.getFailure() != null) { // If the task fails, and other tasks use this cluster, the other task will likely never be // executed at all, so we will never get to un-claim and terminate it. - // The downside is that with multi project builds if that other task is in a different - // project and executing right now, we may terminate the cluster while it's running it. clustersUsedByTask.forEach(each -> each.stop(true)); } else { - clustersUsedByTask.forEach(each -> { - synchronized (claimsInventory) { - claimsInventory.put(each, claimsInventory.get(each) - 1); - } - }); - final List stoppable; - synchronized (runningClusters) { - stoppable = claimsInventory.entrySet().stream() - .filter(entry -> entry.getValue() == 0) - .filter(entry -> runningClusters.contains(entry.getKey())) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - runningClusters.removeAll(stoppable); - } - stoppable.forEach(each -> each.stop(false)); + clustersUsedByTask.forEach( + each -> claimsInventory.put(each, claimsInventory.getOrDefault(each, 0) - 1) + ); + claimsInventory.entrySet().stream() + .filter(entry -> entry.getValue() == 0) + .filter(entry -> runningClusters.contains(entry.getKey())) + .map(Map.Entry::getKey) + .forEach(each -> { + each.stop(false); + runningClusters.remove(each); + }); } } @Override @@ -278,10 +274,6 @@ public void beforeExecute(Task task) {} ); } - static File getTestClustersConfigurationExtractDir(Project project) { - return new File(project.getRootProject().getBuildDir(), "testclusters/extract"); - } - /** * Boilerplate to get testClusters container extension * @@ -354,15 +346,9 @@ private static void autoConfigureClusterDependencies( }))); } - private static void configureCleanupHooks(Project project) { - synchronized (runningClusters) { - if (executorService == null || executorService.isTerminated()) { - executorService = Executors.newSingleThreadExecutor(); - } else { - throw new IllegalStateException("Trying to configure executor service twice"); - } - } + private void configureCleanupHooks(Project project) { // When the Gradle daemon is used, it will interrupt all threads when the build concludes. + // This is our signal to clean up executorService.submit(() -> { while (true) { try { @@ -375,17 +361,21 @@ private static void configureCleanupHooks(Project project) { } }); - project.getGradle().buildFinished(buildResult -> { - logger.info("Build finished"); - shutdownExecutorService(); - }); // When the Daemon is not used, or runs into issues, rely on a shutdown hook // When the daemon is used, but does not work correctly and eventually dies off (e.x. due to non interruptible // thread in the build) process will be stopped eventually when the daemon dies. - Runtime.getRuntime().addShutdownHook(new Thread(TestClustersPlugin::shutDownAllClusters)); + Runtime.getRuntime().addShutdownHook(shutdownHook); + + // When we don't run into anything out of the ordinary, and the build completes, makes sure to clean up + project.getGradle().buildFinished(buildResult -> { + shutdownExecutorService(); + if (false == Runtime.getRuntime().removeShutdownHook(shutdownHook)) { + logger.info("Trying to deregister shutdown hook when it was not registered."); + } + }); } - private static void shutdownExecutorService() { + private void shutdownExecutorService() { executorService.shutdownNow(); try { if (executorService.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT, EXECUTOR_SHUTDOWN_TIMEOUT_UNIT) == false) { @@ -400,13 +390,13 @@ private static void shutdownExecutorService() { } } - private static void shutDownAllClusters() { - if (logger.isDebugEnabled()) { - logger.debug("Shutting down all test clusters", new RuntimeException()); - } + private void shutDownAllClusters() { synchronized (runningClusters) { - runningClusters.forEach(each -> each.stop(true)); - runningClusters.clear(); + Iterator iterator = runningClusters.iterator(); + while (iterator.hasNext()) { + iterator.remove(); + iterator.next().stop(true); + } } } From cae627681165bfee869b5a2dd72e45a134b76ddd Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 12 Apr 2019 10:45:09 +0100 Subject: [PATCH 017/260] Clarify initial_master_nodes must match node.name (#41137) ... and emphasize that this includes any trailing qualifiers. --- .../modules/discovery/bootstrapping.asciidoc | 29 +++++++++++++++++++ .../discovery-settings.asciidoc | 8 ++++- 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/docs/reference/modules/discovery/bootstrapping.asciidoc b/docs/reference/modules/discovery/bootstrapping.asciidoc index 972c0b8a0645a..f4a775e12df99 100644 --- a/docs/reference/modules/discovery/bootstrapping.asciidoc +++ b/docs/reference/modules/discovery/bootstrapping.asciidoc @@ -56,6 +56,35 @@ cluster.initial_master_nodes: - master-node-hostname -------------------------------------------------- +[NOTE] +================================================== + +The node names used in this list must exactly match the `node.name` properties +of the nodes. By default the node name is set to the machine's hostname which +may or may not be fully-qualified depending on your system configuration. If +each node name is a fully-qualified domain name such as `master-a.example.com` +then you must use fully-qualified domain names in the +`cluster.initial_master_nodes` list too; conversely if your node names are bare +hostnames (without the `.example.com` suffix) then you must use bare hostnames +in the `cluster.initial_master_nodes` list. If you use a mix of fully-qualifed +and bare hostnames, or there is some other mismatch between `node.name` and +`cluster.initial_master_nodes`, then the cluster will not form successfully and +you will see log messages like the following. + +[source,text] +-------------------------------------------------- +[master-a.example.com] master not discovered yet, this node has +not previously joined a bootstrapped (v7+) cluster, and this +node must discover master-eligible nodes [master-a, master-b] to +bootstrap a cluster: have discovered [{master-b.example.com}{... +-------------------------------------------------- + +This message shows the node names `master-a.example.com` and +`master-b.example.com` as well as the `cluster.initial_master_nodes` entries +`master-a` and `master-b`, and it is apparent that they do not match exactly. + +================================================== + Like all node settings, it is also possible to specify the initial set of master nodes on the command-line that is used to start Elasticsearch: diff --git a/docs/reference/setup/important-settings/discovery-settings.asciidoc b/docs/reference/setup/important-settings/discovery-settings.asciidoc index 9650a2b1abd13..4edf5cfbab5e9 100644 --- a/docs/reference/setup/important-settings/discovery-settings.asciidoc +++ b/docs/reference/setup/important-settings/discovery-settings.asciidoc @@ -58,7 +58,13 @@ cluster.initial_master_nodes: `transport.port` if not specified. <2> If a hostname resolves to multiple IP addresses then the node will attempt to discover other nodes at all resolved addresses. -<3> Initial master nodes can be identified by their <>. +<3> Initial master nodes can be identified by their <>. + Make sure that the value here matches the `node.name` exactly. If you use a + fully-qualified domain name such as `master-node-a.example.com` for your + node names then you must use the fully-qualified name in this list; + conversely if `node.name` is a bare hostname without any trailing + qualifiers then you must also omit the trailing qualifiers in + `cluster.initial_master_nodes`. <4> Initial master nodes can also be identified by their IP address. <5> If multiple master nodes share an IP address then the port must be used to disambiguate them. From 31b5e36770a428b94555094ae2ebcf57a9faf5ef Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 12 Apr 2019 08:20:28 -0400 Subject: [PATCH 018/260] Fix some documentation urls in rest-api-spec (#40618) Fixes some documentation urls in the rest-api-spec. Some of these URLs pointed to 404s and a few others pointed to deprecated documentation when we have better documentation now. I'm not consistent about `master` vs `current` because we're not consistent in other places and I think we should solve all of those at once with something a little more automatic. = --- .../resources/rest-api-spec/api/ingest.delete_pipeline.json | 2 +- .../main/resources/rest-api-spec/api/ingest.get_pipeline.json | 2 +- .../main/resources/rest-api-spec/api/ingest.processor_grok.json | 2 +- .../main/resources/rest-api-spec/api/ingest.put_pipeline.json | 2 +- .../src/main/resources/rest-api-spec/api/ingest.simulate.json | 2 +- .../src/test/resources/rest-api-spec/api/license.delete.json | 2 +- .../src/test/resources/rest-api-spec/api/license.get.json | 2 +- .../resources/rest-api-spec/api/license.get_basic_status.json | 2 +- .../resources/rest-api-spec/api/license.get_trial_status.json | 2 +- .../src/test/resources/rest-api-spec/api/license.post.json | 2 +- .../resources/rest-api-spec/api/license.post_start_basic.json | 2 +- .../resources/rest-api-spec/api/license.post_start_trial.json | 2 +- .../resources/rest-api-spec/api/migration.deprecations.json | 2 +- .../resources/rest-api-spec/api/ml.find_file_structure.json | 2 +- .../src/test/resources/rest-api-spec/api/monitoring.bulk.json | 2 +- .../rest-api-spec/api/security.get_user_privileges.json | 2 +- 16 files changed, 16 insertions(+), 16 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json index 1c515e4509581..c3b51de862097 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.delete_pipeline.json @@ -1,6 +1,6 @@ { "ingest.delete_pipeline": { - "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/delete-pipeline-api.html", "methods": [ "DELETE" ], "url": { "path": "/_ingest/pipeline/{id}", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json index 317250874233b..16a07e072b771 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json @@ -1,6 +1,6 @@ { "ingest.get_pipeline": { - "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/get-pipeline-api.html", "methods": [ "GET" ], "url": { "path": "/_ingest/pipeline/{id}", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json index 55afada728122..bf40be853e2d1 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.processor_grok.json @@ -1,6 +1,6 @@ { "ingest.processor_grok": { - "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/grok-processor.html#grok-processor-rest-get", "methods": [ "GET" ], "url": { "path": "/_ingest/processor/grok", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json index e4c3c2eb3f9a3..1ea77901d8dbd 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.put_pipeline.json @@ -1,6 +1,6 @@ { "ingest.put_pipeline": { - "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/put-pipeline-api.html", "methods": [ "PUT" ], "url": { "path": "/_ingest/pipeline/{id}", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json index d02f97d81dd70..c16008ad6b659 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.simulate.json @@ -1,6 +1,6 @@ { "ingest.simulate": { - "documentation": "https://www.elastic.co/guide/en/elasticsearch/plugins/master/ingest.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/simulate-pipeline-api.html", "methods": [ "GET", "POST" ], "url": { "path": "/_ingest/pipeline/_simulate", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.delete.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.delete.json index 315b283699b62..a85552e557597 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.delete.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.delete.json @@ -1,6 +1,6 @@ { "license.delete": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/delete-license.html", "methods": ["DELETE"], "url": { "path": "/_license", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get.json index 0de1fb48536e8..07a570935a6b7 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get.json @@ -1,6 +1,6 @@ { "license.get": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/get-license.html", "methods": ["GET"], "url": { "path": "/_license", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_basic_status.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_basic_status.json index e9823b449087e..cfb5608ac4063 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_basic_status.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_basic_status.json @@ -1,6 +1,6 @@ { "license.get_basic_status": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/get-basic-status.html", "methods": ["GET"], "url": { "path": "/_license/basic_status", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_trial_status.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_trial_status.json index 54f6b0a8c7d43..daeb4913ad9eb 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_trial_status.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.get_trial_status.json @@ -1,6 +1,6 @@ { "license.get_trial_status": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/get-trial-status.html", "methods": ["GET"], "url": { "path": "/_license/trial_status", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post.json index 23d597a3c1964..1bd78a1fb6af5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post.json @@ -1,6 +1,6 @@ { "license.post": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/update-license.html", "methods": ["PUT", "POST"], "url": { "path": "/_license", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_basic.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_basic.json index 2b9da7d47c685..91da4de9ff0a2 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_basic.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_basic.json @@ -1,6 +1,6 @@ { "license.post_start_basic": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/start-basic.html", "methods": ["POST"], "url": { "path": "/_license/start_basic", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_trial.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_trial.json index d0e3afcbb1e1f..55a73df4946b5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_trial.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/license.post_start_trial.json @@ -1,6 +1,6 @@ { "license.post_start_trial": { - "documentation": "https://www.elastic.co/guide/en/x-pack/current/license-management.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/start-trial.html", "methods": ["POST"], "url": { "path": "/_license/start_trial", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/migration.deprecations.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/migration.deprecations.json index 989b206919ba9..7c7ad54f095a6 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/migration.deprecations.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/migration.deprecations.json @@ -1,6 +1,6 @@ { "migration.deprecations": { - "documentation": "http://www.elastic.co/guide/en/migration/current/migration-api-deprecation.html", + "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current/migration-api-deprecation.html", "methods": [ "GET" ], "url": { "path": "/{index}/_migration/deprecations", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.find_file_structure.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.find_file_structure.json index 94b69951e32ec..4e5550ae824a9 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.find_file_structure.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.find_file_structure.json @@ -1,6 +1,6 @@ { "ml.find_file_structure": { - "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current/ml-file-structure.html", + "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/current/ml-find-file-structure.html", "methods": [ "POST" ], "url": { "path": "/_ml/find_file_structure", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json index 55ce7b9ba6170..a299e2adc4bc7 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json @@ -1,6 +1,6 @@ { "monitoring.bulk": { - "documentation": "http://www.elastic.co/guide/en/monitoring/current/appendix-api-bulk.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/master/es-monitoring.html", "methods": ["POST", "PUT"], "url": { "path": "/_monitoring/bulk", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/security.get_user_privileges.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/security.get_user_privileges.json index 45f1e4a08c6d8..b60298f89e017 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/security.get_user_privileges.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/security.get_user_privileges.json @@ -1,6 +1,6 @@ { "security.get_user_privileges": { - "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/current/security-api-get-user-privileges.html", + "documentation": "https://www.elastic.co/guide/en/elasticsearch/reference/current/security-api-get-privileges.html", "methods": [ "GET" ], "url": { "path": "/_security/user/_privileges", From 2141af6b840eefb98210f914cd849970bba83666 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 12 Apr 2019 11:19:21 -0400 Subject: [PATCH 019/260] [DOCS] Fix code block length for Asciidoctor migration (#41151) --- docs/reference/setup/install/docker.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/setup/install/docker.asciidoc b/docs/reference/setup/install/docker.asciidoc index 83f80b569a817..9d03edb9e7e9d 100644 --- a/docs/reference/setup/install/docker.asciidoc +++ b/docs/reference/setup/install/docker.asciidoc @@ -82,7 +82,7 @@ The `vm.max_map_count` setting should be set permanently in `/etc/sysctl.conf`: -------------------------------------------- $ grep vm.max_map_count /etc/sysctl.conf vm.max_map_count=262144 ----------------------------------- +-------------------------------------------- To apply the setting on a live system type: `sysctl -w vm.max_map_count=262144` -- From eb818550b173eaab5bc015ca4cb4d9abbebb4919 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 12 Apr 2019 11:31:08 -0400 Subject: [PATCH 020/260] Docs: Fix rendering of APIs that mention version (#41154) Fixes rendering the `version_qualified` attribute in the docs. This attribute includes `-SNAPSHOT` but does not include `-alpha` or `-beta` and represents the `version` field as returned by `GET /` or `GET /_cat/plugins`. Without this change we just drop the entire line on the floor rather than render it so the output looks bad. --- docs/Versions.asciidoc | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/Versions.asciidoc b/docs/Versions.asciidoc index bbd001b252e0d..9362ae0aea796 100644 --- a/docs/Versions.asciidoc +++ b/docs/Versions.asciidoc @@ -1,4 +1,8 @@ :version: 8.0.0-alpha1 +//// +bare_version never includes -alpha or -beta +//// +:bare_version: 8.0.0 :major-version: 8.x :prev-major-version: 7.x :lucene_version: 8.0.0 @@ -42,6 +46,7 @@ ifeval::["{release-state}"=="unreleased"] :percolator-client-javadoc: https://snapshots.elastic.co/javadoc/org/elasticsearch/plugin/percolator-client/{version}-SNAPSHOT :matrixstats-client-javadoc: https://snapshots.elastic.co/javadoc/org/elasticsearch/plugin/aggs-matrix-stats-client/{version}-SNAPSHOT :rank-eval-client-javadoc: https://snapshots.elastic.co/javadoc/org/elasticsearch/plugin/rank-eval-client/{version}-SNAPSHOT +:version_qualified: {bare_version}-SNAPSHOT endif::[] ifeval::["{release-state}"!="unreleased"] @@ -55,6 +60,7 @@ ifeval::["{release-state}"!="unreleased"] :percolator-client-javadoc: https://artifacts.elastic.co/javadoc/org/elasticsearch/plugin/percolator-client/{version} :matrixstats-client-javadoc: https://artifacts.elastic.co/javadoc/org/elasticsearch/plugin/aggs-matrix-stats-client/{version} :rank-eval-client-javadoc: https://artifacts.elastic.co/javadoc/org/elasticsearch/plugin/rank-eval-client/{version} +:version_qualified: {bare_version} endif::[] :javadoc-client: {rest-high-level-client-javadoc}/org/elasticsearch/client From 2106c8a1b2a062514eca5b604373f980954d4253 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Fri, 12 Apr 2019 09:49:46 -0600 Subject: [PATCH 021/260] Use environment settings instead of state settings for Watcher config (#41087) * Use environment settings instead of state settings for Watcher config Prior to this we used the settings from cluster state to see whether ILM was enabled of disabled, however, these settings don't accurately reflect the `xpack.ilm.enabled` setting in `elasticsearch.yml`. This commit changes to using the `Environment` settings, which correctly reflect the ILM enabled setting. Resolves #41042 * Rename settings object to nodeSettings * Use correct template list in WatcherRestIT * Use correct template list in other tests --- .../WatcherIndexTemplateRegistryField.java | 5 +++- .../elasticsearch/xpack/watcher/Watcher.java | 2 +- .../support/WatcherIndexTemplateRegistry.java | 10 +++++--- .../WatcherIndexTemplateRegistryTests.java | 23 +++++++++++-------- ...cherWithSecurityClientYamlTestSuiteIT.java | 2 +- .../SmokeTestWatcherTestSuiteIT.java | 2 +- .../smoketest/WatcherRestIT.java | 2 +- .../smoketest/WatcherJiraYamlTestSuiteIT.java | 2 +- .../WatcherPagerDutyYamlTestSuiteIT.java | 2 +- .../WatcherSlackYamlTestSuiteIT.java | 2 +- 10 files changed, 32 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/WatcherIndexTemplateRegistryField.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/WatcherIndexTemplateRegistryField.java index ac4b950ea0524..4007b06ee7eca 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/WatcherIndexTemplateRegistryField.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/WatcherIndexTemplateRegistryField.java @@ -21,7 +21,10 @@ public final class WatcherIndexTemplateRegistryField { public static final String TRIGGERED_TEMPLATE_NAME = ".triggered_watches"; public static final String WATCHES_TEMPLATE_NAME = ".watches"; public static final String[] TEMPLATE_NAMES = new String[] { - HISTORY_TEMPLATE_NAME, TRIGGERED_TEMPLATE_NAME, WATCHES_TEMPLATE_NAME + HISTORY_TEMPLATE_NAME, TRIGGERED_TEMPLATE_NAME, WATCHES_TEMPLATE_NAME + }; + public static final String[] TEMPLATE_NAMES_NO_ILM = new String[] { + HISTORY_TEMPLATE_NAME_NO_ILM, TRIGGERED_TEMPLATE_NAME, WATCHES_TEMPLATE_NAME }; private WatcherIndexTemplateRegistryField() {} diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index 6888019b2699c..f5f12d4fd244a 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -267,7 +267,7 @@ public Collection createComponents(Client client, ClusterService cluster throw new UncheckedIOException(e); } - new WatcherIndexTemplateRegistry(clusterService, threadPool, client, xContentRegistry); + new WatcherIndexTemplateRegistry(environment.settings(), clusterService, threadPool, client, xContentRegistry); // http client httpClient = new HttpClient(settings, getSslService(), cryptoService, clusterService); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java index 735bf04c7216f..4ebcc5a8f4173 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java @@ -17,6 +17,7 @@ import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; @@ -63,14 +64,17 @@ public class WatcherIndexTemplateRegistry implements ClusterStateListener { private static final Logger logger = LogManager.getLogger(WatcherIndexTemplateRegistry.class); + private final Settings nodeSettings; private final Client client; private final ThreadPool threadPool; private final NamedXContentRegistry xContentRegistry; private final ConcurrentMap templateCreationsInProgress = new ConcurrentHashMap<>(); private final AtomicBoolean historyPolicyCreationInProgress = new AtomicBoolean(); - public WatcherIndexTemplateRegistry(ClusterService clusterService, ThreadPool threadPool, Client client, + public WatcherIndexTemplateRegistry(Settings nodeSettings, ClusterService clusterService, + ThreadPool threadPool, Client client, NamedXContentRegistry xContentRegistry) { + this.nodeSettings = nodeSettings; this.client = client; this.threadPool = threadPool; this.xContentRegistry = xContentRegistry; @@ -104,7 +108,7 @@ public void clusterChanged(ClusterChangedEvent event) { } private void addTemplatesIfMissing(ClusterState state) { - boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(state.metaData().settings()); + boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(this.nodeSettings); final TemplateConfig[] indexTemplates = ilmSupported ? TEMPLATE_CONFIGS : TEMPLATE_CONFIGS_NO_ILM; for (TemplateConfig template : indexTemplates) { final String templateName = template.getTemplateName(); @@ -153,7 +157,7 @@ LifecyclePolicy loadWatcherHistoryPolicy() { } private void addIndexLifecyclePolicyIfMissing(ClusterState state) { - boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(state.metaData().settings()); + boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(this.nodeSettings); if (ilmSupported && historyPolicyCreationInProgress.compareAndSet(false, true)) { final LifecyclePolicy policyOnDisk = loadWatcherHistoryPolicy(); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java index 7ede531305348..bd55e75795382 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java @@ -73,11 +73,13 @@ public class WatcherIndexTemplateRegistryTests extends ESTestCase { private WatcherIndexTemplateRegistry registry; private NamedXContentRegistry xContentRegistry; + private ClusterService clusterService; + private ThreadPool threadPool; private Client client; @Before public void createRegistryAndClient() { - ThreadPool threadPool = mock(ThreadPool.class); + threadPool = mock(ThreadPool.class); when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); when(threadPool.generic()).thenReturn(EsExecutors.newDirectExecutorService()); @@ -94,14 +96,14 @@ public void createRegistryAndClient() { return null; }).when(indicesAdminClient).putTemplate(any(PutIndexTemplateRequest.class), any(ActionListener.class)); - ClusterService clusterService = mock(ClusterService.class); + clusterService = mock(ClusterService.class); List entries = new ArrayList<>(ClusterModule.getNamedXWriteables()); entries.addAll(Arrays.asList( new NamedXContentRegistry.Entry(LifecycleType.class, new ParseField(TimeseriesLifecycleType.TYPE), (p) -> TimeseriesLifecycleType.INSTANCE), new NamedXContentRegistry.Entry(LifecycleAction.class, new ParseField(DeleteAction.NAME), DeleteAction::parse))); xContentRegistry = new NamedXContentRegistry(entries); - registry = new WatcherIndexTemplateRegistry(clusterService, threadPool, client, xContentRegistry); + registry = new WatcherIndexTemplateRegistry(Settings.EMPTY, clusterService, threadPool, client, xContentRegistry); } public void testThatNonExistingTemplatesAreAddedImmediately() { @@ -130,9 +132,10 @@ public void testThatNonExistingTemplatesAreAddedEvenWithILMDisabled() { DiscoveryNode node = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Version.CURRENT); DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build(); - ClusterChangedEvent event = createClusterChangedEvent(Settings.builder() + registry = new WatcherIndexTemplateRegistry(Settings.builder() .put(XPackSettings.INDEX_LIFECYCLE_ENABLED.getKey(), false).build(), - Collections.emptyList(), Collections.emptyMap(), nodes); + clusterService, threadPool, client, xContentRegistry); + ClusterChangedEvent event = createClusterChangedEvent(Settings.EMPTY, Collections.emptyList(), Collections.emptyMap(), nodes); registry.clusterChanged(event); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(PutIndexTemplateRequest.class); verify(client.admin().indices(), times(3)).putTemplate(argumentCaptor.capture(), anyObject()); @@ -142,8 +145,9 @@ public void testThatNonExistingTemplatesAreAddedEvenWithILMDisabled() { WatcherIndexTemplateRegistryField.TRIGGERED_TEMPLATE_NAME), nodes); registry.clusterChanged(newEvent); ArgumentCaptor captor = ArgumentCaptor.forClass(PutIndexTemplateRequest.class); - verify(client.admin().indices(), times(4)).putTemplate(captor.capture(), anyObject()); + verify(client.admin().indices(), times(5)).putTemplate(captor.capture(), anyObject()); captor.getAllValues().forEach(req -> assertNull(req.settings().get("index.lifecycle.name"))); + verify(client, times(0)).execute(eq(PutLifecycleAction.INSTANCE), anyObject(), anyObject()); } public void testThatNonExistingPoliciesAreAddedImmediately() { @@ -171,9 +175,10 @@ public void testNoPolicyButILMDisabled() { DiscoveryNode node = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Version.CURRENT); DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build(); - ClusterChangedEvent event = createClusterChangedEvent(Settings.builder() - .put(XPackSettings.INDEX_LIFECYCLE_ENABLED.getKey(), false).build(), - Collections.emptyList(), Collections.emptyMap(), nodes); + registry = new WatcherIndexTemplateRegistry(Settings.builder() + .put(XPackSettings.INDEX_LIFECYCLE_ENABLED.getKey(), false).build(), + clusterService, threadPool, client, xContentRegistry); + ClusterChangedEvent event = createClusterChangedEvent(Settings.EMPTY, Collections.emptyList(), Collections.emptyMap(), nodes); registry.clusterChanged(event); verify(client, times(0)).execute(eq(PutLifecycleAction.INSTANCE), anyObject(), anyObject()); } diff --git a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java index 5eabd512dc525..879be233fa180 100644 --- a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java +++ b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java @@ -76,7 +76,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES) { + for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM) { ClientYamlTestResponse templateExistsResponse = getAdminExecutionContext().callApi("indices.exists_template", singletonMap("name", template), emptyList(), emptyMap()); assertThat(templateExistsResponse.getStatusCode(), is(200)); diff --git a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java index 7066f1b80ac46..e08b2a20278f0 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java +++ b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java @@ -64,7 +64,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES) { + for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM) { Response templateExistsResponse = adminClient().performRequest(new Request("HEAD", "/_template/" + template)); assertThat(templateExistsResponse.getStatusLine().getStatusCode(), is(200)); } diff --git a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java index 1d7759b28b9fe..19c82c8cef799 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java +++ b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java @@ -58,7 +58,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES) { + for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM) { ClientYamlTestResponse templateExistsResponse = getAdminExecutionContext().callApi("indices.exists_template", singletonMap("name", template), emptyList(), emptyMap()); assertThat(templateExistsResponse.getStatusCode(), is(200)); diff --git a/x-pack/qa/third-party/jira/src/test/java/org/elasticsearch/smoketest/WatcherJiraYamlTestSuiteIT.java b/x-pack/qa/third-party/jira/src/test/java/org/elasticsearch/smoketest/WatcherJiraYamlTestSuiteIT.java index 0eca4d03dfd06..8f8792f26971e 100644 --- a/x-pack/qa/third-party/jira/src/test/java/org/elasticsearch/smoketest/WatcherJiraYamlTestSuiteIT.java +++ b/x-pack/qa/third-party/jira/src/test/java/org/elasticsearch/smoketest/WatcherJiraYamlTestSuiteIT.java @@ -37,7 +37,7 @@ public static Iterable parameters() throws Exception { @Before public void startWatcher() throws Exception { - final List watcherTemplates = Arrays.asList(WatcherIndexTemplateRegistryField.TEMPLATE_NAMES); + final List watcherTemplates = Arrays.asList(WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM); assertBusy(() -> { try { getAdminExecutionContext().callApi("watcher.start", emptyMap(), emptyList(), emptyMap()); diff --git a/x-pack/qa/third-party/pagerduty/src/test/java/org/elasticsearch/smoketest/WatcherPagerDutyYamlTestSuiteIT.java b/x-pack/qa/third-party/pagerduty/src/test/java/org/elasticsearch/smoketest/WatcherPagerDutyYamlTestSuiteIT.java index e111bbd10696b..b9a628f71f972 100644 --- a/x-pack/qa/third-party/pagerduty/src/test/java/org/elasticsearch/smoketest/WatcherPagerDutyYamlTestSuiteIT.java +++ b/x-pack/qa/third-party/pagerduty/src/test/java/org/elasticsearch/smoketest/WatcherPagerDutyYamlTestSuiteIT.java @@ -37,7 +37,7 @@ public static Iterable parameters() throws Exception { @Before public void startWatcher() throws Exception { - final List watcherTemplates = Arrays.asList(WatcherIndexTemplateRegistryField.TEMPLATE_NAMES); + final List watcherTemplates = Arrays.asList(WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM); assertBusy(() -> { try { getAdminExecutionContext().callApi("watcher.start", emptyMap(), emptyList(), emptyMap()); diff --git a/x-pack/qa/third-party/slack/src/test/java/org/elasticsearch/smoketest/WatcherSlackYamlTestSuiteIT.java b/x-pack/qa/third-party/slack/src/test/java/org/elasticsearch/smoketest/WatcherSlackYamlTestSuiteIT.java index 7021548109fd5..01eeae442b2e0 100644 --- a/x-pack/qa/third-party/slack/src/test/java/org/elasticsearch/smoketest/WatcherSlackYamlTestSuiteIT.java +++ b/x-pack/qa/third-party/slack/src/test/java/org/elasticsearch/smoketest/WatcherSlackYamlTestSuiteIT.java @@ -37,7 +37,7 @@ public static Iterable parameters() throws Exception { @Before public void startWatcher() throws Exception { - final List watcherTemplates = Arrays.asList(WatcherIndexTemplateRegistryField.TEMPLATE_NAMES); + final List watcherTemplates = Arrays.asList(WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM); assertBusy(() -> { try { getAdminExecutionContext().callApi("watcher.start", emptyMap(), emptyList(), emptyMap()); From b5252f959431a16499f3e21c3aa1913770eec03b Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 12 Apr 2019 12:18:20 -0400 Subject: [PATCH 022/260] [DOCS] Fix code block length for Asciidoctor migration (#41153) --- docs/reference/indices/split-index.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/indices/split-index.asciidoc b/docs/reference/indices/split-index.asciidoc index ade0a8075d582..d79c0b0e9e250 100644 --- a/docs/reference/indices/split-index.asciidoc +++ b/docs/reference/indices/split-index.asciidoc @@ -92,7 +92,7 @@ PUT my_source_index "index.number_of_shards" : 1 } } -------------------------------------------------- +-------------------------------------------------- // CONSOLE In order to split an index, the index must be marked as read-only, From bd3ff4a1f388dfbcc6fc33bb29cd7baa932caed3 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 12 Apr 2019 12:26:39 -0400 Subject: [PATCH 023/260] [DOCS] Fix code block length for Asciidoctor migration (#41152) --- docs/reference/search/rank-eval.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/search/rank-eval.asciidoc b/docs/reference/search/rank-eval.asciidoc index c549b5e7a689b..cf23121f53b33 100644 --- a/docs/reference/search/rank-eval.asciidoc +++ b/docs/reference/search/rank-eval.asciidoc @@ -44,7 +44,7 @@ GET /my_index/_rank_eval "mean_reciprocal_rank": { ... } <3> } } ------------------------------- +----------------------------- // NOTCONSOLE <1> a set of typical search requests, together with their provided ratings @@ -77,7 +77,7 @@ The request section contains several search requests typical to your application ] } ] ------------------------------- +----------------------------- // NOTCONSOLE <1> the search requests id, used to group result details later From 454e4967f14f0a8e5377170be73510882764f0fd Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 12 Apr 2019 10:43:10 -0600 Subject: [PATCH 024/260] Add ILM poll_interval limit to breaking changes (#41095) ILM's poll_interval setting now has a lower limit of 1 second, which is technically a breaking change. --- docs/reference/migration/migrate_8_0.asciidoc | 2 ++ .../migration/migrate_8_0/ilm.asciidoc | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 docs/reference/migration/migrate_8_0/ilm.asciidoc diff --git a/docs/reference/migration/migrate_8_0.asciidoc b/docs/reference/migration/migrate_8_0.asciidoc index 56a9f582142b7..48c883c6197d9 100644 --- a/docs/reference/migration/migrate_8_0.asciidoc +++ b/docs/reference/migration/migrate_8_0.asciidoc @@ -17,6 +17,7 @@ coming[8.0.0] * <> * <> * <> +* <> * <> //NOTE: The notable-breaking-changes tagged regions are re-used in the @@ -46,4 +47,5 @@ include::migrate_8_0/mappings.asciidoc[] include::migrate_8_0/packaging.asciidoc[] include::migrate_8_0/snapshots.asciidoc[] include::migrate_8_0/security.asciidoc[] +include::migrate_8_0/ilm.asciidoc[] include::migrate_8_0/java.asciidoc[] diff --git a/docs/reference/migration/migrate_8_0/ilm.asciidoc b/docs/reference/migration/migrate_8_0/ilm.asciidoc new file mode 100644 index 0000000000000..e18b39a0d7d9e --- /dev/null +++ b/docs/reference/migration/migrate_8_0/ilm.asciidoc @@ -0,0 +1,17 @@ +[float] +[[breaking_80_ilm_changes]] +=== Index Lifecycle Management changes + +//NOTE: The notable-breaking-changes tagged regions are re-used in the +//Installation and Upgrade Guide + +//tag::notable-breaking-changes[] + +// end::notable-breaking-changes[] + +[float] +[[ilm-poll-interval-limit]] +==== `indices.lifecycle.poll_interval` must be greater than 1 second + +The setting `indices.lifecycle.poll_interval`, if set too low, can cause +excessive load on a cluster. This setting must now be set to 1 second or higher. From 07e0f0df21b67d62c18e00a9f4f124df76c2fc15 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Fri, 12 Apr 2019 15:42:05 -0600 Subject: [PATCH 025/260] Remove deprecated transport settings (#40918) This is related to #36652. There are a number of transport settings that were deprecated and replaced with new versions in 7.x. This commit removes them from 8.0. --- docs/reference/migration/migrate_8_0.asciidoc | 6 ++ .../migration/migrate_8_0/http.asciidoc | 15 +++++ .../migration/migrate_8_0/network.asciidoc | 16 ++++++ .../migration/migrate_8_0/transport.asciidoc | 22 ++++++++ ...Netty4TransportMultiPortIntegrationIT.java | 2 +- .../common/network/NetworkService.java | 5 -- .../common/settings/ClusterSettings.java | 11 ---- .../http/HttpTransportSettings.java | 4 +- .../transport/TransportSettings.java | 55 ++++--------------- .../resources/packaging/tests/certgen.bash | 2 +- 10 files changed, 72 insertions(+), 66 deletions(-) create mode 100644 docs/reference/migration/migrate_8_0/http.asciidoc create mode 100644 docs/reference/migration/migrate_8_0/network.asciidoc create mode 100644 docs/reference/migration/migrate_8_0/transport.asciidoc diff --git a/docs/reference/migration/migrate_8_0.asciidoc b/docs/reference/migration/migrate_8_0.asciidoc index 48c883c6197d9..11502be569076 100644 --- a/docs/reference/migration/migrate_8_0.asciidoc +++ b/docs/reference/migration/migrate_8_0.asciidoc @@ -19,6 +19,9 @@ coming[8.0.0] * <> * <> * <> +* <> +* <> +* <> //NOTE: The notable-breaking-changes tagged regions are re-used in the //Installation and Upgrade Guide @@ -49,3 +52,6 @@ include::migrate_8_0/snapshots.asciidoc[] include::migrate_8_0/security.asciidoc[] include::migrate_8_0/ilm.asciidoc[] include::migrate_8_0/java.asciidoc[] +include::migrate_8_0/network.asciidoc[] +include::migrate_8_0/transport.asciidoc[] +include::migrate_8_0/http.asciidoc[] diff --git a/docs/reference/migration/migrate_8_0/http.asciidoc b/docs/reference/migration/migrate_8_0/http.asciidoc new file mode 100644 index 0000000000000..6209beeaac4e2 --- /dev/null +++ b/docs/reference/migration/migrate_8_0/http.asciidoc @@ -0,0 +1,15 @@ +[float] +[[breaking_80_http_changes]] +=== HTTP changes + +//NOTE: The notable-breaking-changes tagged regions are re-used in the +//Installation and Upgrade Guide +//tag::notable-breaking-changes[] + +// end::notable-breaking-changes[] + +[float] +==== Removal of old HTTP settings + +The `http.tcp_no_delay` setting was deprecated in 7.x and has been removed in 8.0. It has been replaced by +`http.tcp.no_delay`. \ No newline at end of file diff --git a/docs/reference/migration/migrate_8_0/network.asciidoc b/docs/reference/migration/migrate_8_0/network.asciidoc new file mode 100644 index 0000000000000..ac77f6e9c0cb8 --- /dev/null +++ b/docs/reference/migration/migrate_8_0/network.asciidoc @@ -0,0 +1,16 @@ +[float] +[[breaking_80_network_changes]] +=== Network changes + +//NOTE: The notable-breaking-changes tagged regions are re-used in the +//Installation and Upgrade Guide +//tag::notable-breaking-changes[] + +// end::notable-breaking-changes[] + +[float] +==== Removal of old network settings + +The `network.tcp.connect_timeout` setting was deprecated in 7.x and has been removed in 8.0. This setting +was a fallback setting for `transport.connect_timeout`. To change the default connection timeout for client +connections `transport.connect_timeout` should be modified. \ No newline at end of file diff --git a/docs/reference/migration/migrate_8_0/transport.asciidoc b/docs/reference/migration/migrate_8_0/transport.asciidoc new file mode 100644 index 0000000000000..f55117ae81c22 --- /dev/null +++ b/docs/reference/migration/migrate_8_0/transport.asciidoc @@ -0,0 +1,22 @@ +[float] +[[breaking_80_transport_changes]] +=== Transport changes + +//tag::notable-breaking-changes[] +[float] +==== Removal of old transport settings + +The following settings have been deprecated in 7.x and removed in 8.0. Each setting has a replacement +setting that was introduced in 6.7. + +- `transport.tcp.port` replaced by `transport.port` +- `transport.tcp.compress` replaced by `transport.compress` +- `transport.tcp.connect_timeout` replaced by `transport.connect_timeout` +- `transport.tcp_no_delay` replaced by `transport.tcp.no_delay` +- `transport.profiles.profile_name.tcp_no_delay` replaced by `transport.profiles.profile_name.tcp.no_delay` +- `transport.profiles.profile_name.tcp_keep_alive` replaced by `transport.profiles.profile_name.tcp.keep_alive` +- `transport.profiles.profile_name.reuse_address` replaced by `transport.profiles.profile_name.tcp.reuse_address` +- `transport.profiles.profile_name.send_buffer_size` replaced by `transport.profiles.profile_name.tcp.send_buffer_size` +- `transport.profiles.profile_name.receive_buffer_size` replaced by `transport.profiles.profile_name.tcp.receive_buffer_size` + +// end::notable-breaking-changes[] \ No newline at end of file diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java index 52ad32efb5645..881fe73112c5a 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4TransportMultiPortIntegrationIT.java @@ -65,7 +65,7 @@ protected Settings nodeSettings(int nodeOrdinal) { .put("transport.profiles.client1.port", randomPortRange) .put("transport.profiles.client1.publish_host", "127.0.0.7") .put("transport.profiles.client1.publish_port", "4321") - .put("transport.profiles.client1.reuse_address", true); + .put("transport.profiles.client1.tcp.reuse_address", true); return builder.build(); } diff --git a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java index babc83a17722c..cdae56a4e59c9 100644 --- a/server/src/main/java/org/elasticsearch/common/network/NetworkService.java +++ b/server/src/main/java/org/elasticsearch/common/network/NetworkService.java @@ -22,7 +22,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import java.io.IOException; import java.net.InetAddress; @@ -32,7 +31,6 @@ import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.concurrent.TimeUnit; import java.util.function.Function; public final class NetworkService { @@ -58,9 +56,6 @@ public final class NetworkService { Setting.byteSizeSetting("network.tcp.send_buffer_size", new ByteSizeValue(-1), Property.NodeScope); public static final Setting TCP_RECEIVE_BUFFER_SIZE = Setting.byteSizeSetting("network.tcp.receive_buffer_size", new ByteSizeValue(-1), Property.NodeScope); - public static final Setting TCP_CONNECT_TIMEOUT = - Setting.timeSetting("network.tcp.connect_timeout", new TimeValue(30, TimeUnit.SECONDS), Property.NodeScope, - Setting.Property.Deprecated); /** * A custom name resolver can support custom lookup keys (my_net_key:ipv4) and also change diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 315e6175a3a36..82ae117c3e24b 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -261,7 +261,6 @@ public void apply(Settings value, Settings current, Settings previous) { HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH, HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT, HttpTransportSettings.SETTING_HTTP_RESET_COOKIES, - HttpTransportSettings.OLD_SETTING_HTTP_TCP_NO_DELAY, HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY, HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE, HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS, @@ -309,32 +308,23 @@ public void apply(Settings value, Settings current, Settings previous) { TransportSettings.PUBLISH_HOST_PROFILE, TransportSettings.BIND_HOST, TransportSettings.BIND_HOST_PROFILE, - TransportSettings.OLD_PORT, TransportSettings.PORT, TransportSettings.PORT_PROFILE, TransportSettings.PUBLISH_PORT, TransportSettings.PUBLISH_PORT_PROFILE, - TransportSettings.OLD_TRANSPORT_COMPRESS, TransportSettings.TRANSPORT_COMPRESS, TransportSettings.PING_SCHEDULE, - TransportSettings.TCP_CONNECT_TIMEOUT, TransportSettings.CONNECT_TIMEOUT, TransportSettings.DEFAULT_FEATURES_SETTING, - TransportSettings.OLD_TCP_NO_DELAY, TransportSettings.TCP_NO_DELAY, - TransportSettings.OLD_TCP_NO_DELAY_PROFILE, TransportSettings.TCP_NO_DELAY_PROFILE, TransportSettings.TCP_KEEP_ALIVE, - TransportSettings.OLD_TCP_KEEP_ALIVE_PROFILE, TransportSettings.TCP_KEEP_ALIVE_PROFILE, TransportSettings.TCP_REUSE_ADDRESS, - TransportSettings.OLD_TCP_REUSE_ADDRESS_PROFILE, TransportSettings.TCP_REUSE_ADDRESS_PROFILE, TransportSettings.TCP_SEND_BUFFER_SIZE, - TransportSettings.OLD_TCP_SEND_BUFFER_SIZE_PROFILE, TransportSettings.TCP_SEND_BUFFER_SIZE_PROFILE, TransportSettings.TCP_RECEIVE_BUFFER_SIZE, - TransportSettings.OLD_TCP_RECEIVE_BUFFER_SIZE_PROFILE, TransportSettings.TCP_RECEIVE_BUFFER_SIZE_PROFILE, TransportSettings.CONNECTIONS_PER_NODE_RECOVERY, TransportSettings.CONNECTIONS_PER_NODE_BULK, @@ -352,7 +342,6 @@ public void apply(Settings value, Settings current, Settings previous) { NetworkService.TCP_REUSE_ADDRESS, NetworkService.TCP_SEND_BUFFER_SIZE, NetworkService.TCP_RECEIVE_BUFFER_SIZE, - NetworkService.TCP_CONNECT_TIMEOUT, IndexSettings.QUERY_STRING_ANALYZE_WILDCARD, IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD, ScriptService.SCRIPT_CACHE_SIZE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/http/HttpTransportSettings.java b/server/src/main/java/org/elasticsearch/http/HttpTransportSettings.java index 2a5639f2e72ce..58d30edf79944 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpTransportSettings.java +++ b/server/src/main/java/org/elasticsearch/http/HttpTransportSettings.java @@ -107,10 +107,8 @@ public final class HttpTransportSettings { // Tcp socket settings - public static final Setting OLD_SETTING_HTTP_TCP_NO_DELAY = - boolSetting("http.tcp_no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope, Setting.Property.Deprecated); public static final Setting SETTING_HTTP_TCP_NO_DELAY = - boolSetting("http.tcp.no_delay", OLD_SETTING_HTTP_TCP_NO_DELAY, Setting.Property.NodeScope); + boolSetting("http.tcp.no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope); public static final Setting SETTING_HTTP_TCP_KEEP_ALIVE = boolSetting("http.tcp.keep_alive", NetworkService.TCP_KEEP_ALIVE, Setting.Property.NodeScope); public static final Setting SETTING_HTTP_TCP_REUSE_ADDRESS = diff --git a/server/src/main/java/org/elasticsearch/transport/TransportSettings.java b/server/src/main/java/org/elasticsearch/transport/TransportSettings.java index 1dd5541b0e2e4..b89c06de84a19 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportSettings.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportSettings.java @@ -27,6 +27,7 @@ import java.util.Arrays; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import static java.util.Collections.emptyList; @@ -52,84 +53,48 @@ public final class TransportSettings { listSetting("transport.bind_host", HOST, Function.identity(), Setting.Property.NodeScope); public static final Setting.AffixSetting> BIND_HOST_PROFILE = affixKeySetting("transport.profiles.", "bind_host", key -> listSetting(key, BIND_HOST, Function.identity(), Setting.Property.NodeScope)); - public static final Setting OLD_PORT = - new Setting<>("transport.tcp.port", "9300-9400", Function.identity(), Setting.Property.NodeScope, Setting.Property.Deprecated); public static final Setting PORT = - new Setting<>("transport.port", OLD_PORT, Function.identity(), Setting.Property.NodeScope); + new Setting<>("transport.port", "9300-9400", Function.identity(), Setting.Property.NodeScope); public static final Setting.AffixSetting PORT_PROFILE = affixKeySetting("transport.profiles.", "port", key -> new Setting<>(key, PORT, Function.identity(), Setting.Property.NodeScope)); public static final Setting PUBLISH_PORT = intSetting("transport.publish_port", -1, -1, Setting.Property.NodeScope); public static final Setting.AffixSetting PUBLISH_PORT_PROFILE = affixKeySetting("transport.profiles.", "publish_port", key -> intSetting(key, -1, -1, Setting.Property.NodeScope)); - public static final Setting OLD_TRANSPORT_COMPRESS = - boolSetting("transport.tcp.compress", false, Setting.Property.NodeScope, Setting.Property.Deprecated); public static final Setting TRANSPORT_COMPRESS = - boolSetting("transport.compress", OLD_TRANSPORT_COMPRESS, Setting.Property.NodeScope); + boolSetting("transport.compress", false, Setting.Property.NodeScope); // the scheduled internal ping interval setting, defaults to disabled (-1) public static final Setting PING_SCHEDULE = timeSetting("transport.ping_schedule", TimeValue.timeValueSeconds(-1), Setting.Property.NodeScope); - public static final Setting TCP_CONNECT_TIMEOUT = - timeSetting("transport.tcp.connect_timeout", NetworkService.TCP_CONNECT_TIMEOUT, Setting.Property.NodeScope, - Setting.Property.Deprecated); public static final Setting CONNECT_TIMEOUT = - timeSetting("transport.connect_timeout", TCP_CONNECT_TIMEOUT, Setting.Property.NodeScope); + timeSetting("transport.connect_timeout", new TimeValue(30, TimeUnit.SECONDS), Setting.Property.NodeScope); public static final Setting DEFAULT_FEATURES_SETTING = Setting.groupSetting(FEATURE_PREFIX + ".", Setting.Property.NodeScope); // Tcp socket settings - public static final Setting OLD_TCP_NO_DELAY = - boolSetting("transport.tcp_no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope, Setting.Property.Deprecated); public static final Setting TCP_NO_DELAY = - boolSetting("transport.tcp.no_delay", OLD_TCP_NO_DELAY, Setting.Property.NodeScope); - public static final Setting.AffixSetting OLD_TCP_NO_DELAY_PROFILE = - affixKeySetting("transport.profiles.", "tcp_no_delay", key -> boolSetting(key, TCP_NO_DELAY, Setting.Property.NodeScope, - Setting.Property.Deprecated)); + boolSetting("transport.tcp.no_delay", NetworkService.TCP_NO_DELAY, Setting.Property.NodeScope); public static final Setting.AffixSetting TCP_NO_DELAY_PROFILE = - affixKeySetting("transport.profiles.", "tcp.no_delay", - key -> boolSetting(key, - fallback(key, OLD_TCP_NO_DELAY_PROFILE, "tcp\\.no_delay$", "tcp_no_delay"), - Setting.Property.NodeScope)); + affixKeySetting("transport.profiles.", "tcp.no_delay", key -> boolSetting(key, TCP_NO_DELAY, Setting.Property.NodeScope)); public static final Setting TCP_KEEP_ALIVE = boolSetting("transport.tcp.keep_alive", NetworkService.TCP_KEEP_ALIVE, Setting.Property.NodeScope); - public static final Setting.AffixSetting OLD_TCP_KEEP_ALIVE_PROFILE = - affixKeySetting("transport.profiles.", "tcp_keep_alive", - key -> boolSetting(key, TCP_KEEP_ALIVE, Setting.Property.NodeScope, Setting.Property.Deprecated)); public static final Setting.AffixSetting TCP_KEEP_ALIVE_PROFILE = affixKeySetting("transport.profiles.", "tcp.keep_alive", - key -> boolSetting(key, - fallback(key, OLD_TCP_KEEP_ALIVE_PROFILE, "tcp\\.keep_alive$", "tcp_keep_alive"), - Setting.Property.NodeScope)); + key -> boolSetting(key, TCP_KEEP_ALIVE, Setting.Property.NodeScope)); public static final Setting TCP_REUSE_ADDRESS = boolSetting("transport.tcp.reuse_address", NetworkService.TCP_REUSE_ADDRESS, Setting.Property.NodeScope); - public static final Setting.AffixSetting OLD_TCP_REUSE_ADDRESS_PROFILE = - affixKeySetting("transport.profiles.", "reuse_address", key -> boolSetting(key, TCP_REUSE_ADDRESS, Setting.Property.NodeScope, - Setting.Property.Deprecated)); public static final Setting.AffixSetting TCP_REUSE_ADDRESS_PROFILE = - affixKeySetting("transport.profiles.", "tcp.reuse_address", - key -> boolSetting(key, - fallback(key, OLD_TCP_REUSE_ADDRESS_PROFILE, "tcp\\.reuse_address$", "reuse_address"), - Setting.Property.NodeScope)); + affixKeySetting("transport.profiles.", "tcp.reuse_address", key -> boolSetting(key, TCP_REUSE_ADDRESS, Setting.Property.NodeScope)); public static final Setting TCP_SEND_BUFFER_SIZE = Setting.byteSizeSetting("transport.tcp.send_buffer_size", NetworkService.TCP_SEND_BUFFER_SIZE, Setting.Property.NodeScope); - public static final Setting.AffixSetting OLD_TCP_SEND_BUFFER_SIZE_PROFILE = - affixKeySetting("transport.profiles.", "send_buffer_size", - key -> Setting.byteSizeSetting(key, TCP_SEND_BUFFER_SIZE, Setting.Property.NodeScope, Setting.Property.Deprecated)); public static final Setting.AffixSetting TCP_SEND_BUFFER_SIZE_PROFILE = affixKeySetting("transport.profiles.", "tcp.send_buffer_size", - key -> Setting.byteSizeSetting(key, - fallback(key, OLD_TCP_SEND_BUFFER_SIZE_PROFILE, "tcp\\.send_buffer_size$", "send_buffer_size"), - Setting.Property.NodeScope)); + key -> Setting.byteSizeSetting(key, TCP_SEND_BUFFER_SIZE, Setting.Property.NodeScope)); public static final Setting TCP_RECEIVE_BUFFER_SIZE = Setting.byteSizeSetting("transport.tcp.receive_buffer_size", NetworkService.TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope); - public static final Setting.AffixSetting OLD_TCP_RECEIVE_BUFFER_SIZE_PROFILE = - affixKeySetting("transport.profiles.", "receive_buffer_size", - key -> Setting.byteSizeSetting(key, TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope, Setting.Property.Deprecated)); public static final Setting.AffixSetting TCP_RECEIVE_BUFFER_SIZE_PROFILE = affixKeySetting("transport.profiles.", "tcp.receive_buffer_size", - key -> Setting.byteSizeSetting(key, - fallback(key, OLD_TCP_RECEIVE_BUFFER_SIZE_PROFILE, "tcp\\.receive_buffer_size$", "receive_buffer_size"), - Setting.Property.NodeScope)); + key -> Setting.byteSizeSetting(key, TCP_RECEIVE_BUFFER_SIZE, Setting.Property.NodeScope)); // Connections per node settings diff --git a/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash b/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash index 13aed28b4c1ba..6cc6fab3d4d1e 100644 --- a/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash +++ b/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash @@ -345,7 +345,7 @@ xpack.security.http.ssl.certificate: $ESCONFIG/certs//node-data/node-data.crt xpack.security.http.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] xpack.security.transport.ssl.enabled: true -transport.tcp.port: 9301 +transport.port: 9301 xpack.security.http.ssl.enabled: true http.port: 9201 From c86e7f17e27624bb5f76f2c1061e99500d55d7bb Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 14 Apr 2019 11:29:51 +0200 Subject: [PATCH 026/260] Simplify TransportShardBulkAction#performOnReplica (#41065) * Simplify TransortShardBulkAction#performOnReplica * Resolve TODO since 8.0 doesn't have to worry about pre 6.x nodes * Remove test for removed method since the logic is now completely internal to `performOnReplica` --- .../action/bulk/TransportShardBulkAction.java | 86 ++++--------------- .../bulk/TransportShardBulkActionTests.java | 43 ---------- 2 files changed, 17 insertions(+), 112 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 da30dedfe5e60..f49521a37463e 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -338,9 +338,8 @@ private static boolean isConflictException(final Exception e) { /** * Creates a new bulk item result from the given requests and result of performing the update operation on the shard. */ - static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, final String concreteIndex, - BulkItemResponse operationResponse, - final UpdateHelper.Result translate) { + private static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, final String concreteIndex, + BulkItemResponse operationResponse, final UpdateHelper.Result translate) { final BulkItemResponse response; DocWriteResponse.Result translatedResult = translate.getResponseResult(); if (operationResponse.isFailed()) { @@ -382,54 +381,6 @@ static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, return response; } - - /** Modes for executing item request on replica depending on corresponding primary execution result */ - public enum ReplicaItemExecutionMode { - - /** - * When primary execution succeeded - */ - NORMAL, - - /** - * When primary execution failed before sequence no was generated - * or primary execution was a noop (only possible when request is originating from pre-6.0 nodes) - */ - NOOP, - - /** - * When primary execution failed after sequence no was generated - */ - FAILURE - } - - /** - * Determines whether a bulk item request should be executed on the replica. - * - * @return {@link ReplicaItemExecutionMode#NORMAL} upon normal primary execution with no failures - * {@link ReplicaItemExecutionMode#FAILURE} upon primary execution failure after sequence no generation - * {@link ReplicaItemExecutionMode#NOOP} upon primary execution failure before sequence no generation or - * when primary execution resulted in noop (only possible for write requests from pre-6.0 nodes) - */ - static ReplicaItemExecutionMode replicaItemExecutionMode(final BulkItemRequest request, final int index) { - final BulkItemResponse primaryResponse = request.getPrimaryResponse(); - assert primaryResponse != null : "expected primary response to be set for item [" + index + "] request [" + request.request() + "]"; - if (primaryResponse.isFailed()) { - return primaryResponse.getFailure().getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO - ? ReplicaItemExecutionMode.FAILURE // we have a seq no generated with the failure, replicate as no-op - : ReplicaItemExecutionMode.NOOP; // no seq no generated, ignore replication - } else { - // TODO: once we know for sure that every operation that has been processed on the primary is assigned a seq# - // (i.e., all nodes on the cluster are on v6.0.0 or higher) we can use the existence of a seq# to indicate whether - // an operation should be processed or be treated as a noop. This means we could remove this method and the - // ReplicaItemExecutionMode enum and have a simple boolean check for seq != UNASSIGNED_SEQ_NO which will work for - // both failures and indexing operations. - return primaryResponse.getResponse().getResult() != DocWriteResponse.Result.NOOP - ? ReplicaItemExecutionMode.NORMAL // execution successful on primary - : ReplicaItemExecutionMode.NOOP; // ignore replication - } - } - @Override public WriteReplicaResult shardOperationOnReplica(BulkShardRequest request, IndexShard replica) throws Exception { final Translog.Location location = performOnReplica(request, replica); @@ -442,25 +393,22 @@ public static Translog.Location performOnReplica(BulkShardRequest request, Index BulkItemRequest item = request.items()[i]; final Engine.Result operationResult; DocWriteRequest docWriteRequest = item.request(); - switch (replicaItemExecutionMode(item, i)) { - case NORMAL: - final DocWriteResponse primaryResponse = item.getPrimaryResponse().getResponse(); - operationResult = performOpOnReplica(primaryResponse, docWriteRequest, replica); - assert operationResult != null : "operation result must never be null when primary response has no failure"; - location = syncOperationResultOrThrow(operationResult, location); - break; - case NOOP: - break; - case FAILURE: - final BulkItemResponse.Failure failure = item.getPrimaryResponse().getFailure(); - assert failure.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO : "seq no must be assigned"; - operationResult = replica.markSeqNoAsNoop(failure.getSeqNo(), failure.getMessage()); - assert operationResult != null : "operation result must never be null when primary response has no failure"; - location = syncOperationResultOrThrow(operationResult, location); - break; - default: - throw new IllegalStateException("illegal replica item execution mode for: " + docWriteRequest); + 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); + } else { + operationResult = replica.markSeqNoAsNoop(seqNum, failure.getMessage()); } + assert operationResult != null : "operation result must never be null when primary response has no failure"; + location = syncOperationResultOrThrow(operationResult, location); } return location; } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java index 62217f7873138..55078840153f2 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.LatchedActionListener; -import org.elasticsearch.action.bulk.TransportShardBulkAction.ReplicaItemExecutionMode; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexRequest; @@ -59,7 +58,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; -import static org.elasticsearch.action.bulk.TransportShardBulkAction.replicaItemExecutionMode; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.notNullValue; @@ -96,47 +94,6 @@ private IndexMetaData indexMetaData() throws IOException { .primaryTerm(0, 1).build(); } - public void testShouldExecuteReplicaItem() throws Exception { - // Successful index request should be replicated - DocWriteRequest writeRequest = new IndexRequest("index", "_doc", "id") - .source(Requests.INDEX_CONTENT_TYPE, "foo", "bar"); - DocWriteResponse response = new IndexResponse(shardId, "type", "id", 1, 17, 1, randomBoolean()); - BulkItemRequest request = new BulkItemRequest(0, writeRequest); - request.setPrimaryResponse(new BulkItemResponse(0, DocWriteRequest.OpType.INDEX, response)); - assertThat(replicaItemExecutionMode(request, 0), - equalTo(ReplicaItemExecutionMode.NORMAL)); - - // Failed index requests without sequence no should not be replicated - writeRequest = new IndexRequest("index", "_doc", "id") - .source(Requests.INDEX_CONTENT_TYPE, "foo", "bar"); - request = new BulkItemRequest(0, writeRequest); - request.setPrimaryResponse( - new BulkItemResponse(0, DocWriteRequest.OpType.INDEX, - new BulkItemResponse.Failure("index", "type", "id", - new IllegalArgumentException("i died")))); - assertThat(replicaItemExecutionMode(request, 0), - equalTo(ReplicaItemExecutionMode.NOOP)); - - // Failed index requests with sequence no should be replicated - request = new BulkItemRequest(0, writeRequest); - request.setPrimaryResponse( - new BulkItemResponse(0, DocWriteRequest.OpType.INDEX, - new BulkItemResponse.Failure("index", "type", "id", - new IllegalArgumentException( - "i died after sequence no was generated"), - 1))); - assertThat(replicaItemExecutionMode(request, 0), - equalTo(ReplicaItemExecutionMode.FAILURE)); - // NOOP requests should not be replicated - DocWriteRequest updateRequest = new UpdateRequest("index", "type", "id"); - response = new UpdateResponse(shardId, "type", "id", 1, DocWriteResponse.Result.NOOP); - request = new BulkItemRequest(0, updateRequest); - request.setPrimaryResponse(new BulkItemResponse(0, DocWriteRequest.OpType.UPDATE, - response)); - assertThat(replicaItemExecutionMode(request, 0), - equalTo(ReplicaItemExecutionMode.NOOP)); - } - public void testExecuteBulkIndexRequest() throws Exception { IndexShard shard = newStartedShard(true); From f0fac9f56b2877bd1f9663172e92d3dfc371d292 Mon Sep 17 00:00:00 2001 From: David Turner Date: Sun, 14 Apr 2019 10:39:50 +0100 Subject: [PATCH 027/260] Further clarify cluster.initial_master_nodes (#41179) The following phrase causes confusion: > Alternatively the IP addresses or hostnames (if node name defaults to the > host name) can be used. This change clarifies the conditions under which you can use a hostname, and adds an anchor to the note introduced in (#41137) so we can link directly to it in conversations with users. --- .../modules/discovery/bootstrapping.asciidoc | 62 ++++++++++--------- .../discovery-settings.asciidoc | 14 +++-- 2 files changed, 42 insertions(+), 34 deletions(-) diff --git a/docs/reference/modules/discovery/bootstrapping.asciidoc b/docs/reference/modules/discovery/bootstrapping.asciidoc index f4a775e12df99..0ba7d4b17cef7 100644 --- a/docs/reference/modules/discovery/bootstrapping.asciidoc +++ b/docs/reference/modules/discovery/bootstrapping.asciidoc @@ -9,16 +9,21 @@ up: nodes that have already joined a cluster store this information in their data folder and freshly-started nodes that are joining an existing cluster obtain this information from the cluster's elected master. -The initial set of master-eligible nodes is defined in the -<>. When you -start a master-eligible node, you can provide this setting on the command line -or in the `elasticsearch.yml` file. After the cluster has formed, this setting -is no longer required and is ignored. It need not be set -on master-ineligible nodes, nor on master-eligible nodes that are started to -join an existing cluster. Note that master-eligible nodes should use storage -that persists across restarts. If they do not, and -`cluster.initial_master_nodes` is set, and a full cluster restart occurs, then -another brand-new cluster will form and this may result in data loss. +The initial set of master-eligible nodes is defined in the +<>. This is a list +of the <> or IP addresses of the master-eligible nodes in +the new cluster. If you do not configure `node.name` then it is set to the +node's hostname, so in this case you can use hostnames in +`cluster.initial_master_nodes` too. + +When you start a master-eligible node, you can provide this setting on the +command line or in the `elasticsearch.yml` file. After the cluster has formed, +this setting is no longer required and is ignored. It need not be set on +master-ineligible nodes, nor on master-eligible nodes that are started to join +an existing cluster. Note that master-eligible nodes should use storage that +persists across restarts. If they do not, and `cluster.initial_master_nodes` is +set, and a full cluster restart occurs, then another brand-new cluster will +form and this may result in data loss. It is technically sufficient to set `cluster.initial_master_nodes` on a single master-eligible node in the cluster, and only to mention that single node in the @@ -42,10 +47,9 @@ cluster.initial_master_nodes: - master-c -------------------------------------------------- -Alternatively the IP addresses or hostnames (<>) can be used. If there is more than one Elasticsearch node -with the same IP address or hostname then the transport ports must also be given -to specify exactly which node is meant: +You can use a mix of IP addresses and node names too. If there is more than one +Elasticsearch node with the same IP address then the transport port must also +be given to specify exactly which node is meant: [source,yaml] -------------------------------------------------- @@ -56,14 +60,23 @@ cluster.initial_master_nodes: - master-node-hostname -------------------------------------------------- +Like all node settings, it is also possible to specify the initial set of master +nodes on the command-line that is used to start Elasticsearch: + +[source,bash] +-------------------------------------------------- +$ bin/elasticsearch -Ecluster.initial_master_nodes=master-a,master-b,master-c +-------------------------------------------------- + [NOTE] ================================================== -The node names used in this list must exactly match the `node.name` properties -of the nodes. By default the node name is set to the machine's hostname which -may or may not be fully-qualified depending on your system configuration. If -each node name is a fully-qualified domain name such as `master-a.example.com` -then you must use fully-qualified domain names in the +[[modules-discovery-bootstrap-cluster-fqdns]] The node names used in the +`cluster.initial_master_nodes` list must exactly match the `node.name` +properties of the nodes. By default the node name is set to the machine's +hostname which may or may not be fully-qualified depending on your system +configuration. If each node name is a fully-qualified domain name such as +`master-a.example.com` then you must use fully-qualified domain names in the `cluster.initial_master_nodes` list too; conversely if your node names are bare hostnames (without the `.example.com` suffix) then you must use bare hostnames in the `cluster.initial_master_nodes` list. If you use a mix of fully-qualifed @@ -81,18 +94,11 @@ bootstrap a cluster: have discovered [{master-b.example.com}{... This message shows the node names `master-a.example.com` and `master-b.example.com` as well as the `cluster.initial_master_nodes` entries -`master-a` and `master-b`, and it is apparent that they do not match exactly. +`master-a` and `master-b`, and it is clear from this message that they do not +match exactly. ================================================== -Like all node settings, it is also possible to specify the initial set of master -nodes on the command-line that is used to start Elasticsearch: - -[source,bash] --------------------------------------------------- -$ bin/elasticsearch -Ecluster.initial_master_nodes=master-a,master-b,master-c --------------------------------------------------- - [float] ==== Choosing a cluster name diff --git a/docs/reference/setup/important-settings/discovery-settings.asciidoc b/docs/reference/setup/important-settings/discovery-settings.asciidoc index 4edf5cfbab5e9..5709ae3bb9345 100644 --- a/docs/reference/setup/important-settings/discovery-settings.asciidoc +++ b/docs/reference/setup/important-settings/discovery-settings.asciidoc @@ -58,15 +58,17 @@ cluster.initial_master_nodes: `transport.port` if not specified. <2> If a hostname resolves to multiple IP addresses then the node will attempt to discover other nodes at all resolved addresses. -<3> Initial master nodes can be identified by their <>. - Make sure that the value here matches the `node.name` exactly. If you use a - fully-qualified domain name such as `master-node-a.example.com` for your +<3> Initial master nodes can be identified by their <>, + which defaults to the hostname. Make sure that the value in + `cluster.initial_master_nodes` matches the `node.name` exactly. If you use + a fully-qualified domain name such as `master-node-a.example.com` for your node names then you must use the fully-qualified name in this list; conversely if `node.name` is a bare hostname without any trailing qualifiers then you must also omit the trailing qualifiers in `cluster.initial_master_nodes`. <4> Initial master nodes can also be identified by their IP address. -<5> If multiple master nodes share an IP address then the port must be used to - disambiguate them. +<5> If multiple master nodes share an IP address then the transport port must + be used to distinguish between them. -For more information, see <>. +For more information, see <> and +<>. From 7543478fafe075c6b5bd7f676b78968811563929 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 15 Apr 2019 09:28:58 +0200 Subject: [PATCH 028/260] Remove xpack dependencies from qa rest modules (#41134) This commit removes xpack dependencies of many xpack qa modules. (for some qa modules this will require some more work) The reason behind this change is that qa rest modules should not depend on the x-pack plugins, because the plugins are an implementation detail and the tests should only know about the rest interface and qa cluster that is being tested. Also some qa modules rely on xpack plugins and hlrc (which is a valid dependency for rest qa tests) creates a cyclic dependency and this is something that we should avoid. Also Eclipse can't handle gradle cyclic dependencies (see #41064). * don't copy xpack-core's plugin property into the test resource of qa modules. Otherwise installing security manager fails, because it tries to find the XPackPlugin class. --- .../support}/WatcherTemplateTests.java | 2 +- x-pack/qa/build.gradle | 7 ++++ .../build.gradle | 3 +- ...CoreWithSecurityClientYamlTestSuiteIT.java | 2 +- x-pack/qa/full-cluster-restart/build.gradle | 27 +++--------- .../xpack/restart/FullClusterRestartIT.java | 35 +++++++--------- .../MlMigrationFullClusterRestartIT.java | 3 +- .../build.gradle | 4 +- ...sterSearchWithSecurityYamlTestSuiteIT.java | 2 +- x-pack/qa/multi-node/build.gradle | 2 +- .../GlobalCheckpointSyncActionIT.java | 2 +- .../elasticsearch/multi_node/RollupIT.java | 2 +- x-pack/qa/rolling-upgrade-basic/build.gradle | 13 +----- .../build.gradle | 4 +- x-pack/qa/rolling-upgrade/build.gradle | 14 +------ .../upgrades/AbstractUpgradeTestCase.java | 6 +-- .../upgrades/MlMappingsUpgradeIT.java | 6 +-- .../UpgradeClusterClientYamlTestSuiteIT.java | 3 +- x-pack/qa/saml-idp-tests/build.gradle | 7 +--- .../build.gradle | 4 +- .../MonitoringWithWatcherRestIT.java | 33 ++++++++------- x-pack/qa/smoke-test-plugins/build.gradle | 2 +- ...SmokeTestPluginsClientYamlTestSuiteIT.java | 2 +- .../build.gradle | 5 +-- ...rityWithMustacheClientYamlTestSuiteIT.java | 2 +- .../build.gradle | 2 +- ...cherWithSecurityClientYamlTestSuiteIT.java | 6 +-- .../SmokeTestWatcherWithSecurityIT.java | 6 +-- x-pack/qa/smoke-test-watcher/build.gradle | 5 +-- .../SmokeTestWatcherTestSuiteIT.java | 6 +-- .../smoketest/WatcherRestIT.java | 4 +- .../test/SecuritySettingsSourceField.java | 42 +++++++++++++++++++ .../test/rest/XPackRestTestConstants.java | 38 +++++++++++++++++ .../xpack/test/rest/XPackRestTestHelper.java | 19 +++------ 34 files changed, 173 insertions(+), 147 deletions(-) rename x-pack/{qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest => plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support}/WatcherTemplateTests.java (99%) create mode 100644 x-pack/qa/src/main/java/org/elasticsearch/xpack/test/SecuritySettingsSourceField.java create mode 100644 x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestConstants.java rename x-pack/{plugin/core/src/test => qa/src/main}/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java (81%) diff --git a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherTemplateTests.java similarity index 99% rename from x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateTests.java rename to x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherTemplateTests.java index df98e73118711..8c45bc15b4c9e 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherTemplateTests.java @@ -3,7 +3,7 @@ * 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.smoketest; +package org.elasticsearch.xpack.watcher.support; import com.fasterxml.jackson.core.io.JsonStringEncoder; import org.elasticsearch.common.Nullable; diff --git a/x-pack/qa/build.gradle b/x-pack/qa/build.gradle index 7d705d5b0dc9f..2555b0ef729dc 100644 --- a/x-pack/qa/build.gradle +++ b/x-pack/qa/build.gradle @@ -3,6 +3,13 @@ import org.elasticsearch.gradle.test.RestIntegTestTask +apply plugin: 'elasticsearch.build' +test.enabled = false + +dependencies { + compile project(':test:framework') +} + subprojects { // HACK: please fix this // we want to add the rest api specs for xpack to qa tests, but we diff --git a/x-pack/qa/core-rest-tests-with-security/build.gradle b/x-pack/qa/core-rest-tests-with-security/build.gradle index dbf5aa6fc227c..72b62f94b5542 100644 --- a/x-pack/qa/core-rest-tests-with-security/build.gradle +++ b/x-pack/qa/core-rest-tests-with-security/build.gradle @@ -2,8 +2,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') - testCompile project(path: xpackModule('security'), configuration: 'testArtifacts') + testCompile project(':x-pack:qa') } integTest { diff --git a/x-pack/qa/core-rest-tests-with-security/src/test/java/org/elasticsearch/xpack/security/CoreWithSecurityClientYamlTestSuiteIT.java b/x-pack/qa/core-rest-tests-with-security/src/test/java/org/elasticsearch/xpack/security/CoreWithSecurityClientYamlTestSuiteIT.java index 212a342479d3f..1d98c4fd59c76 100644 --- a/x-pack/qa/core-rest-tests-with-security/src/test/java/org/elasticsearch/xpack/security/CoreWithSecurityClientYamlTestSuiteIT.java +++ b/x-pack/qa/core-rest-tests-with-security/src/test/java/org/elasticsearch/xpack/security/CoreWithSecurityClientYamlTestSuiteIT.java @@ -18,7 +18,7 @@ import java.util.Objects; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; @TimeoutSuite(millis = 30 * TimeUnits.MINUTE) // as default timeout seems not enough on the jenkins VMs public class CoreWithSecurityClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { diff --git a/x-pack/qa/full-cluster-restart/build.gradle b/x-pack/qa/full-cluster-restart/build.gradle index 964cc2fb43cc7..7f0e14d2a53bf 100644 --- a/x-pack/qa/full-cluster-restart/build.gradle +++ b/x-pack/qa/full-cluster-restart/build.gradle @@ -10,22 +10,12 @@ apply plugin: 'elasticsearch.standalone-test' test.enabled = false dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') - testCompile (project(path: xpackModule('security'), configuration: 'runtime')) { - // Need to drop the guava dependency here or we get a conflict with watcher's guava dependency. - // This is total #$%, but the solution is to get the SAML realm (which uses guava) out of security proper - exclude group: "com.google.guava", module: "guava" - } - testCompile project(path: xpackModule('watcher'), configuration: 'runtime') - + // 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) testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') - testCompile (project(path: xpackModule('security'), configuration: 'testArtifacts')) { - // Need to drop the guava dependency here or we get a conflict with watcher's guava dependency. - // This is total #$%, but the solution is to get the SAML realm (which uses guava) out of security proper - exclude group: "com.google.guava", module: "guava" - } + testCompile project(path: ':qa:full-cluster-restart', configuration: 'testArtifacts') + testCompile project(':x-pack:qa') } Closure waitWithAuth = { NodeInfo node, AntBuilder ant -> @@ -225,11 +215,4 @@ task copyXPackRestSpec(type: Copy) { include 'rest-api-spec/api/**' into project.sourceSets.test.output.resourcesDir } - -task copyXPackPluginProps(type: Copy) { - dependsOn(copyXPackRestSpec) - from project(xpackModule('core')).file('src/main/plugin-metadata') - from project(xpackModule('core')).tasks.pluginProperties - into outputDir -} -project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackPluginProps) +project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackRestSpec) diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java index 29f3237dcedb5..7ed204e152b64 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java @@ -13,22 +13,12 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ObjectPath; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.action.search.RestSearchAction; import org.elasticsearch.test.StreamsUtils; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.upgrades.AbstractFullClusterRestartTestCase; -import org.elasticsearch.xpack.core.upgrade.UpgradeField; -import org.elasticsearch.xpack.core.watcher.client.WatchSourceBuilder; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; -import org.elasticsearch.xpack.watcher.actions.index.IndexAction; -import org.elasticsearch.xpack.watcher.actions.logging.LoggingAction; -import org.elasticsearch.xpack.watcher.common.text.TextTemplate; -import org.elasticsearch.xpack.watcher.condition.InternalAlwaysCondition; -import org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule; -import org.elasticsearch.xpack.watcher.trigger.schedule.ScheduleTrigger; import org.hamcrest.Matcher; import java.io.IOException; @@ -54,6 +44,12 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase { + public static final String INDEX_ACTION_TYPES_DEPRECATION_MESSAGE = + "[types removal] Specifying types in a watcher index action is deprecated."; + + public static final int UPGRADE_FIELD_EXPECTED_INDEX_FORMAT_VERSION = 6; + public static final int SECURITY_EXPECTED_INDEX_FORMAT_VERSION = 6; + @Override protected Settings restClientSettings() { String token = "Basic " + Base64.getEncoder().encodeToString("test_user:x-pack-test-password".getBytes(StandardCharsets.UTF_8)); @@ -106,7 +102,7 @@ public void testSecurityNativeRealm() throws Exception { if (settingsMap.containsKey("index")) { @SuppressWarnings("unchecked") int format = Integer.parseInt(String.valueOf(((Map)settingsMap.get("index")).get("format"))); - assertEquals("The security index needs to be upgraded", SecurityIndexManager.INTERNAL_INDEX_FORMAT, format); + assertEquals("The security index needs to be upgraded", SECURITY_EXPECTED_INDEX_FORMAT_VERSION, format); } } @@ -127,8 +123,8 @@ public void testWatcher() throws Exception { Request createBwcWatch = new Request("PUT", getWatcherEndpoint() + "/watch/bwc_watch"); Request createBwcThrottlePeriod = new Request("PUT", getWatcherEndpoint() + "/watch/bwc_throttle_period"); if (getOldClusterVersion().onOrAfter(Version.V_7_0_0)) { - createBwcWatch.setOptions(expectWarnings(IndexAction.TYPES_DEPRECATION_MESSAGE)); - createBwcThrottlePeriod.setOptions(expectWarnings(IndexAction.TYPES_DEPRECATION_MESSAGE)); + createBwcWatch.setOptions(expectWarnings(INDEX_ACTION_TYPES_DEPRECATION_MESSAGE)); + createBwcThrottlePeriod.setOptions(expectWarnings(INDEX_ACTION_TYPES_DEPRECATION_MESSAGE)); } createBwcWatch.setJsonEntity(loadWatch("simple-watch.json")); client().performRequest(createBwcWatch); @@ -166,7 +162,7 @@ public void testWatcher() throws Exception { logger.info("settings map {}", settingsMap); if (settingsMap.containsKey("index")) { int format = Integer.parseInt(String.valueOf(((Map)settingsMap.get("index")).get("format"))); - assertEquals("The watches index needs to be upgraded", UpgradeField.EXPECTED_INDEX_FORMAT_VERSION, format); + assertEquals("The watches index needs to be upgraded", UPGRADE_FIELD_EXPECTED_INDEX_FORMAT_VERSION, format); } } @@ -289,7 +285,7 @@ private void assertWatchIndexContentsWork() throws Exception { Request getRequest = new Request("GET", "_watcher/watch/bwc_watch"); getRequest.setOptions( expectWarnings( - IndexAction.TYPES_DEPRECATION_MESSAGE + INDEX_ACTION_TYPES_DEPRECATION_MESSAGE ) ); @@ -310,7 +306,7 @@ private void assertWatchIndexContentsWork() throws Exception { getRequest = new Request("GET", "_watcher/watch/bwc_throttle_period"); getRequest.setOptions( expectWarnings( - IndexAction.TYPES_DEPRECATION_MESSAGE + INDEX_ACTION_TYPES_DEPRECATION_MESSAGE ) ); @@ -352,10 +348,9 @@ private void assertWatchIndexContentsWork() throws Exception { private void assertBasicWatchInteractions() throws Exception { - String watch = new WatchSourceBuilder() - .condition(InternalAlwaysCondition.INSTANCE) - .trigger(ScheduleTrigger.builder(new IntervalSchedule(IntervalSchedule.Interval.seconds(1)))) - .addAction("awesome", LoggingAction.builder(new TextTemplate("test"))).buildAsBytes(XContentType.JSON).utf8ToString(); + String watch = "{\"trigger\":{\"schedule\":{\"interval\":\"1s\"}},\"input\":{\"none\":{}}," + + "\"condition\":{\"always\":{}}," + + "\"actions\":{\"awesome\":{\"logging\":{\"level\":\"info\",\"text\":\"test\"}}}}"; Request createWatchRequest = new Request("PUT", "_watcher/watch/new_watch"); createWatchRequest.setJsonEntity(watch); Map createWatch = entityAsMap(client().performRequest(createWatchRequest)); diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlMigrationFullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlMigrationFullClusterRestartIT.java index 917d73d5af2c4..3c849811f5620 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlMigrationFullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlMigrationFullClusterRestartIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.xpack.core.ml.job.config.DataDescription; import org.elasticsearch.xpack.core.ml.job.config.Detector; import org.elasticsearch.xpack.core.ml.job.config.Job; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.elasticsearch.xpack.test.rest.XPackRestTestHelper; import org.junit.Before; @@ -56,7 +57,7 @@ protected Settings restClientSettings() { @Before public void waitForMlTemplates() throws Exception { - List templatesToWaitFor = XPackRestTestHelper.ML_POST_V660_TEMPLATES; + List templatesToWaitFor = XPackRestTestConstants.ML_POST_V660_TEMPLATES; XPackRestTestHelper.waitForTemplates(client(), templatesToWaitFor); } diff --git a/x-pack/qa/multi-cluster-search-security/build.gradle b/x-pack/qa/multi-cluster-search-security/build.gradle index c31b2c0ad1d5e..8406345575dfd 100644 --- a/x-pack/qa/multi-cluster-search-security/build.gradle +++ b/x-pack/qa/multi-cluster-search-security/build.gradle @@ -3,9 +3,7 @@ import org.elasticsearch.gradle.test.RestIntegTestTask apply plugin: 'elasticsearch.standalone-test' dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') - testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') + testCompile project(':x-pack:qa') } task remoteClusterTest(type: RestIntegTestTask) { diff --git a/x-pack/qa/multi-cluster-search-security/src/test/java/org/elasticsearch/xpack/security/MultiClusterSearchWithSecurityYamlTestSuiteIT.java b/x-pack/qa/multi-cluster-search-security/src/test/java/org/elasticsearch/xpack/security/MultiClusterSearchWithSecurityYamlTestSuiteIT.java index e61ff9062d171..011fe5be92803 100644 --- a/x-pack/qa/multi-cluster-search-security/src/test/java/org/elasticsearch/xpack/security/MultiClusterSearchWithSecurityYamlTestSuiteIT.java +++ b/x-pack/qa/multi-cluster-search-security/src/test/java/org/elasticsearch/xpack/security/MultiClusterSearchWithSecurityYamlTestSuiteIT.java @@ -14,7 +14,7 @@ import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; public class MultiClusterSearchWithSecurityYamlTestSuiteIT extends ESClientYamlSuiteTestCase { diff --git a/x-pack/qa/multi-node/build.gradle b/x-pack/qa/multi-node/build.gradle index 243a6f40438cc..28de4a4c114a9 100644 --- a/x-pack/qa/multi-node/build.gradle +++ b/x-pack/qa/multi-node/build.gradle @@ -2,7 +2,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile "org.elasticsearch.plugin:x-pack-core:${version}" + testCompile project(':x-pack:qa') } integTestCluster { diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/GlobalCheckpointSyncActionIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/GlobalCheckpointSyncActionIT.java index 9d3e88cbc5c8d..02dd7ecb40cc1 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/GlobalCheckpointSyncActionIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/GlobalCheckpointSyncActionIT.java @@ -16,7 +16,7 @@ import org.elasticsearch.test.rest.yaml.ObjectPath; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; import static org.hamcrest.Matchers.equalTo; public class GlobalCheckpointSyncActionIT extends ESRestTestCase { diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java index feddc57f6e014..a5579ad0aa5d8 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; diff --git a/x-pack/qa/rolling-upgrade-basic/build.gradle b/x-pack/qa/rolling-upgrade-basic/build.gradle index bb28821b96fc0..28b6428146e45 100644 --- a/x-pack/qa/rolling-upgrade-basic/build.gradle +++ b/x-pack/qa/rolling-upgrade-basic/build.gradle @@ -4,9 +4,7 @@ import org.elasticsearch.gradle.test.RestIntegTestTask apply plugin: 'elasticsearch.standalone-test' dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') - testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') // to be moved in a later commit + testCompile project(':x-pack:qa') } // This is a top level task which we will add dependencies to below. @@ -141,14 +139,7 @@ task copyXPackRestSpec(type: Copy) { include 'rest-api-spec/api/**' into project.sourceSets.test.output.resourcesDir } - -task copyXPackPluginProps(type: Copy) { - dependsOn(copyXPackRestSpec) - from project(xpackModule('core')).file('src/main/plugin-metadata') - from project(xpackModule('core')).tasks.pluginProperties - into outputDir -} -project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackPluginProps) +project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackRestSpec) repositories { maven { diff --git a/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle b/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle index 19e1f5a716c08..ade76e361e6c4 100644 --- a/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle +++ b/x-pack/qa/rolling-upgrade-multi-cluster/build.gradle @@ -4,9 +4,7 @@ import org.elasticsearch.gradle.test.RestIntegTestTask apply plugin: 'elasticsearch.standalone-test' dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') - testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') // to be moved in a later commit + testCompile project(':x-pack:qa') } // This is a top level task which we will add dependencies to below. diff --git a/x-pack/qa/rolling-upgrade/build.gradle b/x-pack/qa/rolling-upgrade/build.gradle index b3d6bbd221d56..53810b89386f1 100644 --- a/x-pack/qa/rolling-upgrade/build.gradle +++ b/x-pack/qa/rolling-upgrade/build.gradle @@ -10,10 +10,7 @@ apply plugin: 'elasticsearch.standalone-test' test.enabled = false dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') - testCompile project(path: xpackModule('security'), configuration: 'runtime') - testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') // to be moved in a later commit + testCompile project(':x-pack:qa') testCompile ("org.elasticsearch.client:elasticsearch-rest-high-level-client:${version}") } @@ -307,11 +304,4 @@ task copyXPackRestSpec(type: Copy) { include 'rest-api-spec/api/**' into project.sourceSets.test.output.resourcesDir } - -task copyXPackPluginProps(type: Copy) { - dependsOn(copyXPackRestSpec) - from project(xpackModule('core')).file('src/main/plugin-metadata') - from project(xpackModule('core')).tasks.pluginProperties - into outputDir -} -project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackPluginProps) +project.sourceSets.test.output.dir(outputDir, builtBy: copyXPackRestSpec) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java index 0d0f05bcf9c6d..64c3a785d14e4 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java @@ -8,7 +8,7 @@ import org.elasticsearch.client.Request; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.test.SecuritySettingsSourceField; +import org.elasticsearch.xpack.test.SecuritySettingsSourceField; import org.elasticsearch.test.rest.ESRestTestCase; import org.junit.Before; @@ -16,12 +16,12 @@ import java.util.Collection; import java.util.Collections; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; public abstract class AbstractUpgradeTestCase extends ESRestTestCase { private static final String BASIC_AUTH_VALUE = - basicAuthHeaderValue("test_user", SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING); + basicAuthHeaderValue("test_user", SecuritySettingsSourceField.TEST_PASSWORD); @Override protected boolean preserveIndicesUponCompletion() { 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 4bded9a25c56c..13ed2dafc5f31 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 @@ -14,7 +14,7 @@ import org.elasticsearch.client.ml.job.config.Job; import org.elasticsearch.common.Strings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.elasticsearch.xpack.test.rest.XPackRestTestHelper; import java.io.IOException; @@ -30,7 +30,7 @@ public class MlMappingsUpgradeIT extends AbstractUpgradeTestCase { @Override protected Collection templatesToWaitFor() { - return Stream.concat(XPackRestTestHelper.ML_POST_V660_TEMPLATES.stream(), + return Stream.concat(XPackRestTestConstants.ML_POST_V660_TEMPLATES.stream(), super.templatesToWaitFor().stream()).collect(Collectors.toSet()); } @@ -81,7 +81,7 @@ private void createAndOpenTestJob() throws IOException { private void assertUpgradedMappings() throws Exception { assertBusy(() -> { - Request getMappings = new Request("GET", AnomalyDetectorsIndex.resultsWriteAlias(JOB_ID) + "/_mappings"); + Request getMappings = new Request("GET", XPackRestTestHelper.resultsWriteAlias(JOB_ID) + "/_mappings"); Response response = client().performRequest(getMappings); Map responseLevel = entityAsMap(response); diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java index 9374346449c95..33f256ac1e198 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/UpgradeClusterClientYamlTestSuiteIT.java @@ -13,6 +13,7 @@ import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.elasticsearch.xpack.test.rest.XPackRestTestHelper; import org.junit.Before; @@ -28,7 +29,7 @@ public class UpgradeClusterClientYamlTestSuiteIT extends ESClientYamlSuiteTestCa */ @Before public void waitForTemplates() throws Exception { - XPackRestTestHelper.waitForTemplates(client(), XPackRestTestHelper.ML_POST_V660_TEMPLATES); + XPackRestTestHelper.waitForTemplates(client(), XPackRestTestConstants.ML_POST_V660_TEMPLATES); } @Override diff --git a/x-pack/qa/saml-idp-tests/build.gradle b/x-pack/qa/saml-idp-tests/build.gradle index 7b76321fe9d4f..4355ac0b5b825 100644 --- a/x-pack/qa/saml-idp-tests/build.gradle +++ b/x-pack/qa/saml-idp-tests/build.gradle @@ -6,10 +6,7 @@ apply plugin: 'elasticsearch.rest-test' apply plugin: 'elasticsearch.test.fixtures' dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') - testCompile project(path: xpackModule('security'), configuration: 'testArtifacts') testCompile 'com.google.jimfs:jimfs:1.1' } testFixtures.useFixture ":x-pack:test:idp-fixture" @@ -103,9 +100,7 @@ thirdPartyAudit { 'com.google.common.cache.Striped64$1', 'com.google.common.cache.Striped64$Cell', 'com.google.common.primitives.UnsignedBytes$LexicographicalComparatorHolder$UnsafeComparator', - 'com.google.common.primitives.UnsignedBytes$LexicographicalComparatorHolder$UnsafeComparator$1', - 'com.google.common.util.concurrent.AbstractFuture$UnsafeAtomicHelper', - 'com.google.common.util.concurrent.AbstractFuture$UnsafeAtomicHelper$1' + 'com.google.common.primitives.UnsignedBytes$LexicographicalComparatorHolder$UnsafeComparator$1' ) ignoreMissingClasses ( diff --git a/x-pack/qa/smoke-test-monitoring-with-watcher/build.gradle b/x-pack/qa/smoke-test-monitoring-with-watcher/build.gradle index 57be337f634f2..18bf5656f1991 100644 --- a/x-pack/qa/smoke-test-monitoring-with-watcher/build.gradle +++ b/x-pack/qa/smoke-test-monitoring-with-watcher/build.gradle @@ -2,9 +2,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile "org.elasticsearch.plugin:x-pack-core:${version}" - testCompile project(path: xpackModule('watcher')) - testCompile project(path: xpackModule('monitoring')) + testCompile project(':x-pack:qa') } integTestCluster { diff --git a/x-pack/qa/smoke-test-monitoring-with-watcher/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java b/x-pack/qa/smoke-test-monitoring-with-watcher/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java index 66b52afed1652..40ed71df842f3 100644 --- a/x-pack/qa/smoke-test-monitoring-with-watcher/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java +++ b/x-pack/qa/smoke-test-monitoring-with-watcher/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java @@ -9,28 +9,32 @@ import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; -import org.elasticsearch.xpack.monitoring.exporter.ClusterAlertsUtil; -import org.elasticsearch.xpack.watcher.actions.ActionBuilders; -import org.elasticsearch.xpack.watcher.client.WatchSourceBuilders; -import org.elasticsearch.xpack.watcher.trigger.TriggerBuilders; -import org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule; import org.junit.After; import java.io.IOException; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.xpack.watcher.input.InputBuilders.simpleInput; -import static org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule.Interval.Unit.MINUTES; import static org.hamcrest.Matchers.is; @TestLogging("org.elasticsearch.client:TRACE,tracer:TRACE") @AwaitsFix(bugUrl = "flaky tests") public class MonitoringWithWatcherRestIT extends ESRestTestCase { + /** + * An unsorted list of Watch IDs representing resource files for Monitoring Cluster Alerts. + */ + public static final String[] WATCH_IDS = { + "elasticsearch_cluster_status", + "elasticsearch_version_mismatch", + "kibana_version_mismatch", + "logstash_version_mismatch", + "xpack_license_expiration", + "elasticsearch_nodes", + }; + @After public void cleanExporters() throws Exception { Request request = new Request("PUT", "/_cluster/settings"); @@ -53,7 +57,7 @@ public void testThatLocalExporterAddsWatches() throws Exception { .endObject().endObject())); adminClient().performRequest(request); - assertTotalWatchCount(ClusterAlertsUtil.WATCH_IDS.length); + assertTotalWatchCount(WATCH_IDS.length); assertMonitoringWatchHasBeenOverWritten(watchId); } @@ -71,7 +75,7 @@ public void testThatHttpExporterAddsWatches() throws Exception { .endObject().endObject())); adminClient().performRequest(request); - assertTotalWatchCount(ClusterAlertsUtil.WATCH_IDS.length); + assertTotalWatchCount(WATCH_IDS.length); assertMonitoringWatchHasBeenOverWritten(watchId); } @@ -95,11 +99,10 @@ private String createMonitoringWatch() throws Exception { String clusterUUID = getClusterUUID(); String watchId = clusterUUID + "_kibana_version_mismatch"; Request request = new Request("PUT", "/_watcher/watch/" + watchId); - request.setJsonEntity(WatchSourceBuilders.watchBuilder() - .trigger(TriggerBuilders.schedule(new IntervalSchedule(new IntervalSchedule.Interval(1000, MINUTES)))) - .input(simpleInput()) - .addAction("logme", ActionBuilders.loggingAction("foo")) - .buildAsBytes(XContentType.JSON).utf8ToString()); + String watch = "{\"trigger\":{\"schedule\":{\"interval\":\"1000m\"}},\"input\":{\"simple\":{}}," + + "\"condition\":{\"always\":{}}," + + "\"actions\":{\"logme\":{\"logging\":{\"level\":\"info\",\"text\":\"foo\"}}}}"; + request.setJsonEntity(watch); client().performRequest(request); return watchId; } diff --git a/x-pack/qa/smoke-test-plugins/build.gradle b/x-pack/qa/smoke-test-plugins/build.gradle index 3b7661eeeb05a..5aa3adbdf09f5 100644 --- a/x-pack/qa/smoke-test-plugins/build.gradle +++ b/x-pack/qa/smoke-test-plugins/build.gradle @@ -4,7 +4,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile "org.elasticsearch.plugin:x-pack-core:${version}" + testCompile project(':x-pack:qa') } ext.pluginsCount = 0 diff --git a/x-pack/qa/smoke-test-plugins/src/test/java/org/elasticsearch/smoketest/XSmokeTestPluginsClientYamlTestSuiteIT.java b/x-pack/qa/smoke-test-plugins/src/test/java/org/elasticsearch/smoketest/XSmokeTestPluginsClientYamlTestSuiteIT.java index 29671386f5ba0..6a676be7430df 100644 --- a/x-pack/qa/smoke-test-plugins/src/test/java/org/elasticsearch/smoketest/XSmokeTestPluginsClientYamlTestSuiteIT.java +++ b/x-pack/qa/smoke-test-plugins/src/test/java/org/elasticsearch/smoketest/XSmokeTestPluginsClientYamlTestSuiteIT.java @@ -14,7 +14,7 @@ import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; public class XSmokeTestPluginsClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { diff --git a/x-pack/qa/smoke-test-security-with-mustache/build.gradle b/x-pack/qa/smoke-test-security-with-mustache/build.gradle index 48b525ba3dae9..bf2e6c325730e 100644 --- a/x-pack/qa/smoke-test-security-with-mustache/build.gradle +++ b/x-pack/qa/smoke-test-security-with-mustache/build.gradle @@ -2,10 +2,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here - testCompile project(path: xpackModule('core'), configuration: 'default') - testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') - testCompile project(path: ':modules:lang-mustache', configuration: 'runtime') + testCompile project(':x-pack:qa') } integTestCluster { diff --git a/x-pack/qa/smoke-test-security-with-mustache/src/test/java/org/elasticsearch/smoketest/SmokeTestSecurityWithMustacheClientYamlTestSuiteIT.java b/x-pack/qa/smoke-test-security-with-mustache/src/test/java/org/elasticsearch/smoketest/SmokeTestSecurityWithMustacheClientYamlTestSuiteIT.java index 46daddae62b0d..f65fd64be29f5 100644 --- a/x-pack/qa/smoke-test-security-with-mustache/src/test/java/org/elasticsearch/smoketest/SmokeTestSecurityWithMustacheClientYamlTestSuiteIT.java +++ b/x-pack/qa/smoke-test-security-with-mustache/src/test/java/org/elasticsearch/smoketest/SmokeTestSecurityWithMustacheClientYamlTestSuiteIT.java @@ -14,7 +14,7 @@ import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; public class SmokeTestSecurityWithMustacheClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { diff --git a/x-pack/qa/smoke-test-watcher-with-security/build.gradle b/x-pack/qa/smoke-test-watcher-with-security/build.gradle index 2ff2ff9272fa5..0b622fc446b38 100644 --- a/x-pack/qa/smoke-test-watcher-with-security/build.gradle +++ b/x-pack/qa/smoke-test-watcher-with-security/build.gradle @@ -2,7 +2,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile "org.elasticsearch.plugin:x-pack-core:${version}" + testCompile project(':x-pack:qa') } // bring in watcher rest test suite diff --git a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java index 879be233fa180..679bc08f01f38 100644 --- a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java +++ b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityClientYamlTestSuiteIT.java @@ -14,7 +14,7 @@ import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ClientYamlTestResponse; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; -import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.junit.After; import org.junit.Before; @@ -23,7 +23,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.singletonMap; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; import static org.hamcrest.Matchers.is; public class SmokeTestWatcherWithSecurityClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase { @@ -76,7 +76,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM) { + for (String template : XPackRestTestConstants.TEMPLATE_NAMES_NO_ILM) { ClientYamlTestResponse templateExistsResponse = getAdminExecutionContext().callApi("indices.exists_template", singletonMap("name", template), emptyList(), emptyMap()); assertThat(templateExistsResponse.getStatusCode(), is(200)); diff --git a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java index da8eca37d5476..902115e82925d 100644 --- a/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java +++ b/x-pack/qa/smoke-test-watcher-with-security/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherWithSecurityIT.java @@ -16,7 +16,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; -import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.junit.After; import org.junit.Before; @@ -26,7 +26,7 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.rest.action.search.RestSearchAction.TOTAL_HITS_AS_INT_PARAM; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasEntry; @@ -83,7 +83,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES) { + for (String template : XPackRestTestConstants.TEMPLATE_NAMES) { assertOK(adminClient().performRequest(new Request("HEAD", "_template/" + template))); } }); diff --git a/x-pack/qa/smoke-test-watcher/build.gradle b/x-pack/qa/smoke-test-watcher/build.gradle index fb2e4c06ced88..9194c46daed01 100644 --- a/x-pack/qa/smoke-test-watcher/build.gradle +++ b/x-pack/qa/smoke-test-watcher/build.gradle @@ -2,10 +2,7 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile "org.elasticsearch.plugin:x-pack-core:${version}" - testCompile project(path: xpackModule('watcher'), configuration: 'runtime') - testCompile project(path: ':modules:lang-mustache', configuration: 'runtime') - testCompile project(path: ':modules:lang-painless', configuration: 'runtime') + testCompile project(':x-pack:qa') } integTestCluster { diff --git a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java index e08b2a20278f0..ec5015345fcb2 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java +++ b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/SmokeTestWatcherTestSuiteIT.java @@ -15,7 +15,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; -import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.junit.After; import org.junit.Before; @@ -25,7 +25,7 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.rest.action.search.RestSearchAction.TOTAL_HITS_AS_INT_PARAM; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.elasticsearch.xpack.test.SecuritySettingsSourceField.basicAuthHeaderValue; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.is; @@ -64,7 +64,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM) { + for (String template : XPackRestTestConstants.TEMPLATE_NAMES_NO_ILM) { Response templateExistsResponse = adminClient().performRequest(new Request("HEAD", "/_template/" + template)); assertThat(templateExistsResponse.getStatusLine().getStatusCode(), is(200)); } diff --git a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java index 19c82c8cef799..2dd5cc86a89c6 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java +++ b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherRestIT.java @@ -10,7 +10,7 @@ import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ClientYamlTestResponse; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; -import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField; +import org.elasticsearch.xpack.test.rest.XPackRestTestConstants; import org.junit.After; import org.junit.Before; @@ -58,7 +58,7 @@ public void startWatcher() throws Exception { }); assertBusy(() -> { - for (String template : WatcherIndexTemplateRegistryField.TEMPLATE_NAMES_NO_ILM) { + for (String template : XPackRestTestConstants.TEMPLATE_NAMES_NO_ILM) { ClientYamlTestResponse templateExistsResponse = getAdminExecutionContext().callApi("indices.exists_template", singletonMap("name", template), emptyList(), emptyMap()); assertThat(templateExistsResponse.getStatusCode(), is(200)); diff --git a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/SecuritySettingsSourceField.java b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/SecuritySettingsSourceField.java new file mode 100644 index 0000000000000..8747d7809896b --- /dev/null +++ b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/SecuritySettingsSourceField.java @@ -0,0 +1,42 @@ +/* + * 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.test; + +import org.elasticsearch.common.CharArrays; +import org.elasticsearch.common.settings.SecureString; + +import java.nio.CharBuffer; +import java.util.Arrays; +import java.util.Base64; + +public final class SecuritySettingsSourceField { + public static final SecureString TEST_PASSWORD_SECURE_STRING = new SecureString("x-pack-test-password".toCharArray()); + public static final String TEST_PASSWORD = "x-pack-test-password"; + + private SecuritySettingsSourceField() {} + + public static String basicAuthHeaderValue(String username, String passwd) { + return basicAuthHeaderValue(username, new SecureString(passwd.toCharArray())); + } + + public static String basicAuthHeaderValue(String username, SecureString passwd) { + CharBuffer chars = CharBuffer.allocate(username.length() + passwd.length() + 1); + byte[] charBytes = null; + try { + chars.put(username).put(':').put(passwd.getChars()); + charBytes = CharArrays.toUtf8Bytes(chars.array()); + + //TODO we still have passwords in Strings in headers. Maybe we can look into using a CharSequence? + String basicToken = Base64.getEncoder().encodeToString(charBytes); + return "Basic " + basicToken; + } finally { + Arrays.fill(chars.array(), (char) 0); + if (charBytes != null) { + Arrays.fill(charBytes, (byte) 0); + } + } + } +} diff --git a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestConstants.java b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestConstants.java new file mode 100644 index 0000000000000..1a6a59fbc696b --- /dev/null +++ b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestConstants.java @@ -0,0 +1,38 @@ +/* + * 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.test.rest; + +import java.util.List; + +public final class XPackRestTestConstants { + + // Watcher constants: + public static final String INDEX_TEMPLATE_VERSION = "9"; + public static final String HISTORY_TEMPLATE_NAME = ".watch-history-" + INDEX_TEMPLATE_VERSION; + public static final String HISTORY_TEMPLATE_NAME_NO_ILM = ".watch-history-no-ilm-" + INDEX_TEMPLATE_VERSION; + public static final String TRIGGERED_TEMPLATE_NAME = ".triggered_watches"; + public static final String WATCHES_TEMPLATE_NAME = ".watches"; + public static final String[] TEMPLATE_NAMES = new String[] { + HISTORY_TEMPLATE_NAME, TRIGGERED_TEMPLATE_NAME, WATCHES_TEMPLATE_NAME + }; + public static final String[] TEMPLATE_NAMES_NO_ILM = new String[] { + HISTORY_TEMPLATE_NAME_NO_ILM, TRIGGERED_TEMPLATE_NAME, WATCHES_TEMPLATE_NAME + }; + + // ML constants: + public static final String ML_META_INDEX_NAME = ".ml-meta"; + public static final String AUDITOR_NOTIFICATIONS_INDEX = ".ml-notifications"; + public static final String CONFIG_INDEX = ".ml-config"; + public static final String RESULTS_INDEX_PREFIX = ".ml-anomalies-"; + public static final String STATE_INDEX_PREFIX = ".ml-state"; + public static final String RESULTS_INDEX_DEFAULT = "shared"; + + public static final List ML_POST_V660_TEMPLATES = + List.of(AUDITOR_NOTIFICATIONS_INDEX, ML_META_INDEX_NAME, STATE_INDEX_PREFIX, RESULTS_INDEX_PREFIX, CONFIG_INDEX); + + private XPackRestTestConstants() { + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java similarity index 81% rename from x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java rename to x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java index 24804129bde6c..6ad16d512ef0b 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java +++ b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/XPackRestTestHelper.java @@ -14,14 +14,8 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.core.ml.MlMetaIndex; -import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; -import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields; -import org.elasticsearch.xpack.core.ml.notifications.AuditorField; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; @@ -30,13 +24,6 @@ public final class XPackRestTestHelper { - public static final List ML_POST_V660_TEMPLATES = Collections.unmodifiableList( - Arrays.asList(AuditorField.NOTIFICATIONS_INDEX, - MlMetaIndex.INDEX_NAME, - AnomalyDetectorsIndexFields.STATE_INDEX_PREFIX, - AnomalyDetectorsIndex.jobResultsIndexPrefix(), - AnomalyDetectorsIndex.configIndexName())); - private XPackRestTestHelper() { } @@ -89,4 +76,10 @@ public static void waitForTemplates(RestClient client, List templateName }); } } + + public static String resultsWriteAlias(String jobId) { + // ".write" rather than simply "write" to avoid the danger of clashing + // with the read alias of a job whose name begins with "write-" + return XPackRestTestConstants.RESULTS_INDEX_PREFIX + ".write-" + jobId; + } } From cb3b21308636edccbcc529f603699ef2646bef48 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 15 Apr 2019 09:41:34 +0200 Subject: [PATCH 029/260] Clarify some ToXContent implementations behaviour (#41000) This change adds either ToXContentObject or ToXContentFragment to classes directly implementing ToXContent currently. This helps in reasoning about whether those implementations output full xcontent object or just fragments. Relates to #16347 --- .../client/graph/GraphExploreResponse.java | 2 - .../indices/PutIndexTemplateRequest.java | 34 ++++++------- .../client/ml/FindFileStructureRequest.java | 4 +- .../client/RestHighLevelClientTests.java | 14 +++--- .../action/PainlessExecuteAction.java | 3 +- .../action/ShardOperationFailedException.java | 4 +- .../storedscripts/PutStoredScriptRequest.java | 4 +- .../shards/IndicesShardStoresResponse.java | 7 +-- .../template/put/PutIndexTemplateRequest.java | 49 ++++++++++--------- .../search/SearchPhaseExecutionException.java | 2 - .../action/search/SearchResponse.java | 4 +- .../action/search/ShardSearchFailure.java | 18 ++++--- .../DefaultShardOperationFailedException.java | 7 +++ .../routing/allocation/decider/Decision.java | 6 ++- .../index/query/IntervalsSourceProvider.java | 4 +- .../index/seqno/RetentionLease.java | 4 +- .../index/seqno/RetentionLeases.java | 3 +- .../rest/action/RestActions.java | 2 - .../elasticsearch/snapshots/SnapshotInfo.java | 4 -- .../snapshots/SnapshotShardFailure.java | 2 + .../common/xcontent/BaseXContentTestCase.java | 5 +- .../xpack/graph/GraphExploreResponse.java | 2 - .../rollup/action/GetRollupCapsAction.java | 5 +- .../action/GetRollupIndexCapsAction.java | 5 +- .../rollup/action/GetRollupJobsAction.java | 5 +- .../rollup/action/StartRollupJobAction.java | 5 +- .../rollup/action/StopRollupJobAction.java | 5 +- .../action/token/InvalidateTokenResponse.java | 3 +- .../support/mapper/TemplateRoleName.java | 4 +- .../actions/get/GetWatchResponse.java | 6 ++- .../rest/action/RestGetWatchAction.java | 2 - 31 files changed, 120 insertions(+), 104 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/graph/GraphExploreResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/graph/GraphExploreResponse.java index dddc4bedfe466..2171731290151 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/graph/GraphExploreResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/graph/GraphExploreResponse.java @@ -133,9 +133,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startArray(FAILURES.getPreferredName()); if (shardFailures != null) { for (ShardOperationFailedException shardFailure : shardFailures) { - builder.startObject(); shardFailure.toXContent(builder, params); - builder.endObject(); } } builder.endArray(); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/PutIndexTemplateRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/PutIndexTemplateRequest.java index 5f22691b046eb..7008a719b7b36 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/PutIndexTemplateRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indices/PutIndexTemplateRequest.java @@ -31,7 +31,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; @@ -56,7 +56,7 @@ /** * A request to create an index template. */ -public class PutIndexTemplateRequest extends MasterNodeRequest implements IndicesRequest, ToXContent { +public class PutIndexTemplateRequest extends MasterNodeRequest implements IndicesRequest, ToXContentFragment { private String name; @@ -191,7 +191,7 @@ public PutIndexTemplateRequest settings(Map source) { public Settings settings() { return this.settings; } - + /** * Adds mapping that will be added when the index gets created. * @@ -201,7 +201,7 @@ public Settings settings() { public PutIndexTemplateRequest mapping(String source, XContentType xContentType) { internalMapping(XContentHelper.convertToMap(new BytesArray(source), true, xContentType).v2()); return this; - } + } /** * The cause for this index template creation. @@ -221,11 +221,11 @@ public String cause() { * @param source The mapping source */ public PutIndexTemplateRequest mapping(XContentBuilder source) { - internalMapping(XContentHelper.convertToMap(BytesReference.bytes(source), + internalMapping(XContentHelper.convertToMap(BytesReference.bytes(source), true, source.contentType()).v2()); - return this; - } - + return this; + } + /** * Adds mapping that will be added when the index gets created. * @@ -235,8 +235,8 @@ public PutIndexTemplateRequest mapping(XContentBuilder source) { public PutIndexTemplateRequest mapping(BytesReference source, XContentType xContentType) { internalMapping(XContentHelper.convertToMap(source, true, xContentType).v2()); return this; - } - + } + /** * Adds mapping that will be added when the index gets created. * @@ -244,7 +244,7 @@ public PutIndexTemplateRequest mapping(BytesReference source, XContentType xCont */ public PutIndexTemplateRequest mapping(Map source) { return internalMapping(source); - } + } private PutIndexTemplateRequest internalMapping(Map source) { try { @@ -257,12 +257,12 @@ private PutIndexTemplateRequest internalMapping(Map source) { return this; } catch (IOException e) { throw new UncheckedIOException("failed to convert source to json", e); - } + } } catch (IOException e) { throw new ElasticsearchGenerationException("Failed to generate [" + source + "]", e); } - } - + } + public BytesReference mappings() { return this.mappings; } @@ -349,8 +349,8 @@ public PutIndexTemplateRequest source(byte[] source, int offset, int length, XCo */ public PutIndexTemplateRequest source(BytesReference source, XContentType xContentType) { return source(XContentHelper.convertToMap(source, true, xContentType).v2()); - } - + } + public Set aliases() { return this.aliases; @@ -441,7 +441,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.copyCurrentStructure(parser); } } - + builder.startObject("aliases"); for (Alias alias : aliases) { alias.toXContent(builder, params); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java index 90e0c720e8811..adfee92bd6171 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/FindFileStructureRequest.java @@ -25,7 +25,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; @@ -34,7 +34,7 @@ import java.util.Objects; import java.util.Optional; -public class FindFileStructureRequest implements Validatable, ToXContent { +public class FindFileStructureRequest implements Validatable, ToXContentFragment { public static final ParseField LINES_TO_SAMPLE = new ParseField("lines_to_sample"); public static final ParseField TIMEOUT = new ParseField("timeout"); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java index 90d440fe723bc..ed5d7b66d80c1 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.client; import com.fasterxml.jackson.core.JsonParseException; + import org.apache.http.HttpEntity; import org.apache.http.HttpHost; import org.apache.http.HttpResponse; @@ -61,6 +62,7 @@ import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.cbor.CborXContent; @@ -176,7 +178,7 @@ public void testInfo() throws IOException { MainResponse testInfo = new MainResponse("nodeName", new MainResponse.Version("number", "buildFlavor", "buildType", "buildHash", "buildDate", true, "luceneVersion", "minimumWireCompatibilityVersion", "minimumIndexCompatibilityVersion"), "clusterName", "clusterUuid", "You Know, for Search"); - mockResponse((builder, params) -> { + mockResponse((ToXContentFragment) (builder, params) -> { // taken from the server side MainResponse builder.field("name", testInfo.getNodeName()); builder.field("cluster_name", testInfo.getClusterName()); @@ -762,12 +764,12 @@ public void testApiNamingConventions() throws Exception { Collectors.mapping(Tuple::v2, Collectors.toSet()))); // TODO remove in 8.0 - we will undeprecate indices.get_template because the current getIndexTemplate - // impl will replace the existing getTemplate method. + // impl will replace the existing getTemplate method. // The above general-purpose code ignores all deprecated methods which in this case leaves `getTemplate` - // looking like it doesn't have a valid implementatation when it does. + // looking like it doesn't have a valid implementatation when it does. apiUnsupported.remove("indices.get_template"); - - + + for (Map.Entry> entry : methods.entrySet()) { String apiName = entry.getKey(); @@ -830,7 +832,7 @@ private static void assertSyncMethod(Method method, String apiName, List assertThat("the return type for method [" + method + "] is incorrect", method.getReturnType().getSimpleName(), equalTo("boolean")); } else { - // It's acceptable for 404s to be represented as empty Optionals + // It's acceptable for 404s to be represented as empty Optionals if (!method.getReturnType().isAssignableFrom(Optional.class)) { assertThat("the return type for method [" + method + "] is incorrect", method.getReturnType().getSimpleName(), endsWith("Response")); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java index 81e3bdcd5c4c3..e8d93b8ef772e 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java @@ -55,7 +55,6 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; @@ -107,7 +106,7 @@ public Response newResponse() { return new Response(); } - public static class Request extends SingleShardRequest implements ToXContent { + public static class Request extends SingleShardRequest implements ToXContentObject { private static final ParseField SCRIPT_FIELD = new ParseField("script"); private static final ParseField CONTEXT_FIELD = new ParseField("context"); diff --git a/server/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java b/server/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java index 490a1760abeaa..34a8ccd7ad148 100644 --- a/server/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java +++ b/server/src/main/java/org/elasticsearch/action/ShardOperationFailedException.java @@ -21,7 +21,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.Streamable; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.rest.RestStatus; import java.util.Objects; @@ -30,7 +30,7 @@ * An exception indicating that a failure occurred performing an operation on the shard. * */ -public abstract class ShardOperationFailedException implements Streamable, ToXContent { +public abstract class ShardOperationFailedException implements Streamable, ToXContentObject { protected String index; protected int shardId = -1; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java index 29171f12a1859..9904fc5dcde2b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java @@ -24,7 +24,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; @@ -35,7 +35,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class PutStoredScriptRequest extends AcknowledgedRequest implements ToXContent { +public class PutStoredScriptRequest extends AcknowledgedRequest implements ToXContentFragment { private String id; private String context; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java index a200bc82476c3..86198cb74391d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java @@ -21,6 +21,7 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; + import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.DefaultShardOperationFailedException; @@ -258,8 +259,10 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); builder.field("node", nodeId()); - super.toXContent(builder, params); + super.innerToXContent(builder, params); + builder.endObject(); return builder; } } @@ -352,9 +355,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (failures.size() > 0) { builder.startArray(Fields.FAILURES); for (Failure failure : failures) { - builder.startObject(); failure.toXContent(builder, params); - builder.endObject(); } builder.endArray(); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index 7e3c549cfcf76..a23fac0ac12b3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -38,7 +38,7 @@ import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; @@ -59,14 +59,14 @@ import java.util.stream.Collectors; import static org.elasticsearch.action.ValidateActions.addValidationError; -import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS; import static org.elasticsearch.common.settings.Settings.readSettingsFromStream; import static org.elasticsearch.common.settings.Settings.writeSettingsToStream; +import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS; /** * A request to create an index template. */ -public class PutIndexTemplateRequest extends MasterNodeRequest implements IndicesRequest, ToXContent { +public class PutIndexTemplateRequest extends MasterNodeRequest implements IndicesRequest, ToXContentObject { private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(PutIndexTemplateRequest.class)); @@ -498,32 +498,35 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.field("index_patterns", indexPatterns); - builder.field("order", order); - if (version != null) { - builder.field("version", version); - } + builder.startObject(); + { + builder.field("index_patterns", indexPatterns); + builder.field("order", order); + if (version != null) { + builder.field("version", version); + } - builder.startObject("settings"); - settings.toXContent(builder, params); - builder.endObject(); + builder.startObject("settings"); + settings.toXContent(builder, params); + builder.endObject(); - builder.startObject("mappings"); - for (Map.Entry entry : mappings.entrySet()) { - builder.field(entry.getKey()); - try (XContentParser parser = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, - DeprecationHandler.THROW_UNSUPPORTED_OPERATION, entry.getValue())) { - builder.copyCurrentStructure(parser); + builder.startObject("mappings"); + for (Map.Entry entry : mappings.entrySet()) { + builder.field(entry.getKey()); + try (XContentParser parser = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, entry.getValue())) { + builder.copyCurrentStructure(parser); + } } - } - builder.endObject(); + builder.endObject(); - builder.startObject("aliases"); - for (Alias alias : aliases) { - alias.toXContent(builder, params); + builder.startObject("aliases"); + for (Alias alias : aliases) { + alias.toXContent(builder, params); + } + builder.endObject(); } builder.endObject(); - return builder; } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseExecutionException.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseExecutionException.java index e3247c4f55199..8b4187c7cf0e9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseExecutionException.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseExecutionException.java @@ -140,9 +140,7 @@ protected void metadataToXContent(XContentBuilder builder, Params params) throws builder.startArray(); ShardOperationFailedException[] failures = ExceptionsHelper.groupBy(shardFailures); for (ShardOperationFailedException failure : failures) { - builder.startObject(); failure.toXContent(builder, params); - builder.endObject(); } builder.endArray(); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java index 0e6773f5fbb43..b9b1887be2da8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -29,7 +29,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.StatusToXContentObject; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; @@ -401,7 +401,7 @@ public String toString() { * Holds info about the clusters that the search was executed on: how many in total, how many of them were successful * and how many of them were skipped. */ - public static class Clusters implements ToXContent, Writeable { + public static class Clusters implements ToXContentFragment, Writeable { public static final Clusters EMPTY = new Clusters(0, 0, 0); diff --git a/server/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java b/server/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java index 451ceda70fdc4..cfd23e3c77368 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java +++ b/server/src/main/java/org/elasticsearch/action/search/ShardSearchFailure.java @@ -118,14 +118,18 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.field(SHARD_FIELD, shardId()); - builder.field(INDEX_FIELD, index()); - if (shardTarget != null) { - builder.field(NODE_FIELD, shardTarget.getNodeId()); - } - builder.field(REASON_FIELD); builder.startObject(); - ElasticsearchException.generateThrowableXContent(builder, params, cause); + { + builder.field(SHARD_FIELD, shardId()); + builder.field(INDEX_FIELD, index()); + if (shardTarget != null) { + builder.field(NODE_FIELD, shardTarget.getNodeId()); + } + builder.field(REASON_FIELD); + builder.startObject(); + ElasticsearchException.generateThrowableXContent(builder, params, cause); + builder.endObject(); + } builder.endObject(); return builder; } diff --git a/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java b/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java index 85d8a2c1a38db..7aa7dfb62a63c 100644 --- a/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java +++ b/server/src/main/java/org/elasticsearch/action/support/DefaultShardOperationFailedException.java @@ -91,6 +91,13 @@ public String toString() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + innerToXContent(builder, params); + builder.endObject(); + return builder; + } + + protected XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException { builder.field("shard", shardId()); builder.field("index", index()); builder.field("status", status.name()); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java index 725df82e51599..5311cd9c4a3fa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java @@ -24,6 +24,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; @@ -150,7 +152,7 @@ public boolean higherThan(Type other) { /** * Simple class representing a single decision */ - public static class Single extends Decision { + public static class Single extends Decision implements ToXContentObject { private Type type; private String label; private String explanation; @@ -269,7 +271,7 @@ public void writeTo(StreamOutput out) throws IOException { /** * Simple class representing a list of decisions */ - public static class Multi extends Decision { + public static class Multi extends Decision implements ToXContentFragment { private final List decisions = new ArrayList<>(); diff --git a/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java b/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java index 8aef53bc20e31..6aa8f2d700e7e 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java +++ b/server/src/main/java/org/elasticsearch/index/query/IntervalsSourceProvider.java @@ -31,8 +31,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentFragment; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.analysis.NamedAnalyzer; @@ -457,7 +457,7 @@ protected boolean accept(IntervalIterator it) { } } - public static class IntervalFilter implements ToXContent, Writeable { + public static class IntervalFilter implements ToXContentObject, Writeable { public static final String NAME = "filter"; diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index e6d6ed3fe825f..9cfad7c36ea06 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -24,7 +24,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -37,7 +37,7 @@ * otherwise merge away operations that have been soft deleted). Each retention lease contains a unique identifier, the retaining sequence * number, the timestamp of when the lease was created or renewed, and the source of the retention lease (e.g., "ccr"). */ -public final class RetentionLease implements ToXContent, Writeable { +public final class RetentionLease implements ToXContentObject, Writeable { private final String id; diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java index 3bad887282502..7c3b9e3c7b9c9 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.gateway.MetaDataStateFormat; @@ -42,7 +43,7 @@ * Represents a versioned collection of retention leases. We version the collection of retention leases to ensure that sync requests that * arrive out of order on the replica, using the version to ensure that older sync requests are rejected. */ -public class RetentionLeases implements ToXContent, Writeable { +public class RetentionLeases implements ToXContentFragment, Writeable { private final long primaryTerm; diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestActions.java b/server/src/main/java/org/elasticsearch/rest/action/RestActions.java index f25fd107e51db..f6ad30706f757 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestActions.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestActions.java @@ -91,9 +91,7 @@ public static void buildBroadcastShardsHeader(XContentBuilder builder, Params pa if (shardFailures != null && shardFailures.length > 0) { builder.startArray(FAILURES_FIELD.getPreferredName()); for (ShardOperationFailedException shardFailure : ExceptionsHelper.groupBy(shardFailures)) { - builder.startObject(); shardFailure.toXContent(builder, params); - builder.endObject(); } builder.endArray(); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index a8402dfa72afd..ca743f77aa3ba 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -510,9 +510,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (verbose || !shardFailures.isEmpty()) { builder.startArray(FAILURES); for (SnapshotShardFailure shardFailure : shardFailures) { - builder.startObject(); shardFailure.toXContent(builder, params); - builder.endObject(); } builder.endArray(); } @@ -551,9 +549,7 @@ private XContentBuilder toXContentInternal(final XContentBuilder builder, final builder.field(SUCCESSFUL_SHARDS, successfulShards); builder.startArray(FAILURES); for (SnapshotShardFailure shardFailure : shardFailures) { - builder.startObject(); shardFailure.toXContent(builder, params); - builder.endObject(); } builder.endArray(); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java index a497160967287..10e92b617d353 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java @@ -187,6 +187,7 @@ public static SnapshotShardFailure fromXContent(XContentParser parser) throws IO @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); builder.field("index", shardId.getIndexName()); builder.field("index_uuid", shardId.getIndexName()); builder.field("shard_id", shardId.id()); @@ -195,6 +196,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("node_id", nodeId); } builder.field("status", status.name()); + builder.endObject(); return builder; } diff --git a/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java b/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java index bafe3b7403d6f..7c5cc2bc80264 100644 --- a/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java +++ b/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonGenerationException; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParseException; + import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -752,7 +753,7 @@ public void testToXContent() throws Exception { .field("xcontent", xcontent0) .endObject()); - ToXContent xcontent1 = (builder, params) -> { + ToXContentObject xcontent1 = (builder, params) -> { builder.startObject(); builder.field("field", "value"); builder.startObject("foo"); @@ -762,7 +763,7 @@ public void testToXContent() throws Exception { return builder; }; - ToXContent xcontent2 = (builder, params) -> { + ToXContentObject xcontent2 = (builder, params) -> { builder.startObject(); builder.field("root", xcontent0); builder.array("childs", xcontent0, xcontent1); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/graph/GraphExploreResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/graph/GraphExploreResponse.java index b44e192f407ac..5f6a4b35a9ec1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/graph/GraphExploreResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/protocol/xpack/graph/GraphExploreResponse.java @@ -168,9 +168,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startArray(FAILURES.getPreferredName()); if (shardFailures != null) { for (ShardOperationFailedException shardFailure : shardFailures) { - builder.startObject(); shardFailure.toXContent(builder, params); - builder.endObject(); } } builder.endArray(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupCapsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupCapsAction.java index d28d14a0ac02d..f544c21a15c9a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupCapsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupCapsAction.java @@ -17,7 +17,8 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContent.Params; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.rollup.RollupField; @@ -43,7 +44,7 @@ public Response newResponse() { return new Response(); } - public static class Request extends ActionRequest implements ToXContent { + public static class Request extends ActionRequest implements ToXContentFragment { private String indexPattern; public Request(String indexPattern) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupIndexCapsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupIndexCapsAction.java index 4f95919c4986b..9dcd673c39fb2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupIndexCapsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupIndexCapsAction.java @@ -19,7 +19,8 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContent.Params; +import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.rollup.RollupField; @@ -47,7 +48,7 @@ public Response newResponse() { return new Response(); } - public static class Request extends ActionRequest implements IndicesRequest.Replaceable, ToXContent { + public static class Request extends ActionRequest implements IndicesRequest.Replaceable, ToXContentFragment { private String[] indices; private IndicesOptions options; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java index 4b33e018826d7..913e544e74190 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/GetRollupJobsAction.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; @@ -58,7 +57,7 @@ public Writeable.Reader getResponseReader() { return Response::new; } - public static class Request extends BaseTasksRequest implements ToXContent { + public static class Request extends BaseTasksRequest implements ToXContentObject { private String id; public Request(String id) { @@ -107,7 +106,9 @@ public ActionRequestValidationException validate() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); builder.field(RollupField.ID.getPreferredName(), id); + builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StartRollupJobAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StartRollupJobAction.java index ff803b136281a..ca2a5cd8d7264 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StartRollupJobAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StartRollupJobAction.java @@ -15,7 +15,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; @@ -44,7 +43,7 @@ public Writeable.Reader getResponseReader() { return Response::new; } - public static class Request extends BaseTasksRequest implements ToXContent { + public static class Request extends BaseTasksRequest implements ToXContentObject { private String id; public Request(String id) { @@ -75,7 +74,9 @@ public ActionRequestValidationException validate() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); builder.field(RollupField.ID.getPreferredName(), id); + builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StopRollupJobAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StopRollupJobAction.java index dadc54726b522..6fc079e0328c9 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StopRollupJobAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/action/StopRollupJobAction.java @@ -18,7 +18,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; @@ -51,7 +50,7 @@ public Writeable.Reader getResponseReader() { return Response::new; } - public static class Request extends BaseTasksRequest implements ToXContent { + public static class Request extends BaseTasksRequest implements ToXContentObject { private String id; private boolean waitForCompletion = false; private TimeValue timeout = null; @@ -106,11 +105,13 @@ public ActionRequestValidationException validate() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); builder.field(RollupField.ID.getPreferredName(), id); builder.field(WAIT_FOR_COMPLETION.getPreferredName(), waitForCompletion); if (timeout != null) { builder.field(TIMEOUT.getPreferredName(), timeout); } + builder.endObject(); return builder; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/token/InvalidateTokenResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/token/InvalidateTokenResponse.java index 9f11c48c96aba..7e25683a2a933 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/token/InvalidateTokenResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/token/InvalidateTokenResponse.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.security.authc.support.TokensInvalidationResult; @@ -18,7 +19,7 @@ /** * Response for a invalidation of one or multiple tokens. */ -public final class InvalidateTokenResponse extends ActionResponse implements ToXContent { +public final class InvalidateTokenResponse extends ActionResponse implements ToXContentObject { private TokensInvalidationResult result; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/TemplateRoleName.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/TemplateRoleName.java index d77882d6454d7..59f9eafec1c00 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/TemplateRoleName.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/support/mapper/TemplateRoleName.java @@ -16,7 +16,7 @@ import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; @@ -44,7 +44,7 @@ /** * Representation of a Mustache template for expressing one or more roles names in a {@link ExpressionRoleMapping}. */ -public class TemplateRoleName implements ToXContent, Writeable { +public class TemplateRoleName implements ToXContentObject, Writeable { private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "role-mapping-template", false, arr -> new TemplateRoleName((BytesReference) arr[0], (Format) arr[1])); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/transport/actions/get/GetWatchResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/transport/actions/get/GetWatchResponse.java index 18ec33f5dfb0d..fe79fdc33f938 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/transport/actions/get/GetWatchResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/transport/actions/get/GetWatchResponse.java @@ -10,7 +10,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.xpack.core.watcher.support.xcontent.XContentSource; @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.Objects; -public class GetWatchResponse extends ActionResponse implements ToXContent { +public class GetWatchResponse extends ActionResponse implements ToXContentObject { private String id; private WatchStatus status; @@ -122,6 +122,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); builder.field("found", found); builder.field("_id", id); if (found) { @@ -131,6 +132,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("status", status, params); builder.field("watch", source, params); } + builder.endObject(); return builder; } diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/rest/action/RestGetWatchAction.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/rest/action/RestGetWatchAction.java index 15cb9612445e1..0d9b1ee6a1991 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/rest/action/RestGetWatchAction.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/rest/action/RestGetWatchAction.java @@ -48,9 +48,7 @@ protected RestChannelConsumer doPrepareRequest(final RestRequest request, Watche return channel -> client.getWatch(getWatchRequest, new RestBuilderListener(channel) { @Override public RestResponse buildResponse(GetWatchResponse response, XContentBuilder builder) throws Exception { - builder.startObject(); response.toXContent(builder, request); - builder.endObject(); RestStatus status = response.isFound() ? OK : NOT_FOUND; return new BytesRestResponse(status, builder); } From f70d8cb9a56148047e2e90bfeefb2d9af29549d0 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Mon, 15 Apr 2019 09:46:50 +0200 Subject: [PATCH 030/260] Full text queries should not always ignore unmapped fields (#41062) Full text queries ignore unmapped fields since https://github.com/elastic/elasticsearch/issues/41022 even if all fields in the query are unmapped. This change makes sure that we ignore unmapped fields only if they are mixed with mapped fields and returns a MatchNoDocsQuery otherwise. Closes #41022 --- .../common/lucene/search/Queries.java | 8 ++++- .../index/search/MultiMatchQuery.java | 8 ++++- .../query/QueryStringQueryBuilderTests.java | 31 +++++++++++++++++-- .../query/SimpleQueryStringBuilderTests.java | 11 +++++++ 4 files changed, 53 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java b/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java index dd4389c2d6b0f..56d1b5cedc33c 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/search/Queries.java @@ -36,6 +36,8 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.TypeFieldMapper; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.regex.Pattern; @@ -52,7 +54,11 @@ public static Query newMatchNoDocsQuery(String reason) { public static Query newUnmappedFieldQuery(String field) { - return Queries.newMatchNoDocsQuery("unmapped field [" + (field != null ? field : "null") + "]"); + return newUnmappedFieldsQuery(Collections.singletonList(field)); + } + + public static Query newUnmappedFieldsQuery(Collection fields) { + return Queries.newMatchNoDocsQuery("unmapped fields " + fields); } public static Query newLenientFieldQuery(String field, RuntimeException e) { diff --git a/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java b/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java index 667d3a3823db8..b9943870df727 100644 --- a/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java +++ b/server/src/main/java/org/elasticsearch/index/search/MultiMatchQuery.java @@ -59,6 +59,12 @@ public void setTieBreaker(float tieBreaker) { public Query parse(MultiMatchQueryBuilder.Type type, Map fieldNames, Object value, String minimumShouldMatch) throws IOException { + boolean hasMappedField = fieldNames.keySet().stream() + .anyMatch(k -> context.fieldMapper(k) != null); + if (hasMappedField == false) { + // all query fields are unmapped + return Queries.newUnmappedFieldsQuery(fieldNames.keySet()); + } final float tieBreaker = groupTieBreaker == null ? type.tieBreaker() : groupTieBreaker; final List queries; switch (type) { @@ -91,7 +97,7 @@ private Query combineGrouped(List groupQuery, float tieBreaker) { } private List buildFieldQueries(MultiMatchQueryBuilder.Type type, Map fieldNames, - Object value, String minimumShouldMatch) throws IOException{ + Object value, String minimumShouldMatch) throws IOException { List queries = new ArrayList<>(); for (String fieldName : fieldNames.keySet()) { if (context.fieldMapper(fieldName) == null) { diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java index c85ef842d6870..608eb5d9d6cd2 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryStringQueryBuilderTests.java @@ -1210,13 +1210,13 @@ public void testUnmappedFieldRewriteToMatchNoDocs() throws IOException { .field("unmapped_field") .lenient(true) .toQuery(createShardContext()); - assertEquals(new BooleanQuery.Builder().build(), query); + assertEquals(new MatchNoDocsQuery(), query); // Unmapped prefix field query = new QueryStringQueryBuilder("unmapped_field:hello") .lenient(true) .toQuery(createShardContext()); - assertEquals(new BooleanQuery.Builder().build(), query); + assertEquals(new MatchNoDocsQuery(), query); // Unmapped fields query = new QueryStringQueryBuilder("hello") @@ -1224,7 +1224,32 @@ public void testUnmappedFieldRewriteToMatchNoDocs() throws IOException { .field("unmapped_field") .field("another_field") .toQuery(createShardContext()); - assertEquals(new BooleanQuery.Builder().build(), query); + assertEquals(new MatchNoDocsQuery(), query); + + // Multi block + query = new QueryStringQueryBuilder("first unmapped:second") + .field(STRING_FIELD_NAME) + .field("unmapped") + .field("another_unmapped") + .defaultOperator(Operator.AND) + .toQuery(createShardContext()); + BooleanQuery expected = new BooleanQuery.Builder() + .add(new TermQuery(new Term(STRING_FIELD_NAME, "first")), BooleanClause.Occur.MUST) + .add(new MatchNoDocsQuery(), BooleanClause.Occur.MUST) + .build(); + assertEquals(expected, query); + + query = new SimpleQueryStringBuilder("first unknown:second") + .field("unmapped") + .field("another_unmapped") + .defaultOperator(Operator.AND) + .toQuery(createShardContext()); + expected = new BooleanQuery.Builder() + .add(new MatchNoDocsQuery(), BooleanClause.Occur.MUST) + .add(new MatchNoDocsQuery(), BooleanClause.Occur.MUST) + .build(); + assertEquals(expected, query); + } public void testDefaultField() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java index 3242f343379aa..2bb289ddc11fa 100644 --- a/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java @@ -717,6 +717,17 @@ public void testUnmappedFieldNoTokenWithAndOperator() throws IOException { .add(new TermQuery(new Term(STRING_FIELD_NAME, "second")), BooleanClause.Occur.MUST) .build(); assertEquals(expected, query); + query = new SimpleQueryStringBuilder("first & second") + .field("unmapped") + .field("another_unmapped") + .defaultOperator(Operator.AND) + .toQuery(createShardContext()); + expected = new BooleanQuery.Builder() + .add(new MatchNoDocsQuery(), BooleanClause.Occur.MUST) + .add(new MatchNoDocsQuery(), BooleanClause.Occur.MUST) + .add(new MatchNoDocsQuery(), BooleanClause.Occur.MUST) + .build(); + assertEquals(expected, query); } public void testNegativeFieldBoost() { From 9f74a932eb558897009754cccda41621d423128e Mon Sep 17 00:00:00 2001 From: Guilherme Ferreira Date: Mon, 15 Apr 2019 14:09:46 +0200 Subject: [PATCH 031/260] [Docs] Correct spelling the "_none_" stopwords element (#41191) --- docs/reference/analysis/analyzers/standard-analyzer.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/analysis/analyzers/standard-analyzer.asciidoc b/docs/reference/analysis/analyzers/standard-analyzer.asciidoc index 3097ece21db23..5117763b66819 100644 --- a/docs/reference/analysis/analyzers/standard-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/standard-analyzer.asciidoc @@ -132,7 +132,7 @@ The `standard` analyzer accepts the following parameters: `stopwords`:: A pre-defined stop words list like `_english_` or an array containing a - list of stop words. Defaults to `\_none_`. + list of stop words. Defaults to `_none_`. `stopwords_path`:: From 923e2c738daa640621b70d10c5f23fef8fd3eb2f Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Mon, 15 Apr 2019 15:24:01 +0300 Subject: [PATCH 032/260] Mute failing test Tracked in #41172 --- .../rest-api-spec/test/mustache/50_webhook_url_escaping.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/50_webhook_url_escaping.yml b/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/50_webhook_url_escaping.yml index 01326f9764fa1..e11809a79baa5 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/50_webhook_url_escaping.yml +++ b/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/50_webhook_url_escaping.yml @@ -1,5 +1,8 @@ --- "Test url escaping with url mustache function": + - skip: + version: "all" + reason: "AwaitsFix https://github.com/elastic/elasticsearch/issues/41172" - do: cluster.health: wait_for_status: yellow From 17463d2be480a92a3fe8d9b8e6563548c0b0b704 Mon Sep 17 00:00:00 2001 From: Guilherme Ferreira Date: Mon, 15 Apr 2019 15:10:57 +0200 Subject: [PATCH 033/260] [Docs] Correct spelling of "_none_" (#41192) --- docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc index 3167a4342ac2d..259bf785b5b30 100644 --- a/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc @@ -78,4 +78,4 @@ Elasticsearch provides the following predefined list of languages: `_portuguese_`, `_romanian_`, `_russian_`, `_sorani_`, `_spanish_`, `_swedish_`, `_thai_`, `_turkish_`. -For the empty stopwords list (to disable stopwords) use: `\_none_`. +For the empty stopwords list (to disable stopwords) use: `_none_`. From 3d5a407e6ab1e0cdc3f83f59e18161f5ce95e672 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Mon, 15 Apr 2019 14:30:30 +0100 Subject: [PATCH 034/260] [ML] Allow xpack.ml.max_machine_memory_percent higher than 100% (#41193) Values higher than 100% are now allowed to accommodate use cases where swapping has been determined to be acceptable. Anomaly detector jobs only use their full model memory during background persistence, and this is deliberately staggered, so with large numbers of jobs few will generally be persisting state at the same time. Settings higher than available memory are only recommended for OEM type situations where a wrapper tightly controls the types of jobs that can be created, and each job alone is considerably smaller than what each node can handle. --- .../xpack/ml/MachineLearning.java | 8 ++++++- .../xpack/ml/MachineLearningTests.java | 23 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index b69e7b786a77e..281a374b97b2f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -271,8 +271,14 @@ public class MachineLearning extends Plugin implements ActionPlugin, AnalysisPlu public static final String MACHINE_MEMORY_NODE_ATTR = "ml.machine_memory"; public static final Setting CONCURRENT_JOB_ALLOCATIONS = Setting.intSetting("xpack.ml.node_concurrent_job_allocations", 2, 0, Property.Dynamic, Property.NodeScope); + // Values higher than 100% are allowed to accommodate use cases where swapping has been determined to be acceptable. + // Anomaly detector jobs only use their full model memory during background persistence, and this is deliberately + // staggered, so with large numbers of jobs few will generally be persisting state at the same time. + // Settings higher than available memory are only recommended for OEM type situations where a wrapper tightly + // controls the types of jobs that can be created, and each job alone is considerably smaller than what each node + // can handle. public static final Setting MAX_MACHINE_MEMORY_PERCENT = - Setting.intSetting("xpack.ml.max_machine_memory_percent", 30, 5, 90, Property.Dynamic, Property.NodeScope); + Setting.intSetting("xpack.ml.max_machine_memory_percent", 30, 5, 200, Property.Dynamic, Property.NodeScope); public static final Setting MAX_LAZY_ML_NODES = Setting.intSetting("xpack.ml.max_lazy_ml_nodes", 0, 0, 3, Property.Dynamic, Property.NodeScope); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MachineLearningTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MachineLearningTests.java index 2c296691c249d..9504cbe7a7011 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MachineLearningTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MachineLearningTests.java @@ -31,6 +31,29 @@ public void testMaxOpenWorkersSetting_givenSetting() { assertEquals(7, maxOpenWorkers); } + public void testMaxMachineMemoryPercent_givenDefault() { + int maxMachineMemoryPercent = MachineLearning.MAX_MACHINE_MEMORY_PERCENT.get(Settings.EMPTY); + assertEquals(30, maxMachineMemoryPercent); + } + + public void testMaxMachineMemoryPercent_givenValidSetting() { + Settings.Builder settings = Settings.builder(); + int expectedMaxMachineMemoryPercent = randomIntBetween(5, 200); + settings.put(MachineLearning.MAX_MACHINE_MEMORY_PERCENT.getKey(), expectedMaxMachineMemoryPercent); + int maxMachineMemoryPercent = MachineLearning.MAX_MACHINE_MEMORY_PERCENT.get(settings.build()); + assertEquals(expectedMaxMachineMemoryPercent, maxMachineMemoryPercent); + } + + public void testMaxMachineMemoryPercent_givenInvalidSetting() { + Settings.Builder settings = Settings.builder(); + int invalidMaxMachineMemoryPercent = randomFrom(4, 201); + settings.put(MachineLearning.MAX_MACHINE_MEMORY_PERCENT.getKey(), invalidMaxMachineMemoryPercent); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> MachineLearning.MAX_MACHINE_MEMORY_PERCENT.get(settings.build())); + assertThat(e.getMessage(), startsWith("Failed to parse value [" + invalidMaxMachineMemoryPercent + + "] for setting [xpack.ml.max_machine_memory_percent] must be")); + } + public void testNoAttributes_givenNoClash() { Settings.Builder builder = Settings.builder(); if (randomBoolean()) { From 3ad2b295466979991f16e9542ba4ac557298b208 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Mon, 15 Apr 2019 14:33:16 +0100 Subject: [PATCH 035/260] [ML DataFrame] Data Frame stop all (#41156) Wild card support for the data frame stop API --- .../dataframe/stop_data_frame.asciidoc | 4 +- .../data-frames/apis/stop-transform.asciidoc | 12 +++- .../action/StopDataFrameTransformAction.java | 36 ++++++++++-- ...pDataFrameTransformActionRequestTests.java | 35 +++++++++++- ...TransportStopDataFrameTransformAction.java | 51 ++++++++++++++--- ...portStopDataFrameTransformActionTests.java | 55 +++++++++++++++++++ .../test/data_frame/transforms_start_stop.yml | 42 ++++++++++++++ 7 files changed, 219 insertions(+), 16 deletions(-) create mode 100644 x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java diff --git a/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc b/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc index c91c228e0ea90..9b05687c00875 100644 --- a/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc +++ b/docs/java-rest/high-level/dataframe/stop_data_frame.asciidoc @@ -12,7 +12,9 @@ It accepts a +{request}+ object and responds with a +{response}+ object. [id="{upid}-{api}-request"] ==== Stop Data Frame Request -A +{request}+ object requires a non-null `id`. +A +{request}+ object requires a non-null `id`. `id` can be a comma separated list of Ids +or a single Id. Wildcards, `*` and `_all` are also accepted. + ["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- diff --git a/docs/reference/data-frames/apis/stop-transform.asciidoc b/docs/reference/data-frames/apis/stop-transform.asciidoc index 179646ddb47ba..2eb5a2d641bd4 100644 --- a/docs/reference/data-frames/apis/stop-transform.asciidoc +++ b/docs/reference/data-frames/apis/stop-transform.asciidoc @@ -10,9 +10,17 @@ Stops one or more {dataframe-transforms}. ==== Request -`POST _data_frame/transforms//_stop` +`POST _data_frame/transforms//_stop` + -//==== Description +`POST _data_frame/transforms/,/_stop` + + +`POST _data_frame/transforms/_all/_stop` + + +==== Description +You can stop multiple {dataframe-transforms} in a single API request by using a +comma-separated list of {dataframe-transforms} or a wildcard expression. +All {dataframe-transforms} can be stopped by using `_all` or `*` as the ``. ==== Path Parameters diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java index 7fa437bd15606..54153aab91ced 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java @@ -21,8 +21,11 @@ import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Objects; +import java.util.Set; import java.util.concurrent.TimeUnit; public class StopDataFrameTransformAction extends Action { @@ -45,6 +48,7 @@ public static class Request extends BaseTasksRequest { private String id; private final boolean waitForCompletion; private final boolean force; + private Set expandedIds; public Request(String id, boolean waitForCompletion, boolean force, @Nullable TimeValue timeout) { this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName()); @@ -64,6 +68,9 @@ public Request(StreamInput in) throws IOException { id = in.readString(); waitForCompletion = in.readBoolean(); force = in.readBoolean(); + if (in.readBoolean()) { + expandedIds = new HashSet<>(Arrays.asList(in.readStringArray())); + } } public String getId() { @@ -82,12 +89,25 @@ public boolean isForce() { return force; } + public Set getExpandedIds() { + return expandedIds; + } + + public void setExpandedIds(Set expandedIds ) { + this.expandedIds = expandedIds; + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(id); out.writeBoolean(waitForCompletion); out.writeBoolean(force); + boolean hasExpandedIds = expandedIds != null; + out.writeBoolean(hasExpandedIds); + if (hasExpandedIds) { + out.writeStringArray(expandedIds.toArray(new String[0])); + } } @Override @@ -98,7 +118,7 @@ public ActionRequestValidationException validate() { @Override public int hashCode() { // the base class does not implement hashCode, therefore we need to hash timeout ourselves - return Objects.hash(id, waitForCompletion, force, this.getTimeout()); + return Objects.hash(id, waitForCompletion, force, expandedIds, this.getTimeout()); } @Override @@ -118,15 +138,21 @@ public boolean equals(Object obj) { } return Objects.equals(id, other.id) && - Objects.equals(waitForCompletion, other.waitForCompletion) && - Objects.equals(force, other.force); + Objects.equals(waitForCompletion, other.waitForCompletion) && + Objects.equals(force, other.force) && + Objects.equals(expandedIds, other.expandedIds); } @Override public boolean match(Task task) { - String expectedDescription = DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + id; + if (task.getDescription().startsWith(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX)) { + String id = task.getDescription().substring(DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX.length()); + if (expandedIds != null) { + return expandedIds.contains(id); + } + } - return task.getDescription().equals(expectedDescription); + return false; } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformActionRequestTests.java index c117e249aef91..81f03902980ac 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformActionRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformActionRequestTests.java @@ -8,15 +8,27 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction.Request; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + public class StopDataFrameTransformActionRequestTests extends AbstractWireSerializingTestCase { @Override protected Request createTestInstance() { TimeValue timeout = randomBoolean() ? TimeValue.timeValueMinutes(randomIntBetween(1, 10)) : null; - return new Request(randomAlphaOfLengthBetween(1, 10), randomBoolean(), randomBoolean(), timeout); + Request request = new Request(randomAlphaOfLengthBetween(1, 10), randomBoolean(), randomBoolean(), timeout); + if (randomBoolean()) { + request.setExpandedIds(new HashSet<>(Arrays.asList(generateRandomStringArray(5, 6, false)))); + } + return request; } @Override @@ -35,4 +47,25 @@ public void testSameButDifferentTimeout() { assertNotEquals(r1,r2); assertNotEquals(r1.hashCode(),r2.hashCode()); } + + public void testMatch() { + String dataFrameId = "dataframe-id"; + + Task dataFrameTask = new Task(1L, "persistent", "action", + DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + dataFrameId, + TaskId.EMPTY_TASK_ID, Collections.emptyMap()); + + Request request = new Request("unrelated", false, false, null); + request.setExpandedIds(Set.of("foo", "bar")); + assertFalse(request.match(dataFrameTask)); + + Request matchingRequest = new Request(dataFrameId, false, false, null); + matchingRequest.setExpandedIds(Set.of(dataFrameId)); + assertTrue(matchingRequest.match(dataFrameTask)); + + Task notADataFrameTask = new Task(1L, "persistent", "action", + "some other task, say monitoring", + TaskId.EMPTY_TASK_ID, Collections.emptyMap()); + assertFalse(matchingRequest.match(notADataFrameTask)); + } } 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 2234226a50134..2092493caaf4c 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 @@ -13,20 +13,27 @@ import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.tasks.TransportTasksAction; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.action.util.PageParams; +import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask; +import java.util.Collection; +import java.util.HashSet; import java.util.List; +import java.util.Set; import static org.elasticsearch.ExceptionsHelper.convertToElastic; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; @@ -52,17 +59,28 @@ public TransportStopDataFrameTransformAction(TransportService transportService, @Override protected void doExecute(Task task, StopDataFrameTransformAction.Request request, ActionListener listener) { - // Need to verify that the config actually exists - dataFrameTransformsConfigManager.getTransformConfiguration(request.getId(), ActionListener.wrap( - config -> super.doExecute(task, request, listener), - listener::onFailure + + dataFrameTransformsConfigManager.expandTransformIds(request.getId(), new PageParams(0, 10_000), ActionListener.wrap( + expandedIds -> { + request.setExpandedIds(new HashSet<>(expandedIds)); + request.setNodes(dataframeNodes(expandedIds, clusterService.state())); + super.doExecute(task, request, listener); + }, + listener::onFailure )); } @Override protected void taskOperation(StopDataFrameTransformAction.Request request, DataFrameTransformTask transformTask, ActionListener listener) { - if (transformTask.getTransformId().equals(request.getId())) { + + Set ids = request.getExpandedIds(); + if (ids == null) { + listener.onFailure(new IllegalStateException("Request does not have expandedIds set")); + return; + } + + if (ids.contains(transformTask.getTransformId())) { if (transformTask.getState().getTaskState() == DataFrameTransformTaskState.FAILED && request.isForce() == false) { listener.onFailure( new ElasticsearchStatusException("Unable to stop data frame transform [" + request.getId() @@ -138,9 +156,28 @@ protected StopDataFrameTransformAction.Response newResponse(StopDataFrameTransfo } } - assert tasks.size() == 1; - boolean allStopped = tasks.stream().allMatch(StopDataFrameTransformAction.Response::isStopped); return new StopDataFrameTransformAction.Response(allStopped); } + + static String[] dataframeNodes(List dataFrameIds, ClusterState clusterState) { + + Set executorNodes = new HashSet<>(); + + PersistentTasksCustomMetaData tasksMetaData = + PersistentTasksCustomMetaData.getPersistentTasksCustomMetaData(clusterState); + + if (tasksMetaData != null) { + Set dataFrameIdsSet = new HashSet<>(dataFrameIds); + + Collection> tasks = + tasksMetaData.findTasks(DataFrameField.TASK_NAME, t -> dataFrameIdsSet.contains(t.getId())); + + for (PersistentTasksCustomMetaData.PersistentTask task : tasks) { + executorNodes.add(task.getExecutorNode()); + } + } + + return executorNodes.toArray(new String[0]); + } } diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java new file mode 100644 index 0000000000000..4be532de2a2be --- /dev/null +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java @@ -0,0 +1,55 @@ +/* + * 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.action; + +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.persistent.PersistentTasksCustomMetaData; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.dataframe.DataFrameField; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; +import org.elasticsearch.xpack.core.ml.MlTasks; +import org.elasticsearch.xpack.core.ml.action.OpenJobAction; + +import java.util.Arrays; +import java.util.Collections; + +import static org.hamcrest.Matchers.hasItemInArray; + +public class TransportStopDataFrameTransformActionTests extends ESTestCase { + + public void testDataframeNodes() { + String dataFrameIdFoo = "df-id-foo"; + String dataFrameIdBar = "df-id-bar"; + + PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); + tasksBuilder.addTask(dataFrameIdFoo, + DataFrameField.TASK_NAME, new DataFrameTransform(dataFrameIdFoo), + new PersistentTasksCustomMetaData.Assignment("node-1", "test assignment")); + tasksBuilder.addTask(dataFrameIdBar, + DataFrameField.TASK_NAME, new DataFrameTransform(dataFrameIdBar), + new PersistentTasksCustomMetaData.Assignment("node-2", "test assignment")); + tasksBuilder.addTask(MlTasks.jobTaskId("foo-1"), MlTasks.JOB_TASK_NAME, new OpenJobAction.JobParams("foo-1"), + new PersistentTasksCustomMetaData.Assignment("node-3", "test assignment")); + + ClusterState cs = ClusterState.builder(new ClusterName("_name")) + .metaData(MetaData.builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())) + .build(); + + String[] nodes = TransportStopDataFrameTransformAction.dataframeNodes(Arrays.asList(dataFrameIdFoo, dataFrameIdBar), cs); + assertEquals(2, nodes.length); + assertThat(nodes, hasItemInArray("node-1")); + assertThat(nodes, hasItemInArray("node-2")); + } + + public void testDataframeNodes_NoTasks() { + ClusterState emptyState = ClusterState.builder(new ClusterName("_name")).build(); + String[] nodes = TransportStopDataFrameTransformAction.dataframeNodes(Collections.singletonList("df-id"), emptyState); + assertEquals(0, nodes.length); + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml index 23a28e14a86c4..96f6b6d0a4150 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml @@ -179,3 +179,45 @@ teardown: - do: data_frame.delete_data_frame_transform: transform_id: "airline-transform-start-later" + +--- +"Test stop all": + - do: + data_frame.put_data_frame_transform: + transform_id: "airline-transform-stop-all" + body: > + { + "source": { "index": "airline-data" }, + "dest": { "index": "airline-data-start-later" }, + "pivot": { + "group_by": { "airline": {"terms": {"field": "airline"}}}, + "aggs": {"avg_response": {"avg": {"field": "responsetime"}}} + } + } + - do: + data_frame.start_data_frame_transform: + transform_id: "airline-transform-stop-all" + - match: { started: true } + + - do: + data_frame.start_data_frame_transform: + transform_id: "airline-transform-start-stop" + - match: { started: true } + + - do: + data_frame.stop_data_frame_transform: + transform_id: "_all" + - match: { stopped: true } + + - do: + data_frame.get_data_frame_transform_stats: + transform_id: "*" + - match: { count: 2 } + - match: { transforms.0.state.indexer_state: "stopped" } + - match: { transforms.0.state.task_state: "stopped" } + - match: { transforms.1.state.indexer_state: "stopped" } + - match: { transforms.1.state.task_state: "stopped" } + + - do: + data_frame.delete_data_frame_transform: + transform_id: "airline-transform-stop-all" From 2619d8f5b4e500b4ec29b33339a09d7c94cf4800 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Mon, 15 Apr 2019 10:35:20 -0400 Subject: [PATCH 036/260] Better error messages when pipelines reference incompatible aggs (#40068) Pipelines require single-valued agg or a numeric to be returned. If they don't get that, they throw an exception. Unfortunately, this exception text is very confusing to users because it usually arises from pathing "through" multiple terms aggs. The final target is a numeric, but it's the intermediary aggs that cause the problem. This commit adds the current agg name to the exception message so the user knows which "level" is the issue. --- .../test/search.aggregation/300_pipeline.yml | 100 ++++++++++ .../aggregations/pipeline/BucketHelpers.java | 30 ++- .../pipeline/BucketHelpersTests.java | 185 ++++++++++++++++++ 3 files changed, 312 insertions(+), 3 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/300_pipeline.yml create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/300_pipeline.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/300_pipeline.yml new file mode 100644 index 0000000000000..0016c9f989486 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/300_pipeline.yml @@ -0,0 +1,100 @@ +setup: + - skip: + version: " - 7.99.99" #TODO change this after backport + reason: These new error messages were added in 7.1 + + - do: + indices.create: + index: test_1 + body: + settings: + number_of_replicas: 0 + mappings: + properties: + int_field: + type : integer + + - do: + bulk: + refresh: true + body: + - index: + _index: test_1 + _id: 1 + - int_field: 1 + - index: + _index: test_1 + _id: 2 + - int_field: 2 + - index: + _index: test_1 + _id: 3 + - int_field: 3 + - index: + _index: test_1 + _id: 4 + - int_field: 4 + +--- +"Max pipeline through terms agg": + + - do: + catch: /\[Object\[\]\] at aggregation \[the_terms_2\]/ + search: + rest_total_hits_as_int: true + body: + aggs: + the_terms: + terms: + field: "int_field" + aggs: + the_terms_2: + terms: + field: "int_field" + aggs: + the_max: + max: + field: "int_field" + the_bad_max: + max_bucket: + buckets_path: "the_terms>the_terms_2>the_max" + +--- +"Max pipeline on terms agg": + + - do: + catch: /\[LongTerms\] at aggregation \[the_terms_2\]/ + search: + rest_total_hits_as_int: true + body: + aggs: + the_terms: + terms: + field: "int_field" + aggs: + the_terms_2: + terms: + field: "int_field" + the_bad_max: + max_bucket: + buckets_path: "the_terms>the_terms_2" + +--- +"Max pipeline on percentiles agg without specifying percent": + + - do: + catch: /buckets_path must reference either a number value or a single value numeric metric aggregation, but \[the_percentiles\] contains multiple values. Please specify which to use\./ + search: + rest_total_hits_as_int: true + body: + aggs: + the_terms: + terms: + field: "int_field" + aggs: + the_percentiles: + percentiles: + field: "int_field" + the_bad_max: + max_bucket: + buckets_path: "the_terms>the_percentiles" diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java index d2c973ebec26c..1a863a209828b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpers.java @@ -156,6 +156,7 @@ public static Double resolveBucketValue(MultiBucketsAggregation agg, InternalMultiBucketAggregation.InternalBucket bucket, List aggPathAsList, GapPolicy gapPolicy) { try { Object propertyValue = bucket.getProperty(agg.getName(), aggPathAsList); + if (propertyValue == null) { throw new AggregationExecutionException(AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + " must reference either a number value or a single value numeric metric aggregation"); @@ -166,9 +167,7 @@ public static Double resolveBucketValue(MultiBucketsAggregation agg, } else if (propertyValue instanceof InternalNumericMetricsAggregation.SingleValue) { value = ((InternalNumericMetricsAggregation.SingleValue) propertyValue).value(); } else { - throw new AggregationExecutionException(AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() - + " must reference either a number value or a single value numeric metric aggregation, got: " - + propertyValue.getClass().getCanonicalName()); + throw formatResolutionError(agg, aggPathAsList, propertyValue); } // doc count never has missing values so gap policy doesn't apply here boolean isDocCountProperty = aggPathAsList.size() == 1 && "_count".equals(aggPathAsList.get(0)); @@ -188,4 +187,29 @@ public static Double resolveBucketValue(MultiBucketsAggregation agg, return null; } } + + /** + * Inspects where we are in the agg tree and tries to format a helpful error + */ + private static AggregationExecutionException formatResolutionError(MultiBucketsAggregation agg, + List aggPathAsList, Object propertyValue) { + String currentAggName; + Object currentAgg; + if (aggPathAsList.isEmpty()) { + currentAggName = agg.getName(); + currentAgg = agg; + } else { + currentAggName = aggPathAsList.get(0); + currentAgg = propertyValue; + } + if (currentAgg instanceof InternalNumericMetricsAggregation.MultiValue) { + return new AggregationExecutionException(AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + + " must reference either a number value or a single value numeric metric aggregation, but [" + currentAggName + + "] contains multiple values. Please specify which to use."); + } else { + return new AggregationExecutionException(AbstractPipelineAggregationBuilder.BUCKETS_PATH_FIELD.getPreferredName() + + " must reference either a number value or a single value numeric metric aggregation, got: [" + + propertyValue.getClass().getSimpleName() + "] at aggregation [" + currentAggName + "]"); + } + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java new file mode 100644 index 0000000000000..fbf8ad9d65a21 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketHelpersTests.java @@ -0,0 +1,185 @@ +/* + * 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.pipeline; + +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; + +public class BucketHelpersTests extends ESTestCase { + + public void testReturnsObjectArray() { + + MultiBucketsAggregation agg = new MultiBucketsAggregation() { + @Override + public List getBuckets() { + return null; + } + + @Override + public String getName() { + return "foo"; + } + + @Override + public String getType() { + return null; + } + + @Override + public Map getMetaData() { + return null; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + }; + + InternalMultiBucketAggregation.InternalBucket bucket = new InternalMultiBucketAggregation.InternalBucket() { + @Override + public void writeTo(StreamOutput out) throws IOException { + + } + + @Override + public Object getKey() { + return null; + } + + @Override + public String getKeyAsString() { + return null; + } + + @Override + public long getDocCount() { + return 0; + } + + @Override + public Aggregations getAggregations() { + return null; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + + @Override + public Object getProperty(String containingAggName, List path) { + return new Object[0]; + } + }; + + AggregationExecutionException e = expectThrows(AggregationExecutionException.class, + () -> BucketHelpers.resolveBucketValue(agg, bucket, "foo>bar", BucketHelpers.GapPolicy.SKIP)); + + assertThat(e.getMessage(), equalTo("buckets_path must reference either a number value or a single value numeric " + + "metric aggregation, got: [Object[]] at aggregation [foo]")); + } + + public void testReturnMultiValueObject() { + + MultiBucketsAggregation agg = new MultiBucketsAggregation() { + @Override + public List getBuckets() { + return null; + } + + @Override + public String getName() { + return "foo"; + } + + @Override + public String getType() { + return null; + } + + @Override + public Map getMetaData() { + return null; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + }; + + InternalMultiBucketAggregation.InternalBucket bucket = new InternalMultiBucketAggregation.InternalBucket() { + @Override + public void writeTo(StreamOutput out) throws IOException { + + } + + @Override + public Object getKey() { + return null; + } + + @Override + public String getKeyAsString() { + return null; + } + + @Override + public long getDocCount() { + return 0; + } + + @Override + public Aggregations getAggregations() { + return null; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + + @Override + public Object getProperty(String containingAggName, List path) { + return mock(InternalTDigestPercentiles.class); + } + }; + + AggregationExecutionException e = expectThrows(AggregationExecutionException.class, + () -> BucketHelpers.resolveBucketValue(agg, bucket, "foo>bar", BucketHelpers.GapPolicy.SKIP)); + + assertThat(e.getMessage(), equalTo("buckets_path must reference either a number value or a single value numeric " + + "metric aggregation, but [foo] contains multiple values. Please specify which to use.")); + } +} From 656cc709b2e51fd8a0bddf1591becf60cc388f39 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 15 Apr 2019 11:27:38 -0400 Subject: [PATCH 037/260] Fix intervals section of auto date-histogram docs (#41203) This section should be at the same sub-level as other sections in the auto date-histogram docs, otherwise it is rendered on to another page and is confusing for users to understand what it's in reference to. --- .../aggregations/bucket/autodatehistogram-aggregation.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc index e371674228bb4..dfc4f62a91b09 100644 --- a/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/autodatehistogram-aggregation.asciidoc @@ -89,7 +89,7 @@ Response: -------------------------------------------------- // TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/] -=== Intervals +==== Intervals The interval of the returned buckets is selected based on the data collected by the aggregation so that the number of buckets returned is less than or equal to the number From 6aa52b1ac5f35c23fb4c39f4f7020966dcbf7c9a Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Mon, 15 Apr 2019 18:48:05 +0300 Subject: [PATCH 038/260] Fix the rpm and deb names for version starting with 7.0.0 (#41198) With the 7.0.0 release, we switched to download the packages instead of using locally built ones. This PR fixes the artifact names to include the architecture as introduced in the 7.0.0 release. --- .../gradle/vagrant/VagrantTestPlugin.groovy | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy index 9e6f7ef250719..121b40821bbc3 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy @@ -214,9 +214,15 @@ class VagrantTestPlugin implements Plugin { } else { UPGRADE_FROM_ARCHIVES.each { // The version of elasticsearch that we upgrade *from* - dependencies.add("downloads.${it}:elasticsearch:${upgradeFromVersion}@${it}") - if (upgradeFromVersion.onOrAfter('6.3.0')) { - dependencies.add("downloads.${it}:elasticsearch-oss:${upgradeFromVersion}@${it}") + if (upgradeFromVersion.onOrAfter('7.0.0')) { + String arch = it == "rpm" ? "x86_64" : "amd64" + dependencies.add("downloads.${it}:elasticsearch:${upgradeFromVersion}-${arch}@${it}") + dependencies.add("downloads.${it}:elasticsearch-oss:${upgradeFromVersion}-${arch}@${it}") + } else { + dependencies.add("downloads.${it}:elasticsearch:${upgradeFromVersion}@${it}") + if (upgradeFromVersion.onOrAfter('6.3.0')) { + dependencies.add("downloads.${it}:elasticsearch-oss:${upgradeFromVersion}@${it}") + } } } } From a4dff365fa2eaef84ad02e160c3a3ce7e6baf210 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 15 Apr 2019 19:22:10 +0100 Subject: [PATCH 039/260] Add 'DO NOT TOUCH' warnings to disco settings docs (#41211) --- .../discovery/discovery-settings.asciidoc | 68 ++++++++++++------- 1 file changed, 45 insertions(+), 23 deletions(-) diff --git a/docs/reference/modules/discovery/discovery-settings.asciidoc b/docs/reference/modules/discovery/discovery-settings.asciidoc index b5b1b4799e87b..541cb15bf1108 100644 --- a/docs/reference/modules/discovery/discovery-settings.asciidoc +++ b/docs/reference/modules/discovery/discovery-settings.asciidoc @@ -17,6 +17,23 @@ Discovery and cluster formation are affected by the following settings: discovery process. By default, it is the <>. +`cluster.initial_master_nodes`:: + + Sets a list of the <> or transport addresses of the + initial set of master-eligible nodes in a brand-new cluster. By default + this list is empty, meaning that this node expects to join a cluster that + has already been bootstrapped. See <>. + +[float] +==== Expert settings + +Discovery and cluster formation are also affected by the following +_expert-level_ settings, although it is not recommended to change any of these +from their default values. + +[WARNING] If you adjust these settings then your cluster may not form correctly +or may become unstable or intolerant of certain failures. + `discovery.cluster_formation_warning_timeout`:: Sets how long a node will try to form a cluster before logging a warning @@ -41,6 +58,7 @@ Discovery and cluster formation are affected by the following settings: handshake. Defaults to `1s`. `discovery.request_peers_timeout`:: + Sets how long a node will wait after asking its peers again before considering the request to have failed. Defaults to `3s`. @@ -67,73 +85,78 @@ Discovery and cluster formation are affected by the following settings: Sets the amount to increase the upper bound on the wait before an election on each election failure. Note that this is _linear_ backoff. This defaults - to `100ms` + to `100ms`. Changing this setting from the default may cause your cluster + to fail to elect a master node. `cluster.election.duration`:: - Sets how long each election is allowed to take before a node considers it to - have failed and schedules a retry. This defaults to `500ms`. + Sets how long each election is allowed to take before a node considers it + to have failed and schedules a retry. This defaults to `500ms`. Changing + this setting from the default may cause your cluster to fail to elect a + master node. `cluster.election.initial_timeout`:: Sets the upper bound on how long a node will wait initially, or after the elected master fails, before attempting its first election. This defaults - to `100ms`. - + to `100ms`. Changing this setting from the default may cause your cluster + to fail to elect a master node. `cluster.election.max_timeout`:: Sets the maximum upper bound on how long a node will wait before attempting an first election, so that an network partition that lasts for a long time - does not result in excessively sparse elections. This defaults to `10s` + does not result in excessively sparse elections. This defaults to `10s`. + Changing this setting from the default may cause your cluster to fail to + elect a master node. [[fault-detection-settings]]`cluster.fault_detection.follower_check.interval`:: Sets how long the elected master waits between follower checks to each - other node in the cluster. Defaults to `1s`. + other node in the cluster. Defaults to `1s`. Changing this setting from the + default may cause your cluster to become unstable. `cluster.fault_detection.follower_check.timeout`:: Sets how long the elected master waits for a response to a follower check - before considering it to have failed. Defaults to `10s`. + before considering it to have failed. Defaults to `10s`. Changing this + setting from the default may cause your cluster to become unstable. `cluster.fault_detection.follower_check.retry_count`:: Sets how many consecutive follower check failures must occur to each node before the elected master considers that node to be faulty and removes it - from the cluster. Defaults to `3`. + from the cluster. Defaults to `3`. Changing this setting from the default + may cause your cluster to become unstable. `cluster.fault_detection.leader_check.interval`:: Sets how long each node waits between checks of the elected master. - Defaults to `1s`. + Defaults to `1s`. Changing this setting from the default may cause your + cluster to become unstable. `cluster.fault_detection.leader_check.timeout`:: Sets how long each node waits for a response to a leader check from the elected master before considering it to have failed. Defaults to `10s`. + Changing this setting from the default may cause your cluster to become + unstable. `cluster.fault_detection.leader_check.retry_count`:: Sets how many consecutive leader check failures must occur before a node considers the elected master to be faulty and attempts to find or elect a - new master. Defaults to `3`. + new master. Defaults to `3`. Changing this setting from the default may + cause your cluster to become unstable. `cluster.follower_lag.timeout`:: Sets how long the master node waits to receive acknowledgements for cluster - state updates from lagging nodes. The default value is `90s`. If a node does - not successfully apply the cluster state update within this period of time, - it is considered to have failed and is removed from the cluster. See + state updates from lagging nodes. The default value is `90s`. If a node + does not successfully apply the cluster state update within this period of + time, it is considered to have failed and is removed from the cluster. See <>. -`cluster.initial_master_nodes`:: - - Sets a list of the <> or transport addresses of the - initial set of master-eligible nodes in a brand-new cluster. By default - this list is empty, meaning that this node expects to join a cluster that - has already been bootstrapped. See <>. - `cluster.join.timeout`:: Sets how long a node will wait after sending a request to join a cluster @@ -149,8 +172,7 @@ Discovery and cluster formation are affected by the following settings: `cluster.publish.timeout`:: Sets how long the master node waits for each cluster state update to be - completely published to all nodes. The default value is `30s`. If this - period of time elapses, the cluster state change is rejected. See + completely published to all nodes. The default value is `30s`. See <>. [[no-master-block]]`cluster.no_master_block`:: From de825d2ecfe4a0cd7ca7e0bb72e5f9f7debf4556 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 15 Apr 2019 14:23:44 -0600 Subject: [PATCH 040/260] Log every use of ILM Move to Step API (#41171) Usage of the ILM Move to Step API can result in some very odd situations, and for diagnosing problems arising from these situations it would be nice to have a record of when this API was called with what parameters. Also, adds a dedicated logger for TransportMoveToStepAction, rather than using the (deprecated) inherited one. --- .../xpack/indexlifecycle/IndexLifecycleRunner.java | 3 +++ .../indexlifecycle/action/TransportMoveToStepAction.java | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java index f6c068d945d79..05ad342f3e779 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunner.java @@ -307,6 +307,9 @@ static ClusterState moveClusterStateToStep(String indexName, ClusterState curren "] with policy [" + indexPolicySetting + "] does not exist"); } + logger.info("moving index [{}] from [{}] to [{}] in policy [{}]", + indexName, currentStepKey, nextStepKey, indexPolicySetting); + return IndexLifecycleRunner.moveClusterStateToNextStep(idxMeta.getIndex(), currentState, currentStepKey, nextStepKey, nowSupplier, forcePhaseDefinitionRefresh); } diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportMoveToStepAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportMoveToStepAction.java index 57f08eba76490..e5cb15f6c3b24 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportMoveToStepAction.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/action/TransportMoveToStepAction.java @@ -6,6 +6,8 @@ package org.elasticsearch.xpack.indexlifecycle.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; @@ -25,6 +27,8 @@ import org.elasticsearch.xpack.indexlifecycle.IndexLifecycleService; public class TransportMoveToStepAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportMoveToStepAction.class); + IndexLifecycleService indexLifecycleService; @Inject public TransportMoveToStepAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, From 540897726386cdc32f1a57c92ace30e2d71a27df Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 15 Apr 2019 14:28:19 -0600 Subject: [PATCH 041/260] Check allocation rules are cleared after ILM Shrink (#41170) Adds some checks to make sure that the allocation rules that ILM adds before a shrink are cleared after the shrink is complete --- .../xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java index 69aca756554ac..b6b317e0c67ef 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java @@ -59,6 +59,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; public class TimeSeriesLifecycleActionsIT extends ESRestTestCase { private String index; @@ -218,6 +219,7 @@ public void testRetryFailedShrinkAction() throws Exception { assertThat(getStepKeyForIndex(shrunkenIndex), equalTo(TerminalPolicyStep.KEY)); assertThat(settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS), equalTo(String.valueOf(expectedFinalShards))); assertThat(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey()), equalTo("true")); + assertThat(settings.get(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_id"), nullValue()); }); expectThrows(ResponseException.class, this::indexDocument); } @@ -461,6 +463,7 @@ public void testShrinkAction() throws Exception { assertThat(getStepKeyForIndex(shrunkenIndex), equalTo(TerminalPolicyStep.KEY)); assertThat(settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS), equalTo(String.valueOf(expectedFinalShards))); assertThat(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey()), equalTo("true")); + assertThat(settings.get(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_id"), nullValue()); }); expectThrows(ResponseException.class, this::indexDocument); } @@ -480,6 +483,7 @@ public void testShrinkSameShards() throws Exception { assertThat(getStepKeyForIndex(index), equalTo(TerminalPolicyStep.KEY)); assertThat(settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS), equalTo(String.valueOf(numberOfShards))); assertNull(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey())); + assertThat(settings.get(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_id"), nullValue()); }); } @@ -523,6 +527,7 @@ public void testShrinkDuringSnapshot() throws Exception { assertThat(getStepKeyForIndex(shrunkenIndex), equalTo(TerminalPolicyStep.KEY)); assertThat(settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS), equalTo(String.valueOf(1))); assertThat(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey()), equalTo("true")); + assertThat(settings.get(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_id"), nullValue()); }, 2, TimeUnit.MINUTES); expectThrows(ResponseException.class, this::indexDocument); // assert that snapshot succeeded From 64768bfb9e18464f8698e6ccfaaaba2050177545 Mon Sep 17 00:00:00 2001 From: Shaunak Kashyap Date: Mon, 15 Apr 2019 18:24:20 -0700 Subject: [PATCH 042/260] Expand beats_system role privileges (#40876) Traditionally we have [recommended](https://www.elastic.co/guide/en/beats/filebeat/current/monitoring.html) that Beats send their monitoring data to the **production** Elasticsearch cluster. Beats do this by calling the `POST _monitoring/bulk` API. When Security is enabled this API call requires the `cluster:admin/xpack/monitoring/bulk` privilege. The built-in `beats_system` role has this privilege. [Going forward](https://github.com/elastic/beats/pull/9260), Beats will be able to send their monitoring data directly to the **monitoring** Elasticsearch cluster. Beats will do this by calling the regular `POST _bulk` API. When Security is enabled this API call requires the `indices:data/write/bulk` privilege. Further, the call has to be able to create any indices that don't exist. This PR expands the built-in `beats_system` role's privileges. Specifically, it adds index-level `write` and `create_index` privileges for `.monitoring-beats-*` indices. This will allow Beats users to continue using the `beats_system` role for the new direct monitoring route when Security is enabled. --- .../authz/store/ReservedRolesStore.java | 7 +++- .../authz/store/ReservedRolesStoreTests.java | 33 +++++++++++-------- 2 files changed, 26 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStore.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStore.java index d9fded1fb2bdd..b767b56086159 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStore.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStore.java @@ -133,7 +133,12 @@ private static Map initializeReservedRoles() { }, null, MetadataUtils.DEFAULT_RESERVED_METADATA)) .put(UsernamesField.BEATS_ROLE, new RoleDescriptor(UsernamesField.BEATS_ROLE, - new String[] { "monitor", MonitoringBulkAction.NAME}, null, null, MetadataUtils.DEFAULT_RESERVED_METADATA)) + new String[] { "monitor", MonitoringBulkAction.NAME}, + new RoleDescriptor.IndicesPrivileges[]{ + RoleDescriptor.IndicesPrivileges.builder() + .indices(".monitoring-beats-*").privileges("create_index", "create").build() + }, + null, MetadataUtils.DEFAULT_RESERVED_METADATA)) .put(UsernamesField.APM_ROLE, new RoleDescriptor(UsernamesField.APM_ROLE, new String[] { "monitor", MonitoringBulkAction.NAME}, null, null, MetadataUtils.DEFAULT_RESERVED_METADATA)) .put("apm_user", new RoleDescriptor("apm_user", diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java index c4c2ec871a548..625e5ddf47c34 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java @@ -838,23 +838,30 @@ public void testBeatsSystemRole() { assertNotNull(roleDescriptor); assertThat(roleDescriptor.getMetadata(), hasEntry("_reserved", true)); - Role logstashSystemRole = Role.builder(roleDescriptor, null).build(); - assertThat(logstashSystemRole.cluster().check(ClusterHealthAction.NAME, request), is(true)); - assertThat(logstashSystemRole.cluster().check(ClusterStateAction.NAME, request), is(true)); - assertThat(logstashSystemRole.cluster().check(ClusterStatsAction.NAME, request), is(true)); - assertThat(logstashSystemRole.cluster().check(PutIndexTemplateAction.NAME, request), is(false)); - assertThat(logstashSystemRole.cluster().check(ClusterRerouteAction.NAME, request), is(false)); - assertThat(logstashSystemRole.cluster().check(ClusterUpdateSettingsAction.NAME, request), is(false)); - assertThat(logstashSystemRole.cluster().check(MonitoringBulkAction.NAME, request), is(true)); + Role beatsSystemRole = Role.builder(roleDescriptor, null).build(); + assertThat(beatsSystemRole.cluster().check(ClusterHealthAction.NAME, request), is(true)); + assertThat(beatsSystemRole.cluster().check(ClusterStateAction.NAME, request), is(true)); + assertThat(beatsSystemRole.cluster().check(ClusterStatsAction.NAME, request), is(true)); + assertThat(beatsSystemRole.cluster().check(PutIndexTemplateAction.NAME, request), is(false)); + assertThat(beatsSystemRole.cluster().check(ClusterRerouteAction.NAME, request), is(false)); + assertThat(beatsSystemRole.cluster().check(ClusterUpdateSettingsAction.NAME, request), is(false)); + assertThat(beatsSystemRole.cluster().check(MonitoringBulkAction.NAME, request), is(true)); - assertThat(logstashSystemRole.runAs().check(randomAlphaOfLengthBetween(1, 30)), is(false)); + assertThat(beatsSystemRole.runAs().check(randomAlphaOfLengthBetween(1, 30)), is(false)); - assertThat(logstashSystemRole.indices().allowedIndicesMatcher(IndexAction.NAME).test("foo"), is(false)); - assertThat(logstashSystemRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(".reporting"), is(false)); - assertThat(logstashSystemRole.indices().allowedIndicesMatcher("indices:foo").test(randomAlphaOfLengthBetween(8, 24)), + + final String index = ".monitoring-beats-" + randomIntBetween(0, 5);; + logger.info("index name [{}]", index); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher(IndexAction.NAME).test("foo"), is(false)); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(".reporting"), is(false)); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher("indices:foo").test(randomAlphaOfLengthBetween(8, 24)), is(false)); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(index), is(true)); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(index), is(true)); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(index), is(false)); + assertThat(beatsSystemRole.indices().allowedIndicesMatcher(BulkAction.NAME).test(index), is(true)); - assertNoAccessAllowed(logstashSystemRole, RestrictedIndicesNames.RESTRICTED_NAMES); + assertNoAccessAllowed(beatsSystemRole, RestrictedIndicesNames.RESTRICTED_NAMES); } public void testAPMSystemRole() { From 01ab82f866d311f3128e8089bc2497d71fdfa6a9 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Tue, 16 Apr 2019 09:29:22 +0200 Subject: [PATCH 043/260] Upgrade to lucene 8.1.0-snapshot-e460356abe (#40952) --- buildSrc/version.properties | 2 +- docs/Versions.asciidoc | 4 +- ...DelimiterGraphTokenFilterFactoryTests.java | 6 +- .../lucene-expressions-8.0.0.jar.sha1 | 1 - ...essions-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-icu-8.0.0.jar.sha1 | 1 - ...ers-icu-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-kuromoji-8.0.0.jar.sha1 | 1 - ...uromoji-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-nori-8.0.0.jar.sha1 | 1 - ...rs-nori-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-phonetic-8.0.0.jar.sha1 | 1 - ...honetic-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-smartcn-8.0.0.jar.sha1 | 1 - ...smartcn-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-stempel-8.0.0.jar.sha1 | 1 - ...stempel-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + ...lucene-analyzers-morfologik-8.0.0.jar.sha1 | 1 - ...fologik-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-analyzers-common-8.0.0.jar.sha1 | 1 - ...-common-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-backward-codecs-8.0.0.jar.sha1 | 1 - ...-codecs-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-core-8.0.0.jar.sha1 | 1 - ...ne-core-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../licenses/lucene-grouping-8.0.0.jar.sha1 | 1 - ...rouping-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-highlighter-8.0.0.jar.sha1 | 1 - ...lighter-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-join-8.0.0.jar.sha1 | 1 - ...ne-join-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-memory-8.0.0.jar.sha1 | 1 - ...-memory-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-misc-8.0.0.jar.sha1 | 1 - ...ne-misc-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-queries-8.0.0.jar.sha1 | 1 - ...queries-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-queryparser-8.0.0.jar.sha1 | 1 - ...yparser-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-sandbox-8.0.0.jar.sha1 | 1 - ...sandbox-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-spatial-8.0.0.jar.sha1 | 1 - ...spatial-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../lucene-spatial-extras-8.0.0.jar.sha1 | 1 - ...-extras-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../licenses/lucene-spatial3d-8.0.0.jar.sha1 | 1 - ...atial3d-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + server/licenses/lucene-suggest-8.0.0.jar.sha1 | 1 - ...suggest-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + .../uhighlight/CustomUnifiedHighlighter.java | 23 +- .../main/java/org/elasticsearch/Version.java | 2 +- .../search/function/FunctionScoreQuery.java | 8 + .../index/codec/CodecService.java | 3 +- .../PerFieldMappingPostingFormatCodec.java | 3 +- .../index/query/IntervalBuilder.java | 12 +- .../search/ESToParentBlockJoinQuery.java | 8 + .../search/profile/query/ProfileScorer.java | 30 --- .../common/lucene/uid/VersionsTests.java | 2 +- .../index/engine/InternalEngineTests.java | 9 +- .../profile/query/QueryProfilerTests.java | 204 ------------------ .../search/query/QueryPhaseTests.java | 1 - .../licenses/lucene-core-8.0.0.jar.sha1 | 1 - ...ne-core-8.1.0-snapshot-e460356abe.jar.sha1 | 1 + 63 files changed, 69 insertions(+), 296 deletions(-) delete mode 100644 modules/lang-expression/licenses/lucene-expressions-8.0.0.jar.sha1 create mode 100644 modules/lang-expression/licenses/lucene-expressions-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-8.0.0.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analyzers-icu-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.0.0.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-nori/licenses/lucene-analyzers-nori-8.0.0.jar.sha1 create mode 100644 plugins/analysis-nori/licenses/lucene-analyzers-nori-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.0.0.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.0.0.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.0.0.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.0.0.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-analyzers-common-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-analyzers-common-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-backward-codecs-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-backward-codecs-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-core-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-grouping-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-grouping-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-highlighter-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-highlighter-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-join-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-join-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-memory-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-memory-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-misc-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-misc-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-queries-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-queries-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-queryparser-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-queryparser-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-sandbox-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-sandbox-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-spatial-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-extras-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-spatial-extras-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-spatial3d-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-spatial3d-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 server/licenses/lucene-suggest-8.0.0.jar.sha1 create mode 100644 server/licenses/lucene-suggest-8.1.0-snapshot-e460356abe.jar.sha1 delete mode 100644 x-pack/plugin/sql/sql-action/licenses/lucene-core-8.0.0.jar.sha1 create mode 100644 x-pack/plugin/sql/sql-action/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index d4ad364f2030b..c723cb1c9e514 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ elasticsearch = 8.0.0 -lucene = 8.0.0 +lucene = 8.1.0-snapshot-e460356abe bundled_jdk = 12+33 diff --git a/docs/Versions.asciidoc b/docs/Versions.asciidoc index 9362ae0aea796..07d735b66547a 100644 --- a/docs/Versions.asciidoc +++ b/docs/Versions.asciidoc @@ -5,8 +5,8 @@ bare_version never includes -alpha or -beta :bare_version: 8.0.0 :major-version: 8.x :prev-major-version: 7.x -:lucene_version: 8.0.0 -:lucene_version_path: 8_0_0 +:lucene_version: 8.1.0 +:lucene_version_path: 8_1_0 :branch: master :jdk: 11.0.2 :jdk_major: 11 diff --git a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java index 0a77c0c42d0a0..d799674f231a1 100644 --- a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java +++ b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/WordDelimiterGraphTokenFilterFactoryTests.java @@ -47,9 +47,9 @@ public void testMultiTerms() throws IOException { TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_word_delimiter"); String source = "PowerShot 500-42 wi-fi wi-fi-4000 j2se O'Neil's"; - String[] expected = new String[] { "PowerShot", "PowerShot", "Power", "Shot", "50042", - "500-42", "500", "42", "wifi", "wi-fi", "wi", "fi", "wifi4000", "wi-fi-4000", "wi", - "fi", "4000", "j2se", "j2se", "j", "2", "se", "ONeil", "O'Neil's", "O", "Neil" }; + String[] expected = new String[] { "PowerShot", "PowerShot", "Power", "Shot", "500-42", + "50042", "500", "42", "wi-fi", "wifi", "wi", "fi", "wi-fi-4000", "wifi4000", "wi", + "fi", "4000", "j2se", "j2se", "j", "2", "se", "O'Neil's", "ONeil", "O", "Neil" }; Tokenizer tokenizer = new WhitespaceTokenizer(); tokenizer.setReader(new StringReader(source)); int[] expectedIncr = new int[] { 1, 0, 0, 1, 1, 0, 0, 1, 1, 0, 0, 1, 1, 0, 0, 1, 1, 1, 0, 0, diff --git a/modules/lang-expression/licenses/lucene-expressions-8.0.0.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-8.0.0.jar.sha1 deleted file mode 100644 index 6174d2bce6456..0000000000000 --- a/modules/lang-expression/licenses/lucene-expressions-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -106b35cf2739f7d2350f3ef5c107d9d066d17cd6 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-8.1.0-snapshot-e460356abe.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..48446e877e309 --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +0a1addebde14147501b7d24a581a7a7288bc585d \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-8.0.0.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-8.0.0.jar.sha1 deleted file mode 100644 index 51bd478ce4cf1..0000000000000 --- a/plugins/analysis-icu/licenses/lucene-analyzers-icu-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1bc195d1b036b59cdf7704f758df8f43c7e229c4 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analyzers-icu-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analyzers-icu-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..c03380c6cf36c --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analyzers-icu-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +b8b7d744e6294706b379ec7fdd2d6f1b6badc95b \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.0.0.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.0.0.jar.sha1 deleted file mode 100644 index 6c60a7f069a22..0000000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -760d9451396c45fdb271750f4e52acc2ff5e7bb2 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..e3195509e493f --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analyzers-kuromoji-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +c9dcc5568ccd4589f4a6871d2019661546106c83 \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analyzers-nori-8.0.0.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analyzers-nori-8.0.0.jar.sha1 deleted file mode 100644 index e52dfa93565a1..0000000000000 --- a/plugins/analysis-nori/licenses/lucene-analyzers-nori-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5bb5a241d95d9753a5e9fb341476306cb96a34a9 \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analyzers-nori-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analyzers-nori-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..77cd0b32ed9ea --- /dev/null +++ b/plugins/analysis-nori/licenses/lucene-analyzers-nori-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +bef6d901a9c8b4c6139de7419b3024e0c9fd4ad3 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.0.0.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.0.0.jar.sha1 deleted file mode 100644 index f802e93d17027..0000000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e3384e1b5c1e5f39026d3d6e48e00df84f614911 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..1f090e9ca523f --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analyzers-phonetic-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +074c06d6f2254edae04acdd53bf6989a4343acc8 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.0.0.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.0.0.jar.sha1 deleted file mode 100644 index 0c427cb38be75..0000000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3e413379d7e61eb43dee64ec5e756cbeb3478a05 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..42a1e22cdfbc0 --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analyzers-smartcn-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +5cd2a341ab4524ec7ff40ba29faa4ead5e805413 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.0.0.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.0.0.jar.sha1 deleted file mode 100644 index ae5d443c7d5ad..0000000000000 --- a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -50d81559e2604da31ca7961581fda41257ab0600 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..c2468bbdd7cac --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analyzers-stempel-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +ba55aba7d278f6201b4ebd6dafbc7edb6fe94f8c \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.0.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.0.0.jar.sha1 deleted file mode 100644 index af556321b7cf2..0000000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a0b165cb156178a0a91baa4b8d2f4c37278d92e0 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.1.0-snapshot-e460356abe.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..176e9533edde9 --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analyzers-morfologik-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +543d99fd2ba4302f3555792236350b201514d821 \ No newline at end of file diff --git a/server/licenses/lucene-analyzers-common-8.0.0.jar.sha1 b/server/licenses/lucene-analyzers-common-8.0.0.jar.sha1 deleted file mode 100644 index 3d51b62b1854d..0000000000000 --- a/server/licenses/lucene-analyzers-common-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -26fdada04adbb02164ef2d0f9abfa3b46ec30a0b \ No newline at end of file diff --git a/server/licenses/lucene-analyzers-common-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-analyzers-common-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..08507536ac134 --- /dev/null +++ b/server/licenses/lucene-analyzers-common-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +c20a8ae0c3bd769aa6c415ebea94ba466d9a631d \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-8.0.0.jar.sha1 b/server/licenses/lucene-backward-codecs-8.0.0.jar.sha1 deleted file mode 100644 index e8ac2167ec45c..0000000000000 --- a/server/licenses/lucene-backward-codecs-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -90bda2357016dc0f4582938b01f5ae1142089d5f \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-backward-codecs-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..6b0a3854c6f38 --- /dev/null +++ b/server/licenses/lucene-backward-codecs-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +6e8921ab37facdcc5c4b71f2612d72300d6de217 \ No newline at end of file diff --git a/server/licenses/lucene-core-8.0.0.jar.sha1 b/server/licenses/lucene-core-8.0.0.jar.sha1 deleted file mode 100644 index 2619abce7979a..0000000000000 --- a/server/licenses/lucene-core-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -407c555efb2d3253f51a676cc2089a5d29a3b7b7 \ No newline at end of file diff --git a/server/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..fea3658cf61bd --- /dev/null +++ b/server/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +3e85f77d8f8ed1db53dba387fbdec55a9f912639 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-8.0.0.jar.sha1 b/server/licenses/lucene-grouping-8.0.0.jar.sha1 deleted file mode 100644 index 86294d2987fd7..0000000000000 --- a/server/licenses/lucene-grouping-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a9004071d79e9f1eb5f2fe81c4b2b736d9d838bf \ No newline at end of file diff --git a/server/licenses/lucene-grouping-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-grouping-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..0bcc1ebab16de --- /dev/null +++ b/server/licenses/lucene-grouping-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +426a1822d888a6341f6bafccaad19e4a2ad88e25 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-8.0.0.jar.sha1 b/server/licenses/lucene-highlighter-8.0.0.jar.sha1 deleted file mode 100644 index ac8a49302cabd..0000000000000 --- a/server/licenses/lucene-highlighter-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -95c55c400dcfd5e08da1bab4f33eb3b6a65b1d16 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-highlighter-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..b2478a52c7a85 --- /dev/null +++ b/server/licenses/lucene-highlighter-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +f83fa4b264198dfb12436a803309a60f5588481d \ No newline at end of file diff --git a/server/licenses/lucene-join-8.0.0.jar.sha1 b/server/licenses/lucene-join-8.0.0.jar.sha1 deleted file mode 100644 index 88f0cdb3238c4..0000000000000 --- a/server/licenses/lucene-join-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9364f8fd4fff476e619e84cb22f4cb0108039eda \ No newline at end of file diff --git a/server/licenses/lucene-join-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-join-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..ea3f6353ce09e --- /dev/null +++ b/server/licenses/lucene-join-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +f381131abef51f77d26bccbb213d1c8563c19ec4 \ No newline at end of file diff --git a/server/licenses/lucene-memory-8.0.0.jar.sha1 b/server/licenses/lucene-memory-8.0.0.jar.sha1 deleted file mode 100644 index 7dc4ebb2f5806..0000000000000 --- a/server/licenses/lucene-memory-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9cce58e14792735cb7dc85fc84239193521d45eb \ No newline at end of file diff --git a/server/licenses/lucene-memory-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-memory-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..0bc96c932c18b --- /dev/null +++ b/server/licenses/lucene-memory-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +8d8733551b9eb71e1f59688b8e78e0b481974d7a \ No newline at end of file diff --git a/server/licenses/lucene-misc-8.0.0.jar.sha1 b/server/licenses/lucene-misc-8.0.0.jar.sha1 deleted file mode 100644 index 98c3fd2026362..0000000000000 --- a/server/licenses/lucene-misc-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c444c2c41ab46744e2bbc61df5bdd2ac62ffe6a5 \ No newline at end of file diff --git a/server/licenses/lucene-misc-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-misc-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..fdde3da39a264 --- /dev/null +++ b/server/licenses/lucene-misc-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +13da0b22f01dff4a01c9907425464a440695104b \ No newline at end of file diff --git a/server/licenses/lucene-queries-8.0.0.jar.sha1 b/server/licenses/lucene-queries-8.0.0.jar.sha1 deleted file mode 100644 index 457b5fc9cba15..0000000000000 --- a/server/licenses/lucene-queries-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -197e4cf95fcbc787f128a33e4675528cfee65065 \ No newline at end of file diff --git a/server/licenses/lucene-queries-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-queries-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..c50232482b5c1 --- /dev/null +++ b/server/licenses/lucene-queries-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +6c3de4dbb98b5cc00875d76e817929374bb9e710 \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-8.0.0.jar.sha1 b/server/licenses/lucene-queryparser-8.0.0.jar.sha1 deleted file mode 100644 index b1764ad6bc05f..0000000000000 --- a/server/licenses/lucene-queryparser-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -aa7d2e07736356405b4bece971d0a9ff1036dac3 \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-queryparser-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..4a6c53845fc24 --- /dev/null +++ b/server/licenses/lucene-queryparser-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +539ef199c74ae6891ac93f55632fe140b9d4c291 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-8.0.0.jar.sha1 b/server/licenses/lucene-sandbox-8.0.0.jar.sha1 deleted file mode 100644 index 0cdbafa3e1ff3..0000000000000 --- a/server/licenses/lucene-sandbox-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -12f32d95596ff55c43c4c2378bf26e9fe3ea7dd9 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-sandbox-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..198b93230fb7c --- /dev/null +++ b/server/licenses/lucene-sandbox-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +0371141f658e2157babd490f0a8ddbcd5114b371 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-8.0.0.jar.sha1 b/server/licenses/lucene-spatial-8.0.0.jar.sha1 deleted file mode 100644 index efa94df2e7319..0000000000000 --- a/server/licenses/lucene-spatial-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5cd61c5c166a69571f39178b50d304d6e3914050 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-spatial-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..ad6558f167d1c --- /dev/null +++ b/server/licenses/lucene-spatial-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +1bae56fbce29d6c597c00889dab1909f51f4aaac \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-8.0.0.jar.sha1 b/server/licenses/lucene-spatial-extras-8.0.0.jar.sha1 deleted file mode 100644 index c3e6b2ac92cee..0000000000000 --- a/server/licenses/lucene-spatial-extras-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -72474064a247566c4c759eda1dfaac4d48778cd1 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-spatial-extras-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..66d5cc808a1ac --- /dev/null +++ b/server/licenses/lucene-spatial-extras-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +6eaed1dea9a18502ab9dffe55f081da6060373f7 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-8.0.0.jar.sha1 b/server/licenses/lucene-spatial3d-8.0.0.jar.sha1 deleted file mode 100644 index f83e1d443a2a0..0000000000000 --- a/server/licenses/lucene-spatial3d-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d40eb969881f58b47bace23865a1d5a9dd4ebf0a \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-spatial3d-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..0e1c69171e07e --- /dev/null +++ b/server/licenses/lucene-spatial3d-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +e54c6be78275637544a3080874dd04b0d92755e5 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-8.0.0.jar.sha1 b/server/licenses/lucene-suggest-8.0.0.jar.sha1 deleted file mode 100644 index 6b60f4196d5f0..0000000000000 --- a/server/licenses/lucene-suggest-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -57ebd0c31e90f5f73aad7dbf7448cd59d8418f03 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-8.1.0-snapshot-e460356abe.jar.sha1 b/server/licenses/lucene-suggest-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..2d1491c40dd0d --- /dev/null +++ b/server/licenses/lucene-suggest-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +e4c95d0bb740f18af520faebcebb968da3e0a687 \ No newline at end of file diff --git a/server/src/main/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighter.java b/server/src/main/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighter.java index d9bf9613cba07..4dd36896bf346 100644 --- a/server/src/main/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighter.java +++ b/server/src/main/java/org/apache/lucene/search/uhighlight/CustomUnifiedHighlighter.java @@ -21,12 +21,10 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.Term; -import org.apache.lucene.queries.CommonTermsQuery; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper; import org.apache.lucene.search.spans.SpanNearQuery; import org.apache.lucene.search.spans.SpanOrQuery; @@ -36,12 +34,9 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lucene.search.MultiPhrasePrefixQuery; -import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; -import org.elasticsearch.index.search.ESToParentBlockJoinQuery; import java.io.IOException; import java.text.BreakIterator; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -151,11 +146,6 @@ protected FieldHighlighter getFieldHighlighter(String field, Query query, Set 0 ? 1 : 0), getFormatter(field), noMatchSize, fieldValue); } - @Override - protected Collection preMultiTermQueryRewrite(Query query) { - return rewriteCustomQuery(query); - } - @Override protected Collection preSpanQueryRewrite(Query query) { return rewriteCustomQuery(query); @@ -175,7 +165,7 @@ private Collection rewriteCustomQuery(Query query) { SpanQuery[] innerQueries = new SpanQuery[terms[i].length]; for (int j = 0; j < terms[i].length; j++) { if (i == sizeMinus1) { - innerQueries[j] = new SpanMultiTermQueryWrapper(new PrefixQuery(terms[i][j])); + innerQueries[j] = new SpanMultiTermQueryWrapper<>(new PrefixQuery(terms[i][j])); } else { innerQueries[j] = new SpanTermQuery(terms[i][j]); } @@ -200,17 +190,6 @@ private Collection rewriteCustomQuery(Query query) { boolean inorder = (mpq.getSlop() == 0); return Collections.singletonList(new SpanNearQuery(positionSpanQueries, mpq.getSlop() + positionGaps, inorder)); - } else if (query instanceof CommonTermsQuery) { - CommonTermsQuery ctq = (CommonTermsQuery) query; - List tqs = new ArrayList<> (); - for (Term term : ctq.getTerms()) { - tqs.add(new TermQuery(term)); - } - return tqs; - } else if (query instanceof FunctionScoreQuery) { - return Collections.singletonList(((FunctionScoreQuery) query).getSubQuery()); - } else if (query instanceof ESToParentBlockJoinQuery) { - return Collections.singletonList(((ESToParentBlockJoinQuery) query).getChildQuery()); } else { return null; } diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index a6cee77522646..e2a4d7a901579 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -135,7 +135,7 @@ public class Version implements Comparable, ToXContentFragment { public static final int V_7_1_0_ID = 7010099; public static final Version V_7_1_0 = new Version(V_7_1_0_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); public static final int V_8_0_0_ID = 8000099; - public static final Version V_8_0_0 = new Version(V_8_0_0_ID, org.apache.lucene.util.Version.LUCENE_8_0_0); + public static final Version V_8_0_0 = new Version(V_8_0_0_ID, org.apache.lucene.util.Version.LUCENE_8_1_0); public static final Version CURRENT = V_8_0_0; diff --git a/server/src/main/java/org/elasticsearch/common/lucene/search/function/FunctionScoreQuery.java b/server/src/main/java/org/elasticsearch/common/lucene/search/function/FunctionScoreQuery.java index 95fbed64a25f5..2592c645111d2 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/search/function/FunctionScoreQuery.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/search/function/FunctionScoreQuery.java @@ -22,10 +22,12 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.FilterScorer; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.ScorerSupplier; import org.apache.lucene.search.Weight; @@ -195,6 +197,12 @@ public CombineFunction getCombineFunction() { return combineFunction; } + @Override + public void visit(QueryVisitor visitor) { + // Highlighters must visit the child query to extract terms + subQuery.visit(visitor.getSubVisitor(BooleanClause.Occur.MUST, this)); + } + @Override public Query rewrite(IndexReader reader) throws IOException { Query rewritten = super.rewrite(reader); diff --git a/server/src/main/java/org/elasticsearch/index/codec/CodecService.java b/server/src/main/java/org/elasticsearch/index/codec/CodecService.java index efc19587658a2..6c3a2138e89bf 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/CodecService.java +++ b/server/src/main/java/org/elasticsearch/index/codec/CodecService.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.FSTLoadMode; import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode; import org.apache.lucene.codecs.lucene80.Lucene80Codec; import org.elasticsearch.common.Nullable; @@ -48,7 +49,7 @@ public CodecService(@Nullable MapperService mapperService, Logger logger) { final MapBuilder codecs = MapBuilder.newMapBuilder(); if (mapperService == null) { codecs.put(DEFAULT_CODEC, new Lucene80Codec()); - codecs.put(BEST_COMPRESSION_CODEC, new Lucene80Codec(Mode.BEST_COMPRESSION)); + codecs.put(BEST_COMPRESSION_CODEC, new Lucene80Codec(Mode.BEST_COMPRESSION, FSTLoadMode.AUTO)); } else { codecs.put(DEFAULT_CODEC, new PerFieldMappingPostingFormatCodec(Mode.BEST_SPEED, mapperService, logger)); diff --git a/server/src/main/java/org/elasticsearch/index/codec/PerFieldMappingPostingFormatCodec.java b/server/src/main/java/org/elasticsearch/index/codec/PerFieldMappingPostingFormatCodec.java index 4a154abd8eadd..705141f1fb925 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/PerFieldMappingPostingFormatCodec.java +++ b/server/src/main/java/org/elasticsearch/index/codec/PerFieldMappingPostingFormatCodec.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.FSTLoadMode; import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat; import org.apache.lucene.codecs.lucene80.Lucene80Codec; import org.elasticsearch.common.lucene.Lucene; @@ -47,7 +48,7 @@ public class PerFieldMappingPostingFormatCodec extends Lucene80Codec { } public PerFieldMappingPostingFormatCodec(Lucene50StoredFieldsFormat.Mode compressionMode, MapperService mapperService, Logger logger) { - super(compressionMode); + super(compressionMode, FSTLoadMode.AUTO); this.mapperService = mapperService; this.logger = logger; } diff --git a/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java b/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java index e174b3fd49eee..92b4fa664193c 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/IntervalBuilder.java @@ -26,9 +26,9 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute; import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.MatchesIterator; +import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.intervals.IntervalIterator; import org.apache.lucene.search.intervals.Intervals; import org.apache.lucene.search.intervals.IntervalsSource; @@ -37,9 +37,10 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Set; /** * Constructs an IntervalsSource based on analyzed text @@ -287,14 +288,17 @@ public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) { return null; } + @Override + public void visit(String field, QueryVisitor visitor) {} + @Override public int minExtent() { return 0; } @Override - public void extractTerms(String field, Set terms) { - + public Collection pullUpDisjunctions() { + return Collections.emptyList(); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java b/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java index 5e6aa3bb7c456..30cd6e52f0dbd 100644 --- a/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java +++ b/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java @@ -20,8 +20,10 @@ package org.elasticsearch.index.search; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.Weight; import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.search.join.ScoreMode; @@ -74,6 +76,12 @@ public Query rewrite(IndexReader reader) throws IOException { return super.rewrite(reader); } + @Override + public void visit(QueryVisitor visitor) { + // Highlighters must visit the child query to extract terms + query.getChildQuery().visit(visitor.getSubVisitor(BooleanClause.Occur.MUST, this)); + } + @Override public Weight createWeight(IndexSearcher searcher, org.apache.lucene.search.ScoreMode scoreMode, float boost) throws IOException { return query.createWeight(searcher, scoreMode, boost); diff --git a/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java b/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java index ab8fb5dbcae02..b2354667889b6 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java +++ b/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java @@ -19,9 +19,7 @@ package org.elasticsearch.search.profile.query; -import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.Scorable; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.TwoPhaseIterator; import org.apache.lucene.search.Weight; @@ -40,8 +38,6 @@ final class ProfileScorer extends Scorer { private ProfileWeight profileWeight; private final Timer scoreTimer, nextDocTimer, advanceTimer, matchTimer, shallowAdvanceTimer, computeMaxScoreTimer; - private final boolean isConstantScoreQuery; - ProfileScorer(ProfileWeight w, Scorer scorer, QueryProfileBreakdown profile) throws IOException { super(w); @@ -53,26 +49,6 @@ final class ProfileScorer extends Scorer { matchTimer = profile.getTimer(QueryTimingType.MATCH); shallowAdvanceTimer = profile.getTimer(QueryTimingType.SHALLOW_ADVANCE); computeMaxScoreTimer = profile.getTimer(QueryTimingType.COMPUTE_MAX_SCORE); - ProfileScorer profileScorer = null; - if (w.getQuery() instanceof ConstantScoreQuery && scorer instanceof ProfileScorer) { - //Case when we have a totalHits query and it is not cached - profileScorer = (ProfileScorer) scorer; - } else if (w.getQuery() instanceof ConstantScoreQuery && scorer.getChildren().size() == 1) { - //Case when we have a top N query. If the scorer has no children, it is because it is cached - //and in that case we do not do any special treatment - Scorable childScorer = scorer.getChildren().iterator().next().child; - if (childScorer instanceof ProfileScorer) { - profileScorer = (ProfileScorer) childScorer; - } - } - if (profileScorer != null) { - isConstantScoreQuery = true; - profile.setTimer(QueryTimingType.NEXT_DOC, profileScorer.nextDocTimer); - profile.setTimer(QueryTimingType.ADVANCE, profileScorer.advanceTimer); - profile.setTimer(QueryTimingType.MATCH, profileScorer.matchTimer); - } else { - isConstantScoreQuery = false; - } } @Override @@ -102,9 +78,6 @@ public Collection getChildren() throws IOException { @Override public DocIdSetIterator iterator() { - if (isConstantScoreQuery) { - return scorer.iterator(); - } final DocIdSetIterator in = scorer.iterator(); return new DocIdSetIterator() { @@ -142,9 +115,6 @@ public long cost() { @Override public TwoPhaseIterator twoPhaseIterator() { - if (isConstantScoreQuery) { - return scorer.twoPhaseIterator(); - } final TwoPhaseIterator in = scorer.twoPhaseIterator(); if (in == null) { return null; diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java index b108020c026f8..b18daf07bf361 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java @@ -205,7 +205,7 @@ public void testLuceneVersionOnUnknownVersions() { assertEquals(Version.V_6_0_0.luceneVersion.major - 1, version.luceneVersion.major); // future version, should be the same version as today - version = Version.fromString("7.77.1"); + version = Version.fromString("8.77.1"); assertEquals(Version.CURRENT.luceneVersion, version.luceneVersion); } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 53bef02a6ebb2..f741f907ace10 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -27,6 +27,7 @@ import org.apache.logging.log4j.core.LogEvent; import org.apache.logging.log4j.core.appender.AbstractAppender; import org.apache.logging.log4j.core.filter.RegexFilter; +import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat; import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat; import org.apache.lucene.document.Field; import org.apache.lucene.document.LongPoint; @@ -300,7 +301,13 @@ public void testSegments() throws Exception { assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); assertThat(segments.get(0).isCompound(), equalTo(true)); assertThat(segments.get(0).ramTree, nullValue()); - assertThat(segments.get(0).getAttributes().keySet(), Matchers.contains(Lucene50StoredFieldsFormat.MODE_KEY)); + assertThat(segments.get(0).getAttributes().keySet(), + Matchers.contains( + // TODO: Lucene50PostingsFormat#MODE_KEY should be public ? + Lucene50PostingsFormat.class.getSimpleName() + ".fstMode", + Lucene50StoredFieldsFormat.MODE_KEY + ) + ); engine.flush(); diff --git a/server/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java b/server/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java index 78a8049d741bd..275349e15cdf1 100644 --- a/server/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java +++ b/server/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerTests.java @@ -28,7 +28,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; -import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.LeafCollector; @@ -120,209 +119,6 @@ public void testBasic() throws IOException { assertThat(rewriteTime, greaterThan(0L)); } - public void testConstantScoreQuery() throws IOException { - QueryProfiler profiler = new QueryProfiler(); - searcher.setProfiler(profiler); - Query query = new ConstantScoreQuery(new TermQuery(new Term("foo", "bar"))); - searcher.search(query, 1); - List results = profiler.getTree(); - assertEquals(1, results.size()); - Map breakdownConstantScoreQuery = results.get(0).getTimeBreakdown(); - assertEquals(1, results.get(0).getProfiledChildren().size()); - Map breakdownTermQuery = results.get(0).getProfiledChildren().get(0).getTimeBreakdown(); - - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - assertEquals(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), - breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue()); - - long rewriteTime = profiler.getRewriteTime(); - assertThat(rewriteTime, greaterThan(0L)); - } - - public void testConstantScoreTotalHitsBeingCachedQuery() throws IOException { - Query query = new ConstantScoreQuery(new TermQuery(new Term("foo", "bar"))); - //clean cache and make sure queries will be cached - searcher.setQueryCache(IndexSearcher.getDefaultQueryCache()); - searcher.setQueryCachingPolicy(ALWAYS_CACHE_POLICY); - - QueryProfiler profiler = new QueryProfiler(); - searcher.setProfiler(profiler); - TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.search(query, collector); - - List results = profiler.getTree(); - assertEquals(1, results.size()); - Map breakdownConstantScoreQuery = results.get(0).getTimeBreakdown(); - assertEquals(1, results.get(0).getProfiledChildren().size()); - Map breakdownTermQuery = results.get(0).getProfiledChildren().get(0).getTimeBreakdown(); - //In this case scorers for constant score query and term query are disconnected. - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - long rewriteTime = profiler.getRewriteTime(); - assertThat(rewriteTime, greaterThan(0L)); - } - - public void testConstantScoreTotalHitsNotCachedQuery() throws IOException { - Query query = new ConstantScoreQuery(new TermQuery(new Term("foo", "bar"))); - - //clean cache and make sure queries will not be cached - searcher.setQueryCache(IndexSearcher.getDefaultQueryCache()); - searcher.setQueryCachingPolicy(NEVER_CACHE_POLICY); - - QueryProfiler profiler = new QueryProfiler(); - searcher.setProfiler(profiler); - TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.search(query, collector); - - List results = profiler.getTree(); - assertEquals(1, results.size()); - Map breakdownConstantScoreQuery = results.get(0).getTimeBreakdown(); - assertEquals(1, results.get(0).getProfiledChildren().size()); - Map breakdownTermQuery = results.get(0).getProfiledChildren().get(0).getTimeBreakdown(); - //Timing from the scorer of term query are inherited by constant score query scorer. - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - - assertEquals(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), - breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue()); - - - long rewriteTime = profiler.getRewriteTime(); - assertThat(rewriteTime, greaterThan(0L)); - } - - public void testConstantScoreTotalHitsCachedQuery() throws IOException { - Query query = new ConstantScoreQuery(new TermQuery(new Term("foo", "bar"))); - - //clean cache and make sure queries will be cached - searcher.setQueryCache(IndexSearcher.getDefaultQueryCache()); - searcher.setQueryCachingPolicy(ALWAYS_CACHE_POLICY); - //Put query on cache - TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.search(query, collector); - - QueryProfiler profiler = new QueryProfiler(); - searcher.setProfiler(profiler); - collector = new TotalHitCountCollector(); - searcher.search(query, collector); - - List results = profiler.getTree(); - assertEquals(1, results.size()); - Map breakdownConstantScoreQuery = results.get(0).getTimeBreakdown(); - assertEquals(1, results.get(0).getProfiledChildren().size()); - Map breakdownTermQuery = results.get(0).getProfiledChildren().get(0).getTimeBreakdown(); - - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownConstantScoreQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString()).longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString()).longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString()).longValue(), equalTo(0L)); - - assertThat(breakdownTermQuery.get(QueryTimingType.CREATE_WEIGHT.toString() + "_count").longValue(), greaterThan(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.BUILD_SCORER.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.NEXT_DOC.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.ADVANCE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.SCORE.toString() + "_count").longValue(), equalTo(0L)); - assertThat(breakdownTermQuery.get(QueryTimingType.MATCH.toString() + "_count").longValue(), equalTo(0L)); - - long rewriteTime = profiler.getRewriteTime(); - assertThat(rewriteTime, greaterThan(0L)); - } - - public void testNoScoring() throws IOException { QueryProfiler profiler = new QueryProfiler(); searcher.setProfiler(profiler); diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java index 16b18efe62322..e65b4aa377a40 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java @@ -317,7 +317,6 @@ public void testInOrderScrollOptimization() throws Exception { contextSearcher = getAssertingEarlyTerminationSearcher(reader, size); QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); - assertTrue(context.queryResult().terminatedEarly()); assertThat(context.terminateAfter(), equalTo(size)); assertThat(context.queryResult().getTotalHits().value, equalTo((long) numDocs)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs[0].doc, greaterThanOrEqualTo(size)); diff --git a/x-pack/plugin/sql/sql-action/licenses/lucene-core-8.0.0.jar.sha1 b/x-pack/plugin/sql/sql-action/licenses/lucene-core-8.0.0.jar.sha1 deleted file mode 100644 index 2619abce7979a..0000000000000 --- a/x-pack/plugin/sql/sql-action/licenses/lucene-core-8.0.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -407c555efb2d3253f51a676cc2089a5d29a3b7b7 \ No newline at end of file diff --git a/x-pack/plugin/sql/sql-action/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 b/x-pack/plugin/sql/sql-action/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 new file mode 100644 index 0000000000000..fea3658cf61bd --- /dev/null +++ b/x-pack/plugin/sql/sql-action/licenses/lucene-core-8.1.0-snapshot-e460356abe.jar.sha1 @@ -0,0 +1 @@ +3e85f77d8f8ed1db53dba387fbdec55a9f912639 \ No newline at end of file From 778a1d0d630f25d7d1df6ef00478daabc6fe2733 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 16 Apr 2019 10:03:35 +0200 Subject: [PATCH 044/260] Fix some `range` query edge cases (#41160) Currently we throw an error when a range querys minimum value exceeds the maximum value due to the fact that they are neighbouring values and both upper and lower value are excluded from the interval. Since this is a condition that the user usually doesn't specify conciously (at least in the case of float and double values its difficult to see which values are adjacent) we should ignore those "wrong" intervals and create a MatchNoDocsQuery in those cases. We should still throw errors with an actionable message if the user specifies the query interval in a way that min value > max value. This PR adds those checks and tests for those cases. Closes #40937 --- .../index/mapper/RangeFieldMapper.java | 126 +++++++++++------ .../index/mapper/RangeFieldTypeTests.java | 129 +++++++++++++++++- 2 files changed, 209 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java index e5ba55de7bfd0..095f84e61563d 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java @@ -35,6 +35,7 @@ import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.IndexOrDocValuesQuery; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.ByteArrayDataOutput; @@ -63,6 +64,7 @@ import java.time.ZoneId; import java.time.ZoneOffset; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -70,6 +72,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.BiFunction; import static org.elasticsearch.index.query.RangeQueryBuilder.GTE_FIELD; import static org.elasticsearch.index.query.RangeQueryBuilder.GT_FIELD; @@ -516,25 +519,38 @@ public Query dvRangeQuery(String field, QueryType queryType, Object from, Object } @Override - public Query withinQuery(String field, Object from, Object to, boolean includeLower, boolean includeUpper) { - InetAddress lower = (InetAddress)from; - InetAddress upper = (InetAddress)to; - return InetAddressRange.newWithinQuery(field, - includeLower ? lower : nextUp(lower), includeUpper ? upper : nextDown(upper)); + public Query withinQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { + return createQuery(field, from, to, includeFrom, includeTo, + (f, t) -> InetAddressRange.newWithinQuery(field, f, t)); } @Override - public Query containsQuery(String field, Object from, Object to, boolean includeLower, boolean includeUpper) { - InetAddress lower = (InetAddress)from; - InetAddress upper = (InetAddress)to; - return InetAddressRange.newContainsQuery(field, - includeLower ? lower : nextUp(lower), includeUpper ? upper : nextDown(upper)); + public Query containsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { + return createQuery(field, from, to, includeFrom, includeTo, + (f, t) -> InetAddressRange.newContainsQuery(field, f, t )); } @Override - public Query intersectsQuery(String field, Object from, Object to, boolean includeLower, boolean includeUpper) { - InetAddress lower = (InetAddress)from; - InetAddress upper = (InetAddress)to; - return InetAddressRange.newIntersectsQuery(field, - includeLower ? lower : nextUp(lower), includeUpper ? upper : nextDown(upper)); + public Query intersectsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { + return createQuery(field, from, to, includeFrom, includeTo, + (f, t) -> InetAddressRange.newIntersectsQuery(field, f ,t )); + } + + private Query createQuery(String field, Object lower, Object upper, boolean includeLower, boolean includeUpper, + BiFunction querySupplier) { + byte[] lowerBytes = InetAddressPoint.encode((InetAddress) lower); + byte[] upperBytes = InetAddressPoint.encode((InetAddress) upper); + if (Arrays.compareUnsigned(lowerBytes, 0, lowerBytes.length, upperBytes, 0, upperBytes.length) > 0) { + throw new IllegalArgumentException( + "Range query `from` value (" + lower + ") is greater than `to` value (" + upper + ")"); + } + InetAddress correctedFrom = includeLower ? (InetAddress) lower : nextUp(lower); + InetAddress correctedTo = includeUpper ? (InetAddress) upper : nextDown(upper);; + lowerBytes = InetAddressPoint.encode(correctedFrom); + upperBytes = InetAddressPoint.encode(correctedTo); + if (Arrays.compareUnsigned(lowerBytes, 0, lowerBytes.length, upperBytes, 0, upperBytes.length) > 0) { + return new MatchNoDocsQuery("float range didn't intersect anything"); + } else { + return querySupplier.apply(correctedFrom, correctedTo); + } } }, DATE("date_range", NumberType.LONG) { @@ -662,21 +678,18 @@ public Field getRangeField(String name, Range r) { } @Override public Query withinQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return FloatRange.newWithinQuery(field, - new float[] {includeFrom ? (Float)from : Math.nextUp((Float)from)}, - new float[] {includeTo ? (Float)to : Math.nextDown((Float)to)}); + return createQuery(field, (Float) from, (Float) to, includeFrom, includeTo, + (f, t) -> FloatRange.newWithinQuery(field, new float[] { f }, new float[] { t }), RangeType.FLOAT); } @Override public Query containsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return FloatRange.newContainsQuery(field, - new float[] {includeFrom ? (Float)from : Math.nextUp((Float)from)}, - new float[] {includeTo ? (Float)to : Math.nextDown((Float)to)}); + return createQuery(field, (Float) from, (Float) to, includeFrom, includeTo, + (f, t) -> FloatRange.newContainsQuery(field, new float[] { f }, new float[] { t }), RangeType.FLOAT); } @Override public Query intersectsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return FloatRange.newIntersectsQuery(field, - new float[] {includeFrom ? (Float)from : Math.nextUp((Float)from)}, - new float[] {includeTo ? (Float)to : Math.nextDown((Float)to)}); + return createQuery(field, (Float) from, (Float) to, includeFrom, includeTo, + (f, t) -> FloatRange.newIntersectsQuery(field, new float[] { f }, new float[] { t }), RangeType.FLOAT); } }, DOUBLE("double_range", NumberType.DOUBLE) { @@ -724,22 +737,20 @@ public Field getRangeField(String name, Range r) { } @Override public Query withinQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return DoubleRange.newWithinQuery(field, - new double[] {includeFrom ? (Double)from : Math.nextUp((Double)from)}, - new double[] {includeTo ? (Double)to : Math.nextDown((Double)to)}); + return createQuery(field, (Double) from, (Double) to, includeFrom, includeTo, + (f, t) -> DoubleRange.newWithinQuery(field, new double[] { f }, new double[] { t }), RangeType.DOUBLE); } @Override public Query containsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return DoubleRange.newContainsQuery(field, - new double[] {includeFrom ? (Double)from : Math.nextUp((Double)from)}, - new double[] {includeTo ? (Double)to : Math.nextDown((Double)to)}); + return createQuery(field, (Double) from, (Double) to, includeFrom, includeTo, + (f, t) -> DoubleRange.newContainsQuery(field, new double[] { f }, new double[] { t }), RangeType.DOUBLE); } @Override public Query intersectsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return DoubleRange.newIntersectsQuery(field, - new double[] {includeFrom ? (Double)from : Math.nextUp((Double)from)}, - new double[] {includeTo ? (Double)to : Math.nextDown((Double)to)}); + return createQuery(field, (Double) from, (Double) to, includeFrom, includeTo, + (f, t) -> DoubleRange.newIntersectsQuery(field, new double[] { f }, new double[] { t }), RangeType.DOUBLE); } + }, // todo add BYTE support // todo add SHORT support @@ -777,18 +788,18 @@ public Field getRangeField(String name, Range r) { } @Override public Query withinQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return IntRange.newWithinQuery(field, new int[] {(Integer)from + (includeFrom ? 0 : 1)}, - new int[] {(Integer)to - (includeTo ? 0 : 1)}); + return createQuery(field, (Integer) from, (Integer) to, includeFrom, includeTo, + (f, t) -> IntRange.newWithinQuery(field, new int[] { f }, new int[] { t }), RangeType.INTEGER); } @Override public Query containsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return IntRange.newContainsQuery(field, new int[] {(Integer)from + (includeFrom ? 0 : 1)}, - new int[] {(Integer)to - (includeTo ? 0 : 1)}); + return createQuery(field, (Integer) from, (Integer) to, includeFrom, includeTo, + (f, t) -> IntRange.newContainsQuery(field, new int[] { f }, new int[] { t }), RangeType.INTEGER); } @Override public Query intersectsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return IntRange.newIntersectsQuery(field, new int[] {(Integer)from + (includeFrom ? 0 : 1)}, - new int[] {(Integer)to - (includeTo ? 0 : 1)}); + return createQuery(field, (Integer) from, (Integer) to, includeFrom, includeTo, + (f, t) -> IntRange.newIntersectsQuery(field, new int[] { f }, new int[] { t }), RangeType.INTEGER); } }, LONG("long_range", NumberType.LONG) { @@ -837,18 +848,18 @@ public Field getRangeField(String name, Range r) { } @Override public Query withinQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return LongRange.newWithinQuery(field, new long[] {(Long)from + (includeFrom ? 0 : 1)}, - new long[] {(Long)to - (includeTo ? 0 : 1)}); + return createQuery(field, (Long) from, (Long) to, includeFrom, includeTo, + (f, t) -> LongRange.newWithinQuery(field, new long[] { f }, new long[] { t }), RangeType.LONG); } @Override public Query containsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return LongRange.newContainsQuery(field, new long[] {(Long)from + (includeFrom ? 0 : 1)}, - new long[] {(Long)to - (includeTo ? 0 : 1)}); + return createQuery(field, (Long) from, (Long) to, includeFrom, includeTo, + (f, t) -> LongRange.newContainsQuery(field, new long[] { f }, new long[] { t }), RangeType.LONG); } @Override public Query intersectsQuery(String field, Object from, Object to, boolean includeFrom, boolean includeTo) { - return LongRange.newIntersectsQuery(field, new long[] {(Long)from + (includeFrom ? 0 : 1)}, - new long[] {(Long)to - (includeTo ? 0 : 1)}); + return createQuery(field, (Long) from, (Long) to, includeFrom, includeTo, + (f, t) -> LongRange.newIntersectsQuery(field, new long[] { f }, new long[] { t }), RangeType.LONG); } }; @@ -867,6 +878,31 @@ public final String typeName() { return name; } + /** + * Internal helper to create the actual {@link Query} using the provided supplier function. Before creating the query we check if + * the intervals min > max, in which case an {@link IllegalArgumentException} is raised. The method adapts the interval bounds + * based on whether the edges should be included or excluded. In case where after this correction the interval would be empty + * because min > max, we simply return a {@link MatchNoDocsQuery}. + * This helper handles all {@link Number} cases and dates, the IP range type uses its own logic. + */ + private static > Query createQuery(String field, T from, T to, boolean includeFrom, boolean includeTo, + BiFunction querySupplier, RangeType rangeType) { + if (from.compareTo(to) > 0) { + // wrong argument order, this is an error the user should fix + throw new IllegalArgumentException("Range query `from` value (" + from + ") is greater than `to` value (" + to + ")"); + } + + @SuppressWarnings("unchecked") + T correctedFrom = includeFrom ? from : (T) rangeType.nextUp(from); + @SuppressWarnings("unchecked") + T correctedTo = includeTo ? to : (T) rangeType.nextDown(to); + if (correctedFrom.compareTo(correctedTo) > 0) { + return new MatchNoDocsQuery("range didn't intersect anything"); + } else { + return querySupplier.apply(correctedFrom, correctedTo); + } + } + public abstract Field getRangeField(String name, Range range); public List createFields(ParseContext context, String name, Range range, boolean indexed, boolean docValued, boolean stored) { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java index 6ca98fb4db6d2..a26999fa3a6f5 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java @@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.queries.BinaryDocValuesRangeQuery; import org.apache.lucene.search.IndexOrDocValuesQuery; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchParseException; @@ -49,6 +50,7 @@ import java.util.Locale; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.instanceOf; public class RangeFieldTypeTests extends FieldTypeTestCase { RangeType type; @@ -92,11 +94,136 @@ public void testRangeQuery() throws Exception { boolean includeUpper = randomBoolean(); Object from = nextFrom(); Object to = nextTo(from); + if (includeLower == false && includeUpper == false) { + // need to increase once more, otherwise interval is empty because edge values are exclusive + to = nextTo(to); + } assertEquals(getExpectedRangeQuery(relation, from, to, includeLower, includeUpper), ft.rangeQuery(from, to, includeLower, includeUpper, relation, null, null, context)); } + /** + * test the queries are correct if from/to are adjacent and the range is exclusive of those values + */ + public void testRangeQueryIntersectsAdjacentValues() throws Exception { + QueryShardContext context = createContext(); + ShapeRelation relation = randomFrom(ShapeRelation.values()); + RangeFieldType ft = new RangeFieldType(type); + ft.setName(FIELDNAME); + ft.setIndexOptions(IndexOptions.DOCS); + + Object from = null; + Object to = null; + switch (type) { + case LONG: { + long fromValue = randomLong(); + from = fromValue; + to = fromValue + 1; + break; + } + case DATE: { + long fromValue = randomInt(); + from = new DateTime(fromValue); + to = new DateTime(fromValue + 1); + break; + } + case INTEGER: { + int fromValue = randomInt(); + from = fromValue; + to = fromValue + 1; + break; + } + case DOUBLE: { + double fromValue = randomDoubleBetween(0, 100, true); + from = fromValue; + to = Math.nextUp(fromValue); + break; + } + case FLOAT: { + float fromValue = randomFloat(); + from = fromValue; + to = Math.nextUp(fromValue); + break; + } + case IP: { + byte[] ipv4 = new byte[4]; + random().nextBytes(ipv4); + InetAddress fromValue = InetAddress.getByAddress(ipv4); + from = fromValue; + to = InetAddressPoint.nextUp(fromValue); + break; + } + default: + from = nextFrom(); + to = nextTo(from); + } + Query rangeQuery = ft.rangeQuery(from, to, false, false, relation, null, null, context); + assertThat(rangeQuery, instanceOf(IndexOrDocValuesQuery.class)); + assertThat(((IndexOrDocValuesQuery) rangeQuery).getIndexQuery(), instanceOf(MatchNoDocsQuery.class)); + } + + /** + * check that we catch cases where the user specifies larger "from" than "to" value, not counting the include upper/lower settings + */ + public void testFromLargerToErrors() throws Exception { + QueryShardContext context = createContext(); + RangeFieldType ft = new RangeFieldType(type); + ft.setName(FIELDNAME); + ft.setIndexOptions(IndexOptions.DOCS); + + final Object from; + final Object to; + switch (type) { + case LONG: { + long fromValue = randomLong(); + from = fromValue; + to = fromValue - 1L; + break; + } + case DATE: { + long fromValue = randomInt(); + from = new DateTime(fromValue); + to = new DateTime(fromValue - 1); + break; + } + case INTEGER: { + int fromValue = randomInt(); + from = fromValue; + to = fromValue - 1; + break; + } + case DOUBLE: { + double fromValue = randomDoubleBetween(0, 100, true); + from = fromValue; + to = fromValue - 1.0d; + break; + } + case FLOAT: { + float fromValue = randomFloat(); + from = fromValue; + to = fromValue - 1.0f; + break; + } + case IP: { + byte[] ipv4 = new byte[4]; + random().nextBytes(ipv4); + InetAddress fromValue = InetAddress.getByAddress(ipv4); + from = fromValue; + to = InetAddressPoint.nextDown(fromValue); + break; + } + default: + // quit test for other range types + return; + } + ShapeRelation relation = randomFrom(ShapeRelation.values()); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> ft.rangeQuery(from, to, true, true, relation, null, null, context)); + assertTrue(ex.getMessage().contains("Range query `from` value")); + assertTrue(ex.getMessage().contains("is greater than `to` value")); + } + private QueryShardContext createContext() { Settings indexSettings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); @@ -104,7 +231,7 @@ private QueryShardContext createContext() { return new QueryShardContext(0, idxSettings, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null); } - + public void testDateRangeQueryUsingMappingFormat() { QueryShardContext context = createContext(); RangeFieldType fieldType = new RangeFieldType(RangeType.DATE); From 6f176ee76a6b39f898872ee79504a9fa2e413d7f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 16 Apr 2019 04:13:47 -0400 Subject: [PATCH 045/260] Correct flush parameters in engine test Since #40213, we forbid a combination of flush parameters: force=true and wait_if_ongoing=false. Closes #41236 --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f741f907ace10..11b9ef7baa518 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -3095,7 +3095,7 @@ public void run() { break; } case "flush": { - engine.flush(true, false); + engine.flush(true, true); break; } } From 4fb156a579a36ca5e14d8f040f3c0d88be5c6f7c Mon Sep 17 00:00:00 2001 From: David Roberts Date: Tue, 16 Apr 2019 09:16:23 +0100 Subject: [PATCH 046/260] [ML] Unmute MachineLearningIT.testDeleteExpiredData (#41186) The cause of failure was fixed by elastic/ml-cpp#459, so all that remains on the Java side is to unmute the test that was failing. Closes #41070 --- .../test/java/org/elasticsearch/client/MachineLearningIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java index 3fdc00419cda6..f7b7b148f660b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java @@ -915,7 +915,6 @@ private String createExpiredData(String jobId) throws Exception { return forecastJobResponse.getForecastId(); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41070") public void testDeleteExpiredData() throws Exception { String jobId = "test-delete-expired-data"; From 3c66cffecd98ff6cafc74d1d23b078ccb74aa4e2 Mon Sep 17 00:00:00 2001 From: Yogesh Gaikwad <902768+bizybot@users.noreply.github.com> Date: Tue, 16 Apr 2019 18:29:13 +1000 Subject: [PATCH 047/260] Use alias name from rollover request to query indices stats (#40774) In `TransportRolloverAction` before doing rollover we resolve source index name (write index) from the alias in the rollover request. Before evaluating the conditions and executing rollover action, we retrieve stats, but to do so we used the source index name resolved from the alias instead of alias from the index. This fails when the user is assigned a role with index privilege on the alias instead of the concrete index. This commit fixes this by using the alias from the request. After this change, verified that when we retrieve all the stats (including write + read indexes) we are considering only source index. Closes #40771 --- .../rollover/TransportRolloverAction.java | 4 +- .../TransportRolloverActionTests.java | 146 +++++++++++++++++- .../authz/31_rollover_using_alias.yml | 139 +++++++++++++++++ 3 files changed, 281 insertions(+), 8 deletions(-) create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java index ce1f1dc240426..edd59f8b18ca1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java @@ -120,7 +120,7 @@ protected void masterOperation(final RolloverRequest rolloverRequest, final Clus final String rolloverIndexName = indexNameExpressionResolver.resolveDateMathExpression(unresolvedName); MetaDataCreateIndexService.validateIndexName(rolloverIndexName, state); // will fail if the index already exists checkNoDuplicatedAliasInIndexTemplate(metaData, rolloverIndexName, rolloverRequest.getAlias()); - client.admin().indices().prepareStats(sourceIndexName).clear().setDocs(true).execute( + client.admin().indices().prepareStats(rolloverRequest.getAlias()).clear().setDocs(true).execute( new ActionListener() { @Override public void onResponse(IndicesStatsResponse statsResponse) { @@ -249,7 +249,7 @@ static Map evaluateConditions(final Collection> co static Map evaluateConditions(final Collection> conditions, final IndexMetaData metaData, final IndicesStatsResponse statsResponse) { - return evaluateConditions(conditions, statsResponse.getPrimaries().getDocs(), metaData); + return evaluateConditions(conditions, statsResponse.getIndex(metaData.getIndex().getName()).getPrimaries().getDocs(), metaData); } static void validate(MetaData metaData, RolloverRequest request) { diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java index ec3c82ba70b2f..058dcc7243029 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java @@ -20,17 +20,30 @@ package org.elasticsearch.action.admin.indices.rollover; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.stats.CommonStats; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.client.AdminClient; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.IndicesAdminClient; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.AliasAction; import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService; +import org.elasticsearch.cluster.metadata.MetaDataIndexAliasesService; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -39,9 +52,12 @@ import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; import org.mockito.ArgumentCaptor; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -51,7 +67,9 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -64,7 +82,9 @@ public void testDocStatsSelectionFromPrimariesOnly() { long docsInShards = 200; final Condition condition = createTestCondition(); - evaluateConditions(Sets.newHashSet(condition), createMetaData(), createIndicesStatResponse(docsInShards, docsInPrimaryShards)); + String indexName = randomAlphaOfLengthBetween(5, 7); + evaluateConditions(Sets.newHashSet(condition), createMetaData(indexName), + createIndicesStatResponse(indexName, docsInShards, docsInPrimaryShards)); final ArgumentCaptor argument = ArgumentCaptor.forClass(Condition.Stats.class); verify(condition).evaluate(argument.capture()); @@ -286,7 +306,7 @@ public void testRejectDuplicateAlias() { .patterns(Arrays.asList("foo-*", "bar-*")) .putAlias(AliasMetaData.builder("foo-write")).putAlias(AliasMetaData.builder("bar-write").writeIndex(randomBoolean())) .build(); - final MetaData metaData = MetaData.builder().put(createMetaData(), false).put(template).build(); + final MetaData metaData = MetaData.builder().put(createMetaData(randomAlphaOfLengthBetween(5, 7)), false).put(template).build(); String indexName = randomFrom("foo-123", "bar-xyz"); String aliasName = randomFrom("foo-write", "bar-write"); final IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, @@ -294,9 +314,92 @@ public void testRejectDuplicateAlias() { assertThat(ex.getMessage(), containsString("index template [test-template]")); } - private IndicesStatsResponse createIndicesStatResponse(long totalDocs, long primaryDocs) { + public void testConditionEvaluationWhenAliasToWriteAndReadIndicesConsidersOnlyPrimariesFromWriteIndex() { + final TransportService mockTransportService = mock(TransportService.class); + final ClusterService mockClusterService = mock(ClusterService.class); + final ThreadPool mockThreadPool = mock(ThreadPool.class); + final MetaDataCreateIndexService mockCreateIndexService = mock(MetaDataCreateIndexService.class); + final IndexNameExpressionResolver mockIndexNameExpressionResolver = mock(IndexNameExpressionResolver.class); + when(mockIndexNameExpressionResolver.resolveDateMathExpression(any())).thenReturn("logs-index-000003"); + final ActionFilters mockActionFilters = mock(ActionFilters.class); + final MetaDataIndexAliasesService mdIndexAliasesService = mock(MetaDataIndexAliasesService.class); + + final Client mockClient = mock(Client.class); + final AdminClient mockAdminClient = mock(AdminClient.class); + final IndicesAdminClient mockIndicesAdminClient = mock(IndicesAdminClient.class); + when(mockClient.admin()).thenReturn(mockAdminClient); + when(mockAdminClient.indices()).thenReturn(mockIndicesAdminClient); + + final IndicesStatsRequestBuilder mockIndicesStatsBuilder = mock(IndicesStatsRequestBuilder.class); + when(mockIndicesAdminClient.prepareStats(any())).thenReturn(mockIndicesStatsBuilder); + final Map indexStats = new HashMap<>(); + int total = randomIntBetween(500, 1000); + indexStats.put("logs-index-000001", createIndexStats(200L, total)); + indexStats.put("logs-index-000002", createIndexStats(300L, total)); + final IndicesStatsResponse statsResponse = createAliasToMultipleIndicesStatsResponse(indexStats); + when(mockIndicesStatsBuilder.clear()).thenReturn(mockIndicesStatsBuilder); + when(mockIndicesStatsBuilder.setDocs(true)).thenReturn(mockIndicesStatsBuilder); + + assert statsResponse.getPrimaries().getDocs().getCount() == 500L; + assert statsResponse.getTotal().getDocs().getCount() == (total + total); + + doAnswer(invocation -> { + Object[] args = invocation.getArguments(); + assert args.length == 1; + ActionListener listener = (ActionListener) args[0]; + listener.onResponse(statsResponse); + return null; + }).when(mockIndicesStatsBuilder).execute(any(ActionListener.class)); + + final IndexMetaData.Builder indexMetaData = IndexMetaData.builder("logs-index-000001") + .putAlias(AliasMetaData.builder("logs-alias").writeIndex(false).build()).settings(settings(Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(1); + final IndexMetaData.Builder indexMetaData2 = IndexMetaData.builder("logs-index-000002") + .putAlias(AliasMetaData.builder("logs-alias").writeIndex(true).build()).settings(settings(Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(1); + final ClusterState stateBefore = ClusterState.builder(ClusterName.DEFAULT) + .metaData(MetaData.builder().put(indexMetaData).put(indexMetaData2)).build(); + + final TransportRolloverAction transportRolloverAction = new TransportRolloverAction(mockTransportService, mockClusterService, + mockThreadPool, mockCreateIndexService, mockActionFilters, mockIndexNameExpressionResolver, mdIndexAliasesService, + mockClient); + + // For given alias, verify that condition evaluation fails when the condition doc count is greater than the primaries doc count + // (primaries from only write index is considered) + PlainActionFuture future = new PlainActionFuture<>(); + RolloverRequest rolloverRequest = new RolloverRequest("logs-alias", "logs-index-000003"); + rolloverRequest.addMaxIndexDocsCondition(500L); + rolloverRequest.dryRun(true); + transportRolloverAction.masterOperation(rolloverRequest, stateBefore, future); + + RolloverResponse response = future.actionGet(); + assertThat(response.getOldIndex(), equalTo("logs-index-000002")); + assertThat(response.getNewIndex(), equalTo("logs-index-000003")); + assertThat(response.isDryRun(), equalTo(true)); + assertThat(response.isRolledOver(), equalTo(false)); + assertThat(response.getConditionStatus().size(), equalTo(1)); + assertThat(response.getConditionStatus().get("[max_docs: 500]"), is(false)); + + // For given alias, verify that the condition evaluation is successful when condition doc count is less than the primaries doc count + // (primaries from only write index is considered) + future = new PlainActionFuture<>(); + rolloverRequest = new RolloverRequest("logs-alias", "logs-index-000003"); + rolloverRequest.addMaxIndexDocsCondition(300L); + rolloverRequest.dryRun(true); + transportRolloverAction.masterOperation(rolloverRequest, stateBefore, future); + + response = future.actionGet(); + assertThat(response.getOldIndex(), equalTo("logs-index-000002")); + assertThat(response.getNewIndex(), equalTo("logs-index-000003")); + assertThat(response.isDryRun(), equalTo(true)); + assertThat(response.isRolledOver(), equalTo(false)); + assertThat(response.getConditionStatus().size(), equalTo(1)); + assertThat(response.getConditionStatus().get("[max_docs: 300]"), is(true)); + } + + private IndicesStatsResponse createIndicesStatResponse(String indexName, long totalDocs, long primariesDocs) { final CommonStats primaryStats = mock(CommonStats.class); - when(primaryStats.getDocs()).thenReturn(new DocsStats(primaryDocs, 0, between(1, 10000))); + when(primaryStats.getDocs()).thenReturn(new DocsStats(primariesDocs, 0, between(1, 10000))); final CommonStats totalStats = mock(CommonStats.class); when(totalStats.getDocs()).thenReturn(new DocsStats(totalDocs, 0, between(1, 10000))); @@ -304,18 +407,49 @@ private IndicesStatsResponse createIndicesStatResponse(long totalDocs, long prim final IndicesStatsResponse response = mock(IndicesStatsResponse.class); when(response.getPrimaries()).thenReturn(primaryStats); when(response.getTotal()).thenReturn(totalStats); + final IndexStats indexStats = mock(IndexStats.class); + when(response.getIndex(indexName)).thenReturn(indexStats); + when(indexStats.getPrimaries()).thenReturn(primaryStats); + when(indexStats.getTotal()).thenReturn(totalStats); + return response; + } + + private IndicesStatsResponse createAliasToMultipleIndicesStatsResponse(Map indexStats) { + final IndicesStatsResponse response = mock(IndicesStatsResponse.class); + final CommonStats primariesStats = new CommonStats(); + final CommonStats totalStats = new CommonStats(); + for (String indexName : indexStats.keySet()) { + when(response.getIndex(indexName)).thenReturn(indexStats.get(indexName)); + primariesStats.add(indexStats.get(indexName).getPrimaries()); + totalStats.add(indexStats.get(indexName).getTotal()); + } + when(response.getPrimaries()).thenReturn(primariesStats); + when(response.getTotal()).thenReturn(totalStats); return response; } - private static IndexMetaData createMetaData() { + private IndexStats createIndexStats(long primaries, long total) { + final CommonStats primariesCommonStats = mock(CommonStats.class); + when(primariesCommonStats.getDocs()).thenReturn(new DocsStats(primaries, 0, between(1, 10000))); + + final CommonStats totalCommonStats = mock(CommonStats.class); + when(totalCommonStats.getDocs()).thenReturn(new DocsStats(total, 0, between(1, 10000))); + + IndexStats indexStats = mock(IndexStats.class); + when(indexStats.getPrimaries()).thenReturn(primariesCommonStats); + when(indexStats.getTotal()).thenReturn(totalCommonStats); + return indexStats; + } + + private static IndexMetaData createMetaData(String indexName) { final Settings settings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) .build(); - return IndexMetaData.builder(randomAlphaOfLength(10)) + return IndexMetaData.builder(indexName) .creationDate(System.currentTimeMillis() - TimeValue.timeValueHours(3).getMillis()) .settings(settings) .build(); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml new file mode 100644 index 0000000000000..52b6259f7ccf0 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml @@ -0,0 +1,139 @@ +--- + +setup: + - skip: + features: headers + + - do: + cluster.health: + wait_for_status: yellow + + - do: + security.put_role: + name: "alias_write_manage_role" + body: > + { + "indices": [ + { "names": ["write_manage_alias"], "privileges": ["write", "manage"] } + ] + } + + - do: + security.put_user: + username: "test_user" + body: > + { + "password" : "x-pack-test-password", + "roles" : [ "alias_write_manage_role" ], + "full_name" : "user with privileges to write, manage via alias" + } + + - do: + indices.create: + index: logs-000001 + body: + settings: + index: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + indices.put_alias: + index: logs-000001 + name: write_manage_alias + +--- +teardown: + - do: + security.delete_user: + username: "test_user" + ignore: 404 + + - do: + security.delete_role: + name: "alias_write_role" + ignore: 404 + + - do: + indices.delete_alias: + index: "logs-000001" + name: [ "write_manage_alias" ] + ignore: 404 + + - do: + indices.delete: + index: [ "logs-000001" ] + ignore: 404 + +--- +"Test rollover, index via write alias of index": + + # index using alias + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + create: + id: 1 + index: write_manage_alias + body: > + { + "name" : "doc1" + } + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + create: + id: 2 + index: write_manage_alias + body: > + { + "name" : "doc2" + } + + - do: + indices.refresh: {} + + # rollover using alias + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + indices.rollover: + alias: "write_manage_alias" + wait_for_active_shards: 1 + body: + conditions: + max_docs: 1 + + - match: { old_index: logs-000001 } + - match: { new_index: logs-000002 } + - match: { rolled_over: true } + - match: { dry_run: false } + - match: { conditions: { "[max_docs: 1]": true } } + + # ensure new index is created + - do: + indices.exists: + index: logs-000002 + + - is_true: '' + + # index using alias + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + create: + id: 3 + index: write_manage_alias + body: > + { + "name" : "doc3" + } + + - do: + indices.refresh: {} + + # check alias points to the new index and the doc was indexed + - do: + search: + rest_total_hits_as_int: true + index: write_manage_alias + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "logs-000002"} From 8ee91821a817afbca065af6bf88f698d467c0503 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Tue, 16 Apr 2019 13:02:10 +0200 Subject: [PATCH 048/260] BlendedTermQuery should ignore fields that don't exists in the index (#41125) Today the blended term query detects if a term exists in a field by looking at the term statistics in the index. However the value to indicate that a term has no occurence in a field have changed in Lucene. A non-existing term now returns a doc and total term frequency of 0. Because of this disrepancy the blended term query picks 0 as the minimum frequency for a term even if other fields have documents for this terms. This confuses the term queries that the blending creates since some of them contain a custom state that indicates a frequency of 0 even though the term has some occurence in the field. For these terms an exception is thrown because the term query always checks that the term state's frequency is greater than 0 if there are documents associate to it. This change fixes this bug by ignoring terms with a doc freq of 0 when the blended term query picks the minimum term frequency among the requested fields. Closes #41118 --- .../lucene/queries/BlendedTermQuery.java | 30 ++-------- .../lucene/queries/BlendedTermQueryTests.java | 59 +++++++++++++++++++ 2 files changed, 64 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java b/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java index dd3ac992475b9..1700979c32d64 100644 --- a/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java +++ b/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java @@ -113,23 +113,17 @@ protected void blend(final TermStates[] contexts, int maxDoc, IndexReader reader // TODO: Maybe it could also make sense to assume independent distributions of documents and eg. have: // df = df1 + df2 - (df1 * df2 / maxDoc)? max = Math.max(df, max); - if (minSumTTF != -1 && ctx.totalTermFreq() != -1) { + if (ctx.totalTermFreq() > 0) { // we need to find out the minimum sumTTF to adjust the statistics // otherwise the statistics don't match minSumTTF = Math.min(minSumTTF, reader.getSumTotalTermFreq(terms[i].field())); - } else { - minSumTTF = -1; } } - if (minSumTTF != -1 && maxDoc > minSumTTF) { - maxDoc = (int)minSumTTF; - } - if (max == 0) { return; // we are done that term doesn't exist at all } - long sumTTF = minSumTTF == -1 ? -1 : 0; + long sumTTF = 0; final int[] tieBreak = new int[contexts.length]; for (int i = 0; i < tieBreak.length; ++i) { tieBreak[i] = i; @@ -165,11 +159,7 @@ protected int compare(int i, int j) { } contexts[i] = ctx = adjustDF(reader.getContext(), ctx, Math.min(maxDoc, actualDf)); prev = current; - if (sumTTF >= 0 && ctx.totalTermFreq() >= 0) { - sumTTF += ctx.totalTermFreq(); - } else { - sumTTF = -1; // omit once TF is omitted anywhere! - } + sumTTF += ctx.totalTermFreq(); } sumTTF = Math.min(sumTTF, minSumTTF); for (int i = 0; i < contexts.length; i++) { @@ -177,17 +167,12 @@ protected int compare(int i, int j) { if (df == 0) { continue; } - // the blended sumTTF can't be greater than the sumTTTF on the field - final long fixedTTF = sumTTF == -1 ? -1 : sumTTF; - contexts[i] = adjustTTF(reader.getContext(), contexts[i], fixedTTF); + contexts[i] = adjustTTF(reader.getContext(), contexts[i], sumTTF); } } private TermStates adjustTTF(IndexReaderContext readerContext, TermStates termContext, long sumTTF) throws IOException { assert termContext.wasBuiltFor(readerContext); - if (sumTTF == -1 && termContext.totalTermFreq() == -1) { - return termContext; - } TermStates newTermContext = new TermStates(readerContext); List leaves = readerContext.leaves(); final int len; @@ -213,12 +198,7 @@ private TermStates adjustTTF(IndexReaderContext readerContext, TermStates termCo private static TermStates adjustDF(IndexReaderContext readerContext, TermStates ctx, int newDocFreq) throws IOException { assert ctx.wasBuiltFor(readerContext); // Use a value of ttf that is consistent with the doc freq (ie. gte) - long newTTF; - if (ctx.totalTermFreq() < 0) { - newTTF = -1; - } else { - newTTF = Math.max(ctx.totalTermFreq(), newDocFreq); - } + long newTTF = Math.max(ctx.totalTermFreq(), newDocFreq); List leaves = readerContext.leaves(); final int len; if (leaves == null) { diff --git a/server/src/test/java/org/apache/lucene/queries/BlendedTermQueryTests.java b/server/src/test/java/org/apache/lucene/queries/BlendedTermQueryTests.java index 1ad067a7e2b36..ce33c247a3337 100644 --- a/server/src/test/java/org/apache/lucene/queries/BlendedTermQueryTests.java +++ b/server/src/test/java/org/apache/lucene/queries/BlendedTermQueryTests.java @@ -28,10 +28,12 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.Term; +import org.apache.lucene.index.TermStates; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.DisjunctionMaxQuery; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryUtils; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreMode; @@ -52,6 +54,8 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; public class BlendedTermQueryTests extends ESTestCase { public void testDismaxQuery() throws IOException { @@ -114,6 +118,61 @@ public void testDismaxQuery() throws IOException { assertEquals(Integer.toString(1), reader.document(scoreDocs[0].doc).getField("id").stringValue()); } + { + // test with an unknown field + String[] fields = new String[] {"username", "song", "unknown_field"}; + Query query = BlendedTermQuery.dismaxBlendedQuery(toTerms(fields, "foo"), 1.0f); + Query rewrite = searcher.rewrite(query); + assertThat(rewrite, instanceOf(BooleanQuery.class)); + for (BooleanClause clause : (BooleanQuery) rewrite) { + assertThat(clause.getQuery(), instanceOf(TermQuery.class)); + TermQuery termQuery = (TermQuery) clause.getQuery(); + TermStates termStates = termQuery.getTermStates(); + if (termQuery.getTerm().field().equals("unknown_field")) { + assertThat(termStates.docFreq(), equalTo(0)); + assertThat(termStates.totalTermFreq(), equalTo(0L)); + } else { + assertThat(termStates.docFreq(), greaterThan(0)); + assertThat(termStates.totalTermFreq(), greaterThan(0L)); + } + } + assertThat(searcher.search(query, 10).totalHits.value, equalTo((long) iters + username.length)); + } + { + // test with an unknown field and an unknown term + String[] fields = new String[] {"username", "song", "unknown_field"}; + Query query = BlendedTermQuery.dismaxBlendedQuery(toTerms(fields, "unknown_term"), 1.0f); + Query rewrite = searcher.rewrite(query); + assertThat(rewrite, instanceOf(BooleanQuery.class)); + for (BooleanClause clause : (BooleanQuery) rewrite) { + assertThat(clause.getQuery(), instanceOf(TermQuery.class)); + TermQuery termQuery = (TermQuery) clause.getQuery(); + TermStates termStates = termQuery.getTermStates(); + assertThat(termStates.docFreq(), equalTo(0)); + assertThat(termStates.totalTermFreq(), equalTo(0L)); + } + assertThat(searcher.search(query, 10).totalHits.value, equalTo(0L)); + } + { + // test with an unknown field and a term that is present in only one field + String[] fields = new String[] {"username", "song", "id", "unknown_field"}; + Query query = BlendedTermQuery.dismaxBlendedQuery(toTerms(fields, "fan"), 1.0f); + Query rewrite = searcher.rewrite(query); + assertThat(rewrite, instanceOf(BooleanQuery.class)); + for (BooleanClause clause : (BooleanQuery) rewrite) { + assertThat(clause.getQuery(), instanceOf(TermQuery.class)); + TermQuery termQuery = (TermQuery) clause.getQuery(); + TermStates termStates = termQuery.getTermStates(); + if (termQuery.getTerm().field().equals("username")) { + assertThat(termStates.docFreq(), equalTo(1)); + assertThat(termStates.totalTermFreq(), equalTo(1L)); + } else { + assertThat(termStates.docFreq(), equalTo(0)); + assertThat(termStates.totalTermFreq(), equalTo(0L)); + } + } + assertThat(searcher.search(query, 10).totalHits.value, equalTo(1L)); + } reader.close(); w.close(); dir.close(); From 5708796e5131bac7724ea57d6290b6f688a9b47a Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 16 Apr 2019 13:03:55 +0100 Subject: [PATCH 049/260] Inline TransportReplAct#createReplicatedOperation (#41197) `TransportReplicationAction.AsyncPrimaryAction#createReplicatedOperation` exists so it can be overridden in tests. This commit re-works these tests to use a real `ReplicationOperation` and inlines the now-unnecessary method. Relates #40706. --- .../replication/ReplicationOperation.java | 2 +- .../TransportReplicationAction.java | 62 +++---- .../TransportReplicationActionTests.java | 161 +++++++----------- 3 files changed, 93 insertions(+), 132 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index e5c2136aae56d..7917d9c05078b 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -74,7 +74,7 @@ public class ReplicationOperation< private final long primaryTerm; // exposed for tests - final ActionListener resultListener; + private final ActionListener resultListener; private volatile PrimaryResultT primaryResult = null; diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index e9c071e5a0ed1..0c464d27e1957 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -357,37 +357,35 @@ public void handleException(TransportException exp) { }); } else { setPhase(replicationTask, "primary"); - createReplicatedOperation(primaryRequest.getRequest(), - ActionListener.wrap(result -> result.respond( - new ActionListener<>() { - @Override - public void onResponse(Response response) { - if (syncGlobalCheckpointAfterOperation) { - final IndexShard shard = primaryShardReference.indexShard; - try { - shard.maybeSyncGlobalCheckpoint("post-operation"); - } catch (final Exception e) { - // only log non-closed exceptions - if (ExceptionsHelper.unwrap( - e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { - // intentionally swallow, a missed global checkpoint sync should not fail this operation - logger.info( - new ParameterizedMessage( - "{} failed to execute post-operation global checkpoint sync", shard.shardId()), e); - } - } - } - primaryShardReference.close(); // release shard operation lock before responding to caller - setPhase(replicationTask, "finished"); - onCompletionListener.onResponse(response); - } - @Override - public void onFailure(Exception e) { - handleException(primaryShardReference, e); + final ActionListener referenceClosingListener = ActionListener.wrap(response -> { + primaryShardReference.close(); // release shard operation lock before responding to caller + setPhase(replicationTask, "finished"); + onCompletionListener.onResponse(response); + }, e -> handleException(primaryShardReference, e)); + + final ActionListener globalCheckpointSyncingListener = ActionListener.wrap(response -> { + if (syncGlobalCheckpointAfterOperation) { + final IndexShard shard = primaryShardReference.indexShard; + try { + shard.maybeSyncGlobalCheckpoint("post-operation"); + } catch (final Exception e) { + // only log non-closed exceptions + if (ExceptionsHelper.unwrap( + e, AlreadyClosedException.class, IndexShardClosedException.class) == null) { + // intentionally swallow, a missed global checkpoint sync should not fail this operation + logger.info( + new ParameterizedMessage( + "{} failed to execute post-operation global checkpoint sync", shard.shardId()), e); } - }), e -> handleException(primaryShardReference, e) - ), primaryShardReference).execute(); + } + } + referenceClosingListener.onResponse(response); + }, referenceClosingListener::onFailure); + + new ReplicationOperation<>(primaryRequest.getRequest(), primaryShardReference, + ActionListener.wrap(result -> result.respond(globalCheckpointSyncingListener), referenceClosingListener::onFailure), + newReplicasProxy(), logger, actionName, primaryRequest.getPrimaryTerm()).execute(); } } catch (Exception e) { handleException(primaryShardReference, e); @@ -405,12 +403,6 @@ public void onFailure(Exception e) { onCompletionListener.onFailure(e); } - protected ReplicationOperation> createReplicatedOperation( - Request request, ActionListener> listener, - PrimaryShardReference primaryShardReference) { - return new ReplicationOperation<>(request, primaryShardReference, listener, - newReplicasProxy(), logger, actionName, primaryRequest.getPrimaryTerm()); - } } public static class PrimaryResult, diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 2cdd3ad2fe480..ccb23a9111a4e 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -143,6 +143,7 @@ public static R resolveRequest(TransportRequest r if (requestOrWrappedRequest instanceof TransportReplicationAction.ConcreteShardRequest) { requestOrWrappedRequest = ((TransportReplicationAction.ConcreteShardRequest)requestOrWrappedRequest).getRequest(); } + //noinspection unchecked return (R) requestOrWrappedRequest; } @@ -209,7 +210,7 @@ private void setStateWithBlock(final ClusterService clusterService, final Cluste setState(clusterService, ClusterState.builder(clusterService.state()).blocks(blocks).build()); } - public void testBlocksInReroutePhase() throws Exception { + public void testBlocksInReroutePhase() { final ClusterBlock nonRetryableBlock = new ClusterBlock(1, "non retryable", false, true, false, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL); final ClusterBlock retryableBlock = @@ -290,7 +291,6 @@ public ClusterBlockLevel indexBlockLevel() { TestAction testActionWithNoBlocks = new TestAction(Settings.EMPTY, "internal:testActionWithNoBlocks", transportService, clusterService, shardStateAction, threadPool); - listener = new PlainActionFuture<>(); TestAction.ReroutePhase reroutePhase = testActionWithNoBlocks.new ReroutePhase(task, requestWithTimeout, listener); reroutePhase.run(); assertListenerThrows("should fail with an IndexNotFoundException when no blocks", listener, IndexNotFoundException.class); @@ -350,7 +350,7 @@ public void assertIndexShardUninitialized() { assertEquals(0, count.get()); } - public void testNotStartedPrimary() throws InterruptedException, ExecutionException { + public void testNotStartedPrimary() { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); // no replicas in oder to skip the replication part @@ -399,7 +399,7 @@ public void testNotStartedPrimary() throws InterruptedException, ExecutionExcept * This test checks that replication request is not routed back from relocation target to relocation source in case of * stale index routing table on relocation target. */ - public void testNoRerouteOnStaleClusterState() throws InterruptedException, ExecutionException { + public void testNoRerouteOnStaleClusterState() { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); ClusterState state = state(index, true, ShardRoutingState.RELOCATING); @@ -441,7 +441,7 @@ public void testNoRerouteOnStaleClusterState() throws InterruptedException, Exec assertIndexShardCounter(0); } - public void testUnknownIndexOrShardOnReroute() throws InterruptedException { + public void testUnknownIndexOrShardOnReroute() { final String index = "test"; // no replicas in oder to skip the replication part setState(clusterService, state(index, true, @@ -462,10 +462,9 @@ public void testUnknownIndexOrShardOnReroute() throws InterruptedException { reroutePhase.run(); assertListenerThrows("must throw shard not found exception", listener, ShardNotFoundException.class); assertFalse(request.isRetrySet.get()); //TODO I'd have expected this to be true but we fail too early? - } - public void testClosedIndexOnReroute() throws InterruptedException { + public void testClosedIndexOnReroute() { final String index = "test"; // no replicas in oder to skip the replication part ClusterStateChanges clusterStateChanges = new ClusterStateChanges(xContentRegistry(), threadPool); @@ -488,7 +487,7 @@ public void testClosedIndexOnReroute() throws InterruptedException { assertFalse(request.isRetrySet.get()); } - public void testStalePrimaryShardOnReroute() throws InterruptedException { + public void testStalePrimaryShardOnReroute() { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); // no replicas in order to skip the replication part @@ -596,23 +595,17 @@ public void testPrimaryPhaseExecutesOrDelegatesRequestToRelocationTarget() throw } final TransportReplicationAction.ConcreteShardRequest primaryRequest = new TransportReplicationAction.ConcreteShardRequest<>(request, primaryShard.allocationId().getId(), primaryTerm); - action.new AsyncPrimaryAction(primaryRequest, listener, task) { + + new TestAction(Settings.EMPTY, "internal:testAction2", transportService, clusterService, shardStateAction, threadPool) { @Override - protected ReplicationOperation> - createReplicatedOperation( - Request request, - ActionListener> actionListener, - TransportReplicationAction.PrimaryShardReference primaryShardReference) { - return new NoopReplicationOperation(request, actionListener, primaryTerm) { - @Override - public void execute() throws Exception { - assertPhase(task, "primary"); - assertFalse(executed.getAndSet(true)); - super.execute(); - } - }; + protected void shardOperationOnPrimary(Request shardRequest, IndexShard primary, + ActionListener> listener) { + assertPhase(task, "primary"); + assertFalse(executed.getAndSet(true)); + super.shardOperationOnPrimary(shardRequest, primary, listener); } - }.run(); + }.new AsyncPrimaryAction(primaryRequest, listener, task).run(); + if (executeOnPrimary) { assertTrue(executed.get()); assertTrue(listener.isDone()); @@ -626,9 +619,12 @@ public void execute() throws Exception { transport.capturedRequestsByTargetNode().get(primaryShard.relocatingNodeId()); assertThat(requests, notNullValue()); assertThat(requests.size(), equalTo(1)); - assertThat("primary request was not delegated to relocation target", requests.get(0).action, equalTo("internal:testAction[p]")); - assertThat("primary term not properly set on primary delegation", - ((TransportReplicationAction.ConcreteShardRequest)requests.get(0).request).getPrimaryTerm(), equalTo(primaryTerm)); + assertThat("primary request was not delegated to relocation target", + requests.get(0).action, equalTo("internal:testAction2[p]")); + //noinspection unchecked + final TransportReplicationAction.ConcreteShardRequest concreteShardRequest + = (TransportReplicationAction.ConcreteShardRequest) requests.get(0).request; + assertThat("primary term not properly set on primary delegation", concreteShardRequest.getPrimaryTerm(), equalTo(primaryTerm)); assertPhase(task, "primary_delegation"); transport.handleResponse(requests.get(0).requestId, new TestResponse()); assertTrue(listener.isDone()); @@ -638,7 +634,7 @@ public void execute() throws Exception { } } - public void testPrimaryPhaseExecutesDelegatedRequestOnRelocationTarget() throws Exception { + public void testPrimaryPhaseExecutesDelegatedRequestOnRelocationTarget() { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); ClusterState state = state(index, true, ShardRoutingState.RELOCATING); @@ -654,34 +650,24 @@ public void testPrimaryPhaseExecutesDelegatedRequestOnRelocationTarget() throws AtomicBoolean executed = new AtomicBoolean(); final TransportReplicationAction.ConcreteShardRequest primaryRequest = new TransportReplicationAction.ConcreteShardRequest<>(request, primaryShard.allocationId().getRelocationId(), primaryTerm); - action.new AsyncPrimaryAction(primaryRequest, listener, task) { - @Override - protected ReplicationOperation> - createReplicatedOperation( - Request request, - ActionListener> actionListener, - TransportReplicationAction.PrimaryShardReference primaryShardReference) { - return new NoopReplicationOperation(request, actionListener, primaryTerm) { - @Override - public void execute() throws Exception { - assertPhase(task, "primary"); - assertFalse(executed.getAndSet(true)); - super.execute(); - } - }; - } + new TestAction(Settings.EMPTY, "internal:testAction2", transportService, clusterService, shardStateAction, threadPool) { @Override - public void onFailure(Exception e) { - throw new RuntimeException(e); + protected void shardOperationOnPrimary(Request shardRequest, IndexShard primary, + ActionListener> listener) { + assertPhase(task, "primary"); + assertFalse(executed.getAndSet(true)); + super.shardOperationOnPrimary(shardRequest, primary, listener); } - }.run(); + }.new AsyncPrimaryAction(primaryRequest, listener, task).run(); assertThat(executed.get(), equalTo(true)); assertPhase(task, "finished"); assertFalse(request.isRetrySet.get()); + assertTrue(listener.isDone()); + listener.actionGet(); // throws no exception } - public void testPrimaryReference() throws Exception { + public void testPrimaryReference() { final IndexShard shard = mock(IndexShard.class); AtomicBoolean closed = new AtomicBoolean(); @@ -789,6 +775,7 @@ public void testSeqNoIsSetOnPrimary() { inSyncIds, shardRoutingTable.getAllAllocationIds())); doAnswer(invocation -> { + //noinspection unchecked ((ActionListener)invocation.getArguments()[0]).onResponse(() -> {}); return null; }).when(shard).acquirePrimaryOperationPermit(any(), anyString(), anyObject()); @@ -805,6 +792,7 @@ public void testSeqNoIsSetOnPrimary() { action.handlePrimaryRequest(concreteShardRequest, createTransportChannel(listener), null); CapturingTransport.CapturedRequest[] requestsToReplicas = transport.capturedRequests(); assertThat(requestsToReplicas, arrayWithSize(1)); + //noinspection unchecked assertThat(((TransportReplicationAction.ConcreteShardRequest) requestsToReplicas[0].request).getPrimaryTerm(), equalTo(primaryTerm)); } @@ -821,47 +809,38 @@ public void testCounterOnPrimary() throws Exception { Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); ReplicationTask task = maybeTask(); - int i = randomInt(3); - final boolean throwExceptionOnCreation = i == 1; - final boolean throwExceptionOnRun = i == 2; - final boolean respondWithError = i == 3; + int i = randomInt(2); + final boolean throwExceptionOnRun = i == 1; + final boolean respondWithError = i == 2; final TransportReplicationAction.ConcreteShardRequest primaryRequest = new TransportReplicationAction.ConcreteShardRequest<>(request, primaryShard.allocationId().getId(), primaryTerm); - action.new AsyncPrimaryAction(primaryRequest, listener, task) { + + new TestAction(Settings.EMPTY, "internal:testAction2", transportService, clusterService, shardStateAction, threadPool) { @Override - protected ReplicationOperation> - createReplicatedOperation( - Request request, - ActionListener> actionListener, - TransportReplicationAction.PrimaryShardReference primaryShardReference) { + protected void shardOperationOnPrimary(Request shardRequest, IndexShard primary, + ActionListener> listener) { assertIndexShardCounter(1); - if (throwExceptionOnCreation) { - throw new ElasticsearchException("simulated exception, during createReplicatedOperation"); + if (throwExceptionOnRun) { + throw new ElasticsearchException("simulated exception, during shardOperationOnPrimary"); + } else if (respondWithError) { + listener.onFailure(new ElasticsearchException("simulated exception, as a response")); + } else { + super.shardOperationOnPrimary(request, primary, listener); } - return new NoopReplicationOperation(request, actionListener, primaryTerm) { - @Override - public void execute() throws Exception { - assertIndexShardCounter(1); - assertPhase(task, "primary"); - if (throwExceptionOnRun) { - throw new ElasticsearchException("simulated exception, during performOnPrimary"); - } else if (respondWithError) { - this.resultListener.onFailure(new ElasticsearchException("simulated exception, as a response")); - } else { - super.execute(); - } - } - }; } - }.run(); + }.new AsyncPrimaryAction(primaryRequest, listener, task).run(); + assertIndexShardCounter(0); assertTrue(listener.isDone()); assertPhase(task, "finished"); try { listener.get(); + if (throwExceptionOnRun || respondWithError) { + fail("expected exception, but none was thrown"); + } } catch (ExecutionException e) { - if (throwExceptionOnCreation || throwExceptionOnRun || respondWithError) { + if (throwExceptionOnRun || respondWithError) { Throwable cause = e.getCause(); assertThat(cause, instanceOf(ElasticsearchException.class)); assertThat(cause.getMessage(), containsString("simulated")); @@ -871,7 +850,7 @@ public void execute() throws Exception { } } - public void testReplicasCounter() throws Exception { + public void testReplicasCounter() { final ShardId shardId = new ShardId("test", "_na_", 0); final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); setState(clusterService, state); @@ -909,7 +888,7 @@ protected ReplicaResult shardOperationOnReplica(Request request, IndexShard repl * This test ensures that replication operations adhere to the {@link IndexMetaData#SETTING_WAIT_FOR_ACTIVE_SHARDS} setting * when the request is using the default value for waitForActiveShards. */ - public void testDefaultWaitForActiveShardsUsesIndexSetting() throws Exception { + public void testDefaultWaitForActiveShardsUsesIndexSetting() { final String indexName = "test"; final ShardId shardId = new ShardId(indexName, "_na_", 0); @@ -1167,9 +1146,9 @@ private void assertPhase(@Nullable ReplicationTask task, Matcher phaseMa } public static class Request extends ReplicationRequest { - public AtomicBoolean processedOnPrimary = new AtomicBoolean(); - public AtomicInteger processedOnReplicas = new AtomicInteger(); - public AtomicBoolean isRetrySet = new AtomicBoolean(false); + AtomicBoolean processedOnPrimary = new AtomicBoolean(); + AtomicInteger processedOnReplicas = new AtomicInteger(); + AtomicBoolean isRetrySet = new AtomicBoolean(false); Request(StreamInput in) throws IOException { super(in); @@ -1284,6 +1263,7 @@ private IndexService mockIndexService(final IndexMetaData indexMetaData, Cluster return indexService; } + @SuppressWarnings("unchecked") private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService) { final IndexShard indexShard = mock(IndexShard.class); when(indexShard.shardId()).thenReturn(shardId); @@ -1319,21 +1299,10 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); when(indexShard.getPendingPrimaryTerm()).thenAnswer(i -> clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); - return indexShard; - } - - class NoopReplicationOperation extends ReplicationOperation> { - - NoopReplicationOperation(Request request, ActionListener> listener, - long primaryTerm) { - super(request, null, listener, null, TransportReplicationActionTests.this.logger, "noop", primaryTerm); - } - @Override - public void execute() throws Exception { - // Using the diamond operator (<>) prevents Eclipse from being able to compile this code - this.resultListener.onResponse(new TransportReplicationAction.PrimaryResult(null, new TestResponse())); - } + ReplicationGroup replicationGroup = mock(ReplicationGroup.class); + when(indexShard.getReplicationGroup()).thenReturn(replicationGroup); + return indexShard; } /** @@ -1348,12 +1317,12 @@ public String getProfileName() { } @Override - public void sendResponse(TransportResponse response) throws IOException { + public void sendResponse(TransportResponse response) { listener.onResponse(((TestResponse) response)); } @Override - public void sendResponse(Exception exception) throws IOException { + public void sendResponse(Exception exception) { listener.onFailure(exception); } From b53554c8058d4f626b9afdd4e8ae48af506dc9e7 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Tue, 16 Apr 2019 13:43:00 +0100 Subject: [PATCH 050/260] [ML] Write header to autodetect before it is visible to other calls (#41085) --- .../process/autodetect/AutodetectCommunicator.java | 14 ++++++++++++-- .../autodetect/AutodetectProcessManager.java | 7 ++++--- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicator.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicator.java index b3f765d89ce1a..7e778e48524ba 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicator.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectCommunicator.java @@ -90,9 +90,8 @@ public class AutodetectCommunicator implements Closeable { && job.getAnalysisConfig().getCategorizationFieldName() != null; } - public void init(ModelSnapshot modelSnapshot) throws IOException { + public void restoreState(ModelSnapshot modelSnapshot) { autodetectProcess.restoreState(stateStreamer, modelSnapshot); - createProcessWriter(Optional.empty()).writeHeader(); } private DataToProcessWriter createProcessWriter(Optional dataDescription) { @@ -101,6 +100,17 @@ private DataToProcessWriter createProcessWriter(Optional dataDe dataCountsReporter, xContentRegistry); } + /** + * This must be called once before {@link #writeToJob(InputStream, AnalysisRegistry, XContentType, DataLoadParams, BiConsumer)} + * can be used + */ + public void writeHeader() throws IOException { + createProcessWriter(Optional.empty()).writeHeader(); + } + + /** + * Call {@link #writeHeader()} exactly once before using this method + */ public void writeToJob(InputStream inputStream, AnalysisRegistry analysisRegistry, XContentType xContentType, DataLoadParams params, BiConsumer handler) { submitOperation(() -> { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index eb387bfa5a235..1e35530fe1735 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -458,7 +458,7 @@ protected void doRun() { try { createProcessAndSetRunning(processContext, job, params, closeHandler); - processContext.getAutodetectCommunicator().init(params.modelSnapshot()); + processContext.getAutodetectCommunicator().restoreState(params.modelSnapshot()); setJobState(jobTask, JobState.OPENED); } catch (Exception e1) { // No need to log here as the persistent task framework will log it @@ -499,7 +499,7 @@ protected void doRun() { private void createProcessAndSetRunning(ProcessContext processContext, Job job, AutodetectParams params, - BiConsumer handler) { + BiConsumer handler) throws IOException { // At this point we lock the process context until the process has been started. // The reason behind this is to ensure closing the job does not happen before // the process is started as that can result to the job getting seemingly closed @@ -507,6 +507,7 @@ private void createProcessAndSetRunning(ProcessContext processContext, processContext.tryLock(); try { AutodetectCommunicator communicator = create(processContext.getJobTask(), job, params, handler); + communicator.writeHeader(); processContext.setRunning(communicator); } finally { // Now that the process is running and we have updated its state we can unlock. @@ -639,7 +640,7 @@ public void closeJob(JobTask jobTask, boolean restart, String reason) { processContext.tryLock(); try { if (processContext.setDying() == false) { - logger.debug("Cannot close job [{}] as it has already been closed", jobId); + logger.debug("Cannot close job [{}] as it has been marked as dying", jobId); return; } From 00139ae8ae195a5487d2509611cb8e0bfe0ebb9b Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Tue, 16 Apr 2019 16:44:17 +0200 Subject: [PATCH 051/260] Mute TestClustersPluginIT.testMultiNode (#41257) Tracked in #41256 --- .../elasticsearch/gradle/testclusters/TestClustersPluginIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java b/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java index 7e2915e234142..b06bd067edde8 100644 --- a/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java @@ -21,6 +21,7 @@ import org.elasticsearch.gradle.test.GradleIntegrationTestCase; import org.gradle.testkit.runner.BuildResult; import org.gradle.testkit.runner.GradleRunner; +import org.junit.Ignore; import java.util.Arrays; @@ -150,6 +151,7 @@ public void testConfigurationLocked() { ); } + @Ignore // https://github.com/elastic/elasticsearch/issues/41256 public void testMultiNode() { BuildResult result = getTestClustersRunner(":multiNode").build(); assertTaskSuccessful(result, ":multiNode"); From dac6c47aa1064b26cd1d6d16a9d4ea65a0ef51ad Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 16 Apr 2019 16:45:00 +0200 Subject: [PATCH 052/260] Some Minor Cleanups in o.e.snapshots (#40962) * Some Minor Cleanups in o.e.snapshots * Some minor obvious cleanups that became possible now tht we're on JDK11 * Removing redundant `Updates` internal class --- .../restore/RestoreSnapshotResponse.java | 2 +- .../elasticsearch/snapshots/RestoreInfo.java | 11 ---- .../snapshots/RestoreService.java | 51 ++++++++----------- .../snapshots/SnapshotException.java | 6 +-- .../elasticsearch/snapshots/SnapshotInfo.java | 2 +- .../snapshots/SnapshotShardFailure.java | 12 +---- .../snapshots/SnapshotShardsService.java | 4 +- .../snapshots/SnapshotUtils.java | 6 +-- .../snapshots/SnapshotsService.java | 12 ++--- ...etadataLoadingDuringSnapshotRestoreIT.java | 3 -- 10 files changed, 35 insertions(+), 74 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java index 171509c018228..ef661838da476 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotResponse.java @@ -77,7 +77,7 @@ public RestStatus status() { if (restoreInfo == null) { return RestStatus.ACCEPTED; } - return restoreInfo.status(); + return RestStatus.OK; } @Override diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java index bc87c49dcca7a..f76a8fe3e88fe 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreInfo.java @@ -27,7 +27,6 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.rest.RestStatus; import java.io.IOException; import java.util.ArrayList; @@ -51,7 +50,6 @@ public class RestoreInfo implements ToXContentObject, Streamable { private int successfulShards; RestoreInfo() { - } public RestoreInfo(String name, List indices, int totalShards, int successfulShards) { @@ -106,15 +104,6 @@ public int successfulShards() { return successfulShards; } - /** - * REST status of the operation - * - * @return REST status - */ - public RestStatus status() { - return RestStatus.OK; - } - static final class Fields { static final String SNAPSHOT = "snapshot"; static final String INDICES = "indices"; diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index cddf7d85dc7b3..3a81a9956870a 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -163,7 +163,7 @@ public RestoreService(ClusterService clusterService, RepositoriesService reposit this.metaDataIndexUpgradeService = metaDataIndexUpgradeService; clusterService.addStateApplier(this); this.clusterSettings = clusterSettings; - this.cleanRestoreStateTaskExecutor = new CleanRestoreStateTaskExecutor(logger); + this.cleanRestoreStateTaskExecutor = new CleanRestoreStateTaskExecutor(); } /** @@ -221,7 +221,7 @@ public void restoreSnapshot(final RestoreSnapshotRequest request, final ActionLi // Now we can start the actual restore process by adding shards to be recovered in the cluster state // and updating cluster metadata (global and index) as needed clusterService.submitStateUpdateTask("restore_snapshot[" + snapshotName + ']', new ClusterStateUpdateTask() { - String restoreUUID = UUIDs.randomBase64UUID(); + final String restoreUUID = UUIDs.randomBase64UUID(); RestoreInfo restoreInfo = null; @Override @@ -354,7 +354,7 @@ public ClusterState execute(ClusterState currentState) { shards = shardsBuilder.build(); RestoreInProgress.Entry restoreEntry = new RestoreInProgress.Entry( restoreUUID, snapshot, overallState(RestoreInProgress.State.INIT, shards), - Collections.unmodifiableList(new ArrayList<>(indices.keySet())), + List.copyOf(indices.keySet()), shards ); RestoreInProgress.Builder restoreInProgressBuilder; @@ -397,7 +397,7 @@ restoreUUID, snapshot, overallState(RestoreInProgress.State.INIT, shards), if (completed(shards)) { // We don't have any indices to restore - we are done restoreInfo = new RestoreInfo(snapshotId.getName(), - Collections.unmodifiableList(new ArrayList<>(indices.keySet())), + List.copyOf(indices.keySet()), shards.size(), shards.size() - failedShards(shards)); } @@ -595,7 +595,8 @@ public RestoreInfo getRestoreInfo() { } public static class RestoreInProgressUpdater extends RoutingChangesObserver.AbstractRoutingChangesObserver { - private final Map shardChanges = new HashMap<>(); + // Map of RestoreUUID to a of changes to the shards' restore statuses + private final Map> shardChanges = new HashMap<>(); @Override public void shardStarted(ShardRouting initializingShard, ShardRouting startedShard) { @@ -603,7 +604,7 @@ public void shardStarted(ShardRouting initializingShard, ShardRouting startedSha if (initializingShard.primary()) { RecoverySource recoverySource = initializingShard.recoverySource(); if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) { - changes(recoverySource).shards.put( + changes(recoverySource).put( initializingShard.shardId(), new ShardRestoreStatus(initializingShard.currentNodeId(), RestoreInProgress.State.SUCCESS)); } @@ -619,7 +620,7 @@ public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) // to restore this shard on another node if the snapshot files are corrupt. In case where a node just left or crashed, // however, we only want to acknowledge the restore operation once it has been successfully restored on another node. if (unassignedInfo.getFailure() != null && Lucene.isCorruptionException(unassignedInfo.getFailure().getCause())) { - changes(recoverySource).shards.put( + changes(recoverySource).put( failedShard.shardId(), new ShardRestoreStatus(failedShard.currentNodeId(), RestoreInProgress.State.FAILURE, unassignedInfo.getFailure().getCause().getMessage())); } @@ -632,7 +633,7 @@ public void shardInitialized(ShardRouting unassignedShard, ShardRouting initiali // if we force an empty primary, we should also fail the restore entry if (unassignedShard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT && initializedShard.recoverySource().getType() != RecoverySource.Type.SNAPSHOT) { - changes(unassignedShard.recoverySource()).shards.put( + changes(unassignedShard.recoverySource()).put( unassignedShard.shardId(), new ShardRestoreStatus(null, RestoreInProgress.State.FAILURE, "recovery source type changed from snapshot to " + initializedShard.recoverySource()) @@ -646,7 +647,7 @@ public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo n if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) { if (newUnassignedInfo.getLastAllocationStatus() == UnassignedInfo.AllocationStatus.DECIDERS_NO) { String reason = "shard could not be allocated to any of the nodes"; - changes(recoverySource).shards.put( + changes(recoverySource).put( unassignedShard.shardId(), new ShardRestoreStatus(unassignedShard.currentNodeId(), RestoreInProgress.State.FAILURE, reason)); } @@ -657,24 +658,20 @@ public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo n * Helper method that creates update entry for the given recovery source's restore uuid * if such an entry does not exist yet. */ - private Updates changes(RecoverySource recoverySource) { + private Map changes(RecoverySource recoverySource) { assert recoverySource.getType() == RecoverySource.Type.SNAPSHOT; - return shardChanges.computeIfAbsent(((SnapshotRecoverySource) recoverySource).restoreUUID(), k -> new Updates()); - } - - private static class Updates { - private Map shards = new HashMap<>(); + return shardChanges.computeIfAbsent(((SnapshotRecoverySource) recoverySource).restoreUUID(), k -> new HashMap<>()); } public RestoreInProgress applyChanges(final RestoreInProgress oldRestore) { if (shardChanges.isEmpty() == false) { RestoreInProgress.Builder builder = new RestoreInProgress.Builder(); for (RestoreInProgress.Entry entry : oldRestore) { - Updates updates = shardChanges.get(entry.uuid()); + Map updates = shardChanges.get(entry.uuid()); ImmutableOpenMap shardStates = entry.shards(); - if (updates != null && updates.shards.isEmpty() == false) { + if (updates != null && updates.isEmpty() == false) { ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(shardStates); - for (Map.Entry shard : updates.shards.entrySet()) { + for (Map.Entry shard : updates.entrySet()) { ShardId shardId = shard.getKey(); ShardRestoreStatus status = shardStates.get(shardId); if (status == null || status.state().completed() == false) { @@ -720,14 +717,8 @@ public String toString() { } } - private final Logger logger; - - CleanRestoreStateTaskExecutor(Logger logger) { - this.logger = logger; - } - @Override - public ClusterTasksResult execute(final ClusterState currentState, final List tasks) throws Exception { + public ClusterTasksResult execute(final ClusterState currentState, final List tasks) { final ClusterTasksResult.Builder resultBuilder = ClusterTasksResult.builder().successes(tasks); Set completedRestores = tasks.stream().map(e -> e.uuid).collect(Collectors.toSet()); RestoreInProgress.Builder restoreInProgressBuilder = new RestoreInProgress.Builder(); @@ -782,8 +773,8 @@ private void cleanupRestoreState(ClusterChangedEvent event) { } } - public static RestoreInProgress.State overallState(RestoreInProgress.State nonCompletedState, - ImmutableOpenMap shards) { + private static RestoreInProgress.State overallState(RestoreInProgress.State nonCompletedState, + ImmutableOpenMap shards) { boolean hasFailed = false; for (ObjectCursor status : shards.values()) { if (!status.value.state().completed()) { @@ -819,7 +810,7 @@ public static int failedShards(ImmutableOpenMap renamedIndices(RestoreSnapshotRequest request, List filteredIndices) { + private static Map renamedIndices(RestoreSnapshotRequest request, List filteredIndices) { Map renamedIndices = new HashMap<>(); for (String index : filteredIndices) { String renamedIndex = index; @@ -841,7 +832,7 @@ private Map renamedIndices(RestoreSnapshotRequest request, List< * @param repository repository name * @param snapshotInfo snapshot metadata */ - private void validateSnapshotRestorable(final String repository, final SnapshotInfo snapshotInfo) { + private static void validateSnapshotRestorable(final String repository, final SnapshotInfo snapshotInfo) { if (!snapshotInfo.state().restorable()) { throw new SnapshotRestoreException(new Snapshot(repository, snapshotInfo.snapshotId()), "unsupported snapshot state [" + snapshotInfo.state() + "]"); @@ -853,7 +844,7 @@ private void validateSnapshotRestorable(final String repository, final SnapshotI } } - private boolean failed(SnapshotInfo snapshot, String index) { + private static boolean failed(SnapshotInfo snapshot, String index) { for (SnapshotShardFailure failure : snapshot.shardFailures()) { if (index.equals(failure.index())) { return true; diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java index 05db85d6f7211..58c31e2639e3a 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java @@ -58,11 +58,7 @@ public SnapshotException(final String repositoryName, final SnapshotId snapshotI } public SnapshotException(final String repositoryName, final String snapshotName, final String msg) { - this(repositoryName, snapshotName, msg, null); - } - - public SnapshotException(final String repositoryName, final String snapshotName, final String msg, final Throwable cause) { - super("[" + repositoryName + ":" + snapshotName + "] " + msg, cause); + super("[" + repositoryName + ":" + snapshotName + "] " + msg); this.repositoryName = repositoryName; this.snapshotName = snapshotName; } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index ca743f77aa3ba..38aa945bcca47 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -221,7 +221,7 @@ int getSuccessfulShards() { private final int successfulShards; @Nullable - private Boolean includeGlobalState; + private final Boolean includeGlobalState; @Nullable private final Version version; diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java index 10e92b617d353..28202fa2c0b97 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardFailure.java @@ -155,16 +155,6 @@ private static SnapshotShardFailure constructSnapshotShardFailure(Object[] args) ShardId shardId = new ShardId(index, indexUuid != null ? indexUuid : IndexMetaData.INDEX_UUID_NA_VALUE, intShardId); - // Workaround for https://github.com/elastic/elasticsearch/issues/25878 - // Some old snapshot might still have null in shard failure reasons - String nonNullReason; - if (reason != null) { - nonNullReason = reason; - } else { - nonNullReason = ""; - } - - RestStatus restStatus; if (status != null) { restStatus = RestStatus.valueOf(status); @@ -172,7 +162,7 @@ private static SnapshotShardFailure constructSnapshotShardFailure(Object[] args) restStatus = RestStatus.INTERNAL_SERVER_ERROR; } - return new SnapshotShardFailure(nodeId, shardId, nonNullReason, restStatus); + return new SnapshotShardFailure(nodeId, shardId, reason, restStatus); } /** diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index fbb0a876e8f29..959faac70b88c 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -492,7 +492,7 @@ private void notifyFailedSnapshotShard(final Snapshot snapshot, final ShardId sh void sendSnapshotShardUpdate(final Snapshot snapshot, final ShardId shardId, final ShardSnapshotStatus status) { remoteFailedRequestDeduplicator.executeOnce( new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status), - new ActionListener() { + new ActionListener<>() { @Override public void onResponse(Void aVoid) { logger.trace("[{}] [{}] updated snapshot state", snapshot, status); @@ -557,7 +557,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS }); } - private class SnapshotStateExecutor implements ClusterStateTaskExecutor { + private static class SnapshotStateExecutor implements ClusterStateTaskExecutor { @Override public ClusterTasksResult diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java index 616e5ffecc814..0c5e92d782113 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java @@ -23,9 +23,7 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.index.IndexNotFoundException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -116,8 +114,8 @@ public static List filterIndices(List availableIndices, String[] } } if (result == null) { - return Collections.unmodifiableList(new ArrayList<>(Arrays.asList(selectedIndices))); + return List.of(selectedIndices); } - return Collections.unmodifiableList(new ArrayList<>(result)); + return List.copyOf(result); } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 257a0b776924a..1559bae8259b0 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -312,7 +312,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, final Cl if (newSnapshot != null) { final Snapshot current = newSnapshot.snapshot(); assert initializingSnapshots.contains(current); - beginSnapshot(newState, newSnapshot, request.partial(), new ActionListener() { + beginSnapshot(newState, newSnapshot, request.partial(), new ActionListener<>() { @Override public void onResponse(final Snapshot snapshot) { initializingSnapshots.remove(snapshot); @@ -342,7 +342,7 @@ public TimeValue timeout() { * @param snapshotName snapshot name * @param state current cluster state */ - private void validate(String repositoryName, String snapshotName, ClusterState state) { + private static void validate(String repositoryName, String snapshotName, ClusterState state) { RepositoriesMetaData repositoriesMetaData = state.getMetaData().custom(RepositoriesMetaData.TYPE); if (repositoriesMetaData == null || repositoriesMetaData.repository(repositoryName) == null) { throw new RepositoryMissingException(repositoryName); @@ -797,7 +797,7 @@ public ClusterState execute(ClusterState currentState) { entries.add(updatedSnapshot); // Clean up the snapshot that failed to start from the old master - deleteSnapshot(snapshot.snapshot(), new ActionListener() { + deleteSnapshot(snapshot.snapshot(), new ActionListener<>() { @Override public void onResponse(Void aVoid) { logger.debug("cleaned up abandoned snapshot {} in INIT state", snapshot.snapshot()); @@ -938,7 +938,7 @@ private static boolean removedNodesCleanupNeeded(SnapshotsInProgress snapshotsIn * @param shards list of shard statuses * @return list of failed and closed indices */ - private Tuple, Set> indicesWithMissingShards( + private static Tuple, Set> indicesWithMissingShards( ImmutableOpenMap shards, MetaData metaData) { Set missing = new HashSet<>(); Set closed = new HashSet<>(); @@ -1140,7 +1140,7 @@ private void deleteSnapshot(final Snapshot snapshot, final ActionListener boolean waitForSnapshot = false; @Override - public ClusterState execute(ClusterState currentState) throws Exception { + public ClusterState execute(ClusterState currentState) { SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(SnapshotDeletionsInProgress.TYPE); if (deletionsInProgress != null && deletionsInProgress.hasDeletionsInProgress()) { throw new ConcurrentSnapshotExecutionException(snapshot, @@ -1310,7 +1310,7 @@ public static boolean isRepositoryInUse(ClusterState clusterState, String reposi * @param repositoryStateId the unique id representing the state of the repository at the time the deletion began */ private void deleteSnapshotFromRepository(Snapshot snapshot, @Nullable ActionListener listener, long repositoryStateId) { - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable(listener) { + threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable<>(listener) { @Override protected void doRun() { Repository repository = repositoriesService.repository(snapshot.getRepository()); diff --git a/server/src/test/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java index 040f12c956696..aeea321b8536b 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java @@ -109,7 +109,6 @@ public void testWhenMetadataAreLoaded() throws Exception { .setWaitForCompletion(true) .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().status(), equalTo(RestStatus.OK)); assertGlobalMetadataLoads("snap", 0); assertIndexMetadataLoads("snap", "docs", 2); assertIndexMetadataLoads("snap", "others", 2); @@ -122,7 +121,6 @@ public void testWhenMetadataAreLoaded() throws Exception { .setWaitForCompletion(true) .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().status(), equalTo(RestStatus.OK)); assertGlobalMetadataLoads("snap", 0); assertIndexMetadataLoads("snap", "docs", 3); assertIndexMetadataLoads("snap", "others", 2); @@ -136,7 +134,6 @@ public void testWhenMetadataAreLoaded() throws Exception { .setWaitForCompletion(true) .get(); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); - assertThat(restoreSnapshotResponse.getRestoreInfo().status(), equalTo(RestStatus.OK)); assertGlobalMetadataLoads("snap", 1); assertIndexMetadataLoads("snap", "docs", 4); assertIndexMetadataLoads("snap", "others", 3); From 8dbdd0688638ac2759da4e5026a3ceb0ac196048 Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Tue, 16 Apr 2019 16:46:54 +0200 Subject: [PATCH 053/260] Reindex from Remote encoding (#41007) Removed the leniency when encoding remote reindex search requests that was introduced in 7.x. All index-names are now encoded before being sent to the remote host. Follow-up to #40303 --- docs/reference/migration/migrate_8_0.asciidoc | 2 ++ docs/reference/migration/migrate_8_0/reindex.asciidoc | 10 ++++++++++ .../index/reindex/remote/RemoteRequestBuilders.java | 4 ---- .../reindex/remote/RemoteRequestBuildersTests.java | 10 ++++------ 4 files changed, 16 insertions(+), 10 deletions(-) create mode 100644 docs/reference/migration/migrate_8_0/reindex.asciidoc diff --git a/docs/reference/migration/migrate_8_0.asciidoc b/docs/reference/migration/migrate_8_0.asciidoc index 11502be569076..ed40dddaae28e 100644 --- a/docs/reference/migration/migrate_8_0.asciidoc +++ b/docs/reference/migration/migrate_8_0.asciidoc @@ -22,6 +22,7 @@ coming[8.0.0] * <> * <> * <> +* <> //NOTE: The notable-breaking-changes tagged regions are re-used in the //Installation and Upgrade Guide @@ -55,3 +56,4 @@ include::migrate_8_0/java.asciidoc[] include::migrate_8_0/network.asciidoc[] include::migrate_8_0/transport.asciidoc[] include::migrate_8_0/http.asciidoc[] +include::migrate_8_0/reindex.asciidoc[] diff --git a/docs/reference/migration/migrate_8_0/reindex.asciidoc b/docs/reference/migration/migrate_8_0/reindex.asciidoc new file mode 100644 index 0000000000000..ebba0f2aebe1d --- /dev/null +++ b/docs/reference/migration/migrate_8_0/reindex.asciidoc @@ -0,0 +1,10 @@ +[float] +[[breaking_80_reindex_changes]] +=== Reindex changes + +Reindex from remote would previously allow URL encoded index-names and not +re-encode them when generating the search request for the remote host. This +leniency has been removed such that all index-names are correctly encoded when +reindex generates remote search requests. + +Instead, please specify the index-name without any encoding. \ No newline at end of file diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java index 2423de5fd704a..90332c7d55c9c 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuilders.java @@ -171,10 +171,6 @@ private static void addIndices(StringBuilder path, String[] indices) { } private static String encodeIndex(String s) { - if (s.contains("%")) { // already encoded, pass-through to allow this in mixed version clusters - checkIndexOrType("Index", s); - return s; - } try { return URLEncoder.encode(s, "utf-8"); } catch (UnsupportedEncodingException e) { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java index eb6192a043160..bf6856754044d 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/remote/RemoteRequestBuildersTests.java @@ -79,15 +79,13 @@ public void testIntialSearchPath() { assertEquals("/%3Ccat%7Bnow%2Fd%7D%3E,%3C%3E%2F%7B%7D%7C%2B%3A%2C/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); - // pass-through if already escaped. + // re-escape already escaped (no special handling). searchRequest.indices("%2f", "%3a"); - assertEquals("/%2f,%3a/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); - - // do not allow , and / if already escaped. + assertEquals("/%252f,%253a/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("%2fcat,"); - expectBadStartRequest(searchRequest, "Index", ",", "%2fcat,"); + assertEquals("/%252fcat%2C/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("%3ccat/"); - expectBadStartRequest(searchRequest, "Index", "/", "%3ccat/"); + assertEquals("/%253ccat%2F/c,d/_search", initialSearch(searchRequest, query, remoteVersion).getEndpoint()); searchRequest.indices("ok"); searchRequest.types("cat,"); From 9fdd0dd368e0832ccc7b17836cbacee2121f331f Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 16 Apr 2019 16:01:22 +0100 Subject: [PATCH 054/260] Test that join validation checks the cluster UUID (#41250) This is a forward-port of parts of #41063 to `master`, adding a test to show that join validation does indeed verify that the cluster UUIDs match. Relates #37775 --- .../cluster/coordination/JoinHelperTests.java | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java index 877d2a5a487d7..08501c4f55326 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java @@ -20,12 +20,18 @@ import org.apache.logging.log4j.Level; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.NotMasterException; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.test.transport.CapturingTransport.CapturedRequest; +import org.elasticsearch.test.transport.MockTransport; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportResponse; @@ -35,6 +41,7 @@ import java.util.Optional; import static org.elasticsearch.node.Node.NODE_NAME_SETTING; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.core.Is.is; @@ -131,4 +138,40 @@ public void testFailedJoinAttemptLogLevel() { new RemoteTransportException("caused by NotMasterException", new NotMasterException("test"))), is(Level.DEBUG)); } + + public void testJoinValidationRejectsMismatchedClusterUUID() { + DeterministicTaskQueue deterministicTaskQueue = new DeterministicTaskQueue( + Settings.builder().put(NODE_NAME_SETTING.getKey(), "node0").build(), random()); + MockTransport mockTransport = new MockTransport(); + DiscoveryNode localNode = new DiscoveryNode("node0", buildNewFakeTransportAddress(), Version.CURRENT); + + final ClusterState localClusterState = ClusterState.builder(ClusterName.DEFAULT).metaData(MetaData.builder() + .generateClusterUuidIfNeeded().clusterUUIDCommitted(true)).build(); + + TransportService transportService = mockTransport.createTransportService(Settings.EMPTY, + deterministicTaskQueue.getThreadPool(), TransportService.NOOP_TRANSPORT_INTERCEPTOR, + x -> localNode, null, Collections.emptySet()); + new JoinHelper(Settings.EMPTY, null, null, transportService, () -> 0L, () -> localClusterState, + (joinRequest, joinCallback) -> { throw new AssertionError(); }, startJoinRequest -> { throw new AssertionError(); }, + Collections.emptyList()); // registers request handler + transportService.start(); + transportService.acceptIncomingRequests(); + + final ClusterState otherClusterState = ClusterState.builder(ClusterName.DEFAULT).metaData(MetaData.builder() + .generateClusterUuidIfNeeded()).build(); + + final PlainActionFuture future = new PlainActionFuture<>(); + transportService.sendRequest(localNode, JoinHelper.VALIDATE_JOIN_ACTION_NAME, + new ValidateJoinRequest(otherClusterState), + new ActionListenerResponseHandler<>(future, in -> TransportResponse.Empty.INSTANCE)); + deterministicTaskQueue.runAllTasks(); + + final CoordinationStateRejectedException coordinationStateRejectedException + = expectThrows(CoordinationStateRejectedException.class, future::actionGet); + assertThat(coordinationStateRejectedException.getMessage(), + containsString("join validation on cluster state with a different cluster uuid")); + assertThat(coordinationStateRejectedException.getMessage(), containsString(localClusterState.metaData().clusterUUID())); + assertThat(coordinationStateRejectedException.getMessage(), containsString(otherClusterState.metaData().clusterUUID())); + } + } From 8dd5dee7ebf5a44ec4c439a48eeec5ba0eadccf6 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Tue, 16 Apr 2019 18:16:55 +0200 Subject: [PATCH 055/260] Unified highlighter should respect no_match_size with number_of_fragments set to 0 (#41069) The unified highlighter returns the first sentence of the text when number_of_fragments is set to 0 (full highlighting). This is a legacy of the removed postings highlighter that was based on sentence break only. This commit changes this behavior in order to respect the provided no_match_size value when number_of_fragments is set to 0. This means that the behavior will be consistent for any value of the number_of_fragments option. Closes #41066 --- .../highlight/UnifiedHighlighter.java | 37 ------------------- .../highlight/HighlighterSearchIT.java | 8 ++-- 2 files changed, 5 insertions(+), 40 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java index af37fa4edab19..2d570d2b7c793 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java @@ -124,8 +124,6 @@ public HighlightField highlight(HighlighterContext highlighterContext) { "Failed to highlight field [" + highlighterContext.fieldName + "]", e); } - snippets = filterSnippets(snippets, field.fieldOptions().numberOfFragments()); - if (field.fieldOptions().scoreOrdered()) { //let's sort the snippets by score if needed CollectionUtil.introSort(snippets, (o1, o2) -> Double.compare(o2.getScore(), o1.getScore())); @@ -185,41 +183,6 @@ protected BreakIterator getBreakIterator(SearchContextHighlight.Field field) { } } - protected static List filterSnippets(List snippets, int numberOfFragments) { - - //We need to filter the snippets as due to no_match_size we could have - //either highlighted snippets or non highlighted ones and we don't want to mix those up - List filteredSnippets = new ArrayList<>(snippets.size()); - for (Snippet snippet : snippets) { - if (snippet.isHighlighted()) { - filteredSnippets.add(snippet); - } - } - - //if there's at least one highlighted snippet, we return all the highlighted ones - //otherwise we return the first non highlighted one if available - if (filteredSnippets.size() == 0) { - if (snippets.size() > 0) { - Snippet snippet = snippets.get(0); - //if we tried highlighting the whole content using whole break iterator (as number_of_fragments was 0) - //we need to return the first sentence of the content rather than the whole content - if (numberOfFragments == 0) { - BreakIterator bi = BreakIterator.getSentenceInstance(Locale.ROOT); - String text = snippet.getText(); - bi.setText(text); - int next = bi.next(); - if (next != BreakIterator.DONE) { - String newText = text.substring(0, next).trim(); - snippet = new Snippet(newText, snippet.getScore(), snippet.isHighlighted()); - } - } - filteredSnippets.add(snippet); - } - } - - return filteredSnippets; - } - protected static String convertFieldValue(MappedFieldType type, Object value) { if (value instanceof BytesRef) { return type.valueForDisplay(value).toString(); diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java index e111abe0d5132..d1a669695313c 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java @@ -1715,9 +1715,11 @@ public void testHighlightNoMatchSize() throws IOException { assertHighlight(response, 0, "text", 0, 1, equalTo("I am pretty long so some")); // We can also ask for a fragment longer than the input string and get the whole string - field.highlighterType("plain").noMatchSize(text.length() * 2); - response = client().prepareSearch("test").highlighter(new HighlightBuilder().field(field)).get(); - assertHighlight(response, 0, "text", 0, 1, equalTo(text)); + for (String type : new String[] { "plain", "unified" }) { + field.highlighterType(type).noMatchSize(text.length() * 2).numOfFragments(0); + response = client().prepareSearch("test").highlighter(new HighlightBuilder().field(field)).get(); + assertHighlight(response, 0, "text", 0, 1, equalTo(text)); + } field.highlighterType("fvh"); response = client().prepareSearch("test").highlighter(new HighlightBuilder().field(field)).get(); From 545a71e2a32b9337aad2ee0a1ae07509f0d1cb95 Mon Sep 17 00:00:00 2001 From: Karel Minarik Date: Tue, 16 Apr 2019 18:25:33 +0200 Subject: [PATCH 056/260] [DOCS] Update documentation for the Go client (#41238) * Removed the "Work in progress" panel * Added proper installation instructions for current release --- docs/go/index.asciidoc | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/docs/go/index.asciidoc b/docs/go/index.asciidoc index f8977dea91950..bb03ec23d877f 100644 --- a/docs/go/index.asciidoc +++ b/docs/go/index.asciidoc @@ -8,12 +8,6 @@ Full documentation is hosted at https://github.com/elastic/go-elasticsearch[GitH and https://godoc.org/github.com/elastic/go-elasticsearch[GoDoc] -- this page provides only an overview. -.Work In Progress -************************************************************************************ -The client is currently available as a public preview. We encourage you to try the -package in your projects, but please be aware that the public API may change. -************************************************************************************ - === Elasticsearch Version Compatibility The client major versions correspond to the Elasticsearch major versions: @@ -28,7 +22,7 @@ Add the package to your `go.mod` file: [source,text] ------------------------------------ -require github.com/elastic/go-elasticsearch {VERSION} +require github.com/elastic/go-elasticsearch/v7 7.x ------------------------------------ === Usage @@ -40,7 +34,7 @@ package main import ( "log" - "github.com/elastic/go-elasticsearch" + "github.com/elastic/go-elasticsearch/v7" ) func main() { @@ -61,7 +55,7 @@ at https://github.com/elastic/go-elasticsearch/tree/master/_examples. == License -Copyright 2019-present Elasticsearch +Copyright 2019 Elasticsearch Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. From 6dbca5edd335eb1da8e7825389a15e5fe45397d4 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Tue, 16 Apr 2019 19:49:00 +0300 Subject: [PATCH 057/260] SQL: Tweak pattern matching in SYS TABLES (#41243) Yet another improvement to SYS TABLES on differentiating between table types specified as '%' and '' while maintaining legacy support for null Fix #40775 --- .../xpack/sql/parser/CommandBuilder.java | 2 +- .../logical/command/sys/SysTablesTests.java | 50 +++++++++++++++---- 2 files changed, 41 insertions(+), 11 deletions(-) diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/CommandBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/CommandBuilder.java index 04935023747c3..ba2a39069953a 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/CommandBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/CommandBuilder.java @@ -146,7 +146,7 @@ public SysTables visitSysTables(SysTablesContext ctx) { boolean legacyTableType = false; for (StringContext string : ctx.string()) { String value = string(string); - if (value != null) { + if (value != null && value.isEmpty() == false) { // check special ODBC wildcard case if (value.equals(StringUtils.SQL_WILDCARD) && ctx.string().size() == 1) { // treat % as null diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java index e2baeb2d8af98..d7a24681329cb 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java @@ -54,7 +54,30 @@ public class SysTablesTests extends ESTestCase { // // catalog enumeration // - public void testSysTablesCatalogEnumeration() throws Exception { + public void testSysTablesCatalogEnumerationWithEmptyType() throws Exception { + executeCommand("SYS TABLES CATALOG LIKE '%' LIKE '' TYPE ''", r -> { + assertEquals(1, r.size()); + assertEquals(CLUSTER_NAME, r.column(0)); + // everything else should be null + for (int i = 1; i < 10; i++) { + assertNull(r.column(i)); + } + }, index); + } + + public void testSysTablesCatalogAllTypes() throws Exception { + executeCommand("SYS TABLES CATALOG LIKE '%' LIKE '' TYPE '%'", r -> { + assertEquals(1, r.size()); + assertEquals(CLUSTER_NAME, r.column(0)); + // everything else should be null + for (int i = 1; i < 10; i++) { + assertNull(r.column(i)); + } + }, new IndexInfo[0]); + } + + // when types are null, consider them equivalent to '' for compatibility reasons + public void testSysTablesCatalogNoTypes() throws Exception { executeCommand("SYS TABLES CATALOG LIKE '%' LIKE ''", r -> { assertEquals(1, r.size()); assertEquals(CLUSTER_NAME, r.column(0)); @@ -65,24 +88,18 @@ public void testSysTablesCatalogEnumeration() throws Exception { }, index); } + // // table types enumeration // + + // missing type means pattern public void testSysTablesTypesEnumerationWoString() throws Exception { executeCommand("SYS TABLES CATALOG LIKE '' LIKE '' ", r -> { assertEquals(2, r.size()); assertEquals("BASE TABLE", r.column(3)); assertTrue(r.advanceRow()); assertEquals("VIEW", r.column(3)); - }, new IndexInfo[0]); - } - - public void testSysTablesEnumerateTypes() throws Exception { - executeCommand("SYS TABLES CATALOG LIKE '' LIKE '' TYPE '%'", r -> { - assertEquals(2, r.size()); - assertEquals("BASE TABLE", r.column(3)); - assertTrue(r.advanceRow()); - assertEquals("VIEW", r.column(3)); }, alias, index); } @@ -107,6 +124,13 @@ public void testSysTablesTypesEnumeration() throws Exception { }, new IndexInfo[0]); } + // when a type is specified, apply filtering + public void testSysTablesTypesEnumerationAllCatalogsAndSpecifiedView() throws Exception { + executeCommand("SYS TABLES CATALOG LIKE '%' LIKE '' TYPE 'VIEW'", r -> { + assertEquals(0, r.size()); + }, new IndexInfo[0]); + } + public void testSysTablesDifferentCatalog() throws Exception { executeCommand("SYS TABLES CATALOG LIKE 'foo'", r -> { assertEquals(0, r.size()); @@ -262,6 +286,12 @@ public void testSysTablesWithCatalogOnlyAliases() throws Exception { }, alias); } + public void testSysTablesWithEmptyCatalogOnlyAliases() throws Exception { + executeCommand("SYS TABLES CATALOG LIKE '' LIKE 'test' TYPE 'VIEW'", r -> { + assertEquals(0, r.size()); + }, alias); + } + public void testSysTablesWithInvalidType() throws Exception { executeCommand("SYS TABLES LIKE 'test' TYPE 'QUE HORA ES'", r -> { assertEquals(0, r.size()); From ab2613c725b17a5661c126fddfaaca2c2fdc5de3 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Tue, 16 Apr 2019 18:53:36 +0200 Subject: [PATCH 058/260] [ML-DataFrame] adapt page size on circuit breaker responses (#41149) handle circuit breaker response and adapt page size to reduce memory pressure, reduce preview buckets to 100, initial page size to 500 --- .../core/dataframe/DataFrameMessages.java | 5 + .../integration/DataFramePivotRestIT.java | 3 +- ...nsportPreviewDataFrameTransformAction.java | 3 +- .../transforms/DataFrameIndexer.java | 97 ++++++- .../transforms/DataFrameTransformTask.java | 74 +++--- .../dataframe/transforms/pivot/Pivot.java | 23 +- .../transforms/DataFrameIndexerTests.java | 239 ++++++++++++++++++ 7 files changed, 409 insertions(+), 35 deletions(-) create mode 100644 x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java index dbe789ca3aebf..86dce1b331420 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java @@ -56,6 +56,11 @@ public class DataFrameMessages { "Failed to parse group_by for data frame pivot transform"; public static final String LOG_DATA_FRAME_TRANSFORM_CONFIGURATION_BAD_AGGREGATION = "Failed to parse aggregation for data frame pivot transform"; + public static final String LOG_DATA_FRAME_TRANSFORM_PIVOT_REDUCE_PAGE_SIZE = + "Search returned with out of memory error, reducing number of buckets per search from [{0}] to [{1}]"; + public static final String LOG_DATA_FRAME_TRANSFORM_PIVOT_LOW_PAGE_SIZE_FAILURE = + "Search returned with out of memory error after repeated page size reductions to [{0}], unable to continue pivot, " + + "please simplify job or increase heap size on data nodes."; public static final String FAILED_TO_PARSE_TRANSFORM_CHECKPOINTS = "Failed to parse transform checkpoints for [{0}]"; diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java index 0d14851aa7cc3..6ff97e1ed9d26 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java @@ -260,7 +260,8 @@ public void testPreviewTransform() throws Exception { createPreviewRequest.setJsonEntity(config); Map previewDataframeResponse = entityAsMap(client().performRequest(createPreviewRequest)); List> preview = (List>)previewDataframeResponse.get("preview"); - assertThat(preview.size(), equalTo(393)); + // preview is limited to 100 + assertThat(preview.size(), equalTo(100)); Set expectedFields = new HashSet<>(Arrays.asList("reviewer", "by_day", "avg_rating")); preview.forEach(p -> { Set keys = p.keySet(); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java index 2a4ba47f50762..b65830f72e7ca 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java @@ -35,6 +35,7 @@ public class TransportPreviewDataFrameTransformAction extends HandledTransportAction { + private static final int NUMBER_OF_PREVIEW_BUCKETS = 100; private final XPackLicenseState licenseState; private final Client client; private final ThreadPool threadPool; @@ -77,7 +78,7 @@ private void getPreview(Pivot pivot, ActionListener>> l ClientHelper.DATA_FRAME_ORIGIN, client, SearchAction.INSTANCE, - pivot.buildSearchRequest(null), + pivot.buildSearchRequest(null, NUMBER_OF_PREVIEW_BUCKETS), ActionListener.wrap( r -> { final CompositeAggregation agg = r.getAggregations().get(COMPOSITE_AGGREGATION_NAME); 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 c781d05f189b6..c670f32740c28 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 @@ -8,14 +8,21 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; +import org.elasticsearch.xpack.core.common.notifications.Auditor; import org.elasticsearch.xpack.core.dataframe.DataFrameField; +import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; +import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer; @@ -26,6 +33,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Map; +import java.util.Objects; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -35,22 +43,34 @@ public abstract class DataFrameIndexer extends AsyncTwoPhaseIndexer, DataFrameIndexerTransformStats> { + public static final int MINIMUM_PAGE_SIZE = 10; public static final String COMPOSITE_AGGREGATION_NAME = "_data_frame"; private static final Logger logger = LogManager.getLogger(DataFrameIndexer.class); + protected final Auditor auditor; + private Pivot pivot; + private int pageSize = 0; public DataFrameIndexer(Executor executor, + Auditor auditor, AtomicReference initialState, Map initialPosition, DataFrameIndexerTransformStats jobStats) { super(executor, initialState, initialPosition, jobStats); + this.auditor = Objects.requireNonNull(auditor); } protected abstract DataFrameTransformConfig getConfig(); protected abstract Map getFieldMappings(); + protected abstract void failIndexer(String message); + + public int getPageSize() { + return pageSize; + } + /** * Request a checkpoint */ @@ -62,6 +82,11 @@ protected void onStart(long now, ActionListener listener) { QueryBuilder queryBuilder = getConfig().getSource().getQueryConfig().getQuery(); pivot = new Pivot(getConfig().getSource().getIndex(), queryBuilder, getConfig().getPivotConfig()); + // if we haven't set the page size yet, if it is set we might have reduced it after running into an out of memory + if (pageSize == 0) { + pageSize = pivot.getInitialPageSize(); + } + // if run for the 1st time, create checkpoint if (getPosition() == null) { createCheckpoint(listener); @@ -73,6 +98,12 @@ protected void onStart(long now, ActionListener listener) { } } + @Override + protected void onFinish(ActionListener listener) { + // reset the page size, so we do not memorize a low page size forever, the pagesize will be re-calculated on start + pageSize = 0; + } + @Override protected IterationResult> doProcess(SearchResponse searchResponse) { final CompositeAggregation agg = searchResponse.getAggregations().get(COMPOSITE_AGGREGATION_NAME); @@ -121,6 +152,70 @@ private Stream processBucketsToIndexRequests(CompositeAggregation @Override protected SearchRequest buildSearchRequest() { - return pivot.buildSearchRequest(getPosition()); + return pivot.buildSearchRequest(getPosition(), pageSize); + } + + /** + * Handle the circuit breaking case: A search consumed to much memory and got aborted. + * + * Going out of memory we smoothly reduce the page size which reduces memory consumption. + * + * Implementation details: We take the values from the circuit breaker as a hint, but + * note that it breaks early, that's why we also reduce using + * + * @param e Exception thrown, only {@link CircuitBreakingException} are handled + * @return true if exception was handled, false if not + */ + protected boolean handleCircuitBreakingException(Exception e) { + CircuitBreakingException circuitBreakingException = getCircuitBreakingException(e); + + if (circuitBreakingException == null) { + return false; + } + + double reducingFactor = Math.min((double) circuitBreakingException.getByteLimit() / circuitBreakingException.getBytesWanted(), + 1 - (Math.log10(pageSize) * 0.1)); + + int newPageSize = (int) Math.round(reducingFactor * pageSize); + + if (newPageSize < MINIMUM_PAGE_SIZE) { + String message = DataFrameMessages.getMessage(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_PIVOT_LOW_PAGE_SIZE_FAILURE, pageSize); + failIndexer(message); + return true; + } + + String message = DataFrameMessages.getMessage(DataFrameMessages.LOG_DATA_FRAME_TRANSFORM_PIVOT_REDUCE_PAGE_SIZE, pageSize, + newPageSize); + auditor.info(getJobId(), message); + logger.info("Data frame transform [" + getJobId() + "]:" + message); + + pageSize = newPageSize; + return true; + } + + /** + * Inspect exception for circuit breaking exception and return the first one it can find. + * + * @param e Exception + * @return CircuitBreakingException instance if found, null otherwise + */ + private static CircuitBreakingException getCircuitBreakingException(Exception e) { + // circuit breaking exceptions are at the bottom + Throwable unwrappedThrowable = ExceptionsHelper.unwrapCause(e); + + if (unwrappedThrowable instanceof CircuitBreakingException) { + return (CircuitBreakingException) unwrappedThrowable; + } else if (unwrappedThrowable instanceof SearchPhaseExecutionException) { + SearchPhaseExecutionException searchPhaseException = (SearchPhaseExecutionException) e; + for (ShardSearchFailure shardFailure : searchPhaseException.shardFailures()) { + Throwable unwrappedShardFailure = ExceptionsHelper.unwrapCause(shardFailure.getCause()); + + if (unwrappedShardFailure instanceof CircuitBreakingException) { + return (CircuitBreakingException) unwrappedShardFailure; + } + } + } + + return null; } } 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 69ceb32dfc709..4088863a89500 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 @@ -277,25 +277,6 @@ void persistStateToClusterState(DataFrameTransformState state, )); } - private boolean isIrrecoverableFailure(Exception e) { - return e instanceof IndexNotFoundException || e instanceof DataFrameConfigurationException; - } - - synchronized void handleFailure(Exception e) { - if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > MAX_CONTINUOUS_FAILURES) { - String failureMessage = isIrrecoverableFailure(e) ? - "task encountered irrecoverable failure: " + e.getMessage() : - "task encountered more than " + MAX_CONTINUOUS_FAILURES + " failures; latest failure: " + e.getMessage(); - auditor.error(transform.getId(), failureMessage); - stateReason.set(failureMessage); - taskState.set(DataFrameTransformTaskState.FAILED); - persistStateToClusterState(getState(), ActionListener.wrap( - r -> failureCount.set(0), // Successfully marked as failed, reset counter so that task can be restarted - exception -> {} // Noop, internal method logs the failure to update the state - )); - } - } - /** * This is called when the persistent task signals that the allocated task should be terminated. * Termination in the task framework is essentially voluntary, as the allocated task can only be @@ -313,13 +294,11 @@ public synchronized void onCancelled() { protected class ClientDataFrameIndexer extends DataFrameIndexer { private static final int LOAD_TRANSFORM_TIMEOUT_IN_SECONDS = 30; - private static final int CREATE_CHECKPOINT_TIMEOUT_IN_SECONDS = 30; private final Client client; private final DataFrameTransformsConfigManager transformsConfigManager; private final DataFrameTransformsCheckpointService transformsCheckpointService; private final String transformId; - private final Auditor auditor; private volatile DataFrameIndexerTransformStats previouslyPersistedStats = null; // Keeps track of the last exception that was written to our audit, keeps us from spamming the audit index private volatile String lastAuditedExceptionMessage = null; @@ -331,13 +310,12 @@ public ClientDataFrameIndexer(String transformId, DataFrameTransformsConfigManag DataFrameTransformsCheckpointService transformsCheckpointService, AtomicReference initialState, Map initialPosition, Client client, Auditor auditor) { - super(threadPool.executor(ThreadPool.Names.GENERIC), initialState, initialPosition, + super(threadPool.executor(ThreadPool.Names.GENERIC), auditor, initialState, initialPosition, new DataFrameIndexerTransformStats(transformId)); this.transformId = transformId; this.transformsConfigManager = transformsConfigManager; this.transformsCheckpointService = transformsCheckpointService; this.client = client; - this.auditor = auditor; } @Override @@ -474,19 +452,26 @@ protected void doSaveState(IndexerState indexerState, Map positi @Override protected void onFailure(Exception exc) { - // 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 (exc.getMessage().equals(lastAuditedExceptionMessage) == false) { - auditor.warning(transform.getId(), "Data frame transform encountered an exception: " + exc.getMessage()); - lastAuditedExceptionMessage = exc.getMessage(); + // the failure handler must not throw an exception due to internal problems + try { + logger.warn("Data frame transform [" + transform.getId() + "] encountered an exception: ", exc); + + // 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 (exc.getMessage().equals(lastAuditedExceptionMessage) == false) { + auditor.warning(transform.getId(), "Data frame transform encountered an exception: " + exc.getMessage()); + lastAuditedExceptionMessage = exc.getMessage(); + } + handleFailure(exc); + } catch (Exception e) { + logger.error("Data frame transform encountered an unexpected internal exception: " ,e); } - logger.warn("Data frame transform [" + transform.getId() + "] encountered an exception: ", exc); - handleFailure(exc); } @Override protected void onFinish(ActionListener listener) { try { + super.onFinish(listener); long checkpoint = currentCheckpoint.incrementAndGet(); auditor.info(transform.getId(), "Finished indexing for data frame transform checkpoint [" + checkpoint + "]"); logger.info("Finished indexing for data frame transform [" + transform.getId() + "] checkpoint [" + checkpoint + "]"); @@ -515,6 +500,35 @@ protected void createCheckpoint(ActionListener listener) { listener.onFailure(new RuntimeException("Failed to retrieve checkpoint", getCheckPointException)); })); } + + private boolean isIrrecoverableFailure(Exception e) { + return e instanceof IndexNotFoundException || e instanceof DataFrameConfigurationException; + } + + synchronized void handleFailure(Exception e) { + if (handleCircuitBreakingException(e)) { + return; + } + + if (isIrrecoverableFailure(e) || failureCount.incrementAndGet() > MAX_CONTINUOUS_FAILURES) { + String failureMessage = isIrrecoverableFailure(e) ? + "task encountered irrecoverable failure: " + e.getMessage() : + "task encountered more than " + MAX_CONTINUOUS_FAILURES + " failures; latest failure: " + e.getMessage(); + failIndexer(failureMessage); + } + } + + @Override + protected void failIndexer(String failureMessage) { + logger.error("Data frame transform [" + getJobId() + "]:" + failureMessage); + auditor.error(transform.getId(), failureMessage); + stateReason.set(failureMessage); + taskState.set(DataFrameTransformTaskState.FAILED); + persistStateToClusterState(DataFrameTransformTask.this.getState(), ActionListener.wrap( + r -> failureCount.set(0), // Successfully marked as failed, reset counter so that task can be restarted + exception -> {} // Noop, internal method logs the failure to update the state + )); + } } class DataFrameConfigurationException extends RuntimeException { diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java index 0cf3edec16283..aa63ea92e7a53 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java @@ -35,6 +35,8 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; public class Pivot { + public static final int DEFAULT_INITIAL_PAGE_SIZE = 500; + private static final String COMPOSITE_AGGREGATION_NAME = "_data_frame"; private final PivotConfig config; @@ -68,11 +70,29 @@ public void deduceMappings(Client client, final ActionListener position) { + /** + * Get the initial page size for this pivot. + * + * The page size is the main parameter for adjusting memory consumption. Memory consumption mainly depends on + * the page size, the type of aggregations and the data. As the page size is the number of buckets we return + * per page the page size is a multiplier for the costs of aggregating bucket. + * + * Initially this returns a default, in future it might inspect the configuration and base the initial size + * on the aggregations used. + * + * @return the page size + */ + public int getInitialPageSize() { + return DEFAULT_INITIAL_PAGE_SIZE; + } + + public SearchRequest buildSearchRequest(Map position, int pageSize) { if (position != null) { cachedCompositeAggregation.aggregateAfter(position); } + cachedCompositeAggregation.size(pageSize); + return cachedSearchRequest; } @@ -127,7 +147,6 @@ private static CompositeAggregationBuilder createCompositeAggregation(PivotConfi XContentParser parser = builder.generator().contentType().xContent().createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, BytesReference.bytes(builder).streamInput()); compositeAggregation = CompositeAggregationBuilder.parse(COMPOSITE_AGGREGATION_NAME, parser); - compositeAggregation.size(1000); config.getAggregationConfig().getAggregatorFactories().forEach(agg -> compositeAggregation.subAggregation(agg)); } catch (IOException e) { throw new RuntimeException(DataFrameMessages.DATA_FRAME_TRANSFORM_PIVOT_FAILED_TO_CREATE_COMPOSITE_AGGREGATION, e); diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java new file mode 100644 index 0000000000000..b121e8091c138 --- /dev/null +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java @@ -0,0 +1,239 @@ +/* + * 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.action.ActionListener; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.breaker.CircuitBreaker.Durability; +import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.common.notifications.Auditor; +import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfigTests; +import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; +import org.junit.Before; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Function; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class DataFrameIndexerTests extends ESTestCase { + + private Client client; + private static final String TEST_ORIGIN = "test_origin"; + private static final String TEST_INDEX = "test_index"; + + class MockedDataFrameIndexer extends DataFrameIndexer { + + private final DataFrameTransformConfig transformConfig; + private final Map fieldMappings; + private final Function searchFunction; + private final Function bulkFunction; + private final Consumer failureConsumer; + + // used for synchronizing with the test + private CountDownLatch latch; + + MockedDataFrameIndexer( + Executor executor, + DataFrameTransformConfig transformConfig, + Map fieldMappings, + Auditor auditor, + AtomicReference initialState, + Map initialPosition, + DataFrameIndexerTransformStats jobStats, + Function searchFunction, + Function bulkFunction, + Consumer failureConsumer) { + super(executor, auditor, initialState, initialPosition, jobStats); + this.transformConfig = Objects.requireNonNull(transformConfig); + this.fieldMappings = Objects.requireNonNull(fieldMappings); + this.searchFunction = searchFunction; + this.bulkFunction = bulkFunction; + this.failureConsumer = failureConsumer; + } + + public CountDownLatch newLatch(int count) { + return latch = new CountDownLatch(count); + } + + @Override + protected DataFrameTransformConfig getConfig() { + return transformConfig; + } + + @Override + protected Map getFieldMappings() { + return fieldMappings; + } + + @Override + protected void createCheckpoint(ActionListener listener) { + listener.onResponse(null); + } + + @Override + protected String getJobId() { + return transformConfig.getId(); + } + + @Override + protected void doNextSearch(SearchRequest request, ActionListener nextPhase) { + assert latch != null; + try { + latch.await(); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + + try { + SearchResponse response = searchFunction.apply(request); + nextPhase.onResponse(response); + } catch (Exception e) { + nextPhase.onFailure(e); + } + } + + @Override + protected void doNextBulk(BulkRequest request, ActionListener nextPhase) { + assert latch != null; + try { + latch.await(); + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + + try { + BulkResponse response = bulkFunction.apply(request); + nextPhase.onResponse(response); + } catch (Exception e) { + nextPhase.onFailure(e); + } + } + + @Override + protected void doSaveState(IndexerState state, Map position, Runnable next) { + assert state == IndexerState.STARTED || state == IndexerState.INDEXING || state == IndexerState.STOPPED; + next.run(); + } + + @Override + protected void onFailure(Exception exc) { + try { + // mimic same behavior as {@link DataFrameTransformTask} + if (handleCircuitBreakingException(exc)) { + return; + } + + failureConsumer.accept(exc); + } catch (Exception e) { + fail("Internal error: " + e.getMessage()); + } + } + + @Override + protected void onFinish(ActionListener listener) { + super.onFinish(listener); + listener.onResponse(null); + } + + @Override + protected void onAbort() { + fail("onAbort should not be called"); + } + + @Override + protected void failIndexer(String message) { + fail("failIndexer should not be called, received error: " + message); + } + + } + + @Before + public void setUpMocks() { + client = mock(Client.class); + ThreadPool threadPool = mock(ThreadPool.class); + when(client.threadPool()).thenReturn(threadPool); + when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); + } + + public void testPageSizeAdapt() throws InterruptedException { + DataFrameTransformConfig config = DataFrameTransformConfigTests.randomDataFrameTransformConfig(); + AtomicReference state = new AtomicReference<>(IndexerState.STOPPED); + + Function searchFunction = searchRequest -> { + throw new SearchPhaseExecutionException("query", "Partial shards failure", new ShardSearchFailure[] { + new ShardSearchFailure(new CircuitBreakingException("to much memory", 110, 100, Durability.TRANSIENT)) }); + }; + + Function bulkFunction = bulkRequest -> new BulkResponse(new BulkItemResponse[0], 100); + + Consumer failureConsumer = e -> { + fail("expected circuit breaker exception to be handled"); + }; + + final ExecutorService executor = Executors.newFixedThreadPool(1); + try { + Auditor auditor = new Auditor<>(client, "node_1", TEST_INDEX, TEST_ORIGIN, + DataFrameAuditMessage.builder()); + + MockedDataFrameIndexer indexer = new MockedDataFrameIndexer(executor, config, Collections.emptyMap(), auditor, state, null, + new DataFrameIndexerTransformStats(config.getId()), searchFunction, bulkFunction, failureConsumer); + final CountDownLatch latch = indexer.newLatch(1); + indexer.start(); + assertThat(indexer.getState(), equalTo(IndexerState.STARTED)); + assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis())); + assertThat(indexer.getState(), equalTo(IndexerState.INDEXING)); + latch.countDown(); + awaitBusy(() -> indexer.getState() == IndexerState.STOPPED); + long pageSizeAfterFirstReduction = indexer.getPageSize(); + assertTrue(Pivot.DEFAULT_INITIAL_PAGE_SIZE > pageSizeAfterFirstReduction); + assertTrue(pageSizeAfterFirstReduction > DataFrameIndexer.MINIMUM_PAGE_SIZE); + + // run indexer a 2nd time + final CountDownLatch secondRunLatch = indexer.newLatch(1); + indexer.start(); + assertEquals(pageSizeAfterFirstReduction, indexer.getPageSize()); + assertThat(indexer.getState(), equalTo(IndexerState.STARTED)); + assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis())); + assertThat(indexer.getState(), equalTo(IndexerState.INDEXING)); + secondRunLatch.countDown(); + awaitBusy(() -> indexer.getState() == IndexerState.STOPPED); + + // assert that page size has been reduced again + assertTrue(pageSizeAfterFirstReduction > indexer.getPageSize()); + assertTrue(pageSizeAfterFirstReduction > DataFrameIndexer.MINIMUM_PAGE_SIZE); + + } finally { + executor.shutdownNow(); + } + } +} From f426a339b77af6008d41cc000c9199fe384e9269 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 16 Apr 2019 20:39:54 +0300 Subject: [PATCH 059/260] SQL: Translate MIN/MAX on keyword fields as FIRST/LAST (#41247) Although the translation rule was implemented in the `Optimizer`, the rule was not added in the list of rules to be executed. Relates to #41195 Follows #37936 --- .../sql/qa/src/main/resources/agg.csv-spec | 20 ++++++++++++++++ .../xpack/sql/optimizer/Optimizer.java | 1 + .../sql/planner/QueryTranslatorTests.java | 24 +++++++++++++++++++ 3 files changed, 45 insertions(+) diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec index b55c3f66eafd9..5cc70a8cb5ef0 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec @@ -414,6 +414,26 @@ SELECT COUNT(ALL last_name)=COUNT(ALL first_name) AS areEqual, COUNT(ALL first_n false |90 |100 ; +topHitsAsMinAndMax +schema::min:s|max:s|first:s|last:s +SELECT MIN(first_name) as min, MAX(first_name) as max, FIRST(first_name) as first, LAST(first_name) as last FROM test_emp; + + min | max | first | last +---------------+---------------+--------------+---------- + Alejandro | Zvonko | Alejandro | Zvonko +; + +topHitsAsMinAndMaxAndGroupBy +schema::gender:s|min:s|max:s|first:s|last:s +SELECT gender, MIN(first_name) as min, MAX(first_name) as max, FIRST(first_name) as first, LAST(first_name) as last FROM test_emp GROUP BY gender ORDER BY gender; + + gender | min | max | first | last +---------------+---------------+--------------+---------------+---------- +null | Berni | Patricio | Berni | Patricio +F | Alejandro | Xinglin | Alejandro | Xinglin +M | Amabile | Zvonko | Amabile | Zvonko +; + topHitsWithOneArgAndGroupBy schema::gender:s|first:s|last:s SELECT gender, FIRST(first_name) as first, LAST(first_name) as last FROM test_emp GROUP BY gender ORDER BY gender; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java index 6b1954f844ca7..d6e4c4fe07d7e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java @@ -149,6 +149,7 @@ protected Iterable.Batch> batches() { Batch aggregate = new Batch("Aggregation Rewrite", //new ReplaceDuplicateAggsWithReferences(), + new ReplaceMinMaxWithTopHits(), new ReplaceAggsWithMatrixStats(), new ReplaceAggsWithExtendedStats(), new ReplaceAggsWithStats(), 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 c76e0da987d55..85bc20596e9e3 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 @@ -763,6 +763,18 @@ public void testTopHitsAggregationWithOneArg() { "\"explain\":false,\"docvalue_fields\":[{\"field\":\"keyword\"}]," + "\"sort\":[{\"keyword\":{\"order\":\"asc\",\"missing\":\"_last\",\"unmapped_type\":\"keyword\"}}]}}}}}")); } + { + PhysicalPlan p = optimizeAndPlan("SELECT MIN(keyword) FROM test"); + assertEquals(EsQueryExec.class, p.getClass()); + EsQueryExec eqe = (EsQueryExec) p; + assertEquals(1, eqe.output().size()); + assertEquals("MIN(keyword)", eqe.output().get(0).qualifiedName()); + assertEquals(DataType.KEYWORD, eqe.output().get(0).dataType()); + assertThat(eqe.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""), + endsWith("\"top_hits\":{\"from\":0,\"size\":1,\"version\":false,\"seq_no_primary_term\":false," + + "\"explain\":false,\"docvalue_fields\":[{\"field\":\"keyword\"}]," + + "\"sort\":[{\"keyword\":{\"order\":\"asc\",\"missing\":\"_last\",\"unmapped_type\":\"keyword\"}}]}}}}}")); + } { PhysicalPlan p = optimizeAndPlan("SELECT LAST(date) FROM test"); assertEquals(EsQueryExec.class, p.getClass()); @@ -775,6 +787,18 @@ public void testTopHitsAggregationWithOneArg() { "\"explain\":false,\"docvalue_fields\":[{\"field\":\"date\",\"format\":\"epoch_millis\"}]," + "\"sort\":[{\"date\":{\"order\":\"desc\",\"missing\":\"_last\",\"unmapped_type\":\"date\"}}]}}}}}")); } + { + PhysicalPlan p = optimizeAndPlan("SELECT MAX(keyword) FROM test"); + assertEquals(EsQueryExec.class, p.getClass()); + EsQueryExec eqe = (EsQueryExec) p; + assertEquals(1, eqe.output().size()); + assertEquals("MAX(keyword)", eqe.output().get(0).qualifiedName()); + assertEquals(DataType.KEYWORD, eqe.output().get(0).dataType()); + assertThat(eqe.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""), + endsWith("\"top_hits\":{\"from\":0,\"size\":1,\"version\":false,\"seq_no_primary_term\":false," + + "\"explain\":false,\"docvalue_fields\":[{\"field\":\"keyword\"}]," + + "\"sort\":[{\"keyword\":{\"order\":\"desc\",\"missing\":\"_last\",\"unmapped_type\":\"keyword\"}}]}}}}}")); + } } public void testTopHitsAggregationWithTwoArgs() { From 8cd40ead74988f98df6306a5f1d1beaaf2d9970c Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 16 Apr 2019 13:47:39 -0400 Subject: [PATCH 060/260] Drop inline callouts from SQL conditional docs (#41205) Drops "inline callouts" from the docs for SQL conditionals because they aren't supported by Asciidoctor. Relates to #41128 --- .../sql/functions/conditional.asciidoc | 31 ++++++++++++++----- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/docs/reference/sql/functions/conditional.asciidoc b/docs/reference/sql/functions/conditional.asciidoc index ce8d5c3e66ced..135381a1c93bd 100644 --- a/docs/reference/sql/functions/conditional.asciidoc +++ b/docs/reference/sql/functions/conditional.asciidoc @@ -11,7 +11,10 @@ Functions that return one of their arguments by evaluating in an if-else manner. .Synopsis: [source, sql] ---- -COALESCE(expression<1>, expression<2>, ...) +COALESCE( + expression, <1> + expression, <2> + ...) ---- *Input*: @@ -51,7 +54,10 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[coalesceReturnNull] .Synopsis: [source, sql] ---- -GREATEST(expression<1>, expression<2>, ...) +GREATEST( + expression, <1> + expression, <2> + ...) ---- *Input*: @@ -92,7 +98,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[greatestReturnNull] .Synopsis: [source, sql] ---- -IFNULL(expression<1>, expression<2>) +IFNULL( + expression, <1> + expression) <2> ---- *Input*: @@ -129,7 +137,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[ifNullReturnSecond] .Synopsis: [source, sql] ---- -ISNULL(expression<1>, expression<2>) +ISNULL( + expression, <1> + expression) <2> ---- *Input*: @@ -166,7 +176,10 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[isNullReturnSecond] .Synopsis: [source, sql] ---- -LEAST(expression<1>, expression<2>, ...) +LEAST( + expression, <1> + expression, <2> + ...) ---- *Input*: @@ -208,7 +221,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[leastReturnNull] .Synopsis: [source, sql] ---- -NULLIF(expression<1>, expression<2>) +NULLIF( + expression, <1> + expression) <2> ---- *Input*: @@ -243,7 +258,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[nullIfReturnNull] .Synopsis: [source, sql] ---- -NVL(expression<1>, expression<2>) +NVL( + expression, <1> + expression) <2> ---- *Input*: From da504141f5d3d2d642ca6a606d7c03cc1a38ac76 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 16 Apr 2019 15:27:51 -0400 Subject: [PATCH 061/260] Docs: Drop inline callouts from two SQL pages (#41270) Drops inline callouts from the docs for SQL's string and type-conversion functions because they are not compatible with Asciidoctor. --- docs/reference/sql/functions/string.asciidoc | 64 +++++++++++++------ .../sql/functions/type-conversion.asciidoc | 8 ++- 2 files changed, 50 insertions(+), 22 deletions(-) diff --git a/docs/reference/sql/functions/string.asciidoc b/docs/reference/sql/functions/string.asciidoc index 45389cd410e46..7acc358763512 100644 --- a/docs/reference/sql/functions/string.asciidoc +++ b/docs/reference/sql/functions/string.asciidoc @@ -11,7 +11,7 @@ Functions for performing string manipulation. .Synopsis: [source, sql] -------------------------------------------------- -ASCII(string_exp<1>) +ASCII(string_exp) <1> -------------------------------------------------- *Input*: @@ -35,7 +35,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringAscii] .Synopsis: [source, sql] -------------------------------------------------- -BIT_LENGTH(string_exp<1>) +BIT_LENGTH(string_exp) <1> -------------------------------------------------- *Input*: @@ -58,7 +58,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringBitLength] .Synopsis: [source, sql] -------------------------------------------------- -CHAR(code<1>) +CHAR(code) <1> -------------------------------------------------- *Input*: @@ -81,7 +81,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringChar] .Synopsis: [source, sql] -------------------------------------------------- -CHAR_LENGTH(string_exp<1>) +CHAR_LENGTH(string_exp) <1> -------------------------------------------------- *Input*: @@ -104,7 +104,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringCharLength] .Synopsis: [source, sql] -------------------------------------------------- -CONCAT(string_exp1<1>,string_exp2<2>) +CONCAT( + string_exp1, <1> + string_exp2) <2> -------------------------------------------------- *Input*: @@ -128,7 +130,11 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringConcat] .Synopsis: [source, sql] -------------------------------------------------- -INSERT(source<1>, start<2>, length<3>, replacement<4>) +INSERT( + source, <1> + start, <2> + length, <3> + replacement) <4> -------------------------------------------------- *Input*: @@ -154,7 +160,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringInsert] .Synopsis: [source, sql] -------------------------------------------------- -LCASE(string_exp<1>) +LCASE(string_exp) <1> -------------------------------------------------- *Input*: @@ -177,7 +183,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringLCase] .Synopsis: [source, sql] -------------------------------------------------- -LEFT(string_exp<1>, count<2>) +LEFT( + string_exp, <1> + count) <2> -------------------------------------------------- *Input*: @@ -201,7 +209,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringLeft] .Synopsis: [source, sql] -------------------------------------------------- -LENGTH(string_exp<1>) +LENGTH(string_exp) <1> -------------------------------------------------- *Input*: @@ -224,7 +232,11 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringLength] .Synopsis: [source, sql] -------------------------------------------------- -LOCATE(pattern<1>, source<2>[, start]<3>) +LOCATE( + pattern, <1> + source <2> + [, start]<3> +) -------------------------------------------------- *Input*: @@ -254,7 +266,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringLocateWithStart] .Synopsis: [source, sql] -------------------------------------------------- -LTRIM(string_exp<1>) +LTRIM(string_exp) <1> -------------------------------------------------- *Input*: @@ -277,7 +289,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringLTrim] .Synopsis: [source, sql] -------------------------------------------------- -OCTET_LENGTH(string_exp<1>) +OCTET_LENGTH(string_exp) <1> -------------------------------------------------- *Input*: @@ -300,7 +312,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringOctetLength] .Synopsis: [source, sql] -------------------------------------------------- -POSITION(string_exp1<1>, string_exp2<2>) +POSITION( + string_exp1, <1> + string_exp2) <2> -------------------------------------------------- *Input*: @@ -324,7 +338,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringPosition] .Synopsis: [source, sql] -------------------------------------------------- -REPEAT(string_exp<1>, count<2>) +REPEAT( + string_exp, <1> + count) <2> -------------------------------------------------- *Input*: @@ -348,7 +364,10 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringRepeat] .Synopsis: [source, sql] -------------------------------------------------- -REPLACE(source<1>, pattern<2>, replacement<3>) +REPLACE( + source, <1> + pattern, <2> + replacement) <3> -------------------------------------------------- *Input*: @@ -373,7 +392,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringReplace] .Synopsis: [source, sql] -------------------------------------------------- -RIGHT(string_exp<1>, count<2>) +RIGHT( + string_exp, <1> + count) <2> -------------------------------------------------- *Input*: @@ -397,7 +418,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringRight] .Synopsis: [source, sql] -------------------------------------------------- -RTRIM(string_exp<1>) +RTRIM(string_exp) <1> -------------------------------------------------- *Input*: @@ -420,7 +441,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringRTrim] .Synopsis: [source, sql] -------------------------------------------------- -SPACE(count<1>) +SPACE(count) <1> -------------------------------------------------- *Input*: @@ -443,7 +464,10 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringSpace] .Synopsis: [source, sql] -------------------------------------------------- -SUBSTRING(source<1>, start<2>, length<3>) +SUBSTRING( + source, <1> + start, <2> + length) <3> -------------------------------------------------- *Input*: @@ -468,7 +492,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[stringSubString] .Synopsis: [source, sql] -------------------------------------------------- -UCASE(string_exp<1>) +UCASE(string_exp) <1> -------------------------------------------------- *Input*: diff --git a/docs/reference/sql/functions/type-conversion.asciidoc b/docs/reference/sql/functions/type-conversion.asciidoc index 2187d5a2e9293..7f8488be40f64 100644 --- a/docs/reference/sql/functions/type-conversion.asciidoc +++ b/docs/reference/sql/functions/type-conversion.asciidoc @@ -11,7 +11,9 @@ Functions for converting an expression of one data type to another. .Synopsis: [source, sql] ---- -CAST(expression<1> AS data_type<2>) +CAST( + expression <1> + AS data_type) <2> ---- <1> Expression to cast @@ -50,7 +52,9 @@ To obtain an {es} `float`, perform casting to its SQL equivalent, `real` type. .Synopsis: [source, sql] ---- -CONVERT(expression<1>, data_type<2>) +CONVERT( + expression, <1> + data_type) <2> ---- <1> Expression to convert From da3726528d9011b05c0677ece6d11558994eccd9 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Tue, 16 Apr 2019 22:40:29 +0300 Subject: [PATCH 062/260] SQL: Fix LIMIT bug in agg sorting (#41258) When specifying a limit over an agg sorting, the limit will be pushed down to the grouping which affects the custom sorting. This commit fixes that and restricts the limit only to sorting. Fix #40984 --- .../src/main/resources/agg-ordering.sql-spec | 24 +++++++++++++++++++ .../sql/execution/search/SourceGenerator.java | 3 ++- 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg-ordering.sql-spec b/x-pack/plugin/sql/qa/src/main/resources/agg-ordering.sql-spec index ed206da6d1334..79d58c48e4469 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg-ordering.sql-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg-ordering.sql-spec @@ -29,9 +29,24 @@ SELECT MAX(salary) AS max, MIN(salary) AS min FROM test_emp HAVING MIN(salary) > aggWithoutAlias SELECT MAX(salary) AS max FROM test_emp GROUP BY gender ORDER BY MAX(salary); +aggWithoutAliasWithLimit +SELECT MAX(salary) AS max FROM test_emp GROUP BY gender ORDER BY MAX(salary) LIMIT 3; + +aggWithoutAliasWithLimitDesc +SELECT MAX(salary) AS max FROM test_emp GROUP BY gender ORDER BY MAX(salary) DESC LIMIT 3; + aggWithAlias SELECT MAX(salary) AS m FROM test_emp GROUP BY gender ORDER BY m; +aggOrderByCountWithLimit +SELECT MAX(salary) AS max, COUNT(*) AS c FROM test_emp GROUP BY gender ORDER BY c LIMIT 3; + +aggOrderByCountWithLimitDescAndGrouping +SELECT gender, COUNT(*) AS c FROM test_emp GROUP BY gender ORDER BY c DESC LIMIT 5; + +aggOrderByCountWithLimitDesc +SELECT MAX(salary) AS max, COUNT(*) AS c FROM test_emp GROUP BY gender ORDER BY c DESC LIMIT 3; + multipleAggsThatGetRewrittenWithoutAlias SELECT MAX(salary) AS max, MIN(salary) AS min FROM test_emp GROUP BY gender ORDER BY MAX(salary); @@ -56,12 +71,21 @@ SELECT MIN(salary) AS min, COUNT(*) AS c FROM test_emp GROUP BY gender HAVING c aggNotSpecifiedInTheAggregateAndGroupWithHaving SELECT gender, MIN(salary) AS min, COUNT(*) AS c FROM test_emp GROUP BY gender HAVING c > 1 ORDER BY MAX(salary), gender; +aggNotSpecifiedInTheAggregateAndGroupWithHavingWithLimit +SELECT gender, MIN(salary) AS min, COUNT(*) AS c FROM test_emp GROUP BY gender HAVING c > 1 ORDER BY MAX(salary), c LIMIT 5; + +aggNotSpecifiedInTheAggregateAndGroupWithHavingWithLimitAndDirection +SELECT gender, MIN(salary) AS min, COUNT(*) AS c FROM test_emp GROUP BY gender HAVING c > 1 ORDER BY MAX(salary) ASC, c DESC LIMIT 5; + groupAndAggNotSpecifiedInTheAggregateWithHaving SELECT gender, MIN(salary) AS min, COUNT(*) AS c FROM test_emp GROUP BY gender HAVING c > 1 ORDER BY gender, MAX(salary); multipleAggsThatGetRewrittenWithAliasOnAMediumGroupBy SELECT languages, MAX(salary) AS max, MIN(salary) AS min FROM test_emp GROUP BY languages ORDER BY max; +multipleAggsThatGetRewrittenWithAliasOnAMediumGroupByWithLimit +SELECT languages, MAX(salary) AS max, MIN(salary) AS min FROM test_emp GROUP BY languages ORDER BY max DESC LIMIT 5; + multipleAggsThatGetRewrittenWithAliasOnALargeGroupBy SELECT emp_no, MAX(salary) AS max, MIN(salary) AS min FROM test_emp GROUP BY emp_no ORDER BY max; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/SourceGenerator.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/SourceGenerator.java index a8876e441f749..8d9e59617aa4d 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/SourceGenerator.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/SourceGenerator.java @@ -80,7 +80,8 @@ public static SearchSourceBuilder sourceBuilder(QueryContainer container, QueryB if (source.size() == -1) { source.size(sz); } - if (aggBuilder instanceof CompositeAggregationBuilder) { + // limit the composite aggs only for non-local sorting + if (aggBuilder instanceof CompositeAggregationBuilder && container.sortingColumns().isEmpty()) { ((CompositeAggregationBuilder) aggBuilder).size(sz); } } From ebf79203f0ba0eb9577167d7dd0505146cc18a35 Mon Sep 17 00:00:00 2001 From: Yogesh Gaikwad <902768+bizybot@users.noreply.github.com> Date: Wed, 17 Apr 2019 12:19:54 +1000 Subject: [PATCH 063/260] put mapping authorization for alias with write-index and multiple read indices (#40834) When the same alias points to multiple indices we can write to only one index with `is_write_index` value `true`. The special handling in case of the put mapping request(to resolve authorized indices) has a check on indices size for a concrete index. If multiple indices existed then it marked the request as unauthorized. The check has been modified to consider write index flag and only when the requested index matches with the one with write index alias, the alias is considered for authorization. Closes #40831 --- .../xpack/security/PermissionsIT.java | 123 +++++++++++++++++- .../authz/IndicesAndAliasesResolver.java | 12 +- .../authz/IndicesAndAliasesResolverTests.java | 29 ++++- 3 files changed, 158 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java b/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java index 78fc2700f860e..5503e12cb8b0e 100644 --- a/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java +++ b/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java @@ -7,9 +7,12 @@ import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; +import org.elasticsearch.client.Node; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; @@ -19,6 +22,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xpack.core.indexlifecycle.DeleteAction; @@ -26,6 +30,7 @@ import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy; import org.elasticsearch.xpack.core.indexlifecycle.LifecycleSettings; import org.elasticsearch.xpack.core.indexlifecycle.Phase; +import org.elasticsearch.xpack.core.indexlifecycle.RolloverAction; import org.junit.Before; import java.io.IOException; @@ -36,8 +41,10 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; public class PermissionsIT extends ESRestTestCase { + private static final String jsonDoc = "{ \"name\" : \"elasticsearch\", \"body\": \"foo bar\" }"; private String deletePolicy = "deletePolicy"; private Settings indexSettingsWithPolicy; @@ -74,7 +81,7 @@ public void init() throws Exception { .put("number_of_shards", 1) .put("number_of_replicas", 0) .build(); - createNewSingletonPolicy(deletePolicy,"delete", new DeleteAction()); + createNewSingletonPolicy(client(), deletePolicy,"delete", new DeleteAction()); } /** @@ -126,7 +133,62 @@ public void testCanViewExplainOnUnmanagedIndex() throws Exception { assertOK(client().performRequest(request)); } - private void createNewSingletonPolicy(String policy, String phaseName, LifecycleAction action) throws IOException { + /** + * Tests when the user is limited by alias of an index is able to write to index + * which was rolled over by an ILM policy. + */ + public void testWhenUserLimitedByOnlyAliasOfIndexCanWriteToIndexWhichWasRolledoverByILMPolicy() + throws IOException, InterruptedException { + /* + * Setup: + * - ILM policy to rollover index when max docs condition is met + * - Index template to which the ILM policy applies and create Index + * - Create role with just write and manage privileges on alias + * - Create user and assign newly created role. + */ + createNewSingletonPolicy(adminClient(), "foo-policy", "hot", new RolloverAction(null, null, 2L)); + createIndexTemplate("foo-template", "foo-logs-*", "foo_alias", "foo-policy"); + createIndexAsAdmin("foo-logs-000001", "foo_alias", randomBoolean()); + createRole("foo_alias_role", "foo_alias"); + createUser("test_user", "x-pack-test-password", "foo_alias_role"); + + // test_user: index docs using alias in the newly created index + indexDocs("test_user", "x-pack-test-password", "foo_alias", 2); + refresh("foo_alias"); + + // wait so the ILM policy triggers rollover action, verify that the new index exists + assertThat(awaitBusy(() -> { + Request request = new Request("HEAD", "/" + "foo-logs-000002"); + int status; + try { + status = adminClient().performRequest(request).getStatusLine().getStatusCode(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return status == 200; + }), is(true)); + + // test_user: index docs using alias, now should be able write to new index + indexDocs("test_user", "x-pack-test-password", "foo_alias", 1); + refresh("foo_alias"); + + // verify that the doc has been indexed into new write index + awaitBusy(() -> { + Request request = new Request("GET", "/foo-logs-000002/_search"); + Response response; + try { + response = adminClient().performRequest(request); + try (InputStream content = response.getEntity().getContent()) { + Map map = XContentHelper.convertToMap(JsonXContent.jsonXContent, content, false); + return ((Integer) XContentMapValues.extractValue("hits.total.value", map)) == 1; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private void createNewSingletonPolicy(RestClient client, String policy, String phaseName, LifecycleAction action) throws IOException { Phase phase = new Phase(phaseName, TimeValue.ZERO, singletonMap(action.getWriteableName(), action)); LifecyclePolicy lifecyclePolicy = new LifecyclePolicy(policy, singletonMap(phase.getName(), phase)); XContentBuilder builder = jsonBuilder(); @@ -135,7 +197,7 @@ private void createNewSingletonPolicy(String policy, String phaseName, Lifecycle "{ \"policy\":" + Strings.toString(builder) + "}", ContentType.APPLICATION_JSON); Request request = new Request("PUT", "_ilm/policy/" + policy); request.setEntity(entity); - client().performRequest(request); + assertOK(client.performRequest(request)); } private void createIndexAsAdmin(String name, Settings settings, String mapping) throws IOException { @@ -144,4 +206,59 @@ private void createIndexAsAdmin(String name, Settings settings, String mapping) + ", \"mappings\" : {" + mapping + "} }"); assertOK(adminClient().performRequest(request)); } + + private void createIndexAsAdmin(String name, String alias, boolean isWriteIndex) throws IOException { + Request request = new Request("PUT", "/" + name); + request.setJsonEntity("{ \"aliases\": { \""+alias+"\": {" + ((isWriteIndex) ? "\"is_write_index\" : true" : "") + + "} } }"); + assertOK(adminClient().performRequest(request)); + } + + private void createIndexTemplate(String name, String pattern, String alias, String policy) throws IOException { + Request request = new Request("PUT", "/_template/" + name); + request.setJsonEntity("{\n" + + " \"index_patterns\": [\""+pattern+"\"],\n" + + " \"settings\": {\n" + + " \"number_of_shards\": 1,\n" + + " \"number_of_replicas\": 0,\n" + + " \"index.lifecycle.name\": \""+policy+"\",\n" + + " \"index.lifecycle.rollover_alias\": \""+alias+"\"\n" + + " }\n" + + " }"); + assertOK(adminClient().performRequest(request)); + } + + private void createUser(String name, String password, String role) throws IOException { + Request request = new Request("PUT", "/_security/user/" + name); + request.setJsonEntity("{ \"password\": \""+password+"\", \"roles\": [ \""+ role+"\"] }"); + assertOK(adminClient().performRequest(request)); + } + + private void createRole(String name, String alias) throws IOException { + Request request = new Request("PUT", "/_security/role/" + name); + request.setJsonEntity("{ \"indices\": [ { \"names\" : [ \""+ alias+"\"], \"privileges\": [ \"write\", \"manage\" ] } ] }"); + assertOK(adminClient().performRequest(request)); + } + + private void indexDocs(String user, String passwd, String index, int noOfDocs) throws IOException { + RestClientBuilder builder = RestClient.builder(adminClient().getNodes().toArray(new Node[0])); + String token = basicAuthHeaderValue(user, new SecureString(passwd.toCharArray())); + configureClient(builder, Settings.builder() + .put(ThreadContext.PREFIX + ".Authorization", token) + .build()); + builder.setStrictDeprecationMode(true); + try (RestClient userClient = builder.build();) { + + for (int cnt = 0; cnt < noOfDocs; cnt++) { + Request request = new Request("POST", "/" + index + "/_doc"); + request.setJsonEntity(jsonDoc); + assertOK(userClient.performRequest(request)); + } + } + } + + private void refresh(String index) throws IOException { + Request request = new Request("POST", "/" + index + "/_refresh"); + assertOK(adminClient().performRequest(request)); + } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index bb5b92dba43a5..b0965a1c5330e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -249,7 +249,17 @@ static String getPutMappingIndexOrAlias(PutMappingRequest request, List Optional foundAlias = aliasMetaData.stream() .map(AliasMetaData::alias) .filter(authorizedIndicesList::contains) - .filter(aliasName -> metaData.getAliasAndIndexLookup().get(aliasName).getIndices().size() == 1) + .filter(aliasName -> { + AliasOrIndex alias = metaData.getAliasAndIndexLookup().get(aliasName); + List indexMetadata = alias.getIndices(); + if (indexMetadata.size() == 1) { + return true; + } else { + assert alias instanceof AliasOrIndex.Alias; + IndexMetaData idxMeta = ((AliasOrIndex.Alias) alias).getWriteIndex(); + return idxMeta != null && idxMeta.getIndex().getName().equals(concreteIndexName); + } + }) .findFirst(); resolvedAliasOrIndex = foundAlias.orElse(concreteIndexName); } else { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index 2f09b74ac3d53..dc32580980e02 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -72,6 +72,7 @@ import org.junit.Before; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -104,7 +105,6 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { private IndicesAndAliasesResolver defaultIndicesResolver; private IndexNameExpressionResolver indexNameExpressionResolver; private Map roleMap; - private FieldPermissionsCache fieldPermissionsCache; @Before public void setup() { @@ -138,13 +138,15 @@ public void setup() { .put(indexBuilder("-index11").settings(settings)) .put(indexBuilder("-index20").settings(settings)) .put(indexBuilder("-index21").settings(settings)) + .put(indexBuilder("logs-00001").putAlias(AliasMetaData.builder("logs-alias").writeIndex(false)).settings(settings)) + .put(indexBuilder("logs-00002").putAlias(AliasMetaData.builder("logs-alias").writeIndex(false)).settings(settings)) + .put(indexBuilder("logs-00003").putAlias(AliasMetaData.builder("logs-alias").writeIndex(true)).settings(settings)) .put(indexBuilder(securityIndexName).settings(settings)).build(); if (withAlias) { metaData = SecurityTestUtils.addAliasToMetaData(metaData, securityIndexName); } this.metaData = metaData; - this.fieldPermissionsCache = new FieldPermissionsCache(settings); user = new User("user", "role"); userDashIndices = new User("dash", "dash"); @@ -1355,6 +1357,29 @@ public void testDynamicPutMappingRequestFromAlias() { request = new PutMappingRequest(Strings.EMPTY_ARRAY).setConcreteIndex(new Index(index, UUIDs.base64UUID())); putMappingIndexOrAlias = IndicesAndAliasesResolver.getPutMappingIndexOrAlias(request, authorizedIndices, metaData); assertEquals(index, putMappingIndexOrAlias); + + } + + public void testWhenAliasToMultipleIndicesAndUserIsAuthorizedUsingAliasReturnsAliasNameForDynamicPutMappingRequestOnWriteIndex() { + String index = "logs-00003"; // write index + PutMappingRequest request = new PutMappingRequest(Strings.EMPTY_ARRAY).setConcreteIndex(new Index(index, UUIDs.base64UUID())); + List authorizedIndices = Collections.singletonList("logs-alias"); + assert metaData.getAliasAndIndexLookup().get("logs-alias").getIndices().size() == 3; + String putMappingIndexOrAlias = IndicesAndAliasesResolver.getPutMappingIndexOrAlias(request, authorizedIndices, metaData); + String message = "user is authorized to access `logs-alias` and the put mapping request is for a write index" + + "so this should have returned the alias name"; + assertEquals(message, "logs-alias", putMappingIndexOrAlias); + } + + public void testWhenAliasToMultipleIndicesAndUserIsAuthorizedUsingAliasReturnsIndexNameForDynamicPutMappingRequestOnReadIndex() { + String index = "logs-00002"; // read index + PutMappingRequest request = new PutMappingRequest(Strings.EMPTY_ARRAY).setConcreteIndex(new Index(index, UUIDs.base64UUID())); + List authorizedIndices = Collections.singletonList("logs-alias"); + assert metaData.getAliasAndIndexLookup().get("logs-alias").getIndices().size() == 3; + String putMappingIndexOrAlias = IndicesAndAliasesResolver.getPutMappingIndexOrAlias(request, authorizedIndices, metaData); + String message = "user is authorized to access `logs-alias` and the put mapping request is for a read index" + + "so this should have returned the concrete index as fallback"; + assertEquals(message, index, putMappingIndexOrAlias); } // TODO with the removal of DeleteByQuery is there another way to test resolving a write action? From 97fd2ed75770bb747473f000ab8237a5b9c36b7d Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Wed, 17 Apr 2019 07:40:40 +0200 Subject: [PATCH 064/260] Remove pending deletions workaround (#41245) The recent upgrade to lucene 8.1.0 included the fix for LUCENE-8735 (pending deletions did not delegate). Removed the temporary workaround that was previously added for this issue. Follow up to #40345 --- .../org/elasticsearch/index/store/SmbDirectoryWrapper.java | 7 ------- .../org/elasticsearch/index/shard/LocalShardSnapshot.java | 7 ------- .../java/org/elasticsearch/index/shard/StoreRecovery.java | 6 ------ .../index/store/ByteSizeCachingDirectory.java | 7 ------- .../src/main/java/org/elasticsearch/index/store/Store.java | 7 ------- .../org/elasticsearch/index/shard/IndexShardTests.java | 6 ------ .../index/store/ByteSizeCachingDirectoryTests.java | 7 ------- .../elasticsearch/test/store/MockFSDirectoryService.java | 7 ------- 8 files changed, 54 deletions(-) diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/SmbDirectoryWrapper.java b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/SmbDirectoryWrapper.java index c2a3af360935c..9264d3403fb34 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/SmbDirectoryWrapper.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/SmbDirectoryWrapper.java @@ -30,7 +30,6 @@ import java.nio.channels.Channels; import java.nio.file.Files; import java.nio.file.StandardOpenOption; -import java.util.Set; /** * This class is used to wrap an existing {@link org.apache.lucene.store.FSDirectory} so that @@ -80,10 +79,4 @@ public void write(byte[] b, int offset, int length) throws IOException { CHUNK_SIZE); } } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } } diff --git a/server/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java b/server/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java index 72b99f4d4868f..cc9ac40c2744d 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java @@ -33,7 +33,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.Collection; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; final class LocalShardSnapshot implements Closeable { @@ -117,12 +116,6 @@ public Lock obtainLock(String name) throws IOException { public void close() throws IOException { throw new UnsupportedOperationException("nobody should close this directory wrapper"); } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } }; } diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 06b6fa557983e..c97c19eb0f3ec 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -259,12 +259,6 @@ public void readBytes(byte[] b, int offset, int len) throws IOException { assert index.getFileDetails(dest).recovered() == l : index.getFileDetails(dest).toString(); } } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } } /** diff --git a/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java b/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java index 9a202a9b4cd31..60d58abd66e8e 100644 --- a/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java +++ b/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java @@ -32,7 +32,6 @@ import java.io.UncheckedIOException; import java.nio.file.AccessDeniedException; import java.nio.file.NoSuchFileException; -import java.util.Set; final class ByteSizeCachingDirectory extends FilterDirectory { @@ -180,10 +179,4 @@ public void deleteFile(String name) throws IOException { } } } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } } diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index f860e7fd940a8..65d2f8d7812f8 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -760,13 +760,6 @@ public IndexInput openInput(String name, IOContext context) throws IOException { public String toString() { return "store(" + in.toString() + ")"; } - - @Override - public Set getPendingDeletions() throws IOException { - // FilterDirectory.getPendingDeletions does not delegate, working around it here. - // to be removed once fixed in FilterDirectory. - return unwrap(this).getPendingDeletions(); - } } /** diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 05a81c6de3c18..0c7a1033eedb0 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1457,12 +1457,6 @@ public String[] listAll() throws IOException { return super.listAll(); } } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } }; try (Store store = createStore(shardId, new IndexSettings(metaData, Settings.EMPTY), directory)) { diff --git a/server/src/test/java/org/elasticsearch/index/store/ByteSizeCachingDirectoryTests.java b/server/src/test/java/org/elasticsearch/index/store/ByteSizeCachingDirectoryTests.java index 509f5e2a4c41b..49de00dd8bef6 100644 --- a/server/src/test/java/org/elasticsearch/index/store/ByteSizeCachingDirectoryTests.java +++ b/server/src/test/java/org/elasticsearch/index/store/ByteSizeCachingDirectoryTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.test.ESTestCase; import java.io.IOException; -import java.util.Set; @LuceneTestCase.SuppressFileSystems("ExtrasFS") public class ByteSizeCachingDirectoryTests extends ESTestCase { @@ -46,12 +45,6 @@ public long fileLength(String name) throws IOException { numFileLengthCalls++; return super.fileLength(name); } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } } public void testBasics() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java index 2fcdf9be511a3..65a66989cdd97 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -54,7 +54,6 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.Random; -import java.util.Set; public class MockFSDirectoryService extends FsDirectoryService { @@ -179,12 +178,6 @@ public synchronized void crash() throws IOException { super.crash(); } } - - // temporary override until LUCENE-8735 is integrated - @Override - public Set getPendingDeletions() throws IOException { - return in.getPendingDeletions(); - } } static final class CloseableDirectory implements Closeable { From db30aac9dfbadbc92b11435dc054ce65d5509d92 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 17 Apr 2019 08:58:11 +0300 Subject: [PATCH 065/260] fix the packer cache script (#41183) * fix the packer cache script This PR disabled the explicit pull since it seems this always tries to work with a registry. Functionality will not be affected since we will still build the images on pull. --- distribution/docker/build.gradle | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/distribution/docker/build.gradle b/distribution/docker/build.gradle index 07e8926b2a367..71fc62673dca9 100644 --- a/distribution/docker/build.gradle +++ b/distribution/docker/build.gradle @@ -138,3 +138,8 @@ for (final boolean oss : [false, true]) { assemble.dependsOn "buildOssDockerImage" assemble.dependsOn "buildDockerImage" + +// We build the images used in compose locally, but the pull command insists on using a repository +// thus we must disable it to prevent it from doing so. +// Everything will still be pulled since we will build the local images on a pull +composePull.enabled = false From bfbc9f140144b5a35aa29008b58bf58074419853 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 17 Apr 2019 10:12:11 +0300 Subject: [PATCH 066/260] SQL: Allow current_date/time/timestamp to be also used as a function escape pattern (#41254) CURRENT_DATE/CURRENT_TIME/CURRENT_TIMESTAMP can be used as SQL keywords (without parentheses) and therefore there is a special rule in the grammar to accommodate this. Previously, this rule was also catching the parenthesised version of those functions too, not allowing the {fn ()} to be used. E.g.: {fn current_time(2)} or {fn current_timestamp()} Now, the grammar rule catches only the keyword versions and all the parenthesised versions go through the normal function resolution. As a consequence the validation of the precision is moved from the parser lever (ExpressionBuilder) to the function implementations. Fixes: #41240 --- x-pack/plugin/sql/src/main/antlr/SqlBase.g4 | 8 +- .../scalar/datetime/CurrentDateTime.java | 12 +- .../function/scalar/datetime/CurrentTime.java | 13 +- .../xpack/sql/parser/ExpressionBuilder.java | 20 +- .../xpack/sql/parser/SqlBaseParser.java | 722 +++++++++--------- .../xpack/sql/util/DateUtils.java | 25 + .../scalar/datetime/CurrentDateTimeTests.java | 31 +- .../scalar/datetime/CurrentTimeTests.java | 31 +- .../sql/parser/EscapedFunctionsTests.java | 23 + .../xpack/sql/parser/ExpressionTests.java | 14 +- 10 files changed, 468 insertions(+), 431 deletions(-) diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 index fd47efbc5c667..d814e9e60f246 100644 --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 @@ -226,9 +226,9 @@ primaryExpression ; builtinDateTimeFunction - : name=CURRENT_TIMESTAMP ('(' precision=INTEGER_VALUE? ')')? - | name=CURRENT_DATE ('(' ')')? - | name=CURRENT_TIME ('(' precision=INTEGER_VALUE? ')')? + : name=CURRENT_TIMESTAMP + | name=CURRENT_DATE + | name=CURRENT_TIME ; castExpression @@ -340,7 +340,7 @@ string // http://developer.mimer.se/validator/sql-reserved-words.tml nonReserved : ANALYZE | ANALYZED - | CATALOGS | COLUMNS + | CATALOGS | COLUMNS | CURRENT_DATE | CURRENT_TIME | CURRENT_TIMESTAMP | DAY | DEBUG | EXECUTABLE | EXPLAIN | FIRST | FORMAT | FULL | FUNCTIONS diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java index 59f1251096fb7..16791421eb69b 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTime.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.Foldables; import org.elasticsearch.xpack.sql.session.Configuration; import org.elasticsearch.xpack.sql.tree.NodeInfo; import org.elasticsearch.xpack.sql.tree.Source; @@ -15,6 +14,8 @@ import java.time.ZonedDateTime; +import static org.elasticsearch.xpack.sql.util.DateUtils.getNanoPrecision; + public class CurrentDateTime extends CurrentFunction { private final Expression precision; @@ -34,13 +35,6 @@ protected NodeInfo info() { } static ZonedDateTime nanoPrecision(ZonedDateTime zdt, Expression precisionExpression) { - int precision = precisionExpression != null ? Foldables.intValueOf(precisionExpression) : 3; - int nano = zdt.getNano(); - if (precision >= 0 && precision < 10) { - // remove the remainder - nano = nano - nano % (int) Math.pow(10, (9 - precision)); - return zdt.withNano(nano); - } - return zdt; + return zdt.withNano(getNanoPrecision(precisionExpression, zdt.getNano())); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTime.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTime.java index 5ab2dd9e8b417..dac8216e0c5af 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTime.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTime.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.Foldables; import org.elasticsearch.xpack.sql.session.Configuration; import org.elasticsearch.xpack.sql.tree.NodeInfo; import org.elasticsearch.xpack.sql.tree.Source; @@ -15,6 +14,8 @@ import java.time.OffsetTime; +import static org.elasticsearch.xpack.sql.util.DateUtils.getNanoPrecision; + public class CurrentTime extends CurrentFunction { private final Expression precision; @@ -35,13 +36,7 @@ protected NodeInfo info() { } static OffsetTime nanoPrecision(OffsetTime ot, Expression precisionExpression) { - int precision = precisionExpression != null ? Foldables.intValueOf(precisionExpression) : 3; - int nano = ot.getNano(); - if (precision >= 0 && precision < 10) { - // remove the remainder - nano = nano - nano % (int) Math.pow(10, (9 - precision)); - return ot.withNano(nano); - } - return ot; + return ot.withNano(getNanoPrecision(precisionExpression, ot.getNano())); } + } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java index 78f68342b6991..ad2539ab99b6d 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java @@ -109,7 +109,6 @@ import org.elasticsearch.xpack.sql.proto.SqlTypedParamValue; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; -import org.elasticsearch.xpack.sql.type.DataTypeConversion; import org.elasticsearch.xpack.sql.type.DataTypes; import org.elasticsearch.xpack.sql.util.StringUtils; @@ -431,30 +430,15 @@ public Object visitBuiltinDateTimeFunction(BuiltinDateTimeFunctionContext ctx) { // maps CURRENT_XXX to its respective function e.g: CURRENT_TIMESTAMP() // since the functions need access to the Configuration, the parser only registers the definition and not the actual function Source source = source(ctx); - Literal p = null; - - if (ctx.precision != null) { - try { - Source pSource = source(ctx.precision); - short safeShort = DataTypeConversion.safeToShort(StringUtils.parseLong(ctx.precision.getText())); - if (safeShort > 9 || safeShort < 0) { - throw new ParsingException(pSource, "Precision needs to be between [0-9], received [{}]", safeShort); - } - p = Literal.of(pSource, Short.valueOf(safeShort)); - } catch (SqlIllegalArgumentException siae) { - throw new ParsingException(source, siae.getMessage()); - } - } - String functionName = ctx.name.getText(); switch (ctx.name.getType()) { case SqlBaseLexer.CURRENT_TIMESTAMP: - return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, p != null ? singletonList(p) : emptyList()); + return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, emptyList()); case SqlBaseLexer.CURRENT_DATE: return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, emptyList()); case SqlBaseLexer.CURRENT_TIME: - return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, p != null ? singletonList(p) : emptyList()); + return new UnresolvedFunction(source, functionName, ResolutionType.STANDARD, emptyList()); default: throw new ParsingException(source, "Unknown function [{}]", functionName); } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java index 5bad02b1392dd..6769cc88695d6 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java @@ -1,13 +1,27 @@ // ANTLR GENERATED CODE: DO NOT EDIT package org.elasticsearch.xpack.sql.parser; -import org.antlr.v4.runtime.atn.*; + +import org.antlr.v4.runtime.FailedPredicateException; +import org.antlr.v4.runtime.NoViableAltException; +import org.antlr.v4.runtime.Parser; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.RecognitionException; +import org.antlr.v4.runtime.RuleContext; +import org.antlr.v4.runtime.RuntimeMetaData; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.TokenStream; +import org.antlr.v4.runtime.Vocabulary; +import org.antlr.v4.runtime.VocabularyImpl; +import org.antlr.v4.runtime.atn.ATN; +import org.antlr.v4.runtime.atn.ATNDeserializer; +import org.antlr.v4.runtime.atn.ParserATNSimulator; +import org.antlr.v4.runtime.atn.PredictionContextCache; import org.antlr.v4.runtime.dfa.DFA; -import org.antlr.v4.runtime.*; -import org.antlr.v4.runtime.misc.*; -import org.antlr.v4.runtime.tree.*; +import org.antlr.v4.runtime.tree.ParseTreeListener; +import org.antlr.v4.runtime.tree.ParseTreeVisitor; +import org.antlr.v4.runtime.tree.TerminalNode; + import java.util.List; -import java.util.Iterator; -import java.util.ArrayList; @SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"}) class SqlBaseParser extends Parser { @@ -768,6 +782,9 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -844,6 +861,9 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -914,6 +934,9 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -1087,6 +1110,9 @@ public final StatementContext statement() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -4241,7 +4267,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _prevctx = _localctx; setState(577); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { { setState(574); qualifiedName(); @@ -4339,9 +4365,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc public static class BuiltinDateTimeFunctionContext extends ParserRuleContext { public Token name; - public Token precision; public TerminalNode CURRENT_TIMESTAMP() { return getToken(SqlBaseParser.CURRENT_TIMESTAMP, 0); } - public TerminalNode INTEGER_VALUE() { return getToken(SqlBaseParser.INTEGER_VALUE, 0); } public TerminalNode CURRENT_DATE() { return getToken(SqlBaseParser.CURRENT_DATE, 0); } public TerminalNode CURRENT_TIME() { return getToken(SqlBaseParser.CURRENT_TIME, 0); } public BuiltinDateTimeFunctionContext(ParserRuleContext parent, int invokingState) { @@ -4366,83 +4390,28 @@ public T accept(ParseTreeVisitor visitor) { public final BuiltinDateTimeFunctionContext builtinDateTimeFunction() throws RecognitionException { BuiltinDateTimeFunctionContext _localctx = new BuiltinDateTimeFunctionContext(_ctx, getState()); enterRule(_localctx, 62, RULE_builtinDateTimeFunction); - int _la; try { - setState(621); + setState(603); switch (_input.LA(1)) { case CURRENT_TIMESTAMP: enterOuterAlt(_localctx, 1); { setState(600); ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_TIMESTAMP); - setState(606); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,82,_ctx) ) { - case 1: - { - setState(601); - match(T__0); - setState(603); - _la = _input.LA(1); - if (_la==INTEGER_VALUE) { - { - setState(602); - ((BuiltinDateTimeFunctionContext)_localctx).precision = match(INTEGER_VALUE); - } - } - - setState(605); - match(T__1); - } - break; - } } break; case CURRENT_DATE: enterOuterAlt(_localctx, 2); { - setState(608); + setState(601); ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_DATE); - setState(611); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,83,_ctx) ) { - case 1: - { - setState(609); - match(T__0); - setState(610); - match(T__1); - } - break; - } } break; case CURRENT_TIME: enterOuterAlt(_localctx, 3); { - setState(613); + setState(602); ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_TIME); - setState(619); - _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,85,_ctx) ) { - case 1: - { - setState(614); - match(T__0); - setState(616); - _la = _input.LA(1); - if (_la==INTEGER_VALUE) { - { - setState(615); - ((BuiltinDateTimeFunctionContext)_localctx).precision = match(INTEGER_VALUE); - } - } - - setState(618); - match(T__1); - } - break; - } } break; default: @@ -4492,42 +4461,42 @@ public final CastExpressionContext castExpression() throws RecognitionException CastExpressionContext _localctx = new CastExpressionContext(_ctx, getState()); enterRule(_localctx, 64, RULE_castExpression); try { - setState(633); + setState(615); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,87,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,82,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(623); + setState(605); castTemplate(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(624); + setState(606); match(FUNCTION_ESC); - setState(625); + setState(607); castTemplate(); - setState(626); + setState(608); match(ESC_END); } break; case 3: enterOuterAlt(_localctx, 3); { - setState(628); + setState(610); convertTemplate(); } break; case 4: enterOuterAlt(_localctx, 4); { - setState(629); + setState(611); match(FUNCTION_ESC); - setState(630); + setState(612); convertTemplate(); - setState(631); + setState(613); match(ESC_END); } break; @@ -4578,17 +4547,17 @@ public final CastTemplateContext castTemplate() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(635); + setState(617); match(CAST); - setState(636); + setState(618); match(T__0); - setState(637); + setState(619); expression(); - setState(638); + setState(620); match(AS); - setState(639); + setState(621); dataType(); - setState(640); + setState(622); match(T__1); } } @@ -4636,17 +4605,17 @@ public final ConvertTemplateContext convertTemplate() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(642); + setState(624); match(CONVERT); - setState(643); + setState(625); match(T__0); - setState(644); + setState(626); expression(); - setState(645); + setState(627); match(T__2); - setState(646); + setState(628); dataType(); - setState(647); + setState(629); match(T__1); } } @@ -4690,23 +4659,23 @@ public final ExtractExpressionContext extractExpression() throws RecognitionExce ExtractExpressionContext _localctx = new ExtractExpressionContext(_ctx, getState()); enterRule(_localctx, 70, RULE_extractExpression); try { - setState(654); + setState(636); switch (_input.LA(1)) { case EXTRACT: enterOuterAlt(_localctx, 1); { - setState(649); + setState(631); extractTemplate(); } break; case FUNCTION_ESC: enterOuterAlt(_localctx, 2); { - setState(650); + setState(632); match(FUNCTION_ESC); - setState(651); + setState(633); extractTemplate(); - setState(652); + setState(634); match(ESC_END); } break; @@ -4760,17 +4729,17 @@ public final ExtractTemplateContext extractTemplate() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(656); + setState(638); match(EXTRACT); - setState(657); + setState(639); match(T__0); - setState(658); + setState(640); ((ExtractTemplateContext)_localctx).field = identifier(); - setState(659); + setState(641); match(FROM); - setState(660); + setState(642); valueExpression(0); - setState(661); + setState(643); match(T__1); } } @@ -4813,12 +4782,15 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx FunctionExpressionContext _localctx = new FunctionExpressionContext(_ctx, getState()); enterRule(_localctx, 74, RULE_functionExpression); try { - setState(668); + setState(650); switch (_input.LA(1)) { case ANALYZE: case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -4859,18 +4831,18 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 1); { - setState(663); + setState(645); functionTemplate(); } break; case FUNCTION_ESC: enterOuterAlt(_localctx, 2); { - setState(664); + setState(646); match(FUNCTION_ESC); - setState(665); + setState(647); functionTemplate(); - setState(666); + setState(648); match(ESC_END); } break; @@ -4928,45 +4900,45 @@ public final FunctionTemplateContext functionTemplate() throws RecognitionExcept try { enterOuterAlt(_localctx, 1); { - setState(670); + setState(652); functionName(); - setState(671); + setState(653); match(T__0); - setState(683); + setState(665); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ALL) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << DISTINCT) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RIGHT - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TRUE - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (FUNCTION_ESC - 64)) | (1L << (DATE_ESC - 64)) | (1L << (TIME_ESC - 64)) | (1L << (TIMESTAMP_ESC - 64)) | (1L << (GUID_ESC - 64)) | (1L << (PLUS - 64)) | (1L << (MINUS - 64)) | (1L << (ASTERISK - 64)) | (1L << (PARAM - 64)) | (1L << (STRING - 64)) | (1L << (INTEGER_VALUE - 64)) | (1L << (DECIMAL_VALUE - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { { - setState(673); + setState(655); _la = _input.LA(1); if (_la==ALL || _la==DISTINCT) { { - setState(672); + setState(654); setQuantifier(); } } - setState(675); + setState(657); expression(); - setState(680); + setState(662); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(676); + setState(658); match(T__2); - setState(677); + setState(659); expression(); } } - setState(682); + setState(664); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(685); + setState(667); match(T__1); } } @@ -5010,19 +4982,19 @@ public final FunctionNameContext functionName() throws RecognitionException { FunctionNameContext _localctx = new FunctionNameContext(_ctx, getState()); enterRule(_localctx, 78, RULE_functionName); try { - setState(690); + setState(672); switch (_input.LA(1)) { case LEFT: enterOuterAlt(_localctx, 1); { - setState(687); + setState(669); match(LEFT); } break; case RIGHT: enterOuterAlt(_localctx, 2); { - setState(688); + setState(670); match(RIGHT); } break; @@ -5030,6 +5002,9 @@ public final FunctionNameContext functionName() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -5068,7 +5043,7 @@ public final FunctionNameContext functionName() throws RecognitionException { case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 3); { - setState(689); + setState(671); identifier(); } break; @@ -5299,13 +5274,13 @@ public final ConstantContext constant() throws RecognitionException { enterRule(_localctx, 80, RULE_constant); try { int _alt; - setState(718); + setState(700); switch (_input.LA(1)) { case NULL: _localctx = new NullLiteralContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(692); + setState(674); match(NULL); } break; @@ -5313,7 +5288,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new IntervalLiteralContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(693); + setState(675); interval(); } break; @@ -5322,7 +5297,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new NumericLiteralContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(694); + setState(676); number(); } break; @@ -5331,7 +5306,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new BooleanLiteralContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(695); + setState(677); booleanValue(); } break; @@ -5339,7 +5314,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new StringLiteralContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(697); + setState(679); _errHandler.sync(this); _alt = 1; do { @@ -5347,7 +5322,7 @@ public final ConstantContext constant() throws RecognitionException { case 1: { { - setState(696); + setState(678); match(STRING); } } @@ -5355,9 +5330,9 @@ public final ConstantContext constant() throws RecognitionException { default: throw new NoViableAltException(this); } - setState(699); + setState(681); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,94,_ctx); + _alt = getInterpreter().adaptivePredict(_input,89,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); } break; @@ -5365,7 +5340,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new ParamLiteralContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(701); + setState(683); match(PARAM); } break; @@ -5373,11 +5348,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new DateEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(702); + setState(684); match(DATE_ESC); - setState(703); + setState(685); string(); - setState(704); + setState(686); match(ESC_END); } break; @@ -5385,11 +5360,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new TimeEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(706); + setState(688); match(TIME_ESC); - setState(707); + setState(689); string(); - setState(708); + setState(690); match(ESC_END); } break; @@ -5397,11 +5372,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new TimestampEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(710); + setState(692); match(TIMESTAMP_ESC); - setState(711); + setState(693); string(); - setState(712); + setState(694); match(ESC_END); } break; @@ -5409,11 +5384,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new GuidEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 10); { - setState(714); + setState(696); match(GUID_ESC); - setState(715); + setState(697); string(); - setState(716); + setState(698); match(ESC_END); } break; @@ -5466,7 +5441,7 @@ public final ComparisonOperatorContext comparisonOperator() throws RecognitionEx try { enterOuterAlt(_localctx, 1); { - setState(720); + setState(702); _la = _input.LA(1); if ( !(((((_la - 101)) & ~0x3f) == 0 && ((1L << (_la - 101)) & ((1L << (EQ - 101)) | (1L << (NULLEQ - 101)) | (1L << (NEQ - 101)) | (1L << (LT - 101)) | (1L << (LTE - 101)) | (1L << (GT - 101)) | (1L << (GTE - 101)))) != 0)) ) { _errHandler.recoverInline(this); @@ -5515,7 +5490,7 @@ public final BooleanValueContext booleanValue() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(722); + setState(704); _la = _input.LA(1); if ( !(_la==FALSE || _la==TRUE) ) { _errHandler.recoverInline(this); @@ -5583,13 +5558,13 @@ public final IntervalContext interval() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(724); + setState(706); match(INTERVAL); - setState(726); + setState(708); _la = _input.LA(1); if (_la==PLUS || _la==MINUS) { { - setState(725); + setState(707); ((IntervalContext)_localctx).sign = _input.LT(1); _la = _input.LA(1); if ( !(_la==PLUS || _la==MINUS) ) { @@ -5600,35 +5575,35 @@ public final IntervalContext interval() throws RecognitionException { } } - setState(730); + setState(712); switch (_input.LA(1)) { case INTEGER_VALUE: case DECIMAL_VALUE: { - setState(728); + setState(710); ((IntervalContext)_localctx).valueNumeric = number(); } break; case PARAM: case STRING: { - setState(729); + setState(711); ((IntervalContext)_localctx).valuePattern = string(); } break; default: throw new NoViableAltException(this); } - setState(732); + setState(714); ((IntervalContext)_localctx).leading = intervalField(); - setState(735); + setState(717); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,98,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,93,_ctx) ) { case 1: { - setState(733); + setState(715); match(TO); - setState(734); + setState(716); ((IntervalContext)_localctx).trailing = intervalField(); } break; @@ -5685,7 +5660,7 @@ public final IntervalFieldContext intervalField() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(737); + setState(719); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << DAY) | (1L << DAYS) | (1L << HOUR) | (1L << HOURS) | (1L << MINUTE) | (1L << MINUTES) | (1L << MONTH) | (1L << MONTHS))) != 0) || ((((_la - 75)) & ~0x3f) == 0 && ((1L << (_la - 75)) & ((1L << (SECOND - 75)) | (1L << (SECONDS - 75)) | (1L << (YEAR - 75)) | (1L << (YEARS - 75)))) != 0)) ) { _errHandler.recoverInline(this); @@ -5743,7 +5718,7 @@ public final DataTypeContext dataType() throws RecognitionException { _localctx = new PrimitiveDataTypeContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(739); + setState(721); identifier(); } } @@ -5795,25 +5770,25 @@ public final QualifiedNameContext qualifiedName() throws RecognitionException { int _alt; enterOuterAlt(_localctx, 1); { - setState(746); + setState(728); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,99,_ctx); + _alt = getInterpreter().adaptivePredict(_input,94,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(741); + setState(723); identifier(); - setState(742); + setState(724); match(DOT); } } } - setState(748); + setState(730); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,99,_ctx); + _alt = getInterpreter().adaptivePredict(_input,94,_ctx); } - setState(749); + setState(731); identifier(); } } @@ -5858,13 +5833,13 @@ public final IdentifierContext identifier() throws RecognitionException { IdentifierContext _localctx = new IdentifierContext(_ctx, getState()); enterRule(_localctx, 94, RULE_identifier); try { - setState(753); + setState(735); switch (_input.LA(1)) { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 1); { - setState(751); + setState(733); quoteIdentifier(); } break; @@ -5872,6 +5847,9 @@ public final IdentifierContext identifier() throws RecognitionException { case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -5908,7 +5886,7 @@ public final IdentifierContext identifier() throws RecognitionException { case DIGIT_IDENTIFIER: enterOuterAlt(_localctx, 2); { - setState(752); + setState(734); unquoteIdentifier(); } break; @@ -5961,43 +5939,43 @@ public final TableIdentifierContext tableIdentifier() throws RecognitionExceptio enterRule(_localctx, 96, RULE_tableIdentifier); int _la; try { - setState(767); + setState(749); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,103,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,98,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(758); + setState(740); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { { - setState(755); + setState(737); ((TableIdentifierContext)_localctx).catalog = identifier(); - setState(756); + setState(738); match(T__3); } } - setState(760); + setState(742); match(TABLE_IDENTIFIER); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(764); + setState(746); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,102,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,97,_ctx) ) { case 1: { - setState(761); + setState(743); ((TableIdentifierContext)_localctx).catalog = identifier(); - setState(762); + setState(744); match(T__3); } break; } - setState(766); + setState(748); ((TableIdentifierContext)_localctx).name = identifier(); } break; @@ -6064,13 +6042,13 @@ public final QuoteIdentifierContext quoteIdentifier() throws RecognitionExceptio QuoteIdentifierContext _localctx = new QuoteIdentifierContext(_ctx, getState()); enterRule(_localctx, 98, RULE_quoteIdentifier); try { - setState(771); + setState(753); switch (_input.LA(1)) { case QUOTED_IDENTIFIER: _localctx = new QuotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(769); + setState(751); match(QUOTED_IDENTIFIER); } break; @@ -6078,7 +6056,7 @@ public final QuoteIdentifierContext quoteIdentifier() throws RecognitionExceptio _localctx = new BackQuotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(770); + setState(752); match(BACKQUOTED_IDENTIFIER); } break; @@ -6150,13 +6128,13 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce UnquoteIdentifierContext _localctx = new UnquoteIdentifierContext(_ctx, getState()); enterRule(_localctx, 100, RULE_unquoteIdentifier); try { - setState(776); + setState(758); switch (_input.LA(1)) { case IDENTIFIER: _localctx = new UnquotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(773); + setState(755); match(IDENTIFIER); } break; @@ -6164,6 +6142,9 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce case ANALYZED: case CATALOGS: case COLUMNS: + case CURRENT_DATE: + case CURRENT_TIME: + case CURRENT_TIMESTAMP: case DAY: case DEBUG: case EXECUTABLE: @@ -6199,7 +6180,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce _localctx = new UnquotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(774); + setState(756); nonReserved(); } break; @@ -6207,7 +6188,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce _localctx = new DigitIdentifierContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(775); + setState(757); match(DIGIT_IDENTIFIER); } break; @@ -6276,13 +6257,13 @@ public final NumberContext number() throws RecognitionException { NumberContext _localctx = new NumberContext(_ctx, getState()); enterRule(_localctx, 102, RULE_number); try { - setState(780); + setState(762); switch (_input.LA(1)) { case DECIMAL_VALUE: _localctx = new DecimalLiteralContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(778); + setState(760); match(DECIMAL_VALUE); } break; @@ -6290,7 +6271,7 @@ public final NumberContext number() throws RecognitionException { _localctx = new IntegerLiteralContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(779); + setState(761); match(INTEGER_VALUE); } break; @@ -6338,7 +6319,7 @@ public final StringContext string() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(782); + setState(764); _la = _input.LA(1); if ( !(_la==PARAM || _la==STRING) ) { _errHandler.recoverInline(this); @@ -6363,6 +6344,9 @@ public static class NonReservedContext extends ParserRuleContext { public TerminalNode ANALYZED() { return getToken(SqlBaseParser.ANALYZED, 0); } public TerminalNode CATALOGS() { return getToken(SqlBaseParser.CATALOGS, 0); } public TerminalNode COLUMNS() { return getToken(SqlBaseParser.COLUMNS, 0); } + public TerminalNode CURRENT_DATE() { return getToken(SqlBaseParser.CURRENT_DATE, 0); } + public TerminalNode CURRENT_TIME() { return getToken(SqlBaseParser.CURRENT_TIME, 0); } + public TerminalNode CURRENT_TIMESTAMP() { return getToken(SqlBaseParser.CURRENT_TIMESTAMP, 0); } public TerminalNode DAY() { return getToken(SqlBaseParser.DAY, 0); } public TerminalNode DEBUG() { return getToken(SqlBaseParser.DEBUG, 0); } public TerminalNode EXECUTABLE() { return getToken(SqlBaseParser.EXECUTABLE, 0); } @@ -6421,9 +6405,9 @@ public final NonReservedContext nonReserved() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(784); + setState(766); _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)))) != 0)) ) { + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -6481,7 +6465,7 @@ private boolean primaryExpression_sempred(PrimaryExpressionContext _localctx, in } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\u0083\u0315\4\2\t"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\u0083\u0303\4\2\t"+ "\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13"+ "\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31\t\31"+ @@ -6528,115 +6512,114 @@ private boolean primaryExpression_sempred(PrimaryExpressionContext _localctx, in "\n\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\7\37\u0237\n\37"+ "\f\37\16\37\u023a\13\37\3 \3 \3 \3 \3 \3 \3 \3 \5 \u0244\n \3 \3 \3 \3"+ " \3 \3 \3 \3 \3 \3 \3 \5 \u0251\n \3 \3 \3 \7 \u0256\n \f \16 \u0259\13"+ - " \3!\3!\3!\5!\u025e\n!\3!\5!\u0261\n!\3!\3!\3!\5!\u0266\n!\3!\3!\3!\5"+ - "!\u026b\n!\3!\5!\u026e\n!\5!\u0270\n!\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\""+ - "\3\"\3\"\5\"\u027c\n\"\3#\3#\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3$\3$\3%\3"+ - "%\3%\3%\3%\5%\u0291\n%\3&\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\5\'\u029f"+ - "\n\'\3(\3(\3(\5(\u02a4\n(\3(\3(\3(\7(\u02a9\n(\f(\16(\u02ac\13(\5(\u02ae"+ - "\n(\3(\3(\3)\3)\3)\5)\u02b5\n)\3*\3*\3*\3*\3*\6*\u02bc\n*\r*\16*\u02bd"+ - "\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\5*\u02d1\n*\3+\3+"+ - "\3,\3,\3-\3-\5-\u02d9\n-\3-\3-\5-\u02dd\n-\3-\3-\3-\5-\u02e2\n-\3.\3."+ - "\3/\3/\3\60\3\60\3\60\7\60\u02eb\n\60\f\60\16\60\u02ee\13\60\3\60\3\60"+ - "\3\61\3\61\5\61\u02f4\n\61\3\62\3\62\3\62\5\62\u02f9\n\62\3\62\3\62\3"+ - "\62\3\62\5\62\u02ff\n\62\3\62\5\62\u0302\n\62\3\63\3\63\5\63\u0306\n\63"+ - "\3\64\3\64\3\64\5\64\u030b\n\64\3\65\3\65\5\65\u030f\n\65\3\66\3\66\3"+ - "\67\3\67\3\67\2\5.<>8\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,."+ - "\60\62\64\668:<>@BDFHJLNPRTVXZ\\^`bdfhjl\2\22\b\2\7\7\t\t\37\37\67\67"+ - "BBFF\4\2))TT\4\2\t\tBB\4\2&&..\3\2\33\34\3\2no\4\2\7\7xx\4\2\r\r\33\33"+ - "\4\2$$\63\63\4\2\7\7\35\35\3\2pr\3\2gm\4\2##UU\7\2\30\31,-9\u0250"+ - "\3\2\2\2@\u026f\3\2\2\2B\u027b\3\2\2\2D\u027d\3\2\2\2F\u0284\3\2\2\2H"+ - "\u0290\3\2\2\2J\u0292\3\2\2\2L\u029e\3\2\2\2N\u02a0\3\2\2\2P\u02b4\3\2"+ - "\2\2R\u02d0\3\2\2\2T\u02d2\3\2\2\2V\u02d4\3\2\2\2X\u02d6\3\2\2\2Z\u02e3"+ - "\3\2\2\2\\\u02e5\3\2\2\2^\u02ec\3\2\2\2`\u02f3\3\2\2\2b\u0301\3\2\2\2"+ - "d\u0305\3\2\2\2f\u030a\3\2\2\2h\u030e\3\2\2\2j\u0310\3\2\2\2l\u0312\3"+ - "\2\2\2no\5\6\4\2op\7\2\2\3p\3\3\2\2\2qr\5,\27\2rs\7\2\2\3s\5\3\2\2\2t"+ - "\u00da\5\b\5\2u\u0083\7!\2\2v\177\7\3\2\2wx\7H\2\2x~\t\2\2\2yz\7%\2\2"+ - "z~\t\3\2\2{|\7Z\2\2|~\5V,\2}w\3\2\2\2}y\3\2\2\2}{\3\2\2\2~\u0081\3\2\2"+ - "\2\177}\3\2\2\2\177\u0080\3\2\2\2\u0080\u0082\3\2\2\2\u0081\177\3\2\2"+ - "\2\u0082\u0084\7\4\2\2\u0083v\3\2\2\2\u0083\u0084\3\2\2\2\u0084\u0085"+ - "\3\2\2\2\u0085\u00da\5\6\4\2\u0086\u0092\7\32\2\2\u0087\u008e\7\3\2\2"+ - "\u0088\u0089\7H\2\2\u0089\u008d\t\4\2\2\u008a\u008b\7%\2\2\u008b\u008d"+ - "\t\3\2\2\u008c\u0088\3\2\2\2\u008c\u008a\3\2\2\2\u008d\u0090\3\2\2\2\u008e"+ - "\u008c\3\2\2\2\u008e\u008f\3\2\2\2\u008f\u0091\3\2\2\2\u0090\u008e\3\2"+ - "\2\2\u0091\u0093\7\4\2\2\u0092\u0087\3\2\2\2\u0092\u0093\3\2\2\2\u0093"+ - "\u0094\3\2\2\2\u0094\u00da\5\6\4\2\u0095\u0096\7P\2\2\u0096\u0099\7S\2"+ - "\2\u0097\u009a\5\66\34\2\u0098\u009a\5b\62\2\u0099\u0097\3\2\2\2\u0099"+ - "\u0098\3\2\2\2\u0099\u009a\3\2\2\2\u009a\u00da\3\2\2\2\u009b\u009c\7P"+ - "\2\2\u009c\u009d\7\23\2\2\u009d\u00a0\t\5\2\2\u009e\u00a1\5\66\34\2\u009f"+ - "\u00a1\5b\62\2\u00a0\u009e\3\2\2\2\u00a0\u009f\3\2\2\2\u00a1\u00da\3\2"+ - "\2\2\u00a2\u00a5\t\6\2\2\u00a3\u00a6\5\66\34\2\u00a4\u00a6\5b\62\2\u00a5"+ - "\u00a3\3\2\2\2\u00a5\u00a4\3\2\2\2\u00a6\u00da\3\2\2\2\u00a7\u00a8\7P"+ - "\2\2\u00a8\u00aa\7(\2\2\u00a9\u00ab\5\66\34\2\u00aa\u00a9\3\2\2\2\u00aa"+ - "\u00ab\3\2\2\2\u00ab\u00da\3\2\2\2\u00ac\u00ad\7P\2\2\u00ad\u00da\7L\2"+ - "\2\u00ae\u00af\7Q\2\2\u00af\u00b2\7S\2\2\u00b0\u00b1\7\21\2\2\u00b1\u00b3"+ - "\5\66\34\2\u00b2\u00b0\3\2\2\2\u00b2\u00b3\3\2\2\2\u00b3\u00b6\3\2\2\2"+ - "\u00b4\u00b7\5\66\34\2\u00b5\u00b7\5b\62\2\u00b6\u00b4\3\2\2\2\u00b6\u00b5"+ - "\3\2\2\2\u00b6\u00b7\3\2\2\2\u00b7\u00c1\3\2\2\2\u00b8\u00b9\7W\2\2\u00b9"+ - "\u00be\5j\66\2\u00ba\u00bb\7\5\2\2\u00bb\u00bd\5j\66\2\u00bc\u00ba\3\2"+ - "\2\2\u00bd\u00c0\3\2\2\2\u00be\u00bc\3\2\2\2\u00be\u00bf\3\2\2\2\u00bf"+ - "\u00c2\3\2\2\2\u00c0\u00be\3\2\2\2\u00c1\u00b8\3\2\2\2\u00c1\u00c2\3\2"+ - "\2\2\u00c2\u00da\3\2\2\2\u00c3\u00c4\7Q\2\2\u00c4\u00c7\7\23\2\2\u00c5"+ - "\u00c6\7\21\2\2\u00c6\u00c8\5j\66\2\u00c7\u00c5\3\2\2\2\u00c7\u00c8\3"+ - "\2\2\2\u00c8\u00cc\3\2\2\2\u00c9\u00ca\7R\2\2\u00ca\u00cd\5\66\34\2\u00cb"+ - "\u00cd\5b\62\2\u00cc\u00c9\3\2\2\2\u00cc\u00cb\3\2\2\2\u00cc\u00cd\3\2"+ - "\2\2\u00cd\u00cf\3\2\2\2\u00ce\u00d0\5\66\34\2\u00cf\u00ce\3\2\2\2\u00cf"+ - "\u00d0\3\2\2\2\u00d0\u00da\3\2\2\2\u00d1\u00d2\7Q\2\2\u00d2\u00d7\7X\2"+ - "\2\u00d3\u00d5\t\7\2\2\u00d4\u00d3\3\2\2\2\u00d4\u00d5\3\2\2\2\u00d5\u00d6"+ - "\3\2\2\2\u00d6\u00d8\5h\65\2\u00d7\u00d4\3\2\2\2\u00d7\u00d8\3\2\2\2\u00d8"+ - "\u00da\3\2\2\2\u00d9t\3\2\2\2\u00d9u\3\2\2\2\u00d9\u0086\3\2\2\2\u00d9"+ - "\u0095\3\2\2\2\u00d9\u009b\3\2\2\2\u00d9\u00a2\3\2\2\2\u00d9\u00a7\3\2"+ - "\2\2\u00d9\u00ac\3\2\2\2\u00d9\u00ae\3\2\2\2\u00d9\u00c3\3\2\2\2\u00d9"+ - "\u00d1\3\2\2\2\u00da\7\3\2\2\2\u00db\u00dc\7\\\2\2\u00dc\u00e1\5\34\17"+ - "\2\u00dd\u00de\7\5\2\2\u00de\u00e0\5\34\17\2\u00df\u00dd\3\2\2\2\u00e0"+ - "\u00e3\3\2\2\2\u00e1\u00df\3\2\2\2\u00e1\u00e2\3\2\2\2\u00e2\u00e5\3\2"+ - "\2\2\u00e3\u00e1\3\2\2\2\u00e4\u00db\3\2\2\2\u00e4\u00e5\3\2\2\2\u00e5"+ - "\u00e6\3\2\2\2\u00e6\u00e7\5\n\6\2\u00e7\t\3\2\2\2\u00e8\u00f3\5\16\b"+ - "\2\u00e9\u00ea\7D\2\2\u00ea\u00eb\7\17\2\2\u00eb\u00f0\5\20\t\2\u00ec"+ - "\u00ed\7\5\2\2\u00ed\u00ef\5\20\t\2\u00ee\u00ec\3\2\2\2\u00ef\u00f2\3"+ - "\2\2\2\u00f0\u00ee\3\2\2\2\u00f0\u00f1\3\2\2\2\u00f1\u00f4\3\2\2\2\u00f2"+ - "\u00f0\3\2\2\2\u00f3\u00e9\3\2\2\2\u00f3\u00f4\3\2\2\2\u00f4\u00f6\3\2"+ - "\2\2\u00f5\u00f7\5\f\7\2\u00f6\u00f5\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7"+ - "\13\3\2\2\2\u00f8\u00f9\7\66\2\2\u00f9\u00fe\t\b\2\2\u00fa\u00fb\7a\2"+ - "\2\u00fb\u00fc\t\b\2\2\u00fc\u00fe\7f\2\2\u00fd\u00f8\3\2\2\2\u00fd\u00fa"+ - "\3\2\2\2\u00fe\r\3\2\2\2\u00ff\u0105\5\22\n\2\u0100\u0101\7\3\2\2\u0101"+ - "\u0102\5\n\6\2\u0102\u0103\7\4\2\2\u0103\u0105\3\2\2\2\u0104\u00ff\3\2"+ - "\2\2\u0104\u0100\3\2\2\2\u0105\17\3\2\2\2\u0106\u0108\5,\27\2\u0107\u0109"+ - "\t\t\2\2\u0108\u0107\3\2\2\2\u0108\u0109\3\2\2\2\u0109\u010c\3\2\2\2\u010a"+ - "\u010b\7@\2\2\u010b\u010d\t\n\2\2\u010c\u010a\3\2\2\2\u010c\u010d\3\2"+ - "\2\2\u010d\21\3\2\2\2\u010e\u0110\7O\2\2\u010f\u0111\5\36\20\2\u0110\u010f"+ - "\3\2\2\2\u0110\u0111\3\2\2\2\u0111\u0112\3\2\2\2\u0112\u0117\5 \21\2\u0113"+ - "\u0114\7\5\2\2\u0114\u0116\5 \21\2\u0115\u0113\3\2\2\2\u0116\u0119\3\2"+ - "\2\2\u0117\u0115\3\2\2\2\u0117\u0118\3\2\2\2\u0118\u011b\3\2\2\2\u0119"+ - "\u0117\3\2\2\2\u011a\u011c\5\24\13\2\u011b\u011a\3\2\2\2\u011b\u011c\3"+ - "\2\2\2\u011c\u011f\3\2\2\2\u011d\u011e\7[\2\2\u011e\u0120\5.\30\2\u011f"+ - "\u011d\3\2\2\2\u011f\u0120\3\2\2\2\u0120\u0124\3\2\2\2\u0121\u0122\7*"+ - "\2\2\u0122\u0123\7\17\2\2\u0123\u0125\5\26\f\2\u0124\u0121\3\2\2\2\u0124"+ - "\u0125\3\2\2\2\u0125\u0128\3\2\2\2\u0126\u0127\7+\2\2\u0127\u0129\5.\30"+ - "\2\u0128\u0126\3\2\2\2\u0128\u0129\3\2\2\2\u0129\23\3\2\2\2\u012a\u012b"+ - "\7&\2\2\u012b\u0130\5\"\22\2\u012c\u012d\7\5\2\2\u012d\u012f\5\"\22\2"+ - "\u012e\u012c\3\2\2\2\u012f\u0132\3\2\2\2\u0130\u012e\3\2\2\2\u0130\u0131"+ - "\3\2\2\2\u0131\25\3\2\2\2\u0132\u0130\3\2\2\2\u0133\u0135\5\36\20\2\u0134"+ - "\u0133\3\2\2\2\u0134\u0135\3\2\2\2\u0135\u0136\3\2\2\2\u0136\u013b\5\30"+ - "\r\2\u0137\u0138\7\5\2\2\u0138\u013a\5\30\r\2\u0139\u0137\3\2\2\2\u013a"+ - "\u013d\3\2\2\2\u013b\u0139\3\2\2\2\u013b\u013c\3\2\2\2\u013c\27\3\2\2"+ - "\2\u013d\u013b\3\2\2\2\u013e\u013f\5\32\16\2\u013f\31\3\2\2\2\u0140\u0149"+ - "\7\3\2\2\u0141\u0146\5,\27\2\u0142\u0143\7\5\2\2\u0143\u0145\5,\27\2\u0144"+ - "\u0142\3\2\2\2\u0145\u0148\3\2\2\2\u0146\u0144\3\2\2\2\u0146\u0147\3\2"+ - "\2\2\u0147\u014a\3\2\2\2\u0148\u0146\3\2\2\2\u0149\u0141\3\2\2\2\u0149"+ - "\u014a\3\2\2\2\u014a\u014b\3\2\2\2\u014b\u014e\7\4\2\2\u014c\u014e\5,"+ - "\27\2\u014d\u0140\3\2\2\2\u014d\u014c\3\2\2\2\u014e\33\3\2\2\2\u014f\u0150"+ - "\5`\61\2\u0150\u0151\7\f\2\2\u0151\u0152\7\3\2\2\u0152\u0153\5\n\6\2\u0153"+ - "\u0154\7\4\2\2\u0154\35\3\2\2\2\u0155\u0156\t\13\2\2\u0156\37\3\2\2\2"+ - "\u0157\u015c\5,\27\2\u0158\u015a\7\f\2\2\u0159\u0158\3\2\2\2\u0159\u015a"+ - "\3\2\2\2\u015a\u015b\3\2\2\2\u015b\u015d\5`\61\2\u015c\u0159\3\2\2\2\u015c"+ + " \3!\3!\3!\5!\u025e\n!\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\5\"\u026a"+ + "\n\"\3#\3#\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\5%\u027f"+ + "\n%\3&\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\5\'\u028d\n\'\3(\3(\3(\5"+ + "(\u0292\n(\3(\3(\3(\7(\u0297\n(\f(\16(\u029a\13(\5(\u029c\n(\3(\3(\3)"+ + "\3)\3)\5)\u02a3\n)\3*\3*\3*\3*\3*\6*\u02aa\n*\r*\16*\u02ab\3*\3*\3*\3"+ + "*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\5*\u02bf\n*\3+\3+\3,\3,\3-\3"+ + "-\5-\u02c7\n-\3-\3-\5-\u02cb\n-\3-\3-\3-\5-\u02d0\n-\3.\3.\3/\3/\3\60"+ + "\3\60\3\60\7\60\u02d9\n\60\f\60\16\60\u02dc\13\60\3\60\3\60\3\61\3\61"+ + "\5\61\u02e2\n\61\3\62\3\62\3\62\5\62\u02e7\n\62\3\62\3\62\3\62\3\62\5"+ + "\62\u02ed\n\62\3\62\5\62\u02f0\n\62\3\63\3\63\5\63\u02f4\n\63\3\64\3\64"+ + "\3\64\5\64\u02f9\n\64\3\65\3\65\5\65\u02fd\n\65\3\66\3\66\3\67\3\67\3"+ + "\67\2\5.<>8\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,.\60\62\64\66"+ + "8:<>@BDFHJLNPRTVXZ\\^`bdfhjl\2\22\b\2\7\7\t\t\37\37\67\67BBFF\4\2))TT"+ + "\4\2\t\tBB\4\2&&..\3\2\33\34\3\2no\4\2\7\7xx\4\2\r\r\33\33\4\2$$\63\63"+ + "\4\2\7\7\35\35\3\2pr\3\2gm\4\2##UU\7\2\30\31,-9\u0250\3\2\2\2@\u025d"+ + "\3\2\2\2B\u0269\3\2\2\2D\u026b\3\2\2\2F\u0272\3\2\2\2H\u027e\3\2\2\2J"+ + "\u0280\3\2\2\2L\u028c\3\2\2\2N\u028e\3\2\2\2P\u02a2\3\2\2\2R\u02be\3\2"+ + "\2\2T\u02c0\3\2\2\2V\u02c2\3\2\2\2X\u02c4\3\2\2\2Z\u02d1\3\2\2\2\\\u02d3"+ + "\3\2\2\2^\u02da\3\2\2\2`\u02e1\3\2\2\2b\u02ef\3\2\2\2d\u02f3\3\2\2\2f"+ + "\u02f8\3\2\2\2h\u02fc\3\2\2\2j\u02fe\3\2\2\2l\u0300\3\2\2\2no\5\6\4\2"+ + "op\7\2\2\3p\3\3\2\2\2qr\5,\27\2rs\7\2\2\3s\5\3\2\2\2t\u00da\5\b\5\2u\u0083"+ + "\7!\2\2v\177\7\3\2\2wx\7H\2\2x~\t\2\2\2yz\7%\2\2z~\t\3\2\2{|\7Z\2\2|~"+ + "\5V,\2}w\3\2\2\2}y\3\2\2\2}{\3\2\2\2~\u0081\3\2\2\2\177}\3\2\2\2\177\u0080"+ + "\3\2\2\2\u0080\u0082\3\2\2\2\u0081\177\3\2\2\2\u0082\u0084\7\4\2\2\u0083"+ + "v\3\2\2\2\u0083\u0084\3\2\2\2\u0084\u0085\3\2\2\2\u0085\u00da\5\6\4\2"+ + "\u0086\u0092\7\32\2\2\u0087\u008e\7\3\2\2\u0088\u0089\7H\2\2\u0089\u008d"+ + "\t\4\2\2\u008a\u008b\7%\2\2\u008b\u008d\t\3\2\2\u008c\u0088\3\2\2\2\u008c"+ + "\u008a\3\2\2\2\u008d\u0090\3\2\2\2\u008e\u008c\3\2\2\2\u008e\u008f\3\2"+ + "\2\2\u008f\u0091\3\2\2\2\u0090\u008e\3\2\2\2\u0091\u0093\7\4\2\2\u0092"+ + "\u0087\3\2\2\2\u0092\u0093\3\2\2\2\u0093\u0094\3\2\2\2\u0094\u00da\5\6"+ + "\4\2\u0095\u0096\7P\2\2\u0096\u0099\7S\2\2\u0097\u009a\5\66\34\2\u0098"+ + "\u009a\5b\62\2\u0099\u0097\3\2\2\2\u0099\u0098\3\2\2\2\u0099\u009a\3\2"+ + "\2\2\u009a\u00da\3\2\2\2\u009b\u009c\7P\2\2\u009c\u009d\7\23\2\2\u009d"+ + "\u00a0\t\5\2\2\u009e\u00a1\5\66\34\2\u009f\u00a1\5b\62\2\u00a0\u009e\3"+ + "\2\2\2\u00a0\u009f\3\2\2\2\u00a1\u00da\3\2\2\2\u00a2\u00a5\t\6\2\2\u00a3"+ + "\u00a6\5\66\34\2\u00a4\u00a6\5b\62\2\u00a5\u00a3\3\2\2\2\u00a5\u00a4\3"+ + "\2\2\2\u00a6\u00da\3\2\2\2\u00a7\u00a8\7P\2\2\u00a8\u00aa\7(\2\2\u00a9"+ + "\u00ab\5\66\34\2\u00aa\u00a9\3\2\2\2\u00aa\u00ab\3\2\2\2\u00ab\u00da\3"+ + "\2\2\2\u00ac\u00ad\7P\2\2\u00ad\u00da\7L\2\2\u00ae\u00af\7Q\2\2\u00af"+ + "\u00b2\7S\2\2\u00b0\u00b1\7\21\2\2\u00b1\u00b3\5\66\34\2\u00b2\u00b0\3"+ + "\2\2\2\u00b2\u00b3\3\2\2\2\u00b3\u00b6\3\2\2\2\u00b4\u00b7\5\66\34\2\u00b5"+ + "\u00b7\5b\62\2\u00b6\u00b4\3\2\2\2\u00b6\u00b5\3\2\2\2\u00b6\u00b7\3\2"+ + "\2\2\u00b7\u00c1\3\2\2\2\u00b8\u00b9\7W\2\2\u00b9\u00be\5j\66\2\u00ba"+ + "\u00bb\7\5\2\2\u00bb\u00bd\5j\66\2\u00bc\u00ba\3\2\2\2\u00bd\u00c0\3\2"+ + "\2\2\u00be\u00bc\3\2\2\2\u00be\u00bf\3\2\2\2\u00bf\u00c2\3\2\2\2\u00c0"+ + "\u00be\3\2\2\2\u00c1\u00b8\3\2\2\2\u00c1\u00c2\3\2\2\2\u00c2\u00da\3\2"+ + "\2\2\u00c3\u00c4\7Q\2\2\u00c4\u00c7\7\23\2\2\u00c5\u00c6\7\21\2\2\u00c6"+ + "\u00c8\5j\66\2\u00c7\u00c5\3\2\2\2\u00c7\u00c8\3\2\2\2\u00c8\u00cc\3\2"+ + "\2\2\u00c9\u00ca\7R\2\2\u00ca\u00cd\5\66\34\2\u00cb\u00cd\5b\62\2\u00cc"+ + "\u00c9\3\2\2\2\u00cc\u00cb\3\2\2\2\u00cc\u00cd\3\2\2\2\u00cd\u00cf\3\2"+ + "\2\2\u00ce\u00d0\5\66\34\2\u00cf\u00ce\3\2\2\2\u00cf\u00d0\3\2\2\2\u00d0"+ + "\u00da\3\2\2\2\u00d1\u00d2\7Q\2\2\u00d2\u00d7\7X\2\2\u00d3\u00d5\t\7\2"+ + "\2\u00d4\u00d3\3\2\2\2\u00d4\u00d5\3\2\2\2\u00d5\u00d6\3\2\2\2\u00d6\u00d8"+ + "\5h\65\2\u00d7\u00d4\3\2\2\2\u00d7\u00d8\3\2\2\2\u00d8\u00da\3\2\2\2\u00d9"+ + "t\3\2\2\2\u00d9u\3\2\2\2\u00d9\u0086\3\2\2\2\u00d9\u0095\3\2\2\2\u00d9"+ + "\u009b\3\2\2\2\u00d9\u00a2\3\2\2\2\u00d9\u00a7\3\2\2\2\u00d9\u00ac\3\2"+ + "\2\2\u00d9\u00ae\3\2\2\2\u00d9\u00c3\3\2\2\2\u00d9\u00d1\3\2\2\2\u00da"+ + "\7\3\2\2\2\u00db\u00dc\7\\\2\2\u00dc\u00e1\5\34\17\2\u00dd\u00de\7\5\2"+ + "\2\u00de\u00e0\5\34\17\2\u00df\u00dd\3\2\2\2\u00e0\u00e3\3\2\2\2\u00e1"+ + "\u00df\3\2\2\2\u00e1\u00e2\3\2\2\2\u00e2\u00e5\3\2\2\2\u00e3\u00e1\3\2"+ + "\2\2\u00e4\u00db\3\2\2\2\u00e4\u00e5\3\2\2\2\u00e5\u00e6\3\2\2\2\u00e6"+ + "\u00e7\5\n\6\2\u00e7\t\3\2\2\2\u00e8\u00f3\5\16\b\2\u00e9\u00ea\7D\2\2"+ + "\u00ea\u00eb\7\17\2\2\u00eb\u00f0\5\20\t\2\u00ec\u00ed\7\5\2\2\u00ed\u00ef"+ + "\5\20\t\2\u00ee\u00ec\3\2\2\2\u00ef\u00f2\3\2\2\2\u00f0\u00ee\3\2\2\2"+ + "\u00f0\u00f1\3\2\2\2\u00f1\u00f4\3\2\2\2\u00f2\u00f0\3\2\2\2\u00f3\u00e9"+ + "\3\2\2\2\u00f3\u00f4\3\2\2\2\u00f4\u00f6\3\2\2\2\u00f5\u00f7\5\f\7\2\u00f6"+ + "\u00f5\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7\13\3\2\2\2\u00f8\u00f9\7\66\2"+ + "\2\u00f9\u00fe\t\b\2\2\u00fa\u00fb\7a\2\2\u00fb\u00fc\t\b\2\2\u00fc\u00fe"+ + "\7f\2\2\u00fd\u00f8\3\2\2\2\u00fd\u00fa\3\2\2\2\u00fe\r\3\2\2\2\u00ff"+ + "\u0105\5\22\n\2\u0100\u0101\7\3\2\2\u0101\u0102\5\n\6\2\u0102\u0103\7"+ + "\4\2\2\u0103\u0105\3\2\2\2\u0104\u00ff\3\2\2\2\u0104\u0100\3\2\2\2\u0105"+ + "\17\3\2\2\2\u0106\u0108\5,\27\2\u0107\u0109\t\t\2\2\u0108\u0107\3\2\2"+ + "\2\u0108\u0109\3\2\2\2\u0109\u010c\3\2\2\2\u010a\u010b\7@\2\2\u010b\u010d"+ + "\t\n\2\2\u010c\u010a\3\2\2\2\u010c\u010d\3\2\2\2\u010d\21\3\2\2\2\u010e"+ + "\u0110\7O\2\2\u010f\u0111\5\36\20\2\u0110\u010f\3\2\2\2\u0110\u0111\3"+ + "\2\2\2\u0111\u0112\3\2\2\2\u0112\u0117\5 \21\2\u0113\u0114\7\5\2\2\u0114"+ + "\u0116\5 \21\2\u0115\u0113\3\2\2\2\u0116\u0119\3\2\2\2\u0117\u0115\3\2"+ + "\2\2\u0117\u0118\3\2\2\2\u0118\u011b\3\2\2\2\u0119\u0117\3\2\2\2\u011a"+ + "\u011c\5\24\13\2\u011b\u011a\3\2\2\2\u011b\u011c\3\2\2\2\u011c\u011f\3"+ + "\2\2\2\u011d\u011e\7[\2\2\u011e\u0120\5.\30\2\u011f\u011d\3\2\2\2\u011f"+ + "\u0120\3\2\2\2\u0120\u0124\3\2\2\2\u0121\u0122\7*\2\2\u0122\u0123\7\17"+ + "\2\2\u0123\u0125\5\26\f\2\u0124\u0121\3\2\2\2\u0124\u0125\3\2\2\2\u0125"+ + "\u0128\3\2\2\2\u0126\u0127\7+\2\2\u0127\u0129\5.\30\2\u0128\u0126\3\2"+ + "\2\2\u0128\u0129\3\2\2\2\u0129\23\3\2\2\2\u012a\u012b\7&\2\2\u012b\u0130"+ + "\5\"\22\2\u012c\u012d\7\5\2\2\u012d\u012f\5\"\22\2\u012e\u012c\3\2\2\2"+ + "\u012f\u0132\3\2\2\2\u0130\u012e\3\2\2\2\u0130\u0131\3\2\2\2\u0131\25"+ + "\3\2\2\2\u0132\u0130\3\2\2\2\u0133\u0135\5\36\20\2\u0134\u0133\3\2\2\2"+ + "\u0134\u0135\3\2\2\2\u0135\u0136\3\2\2\2\u0136\u013b\5\30\r\2\u0137\u0138"+ + "\7\5\2\2\u0138\u013a\5\30\r\2\u0139\u0137\3\2\2\2\u013a\u013d\3\2\2\2"+ + "\u013b\u0139\3\2\2\2\u013b\u013c\3\2\2\2\u013c\27\3\2\2\2\u013d\u013b"+ + "\3\2\2\2\u013e\u013f\5\32\16\2\u013f\31\3\2\2\2\u0140\u0149\7\3\2\2\u0141"+ + "\u0146\5,\27\2\u0142\u0143\7\5\2\2\u0143\u0145\5,\27\2\u0144\u0142\3\2"+ + "\2\2\u0145\u0148\3\2\2\2\u0146\u0144\3\2\2\2\u0146\u0147\3\2\2\2\u0147"+ + "\u014a\3\2\2\2\u0148\u0146\3\2\2\2\u0149\u0141\3\2\2\2\u0149\u014a\3\2"+ + "\2\2\u014a\u014b\3\2\2\2\u014b\u014e\7\4\2\2\u014c\u014e\5,\27\2\u014d"+ + "\u0140\3\2\2\2\u014d\u014c\3\2\2\2\u014e\33\3\2\2\2\u014f\u0150\5`\61"+ + "\2\u0150\u0151\7\f\2\2\u0151\u0152\7\3\2\2\u0152\u0153\5\n\6\2\u0153\u0154"+ + "\7\4\2\2\u0154\35\3\2\2\2\u0155\u0156\t\13\2\2\u0156\37\3\2\2\2\u0157"+ + "\u015c\5,\27\2\u0158\u015a\7\f\2\2\u0159\u0158\3\2\2\2\u0159\u015a\3\2"+ + "\2\2\u015a\u015b\3\2\2\2\u015b\u015d\5`\61\2\u015c\u0159\3\2\2\2\u015c"+ "\u015d\3\2\2\2\u015d!\3\2\2\2\u015e\u0162\5*\26\2\u015f\u0161\5$\23\2"+ "\u0160\u015f\3\2\2\2\u0161\u0164\3\2\2\2\u0162\u0160\3\2\2\2\u0162\u0163"+ "\3\2\2\2\u0163#\3\2\2\2\u0164\u0162\3\2\2\2\u0165\u0166\5&\24\2\u0166"+ @@ -6723,77 +6706,70 @@ private boolean primaryExpression_sempred(PrimaryExpressionContext _localctx, in "\u024b\3\2\2\2\u0250\u024c\3\2\2\2\u0251\u0257\3\2\2\2\u0252\u0253\f\13"+ "\2\2\u0253\u0254\7s\2\2\u0254\u0256\5\\/\2\u0255\u0252\3\2\2\2\u0256\u0259"+ "\3\2\2\2\u0257\u0255\3\2\2\2\u0257\u0258\3\2\2\2\u0258?\3\2\2\2\u0259"+ - "\u0257\3\2\2\2\u025a\u0260\7\27\2\2\u025b\u025d\7\3\2\2\u025c\u025e\7"+ - "x\2\2\u025d\u025c\3\2\2\2\u025d\u025e\3\2\2\2\u025e\u025f\3\2\2\2\u025f"+ - "\u0261\7\4\2\2\u0260\u025b\3\2\2\2\u0260\u0261\3\2\2\2\u0261\u0270\3\2"+ - "\2\2\u0262\u0265\7\25\2\2\u0263\u0264\7\3\2\2\u0264\u0266\7\4\2\2\u0265"+ - "\u0263\3\2\2\2\u0265\u0266\3\2\2\2\u0266\u0270\3\2\2\2\u0267\u026d\7\26"+ - "\2\2\u0268\u026a\7\3\2\2\u0269\u026b\7x\2\2\u026a\u0269\3\2\2\2\u026a"+ - "\u026b\3\2\2\2\u026b\u026c\3\2\2\2\u026c\u026e\7\4\2\2\u026d\u0268\3\2"+ - "\2\2\u026d\u026e\3\2\2\2\u026e\u0270\3\2\2\2\u026f\u025a\3\2\2\2\u026f"+ - "\u0262\3\2\2\2\u026f\u0267\3\2\2\2\u0270A\3\2\2\2\u0271\u027c\5D#\2\u0272"+ - "\u0273\7`\2\2\u0273\u0274\5D#\2\u0274\u0275\7f\2\2\u0275\u027c\3\2\2\2"+ - "\u0276\u027c\5F$\2\u0277\u0278\7`\2\2\u0278\u0279\5F$\2\u0279\u027a\7"+ - "f\2\2\u027a\u027c\3\2\2\2\u027b\u0271\3\2\2\2\u027b\u0272\3\2\2\2\u027b"+ - "\u0276\3\2\2\2\u027b\u0277\3\2\2\2\u027cC\3\2\2\2\u027d\u027e\7\20\2\2"+ - "\u027e\u027f\7\3\2\2\u027f\u0280\5,\27\2\u0280\u0281\7\f\2\2\u0281\u0282"+ - "\5\\/\2\u0282\u0283\7\4\2\2\u0283E\3\2\2\2\u0284\u0285\7\24\2\2\u0285"+ - "\u0286\7\3\2\2\u0286\u0287\5,\27\2\u0287\u0288\7\5\2\2\u0288\u0289\5\\"+ - "/\2\u0289\u028a\7\4\2\2\u028aG\3\2\2\2\u028b\u0291\5J&\2\u028c\u028d\7"+ - "`\2\2\u028d\u028e\5J&\2\u028e\u028f\7f\2\2\u028f\u0291\3\2\2\2\u0290\u028b"+ - "\3\2\2\2\u0290\u028c\3\2\2\2\u0291I\3\2\2\2\u0292\u0293\7\"\2\2\u0293"+ - "\u0294\7\3\2\2\u0294\u0295\5`\61\2\u0295\u0296\7&\2\2\u0296\u0297\5<\37"+ - "\2\u0297\u0298\7\4\2\2\u0298K\3\2\2\2\u0299\u029f\5N(\2\u029a\u029b\7"+ - "`\2\2\u029b\u029c\5N(\2\u029c\u029d\7f\2\2\u029d\u029f\3\2\2\2\u029e\u0299"+ - "\3\2\2\2\u029e\u029a\3\2\2\2\u029fM\3\2\2\2\u02a0\u02a1\5P)\2\u02a1\u02ad"+ - "\7\3\2\2\u02a2\u02a4\5\36\20\2\u02a3\u02a2\3\2\2\2\u02a3\u02a4\3\2\2\2"+ - "\u02a4\u02a5\3\2\2\2\u02a5\u02aa\5,\27\2\u02a6\u02a7\7\5\2\2\u02a7\u02a9"+ - "\5,\27\2\u02a8\u02a6\3\2\2\2\u02a9\u02ac\3\2\2\2\u02aa\u02a8\3\2\2\2\u02aa"+ - "\u02ab\3\2\2\2\u02ab\u02ae\3\2\2\2\u02ac\u02aa\3\2\2\2\u02ad\u02a3\3\2"+ - "\2\2\u02ad\u02ae\3\2\2\2\u02ae\u02af\3\2\2\2\u02af\u02b0\7\4\2\2\u02b0"+ - "O\3\2\2\2\u02b1\u02b5\7\64\2\2\u02b2\u02b5\7I\2\2\u02b3\u02b5\5`\61\2"+ - "\u02b4\u02b1\3\2\2\2\u02b4\u02b2\3\2\2\2\u02b4\u02b3\3\2\2\2\u02b5Q\3"+ - "\2\2\2\u02b6\u02d1\7?\2\2\u02b7\u02d1\5X-\2\u02b8\u02d1\5h\65\2\u02b9"+ - "\u02d1\5V,\2\u02ba\u02bc\7w\2\2\u02bb\u02ba\3\2\2\2\u02bc\u02bd\3\2\2"+ - "\2\u02bd\u02bb\3\2\2\2\u02bd\u02be\3\2\2\2\u02be\u02d1\3\2\2\2\u02bf\u02d1"+ - "\7v\2\2\u02c0\u02c1\7b\2\2\u02c1\u02c2\5j\66\2\u02c2\u02c3\7f\2\2\u02c3"+ - "\u02d1\3\2\2\2\u02c4\u02c5\7c\2\2\u02c5\u02c6\5j\66\2\u02c6\u02c7\7f\2"+ - "\2\u02c7\u02d1\3\2\2\2\u02c8\u02c9\7d\2\2\u02c9\u02ca\5j\66\2\u02ca\u02cb"+ - "\7f\2\2\u02cb\u02d1\3\2\2\2\u02cc\u02cd\7e\2\2\u02cd\u02ce\5j\66\2\u02ce"+ - "\u02cf\7f\2\2\u02cf\u02d1\3\2\2\2\u02d0\u02b6\3\2\2\2\u02d0\u02b7\3\2"+ - "\2\2\u02d0\u02b8\3\2\2\2\u02d0\u02b9\3\2\2\2\u02d0\u02bb\3\2\2\2\u02d0"+ - "\u02bf\3\2\2\2\u02d0\u02c0\3\2\2\2\u02d0\u02c4\3\2\2\2\u02d0\u02c8\3\2"+ - "\2\2\u02d0\u02cc\3\2\2\2\u02d1S\3\2\2\2\u02d2\u02d3\t\r\2\2\u02d3U\3\2"+ - "\2\2\u02d4\u02d5\t\16\2\2\u02d5W\3\2\2\2\u02d6\u02d8\7\60\2\2\u02d7\u02d9"+ - "\t\7\2\2\u02d8\u02d7\3\2\2\2\u02d8\u02d9\3\2\2\2\u02d9\u02dc\3\2\2\2\u02da"+ - "\u02dd\5h\65\2\u02db\u02dd\5j\66\2\u02dc\u02da\3\2\2\2\u02dc\u02db\3\2"+ - "\2\2\u02dd\u02de\3\2\2\2\u02de\u02e1\5Z.\2\u02df\u02e0\7V\2\2\u02e0\u02e2"+ - "\5Z.\2\u02e1\u02df\3\2\2\2\u02e1\u02e2\3\2\2\2\u02e2Y\3\2\2\2\u02e3\u02e4"+ - "\t\17\2\2\u02e4[\3\2\2\2\u02e5\u02e6\5`\61\2\u02e6]\3\2\2\2\u02e7\u02e8"+ - "\5`\61\2\u02e8\u02e9\7u\2\2\u02e9\u02eb\3\2\2\2\u02ea\u02e7\3\2\2\2\u02eb"+ - "\u02ee\3\2\2\2\u02ec\u02ea\3\2\2\2\u02ec\u02ed\3\2\2\2\u02ed\u02ef\3\2"+ - "\2\2\u02ee\u02ec\3\2\2\2\u02ef\u02f0\5`\61\2\u02f0_\3\2\2\2\u02f1\u02f4"+ - "\5d\63\2\u02f2\u02f4\5f\64\2\u02f3\u02f1\3\2\2\2\u02f3\u02f2\3\2\2\2\u02f4"+ - "a\3\2\2\2\u02f5\u02f6\5`\61\2\u02f6\u02f7\7\6\2\2\u02f7\u02f9\3\2\2\2"+ - "\u02f8\u02f5\3\2\2\2\u02f8\u02f9\3\2\2\2\u02f9\u02fa\3\2\2\2\u02fa\u0302"+ - "\7|\2\2\u02fb\u02fc\5`\61\2\u02fc\u02fd\7\6\2\2\u02fd\u02ff\3\2\2\2\u02fe"+ - "\u02fb\3\2\2\2\u02fe\u02ff\3\2\2\2\u02ff\u0300\3\2\2\2\u0300\u0302\5`"+ - "\61\2\u0301\u02f8\3\2\2\2\u0301\u02fe\3\2\2\2\u0302c\3\2\2\2\u0303\u0306"+ - "\7}\2\2\u0304\u0306\7~\2\2\u0305\u0303\3\2\2\2\u0305\u0304\3\2\2\2\u0306"+ - "e\3\2\2\2\u0307\u030b\7z\2\2\u0308\u030b\5l\67\2\u0309\u030b\7{\2\2\u030a"+ - "\u0307\3\2\2\2\u030a\u0308\3\2\2\2\u030a\u0309\3\2\2\2\u030bg\3\2\2\2"+ - "\u030c\u030f\7y\2\2\u030d\u030f\7x\2\2\u030e\u030c\3\2\2\2\u030e\u030d"+ - "\3\2\2\2\u030fi\3\2\2\2\u0310\u0311\t\20\2\2\u0311k\3\2\2\2\u0312\u0313"+ - "\t\21\2\2\u0313m\3\2\2\2m}\177\u0083\u008c\u008e\u0092\u0099\u00a0\u00a5"+ - "\u00aa\u00b2\u00b6\u00be\u00c1\u00c7\u00cc\u00cf\u00d4\u00d7\u00d9\u00e1"+ - "\u00e4\u00f0\u00f3\u00f6\u00fd\u0104\u0108\u010c\u0110\u0117\u011b\u011f"+ - "\u0124\u0128\u0130\u0134\u013b\u0146\u0149\u014d\u0159\u015c\u0162\u0169"+ - "\u0170\u0173\u0177\u017b\u017f\u0181\u018c\u0191\u0195\u0198\u019e\u01a1"+ - "\u01a7\u01aa\u01ac\u01cf\u01d7\u01d9\u01e0\u01e5\u01e8\u01f0\u01f9\u01ff"+ - "\u0207\u020c\u0212\u0215\u021c\u0224\u022a\u0236\u0238\u0243\u0250\u0257"+ - "\u025d\u0260\u0265\u026a\u026d\u026f\u027b\u0290\u029e\u02a3\u02aa\u02ad"+ - "\u02b4\u02bd\u02d0\u02d8\u02dc\u02e1\u02ec\u02f3\u02f8\u02fe\u0301\u0305"+ - "\u030a\u030e"; + "\u0257\3\2\2\2\u025a\u025e\7\27\2\2\u025b\u025e\7\25\2\2\u025c\u025e\7"+ + "\26\2\2\u025d\u025a\3\2\2\2\u025d\u025b\3\2\2\2\u025d\u025c\3\2\2\2\u025e"+ + "A\3\2\2\2\u025f\u026a\5D#\2\u0260\u0261\7`\2\2\u0261\u0262\5D#\2\u0262"+ + "\u0263\7f\2\2\u0263\u026a\3\2\2\2\u0264\u026a\5F$\2\u0265\u0266\7`\2\2"+ + "\u0266\u0267\5F$\2\u0267\u0268\7f\2\2\u0268\u026a\3\2\2\2\u0269\u025f"+ + "\3\2\2\2\u0269\u0260\3\2\2\2\u0269\u0264\3\2\2\2\u0269\u0265\3\2\2\2\u026a"+ + "C\3\2\2\2\u026b\u026c\7\20\2\2\u026c\u026d\7\3\2\2\u026d\u026e\5,\27\2"+ + "\u026e\u026f\7\f\2\2\u026f\u0270\5\\/\2\u0270\u0271\7\4\2\2\u0271E\3\2"+ + "\2\2\u0272\u0273\7\24\2\2\u0273\u0274\7\3\2\2\u0274\u0275\5,\27\2\u0275"+ + "\u0276\7\5\2\2\u0276\u0277\5\\/\2\u0277\u0278\7\4\2\2\u0278G\3\2\2\2\u0279"+ + "\u027f\5J&\2\u027a\u027b\7`\2\2\u027b\u027c\5J&\2\u027c\u027d\7f\2\2\u027d"+ + "\u027f\3\2\2\2\u027e\u0279\3\2\2\2\u027e\u027a\3\2\2\2\u027fI\3\2\2\2"+ + "\u0280\u0281\7\"\2\2\u0281\u0282\7\3\2\2\u0282\u0283\5`\61\2\u0283\u0284"+ + "\7&\2\2\u0284\u0285\5<\37\2\u0285\u0286\7\4\2\2\u0286K\3\2\2\2\u0287\u028d"+ + "\5N(\2\u0288\u0289\7`\2\2\u0289\u028a\5N(\2\u028a\u028b\7f\2\2\u028b\u028d"+ + "\3\2\2\2\u028c\u0287\3\2\2\2\u028c\u0288\3\2\2\2\u028dM\3\2\2\2\u028e"+ + "\u028f\5P)\2\u028f\u029b\7\3\2\2\u0290\u0292\5\36\20\2\u0291\u0290\3\2"+ + "\2\2\u0291\u0292\3\2\2\2\u0292\u0293\3\2\2\2\u0293\u0298\5,\27\2\u0294"+ + "\u0295\7\5\2\2\u0295\u0297\5,\27\2\u0296\u0294\3\2\2\2\u0297\u029a\3\2"+ + "\2\2\u0298\u0296\3\2\2\2\u0298\u0299\3\2\2\2\u0299\u029c\3\2\2\2\u029a"+ + "\u0298\3\2\2\2\u029b\u0291\3\2\2\2\u029b\u029c\3\2\2\2\u029c\u029d\3\2"+ + "\2\2\u029d\u029e\7\4\2\2\u029eO\3\2\2\2\u029f\u02a3\7\64\2\2\u02a0\u02a3"+ + "\7I\2\2\u02a1\u02a3\5`\61\2\u02a2\u029f\3\2\2\2\u02a2\u02a0\3\2\2\2\u02a2"+ + "\u02a1\3\2\2\2\u02a3Q\3\2\2\2\u02a4\u02bf\7?\2\2\u02a5\u02bf\5X-\2\u02a6"+ + "\u02bf\5h\65\2\u02a7\u02bf\5V,\2\u02a8\u02aa\7w\2\2\u02a9\u02a8\3\2\2"+ + "\2\u02aa\u02ab\3\2\2\2\u02ab\u02a9\3\2\2\2\u02ab\u02ac\3\2\2\2\u02ac\u02bf"+ + "\3\2\2\2\u02ad\u02bf\7v\2\2\u02ae\u02af\7b\2\2\u02af\u02b0\5j\66\2\u02b0"+ + "\u02b1\7f\2\2\u02b1\u02bf\3\2\2\2\u02b2\u02b3\7c\2\2\u02b3\u02b4\5j\66"+ + "\2\u02b4\u02b5\7f\2\2\u02b5\u02bf\3\2\2\2\u02b6\u02b7\7d\2\2\u02b7\u02b8"+ + "\5j\66\2\u02b8\u02b9\7f\2\2\u02b9\u02bf\3\2\2\2\u02ba\u02bb\7e\2\2\u02bb"+ + "\u02bc\5j\66\2\u02bc\u02bd\7f\2\2\u02bd\u02bf\3\2\2\2\u02be\u02a4\3\2"+ + "\2\2\u02be\u02a5\3\2\2\2\u02be\u02a6\3\2\2\2\u02be\u02a7\3\2\2\2\u02be"+ + "\u02a9\3\2\2\2\u02be\u02ad\3\2\2\2\u02be\u02ae\3\2\2\2\u02be\u02b2\3\2"+ + "\2\2\u02be\u02b6\3\2\2\2\u02be\u02ba\3\2\2\2\u02bfS\3\2\2\2\u02c0\u02c1"+ + "\t\r\2\2\u02c1U\3\2\2\2\u02c2\u02c3\t\16\2\2\u02c3W\3\2\2\2\u02c4\u02c6"+ + "\7\60\2\2\u02c5\u02c7\t\7\2\2\u02c6\u02c5\3\2\2\2\u02c6\u02c7\3\2\2\2"+ + "\u02c7\u02ca\3\2\2\2\u02c8\u02cb\5h\65\2\u02c9\u02cb\5j\66\2\u02ca\u02c8"+ + "\3\2\2\2\u02ca\u02c9\3\2\2\2\u02cb\u02cc\3\2\2\2\u02cc\u02cf\5Z.\2\u02cd"+ + "\u02ce\7V\2\2\u02ce\u02d0\5Z.\2\u02cf\u02cd\3\2\2\2\u02cf\u02d0\3\2\2"+ + "\2\u02d0Y\3\2\2\2\u02d1\u02d2\t\17\2\2\u02d2[\3\2\2\2\u02d3\u02d4\5`\61"+ + "\2\u02d4]\3\2\2\2\u02d5\u02d6\5`\61\2\u02d6\u02d7\7u\2\2\u02d7\u02d9\3"+ + "\2\2\2\u02d8\u02d5\3\2\2\2\u02d9\u02dc\3\2\2\2\u02da\u02d8\3\2\2\2\u02da"+ + "\u02db\3\2\2\2\u02db\u02dd\3\2\2\2\u02dc\u02da\3\2\2\2\u02dd\u02de\5`"+ + "\61\2\u02de_\3\2\2\2\u02df\u02e2\5d\63\2\u02e0\u02e2\5f\64\2\u02e1\u02df"+ + "\3\2\2\2\u02e1\u02e0\3\2\2\2\u02e2a\3\2\2\2\u02e3\u02e4\5`\61\2\u02e4"+ + "\u02e5\7\6\2\2\u02e5\u02e7\3\2\2\2\u02e6\u02e3\3\2\2\2\u02e6\u02e7\3\2"+ + "\2\2\u02e7\u02e8\3\2\2\2\u02e8\u02f0\7|\2\2\u02e9\u02ea\5`\61\2\u02ea"+ + "\u02eb\7\6\2\2\u02eb\u02ed\3\2\2\2\u02ec\u02e9\3\2\2\2\u02ec\u02ed\3\2"+ + "\2\2\u02ed\u02ee\3\2\2\2\u02ee\u02f0\5`\61\2\u02ef\u02e6\3\2\2\2\u02ef"+ + "\u02ec\3\2\2\2\u02f0c\3\2\2\2\u02f1\u02f4\7}\2\2\u02f2\u02f4\7~\2\2\u02f3"+ + "\u02f1\3\2\2\2\u02f3\u02f2\3\2\2\2\u02f4e\3\2\2\2\u02f5\u02f9\7z\2\2\u02f6"+ + "\u02f9\5l\67\2\u02f7\u02f9\7{\2\2\u02f8\u02f5\3\2\2\2\u02f8\u02f6\3\2"+ + "\2\2\u02f8\u02f7\3\2\2\2\u02f9g\3\2\2\2\u02fa\u02fd\7y\2\2\u02fb\u02fd"+ + "\7x\2\2\u02fc\u02fa\3\2\2\2\u02fc\u02fb\3\2\2\2\u02fdi\3\2\2\2\u02fe\u02ff"+ + "\t\20\2\2\u02ffk\3\2\2\2\u0300\u0301\t\21\2\2\u0301m\3\2\2\2h}\177\u0083"+ + "\u008c\u008e\u0092\u0099\u00a0\u00a5\u00aa\u00b2\u00b6\u00be\u00c1\u00c7"+ + "\u00cc\u00cf\u00d4\u00d7\u00d9\u00e1\u00e4\u00f0\u00f3\u00f6\u00fd\u0104"+ + "\u0108\u010c\u0110\u0117\u011b\u011f\u0124\u0128\u0130\u0134\u013b\u0146"+ + "\u0149\u014d\u0159\u015c\u0162\u0169\u0170\u0173\u0177\u017b\u017f\u0181"+ + "\u018c\u0191\u0195\u0198\u019e\u01a1\u01a7\u01aa\u01ac\u01cf\u01d7\u01d9"+ + "\u01e0\u01e5\u01e8\u01f0\u01f9\u01ff\u0207\u020c\u0212\u0215\u021c\u0224"+ + "\u022a\u0236\u0238\u0243\u0250\u0257\u025d\u0269\u027e\u028c\u0291\u0298"+ + "\u029b\u02a2\u02ab\u02be\u02c6\u02ca\u02cf\u02da\u02e1\u02e6\u02ec\u02ef"+ + "\u02f3\u02f8\u02fc"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/util/DateUtils.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/util/DateUtils.java index a6b0f12dafbff..0f8afdd155215 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/util/DateUtils.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/util/DateUtils.java @@ -8,6 +8,9 @@ import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateFormatters; +import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Foldables; +import org.elasticsearch.xpack.sql.parser.ParsingException; import org.elasticsearch.xpack.sql.proto.StringUtils; import java.time.Instant; @@ -37,6 +40,7 @@ public final class DateUtils { .toFormatter().withZone(UTC); private static final DateFormatter UTC_DATE_TIME_FORMATTER = DateFormatter.forPattern("date_optional_time").withZone(UTC); + private static final int DEFAULT_PRECISION_FOR_CURRENT_FUNCTIONS = 3; private DateUtils() {} @@ -123,4 +127,25 @@ public static long minDayInterval(long l) { } return l - (l % DAY_IN_MILLIS); } + + public static int getNanoPrecision(Expression precisionExpression, int nano) { + int precision = DEFAULT_PRECISION_FOR_CURRENT_FUNCTIONS; + + if (precisionExpression != null) { + try { + precision = Foldables.intValueOf(precisionExpression); + } catch (Exception e) { + throw new ParsingException(precisionExpression.source(), "invalid precision; " + e.getMessage()); + } + } + + if (precision < 0 || precision > 9) { + throw new ParsingException(precisionExpression.source(), "precision needs to be between [0-9], received [{}]", + precisionExpression.sourceText()); + } + + // remove the remainder + nano = nano - nano % (int) Math.pow(10, (9 - precision)); + return nano; + } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java index 405d4805410e5..166490699bb97 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java @@ -6,11 +6,21 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.TestUtils; +import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer; +import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier; +import org.elasticsearch.xpack.sql.analysis.index.EsIndex; +import org.elasticsearch.xpack.sql.analysis.index.IndexResolution; import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.sql.parser.ParsingException; +import org.elasticsearch.xpack.sql.parser.SqlParser; import org.elasticsearch.xpack.sql.session.Configuration; +import org.elasticsearch.xpack.sql.stats.Metrics; import org.elasticsearch.xpack.sql.tree.AbstractNodeTestCase; +import org.elasticsearch.xpack.sql.type.TypesTests; import java.time.ZoneId; import java.time.ZonedDateTime; @@ -22,7 +32,7 @@ public class CurrentDateTimeTests extends AbstractNodeTestCase { public static CurrentDateTime randomCurrentDateTime() { - return new CurrentDateTime(EMPTY, Literal.of(EMPTY, randomInt(10)), TestUtils.randomConfiguration()); + return new CurrentDateTime(EMPTY, Literal.of(EMPTY, randomInt(9)), TestUtils.randomConfiguration()); } @Override @@ -39,8 +49,8 @@ protected CurrentDateTime copy(CurrentDateTime instance) { protected CurrentDateTime mutate(CurrentDateTime instance) { ZonedDateTime now = instance.configuration().now(); ZoneId mutatedZoneId = randomValueOtherThanMany(o -> Objects.equals(now.getOffset(), o.getRules().getOffset(now.toInstant())), - () -> randomZone()); - return new CurrentDateTime(instance.source(), Literal.of(EMPTY, randomInt(10)), TestUtils.randomConfiguration(mutatedZoneId)); + ESTestCase::randomZone); + return new CurrentDateTime(instance.source(), Literal.of(EMPTY, randomInt(9)), TestUtils.randomConfiguration(mutatedZoneId)); } @Override @@ -75,4 +85,19 @@ public void testDefaultPrecision() { ZonedDateTime zdt = ZonedDateTime.parse("2019-02-26T12:34:56.123456789Z"); assertEquals(123_000_000, CurrentDateTime.nanoPrecision(zdt, null).getNano()); } + + public void testInvalidPrecision() { + SqlParser parser = new SqlParser(); + IndexResolution indexResolution = IndexResolution.valid(new EsIndex("test", + TypesTests.loadMapping("mapping-multi-field-with-nested.json"))); + + Analyzer analyzer = new Analyzer(TestUtils.TEST_CFG, new FunctionRegistry(), indexResolution, new Verifier(new Metrics())); + ParsingException e = expectThrows(ParsingException.class, () -> + analyzer.analyze(parser.createStatement("SELECT CURRENT_TIMESTAMP(100000000000000)"), true)); + assertEquals("line 1:27: invalid precision; [100000000000000] out of [integer] range", e.getMessage()); + + e = expectThrows(ParsingException.class, () -> + analyzer.analyze(parser.createStatement("SELECT CURRENT_TIMESTAMP(100)"), true)); + assertEquals("line 1:27: precision needs to be between [0-9], received [100]", e.getMessage()); + } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTimeTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTimeTests.java index 639ac4b7b9291..8603a3c6dd9c1 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTimeTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTimeTests.java @@ -6,11 +6,21 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.TestUtils; +import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer; +import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier; +import org.elasticsearch.xpack.sql.analysis.index.EsIndex; +import org.elasticsearch.xpack.sql.analysis.index.IndexResolution; import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.sql.parser.ParsingException; +import org.elasticsearch.xpack.sql.parser.SqlParser; import org.elasticsearch.xpack.sql.session.Configuration; +import org.elasticsearch.xpack.sql.stats.Metrics; import org.elasticsearch.xpack.sql.tree.AbstractNodeTestCase; +import org.elasticsearch.xpack.sql.type.TypesTests; import java.time.OffsetTime; import java.time.ZoneId; @@ -23,7 +33,7 @@ public class CurrentTimeTests extends AbstractNodeTestCase { public static CurrentTime randomCurrentTime() { - return new CurrentTime(EMPTY, Literal.of(EMPTY, randomInt(10)), TestUtils.randomConfiguration()); + return new CurrentTime(EMPTY, Literal.of(EMPTY, randomInt(9)), TestUtils.randomConfiguration()); } @Override @@ -40,8 +50,8 @@ protected CurrentTime copy(CurrentTime instance) { protected CurrentTime mutate(CurrentTime instance) { ZonedDateTime now = instance.configuration().now(); ZoneId mutatedZoneId = randomValueOtherThanMany(o -> Objects.equals(now.getOffset(), o.getRules().getOffset(now.toInstant())), - () -> randomZone()); - return new CurrentTime(instance.source(), Literal.of(EMPTY, randomInt(10)), TestUtils.randomConfiguration(mutatedZoneId)); + ESTestCase::randomZone); + return new CurrentTime(instance.source(), Literal.of(EMPTY, randomInt(9)), TestUtils.randomConfiguration(mutatedZoneId)); } @Override @@ -76,4 +86,19 @@ public void testDefaultPrecision() { OffsetTime ot = OffsetTime.parse("12:34:56.123456789Z"); assertEquals(123_000_000, CurrentTime.nanoPrecision(ot, null).getNano()); } + + public void testInvalidPrecision() { + SqlParser parser = new SqlParser(); + IndexResolution indexResolution = IndexResolution.valid(new EsIndex("test", + TypesTests.loadMapping("mapping-multi-field-with-nested.json"))); + + Analyzer analyzer = new Analyzer(TestUtils.TEST_CFG, new FunctionRegistry(), indexResolution, new Verifier(new Metrics())); + ParsingException e = expectThrows(ParsingException.class, () -> + analyzer.analyze(parser.createStatement("SELECT CURRENT_TIME(100000000000000)"), true)); + assertEquals("line 1:22: invalid precision; [100000000000000] out of [integer] range", e.getMessage()); + + e = expectThrows(ParsingException.class, () -> + analyzer.analyze(parser.createStatement("SELECT CURRENT_TIME(100)"), true)); + assertEquals("line 1:22: precision needs to be between [0-9], received [100]", e.getMessage()); + } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java index 6fd4611a43416..c22c9c5702965 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java @@ -226,6 +226,29 @@ public void testGUIDValidationLength() { assertEquals("line 1:8: Invalid GUID, too short", ex.getMessage()); } + public void testCurrentTimestampAsEscapedExpression() { + Expression expr = parser.createExpression("{fn CURRENT_TIMESTAMP(2)}"); + assertEquals(UnresolvedFunction.class, expr.getClass()); + UnresolvedFunction ur = (UnresolvedFunction) expr; + assertEquals("{fn CURRENT_TIMESTAMP(2)}", ur.sourceText()); + assertEquals(1, ur.children().size()); + } + + public void testCurrentDateAsEscapedExpression() { + Expression expr = parser.createExpression("{fn CURRENT_DATE()}"); + assertEquals(UnresolvedFunction.class, expr.getClass()); + UnresolvedFunction ur = (UnresolvedFunction) expr; + assertEquals("{fn CURRENT_DATE()}", ur.sourceText()); + assertEquals(0, ur.children().size()); + } + + public void testCurrentTimeAsEscapedExpression() { + Expression expr = parser.createExpression("{fn CURRENT_TIME(2)}"); + assertEquals(UnresolvedFunction.class, expr.getClass()); + UnresolvedFunction ur = (UnresolvedFunction) expr; + assertEquals("{fn CURRENT_TIME(2)}", ur.sourceText()); + assertEquals(1, ur.children().size()); + } public void testLimit() { Limit limit = limit(10); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java index 658c11a8ca562..98d7922d21839 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java @@ -398,12 +398,7 @@ public void testCurrentTimestampPrecision() { assertEquals(1, ur.children().size()); Expression child = ur.children().get(0); assertEquals(Literal.class, child.getClass()); - assertEquals(Short.valueOf((short) 4), child.fold()); - } - - public void testCurrentTimestampInvalidPrecision() { - ParsingException ex = expectThrows(ParsingException.class, () -> parser.createExpression("CURRENT_TIMESTAMP(100)")); - assertEquals("line 1:20: Precision needs to be between [0-9], received [100]", ex.getMessage()); + assertEquals(4, child.fold()); } public void testCurrentDate() { @@ -438,12 +433,7 @@ public void testCurrentTimePrecision() { assertEquals(1, ur.children().size()); Expression child = ur.children().get(0); assertEquals(Literal.class, child.getClass()); - assertEquals(Short.valueOf((short) 7), child.fold()); - } - - public void testCurrentTimeInvalidPrecision() { - ParsingException ex = expectThrows(ParsingException.class, () -> parser.createExpression("CURRENT_TIME(100)")); - assertEquals("line 1:15: Precision needs to be between [0-9], received [100]", ex.getMessage()); + assertEquals(7, child.fold()); } public void testSourceKeyword() { From dc0599a9662e3af6f400637a7b7d78dcf934f6d6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 17 Apr 2019 08:34:07 +0100 Subject: [PATCH 067/260] Do not create missing directories in readonly repo (#41249) Today we erroneously look for a node setting called `readonly` when deciding whether or not to create a missing directory in a filesystem repository. This change fixes this by using the repository setting instead. Closes #41009 Relates #26909 --- .../common/blobstore/fs/FsBlobStore.java | 13 +++-- .../repositories/fs/FsRepository.java | 2 +- .../fs/FsBlobStoreContainerTests.java | 2 +- .../common/blobstore/fs/FsBlobStoreTests.java | 8 ++- .../fs/FsBlobStoreRepositoryIT.java | 53 +++++++++++++++++++ .../snapshots/BlobStoreFormatIT.java | 3 +- 6 files changed, 68 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobStore.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobStore.java index eea30dd4e530f..8a4d51e4dc93c 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobStore.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobStore.java @@ -40,10 +40,10 @@ public class FsBlobStore implements BlobStore { private final boolean readOnly; - public FsBlobStore(Settings settings, Path path) throws IOException { + public FsBlobStore(Settings settings, Path path, boolean readonly) throws IOException { this.path = path; - this.readOnly = settings.getAsBoolean("readonly", false); - if (!this.readOnly) { + this.readOnly = readonly; + if (this.readOnly == false) { Files.createDirectories(path); } this.bufferSizeInBytes = (int) settings.getAsBytesSize("repositories.fs.buffer_size", @@ -74,6 +74,11 @@ public BlobContainer blobContainer(BlobPath path) { @Override public void delete(BlobPath path) throws IOException { + assert readOnly == false : "should not delete anything from a readonly repository: " + path; + //noinspection ConstantConditions in case assertions are disabled + if (readOnly) { + throw new ElasticsearchException("unexpectedly deleting [" + path + "] from a readonly repository"); + } IOUtils.rm(buildPath(path)); } @@ -84,7 +89,7 @@ public void close() { private synchronized Path buildAndCreate(BlobPath path) throws IOException { Path f = buildPath(path); - if (!readOnly) { + if (readOnly == false) { Files.createDirectories(f); } return f; diff --git a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java index 710e6aad40d16..8f495f2d4842a 100644 --- a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java @@ -108,7 +108,7 @@ public FsRepository(RepositoryMetaData metadata, Environment environment, NamedX protected BlobStore createBlobStore() throws Exception { final String location = REPOSITORIES_LOCATION_SETTING.get(metadata.settings()); final Path locationFile = environment.resolveRepoFile(location); - return new FsBlobStore(environment.settings(), locationFile); + return new FsBlobStore(environment.settings(), locationFile, isReadOnly()); } @Override diff --git a/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreContainerTests.java b/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreContainerTests.java index 9230cded82b1d..7bd24aec8de90 100644 --- a/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreContainerTests.java +++ b/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreContainerTests.java @@ -37,6 +37,6 @@ protected BlobStore newBlobStore() throws IOException { } else { settings = Settings.EMPTY; } - return new FsBlobStore(settings, createTempDir()); + return new FsBlobStore(settings, createTempDir(), false); } } diff --git a/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreTests.java b/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreTests.java index 59e4ffd7927ca..4a1b1e1016fb9 100644 --- a/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreTests.java +++ b/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobStoreTests.java @@ -42,15 +42,14 @@ protected BlobStore newBlobStore() throws IOException { } else { settings = Settings.EMPTY; } - return new FsBlobStore(settings, createTempDir()); + return new FsBlobStore(settings, createTempDir(), false); } public void testReadOnly() throws Exception { - Settings settings = Settings.builder().put("readonly", true).build(); Path tempDir = createTempDir(); Path path = tempDir.resolve("bar"); - try (FsBlobStore store = new FsBlobStore(settings, path)) { + try (FsBlobStore store = new FsBlobStore(Settings.EMPTY, path, true)) { assertFalse(Files.exists(path)); BlobPath blobPath = BlobPath.cleanPath().add("foo"); store.blobContainer(blobPath); @@ -61,8 +60,7 @@ public void testReadOnly() throws Exception { assertFalse(Files.exists(storePath)); } - settings = randomBoolean() ? Settings.EMPTY : Settings.builder().put("readonly", false).build(); - try (FsBlobStore store = new FsBlobStore(settings, path)) { + try (FsBlobStore store = new FsBlobStore(Settings.EMPTY, path, false)) { assertTrue(Files.exists(path)); BlobPath blobPath = BlobPath.cleanPath().add("foo"); BlobContainer container = store.blobContainer(blobPath); diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIT.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIT.java index 1ed42cb24746b..dd4ca7bfd20e4 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIT.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsBlobStoreRepositoryIT.java @@ -18,12 +18,22 @@ */ package org.elasticsearch.repositories.fs; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.blobstore.ESBlobStoreRepositoryIntegTestCase; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.util.concurrent.ExecutionException; +import java.util.stream.Stream; + import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.instanceOf; public class FsBlobStoreRepositoryIT extends ESBlobStoreRepositoryIntegTestCase { @@ -41,4 +51,47 @@ protected void createTestRepository(String name, boolean verify) { protected void afterCreationCheck(Repository repository) { assertThat(repository, instanceOf(FsRepository.class)); } + + public void testMissingDirectoriesNotCreatedInReadonlyRepository() throws IOException, ExecutionException, InterruptedException { + final String repoName = randomAsciiName(); + final Path repoPath = randomRepoPath(); + + logger.info("--> creating repository {} at {}", repoName, repoPath); + + assertAcked(client().admin().cluster().preparePutRepository(repoName).setType("fs").setSettings(Settings.builder() + .put("location", repoPath) + .put("compress", randomBoolean()) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); + + String indexName = randomAsciiName(); + int docCount = iterations(10, 1000); + logger.info("--> create random index {} with {} records", indexName, docCount); + addRandomDocuments(indexName, docCount); + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), docCount); + + final String snapshotName = randomAsciiName(); + logger.info("--> create snapshot {}:{}", repoName, snapshotName); + assertSuccessfulSnapshot(client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true).setIndices(indexName)); + + assertAcked(client().admin().indices().prepareDelete(indexName)); + assertAcked(client().admin().cluster().prepareDeleteRepository(repoName)); + + final Path deletedPath; + try (Stream contents = Files.list(repoPath.resolve("indices"))) { + //noinspection OptionalGetWithoutIsPresent because we know there's a subdirectory + deletedPath = contents.filter(Files::isDirectory).findAny().get(); + IOUtils.rm(deletedPath); + } + assertFalse(Files.exists(deletedPath)); + + assertAcked(client().admin().cluster().preparePutRepository(repoName).setType("fs").setSettings(Settings.builder() + .put("location", repoPath).put("readonly", true))); + + final ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> + client().admin().cluster().prepareRestoreSnapshot(repoName, snapshotName).setWaitForCompletion(randomBoolean()).get()); + assertThat(exception.getRootCause(), instanceOf(NoSuchFileException.class)); + + assertFalse("deleted path is not recreated in readonly repository", Files.exists(deletedPath)); + } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatIT.java b/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatIT.java index 6f4f69ad67e88..4febd0695c936 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatIT.java @@ -238,8 +238,7 @@ public void writeBlobAtomic(String blobName, InputStream inputStream, long blobS } protected BlobStore createTestBlobStore() throws IOException { - Settings settings = Settings.builder().build(); - return new FsBlobStore(settings, randomRepoPath()); + return new FsBlobStore(Settings.EMPTY, randomRepoPath(), false); } protected void randomCorruption(BlobContainer blobContainer, String blobName) throws IOException { From c9bbb29eadd1e5e9886b79efe605f2ba7ac3be8e Mon Sep 17 00:00:00 2001 From: David Kyle Date: Wed, 17 Apr 2019 09:01:15 +0100 Subject: [PATCH 068/260] [ML-DataFrame] Resolve random test failure using deterministic name (#41262) --- .../transforms/DataFrameTransformCheckpointTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformCheckpointTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformCheckpointTests.java index 6fe896872f3fd..786fafc2c07ad 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformCheckpointTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformCheckpointTests.java @@ -91,8 +91,8 @@ public void testMatches() throws IOException { .matches(new DataFrameTransformCheckpoint(id, timestamp, checkpoint, checkpointsByIndex, (timeUpperBound / 2) + 1))); } - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/41076") public void testGetBehind() { + String baseIndexName = randomAlphaOfLength(8); String id = randomAlphaOfLengthBetween(1, 10); long timestamp = randomNonNegativeLong(); @@ -112,7 +112,7 @@ public void testGetBehind() { checkpoints2.add(shardCheckpoint + 10); } - String indexName = randomAlphaOfLengthBetween(1, 10); + String indexName = baseIndexName + i; checkpointsByIndexOld.put(indexName, checkpoints1.stream().mapToLong(l -> l).toArray()); checkpointsByIndexNew.put(indexName, checkpoints2.stream().mapToLong(l -> l).toArray()); From 777408df78f1595c32cfa145d517facb2da42d42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 17 Apr 2019 10:21:44 +0200 Subject: [PATCH 069/260] Some cleanups in NoisyChannelSpellChecker (#40949) One of the two #getCorrections methods is only used in tests, so we can move it and any of the required helper methods to that test. Also reducing the visibility of several methods to package private since the class isn't used elsewhere outside the package. --- .../phrase/NoisyChannelSpellChecker.java | 38 ++------- .../suggest/phrase/PhraseSuggester.java | 12 ++- .../phrase/NoisyChannelSpellCheckerTests.java | 81 +++++++++++-------- 3 files changed, 63 insertions(+), 68 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellChecker.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellChecker.java index 7f225f1c3ea73..9612d29f4f556 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellChecker.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellChecker.java @@ -18,48 +18,34 @@ */ package org.elasticsearch.search.suggest.phrase; -import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.shingle.ShingleFilter; import org.apache.lucene.analysis.synonym.SynonymFilter; import org.apache.lucene.analysis.tokenattributes.TypeAttribute; import org.apache.lucene.codecs.TermStats; -import org.apache.lucene.index.IndexReader; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; -import org.apache.lucene.util.CharsRefBuilder; import org.elasticsearch.search.suggest.phrase.DirectCandidateGenerator.Candidate; import org.elasticsearch.search.suggest.phrase.DirectCandidateGenerator.CandidateSet; -import java.io.CharArrayReader; import java.io.IOException; import java.util.ArrayList; import java.util.List; -//TODO public for tests -public final class NoisyChannelSpellChecker { +final class NoisyChannelSpellChecker { public static final double REAL_WORD_LIKELIHOOD = 0.95d; public static final int DEFAULT_TOKEN_LIMIT = 10; private final double realWordLikelihood; private final boolean requireUnigram; private final int tokenLimit; - public NoisyChannelSpellChecker() { - this(REAL_WORD_LIKELIHOOD); - } - - public NoisyChannelSpellChecker(double nonErrorLikelihood) { - this(nonErrorLikelihood, true, DEFAULT_TOKEN_LIMIT); - } - - public NoisyChannelSpellChecker(double nonErrorLikelihood, boolean requireUnigram, int tokenLimit) { + NoisyChannelSpellChecker(double nonErrorLikelihood, boolean requireUnigram, int tokenLimit) { this.realWordLikelihood = nonErrorLikelihood; this.requireUnigram = requireUnigram; this.tokenLimit = tokenLimit; - } - public Result getCorrections(TokenStream stream, final CandidateGenerator generator, + Result getCorrections(TokenStream stream, final CandidateGenerator generator, float maxErrors, int numCorrections, WordScorer wordScorer, float confidence, int gramSize) throws IOException { final List candidateSetsList = new ArrayList<>(); @@ -131,26 +117,12 @@ public void end() { return new Result(bestCandidates, cutoffScore); } - public Result getCorrections(Analyzer analyzer, BytesRef query, CandidateGenerator generator, - float maxErrors, int numCorrections, IndexReader reader, String analysisField, - WordScorer scorer, float confidence, int gramSize) throws IOException { - - return getCorrections(tokenStream(analyzer, query, new CharsRefBuilder(), analysisField), generator, maxErrors, - numCorrections, scorer, confidence, gramSize); - - } - - public TokenStream tokenStream(Analyzer analyzer, BytesRef query, CharsRefBuilder spare, String field) throws IOException { - spare.copyUTF8Bytes(query); - return analyzer.tokenStream(field, new CharArrayReader(spare.chars(), 0, spare.length())); - } - - public static class Result { + static class Result { public static final Result EMPTY = new Result(Correction.EMPTY, Double.MIN_VALUE); public final Correction[] corrections; public final double cutoffScore; - public Result(Correction[] corrections, double cutoffScore) { + private Result(Correction[] corrections, double cutoffScore) { this.corrections = corrections; this.cutoffScore = cutoffScore; } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java index 413afd155d45b..d80fd68dacbf8 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.suggest.phrase; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.MultiTerms; @@ -45,6 +46,7 @@ import org.elasticsearch.search.suggest.SuggestionSearchContext.SuggestionContext; import org.elasticsearch.search.suggest.phrase.NoisyChannelSpellChecker.Result; +import java.io.CharArrayReader; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -93,11 +95,12 @@ public Suggestion> innerExecute(String name, P WordScorer wordScorer = suggestion.model().newScorer(indexReader, suggestTerms, suggestField, realWordErrorLikelihood, separator); Result checkerResult; - try (TokenStream stream = checker.tokenStream(suggestion.getAnalyzer(), suggestion.getText(), spare, suggestion.getField())) { + try (TokenStream stream = tokenStream(suggestion.getAnalyzer(), suggestion.getText(), spare, + suggestion.getField())) { checkerResult = checker.getCorrections(stream, new MultiCandidateGeneratorWrapper(suggestion.getShardSize(), gens.toArray(new CandidateGenerator[gens.size()])), suggestion.maxErrors(), suggestion.getShardSize(), wordScorer, suggestion.confidence(), suggestion.gramSize()); - } + } PhraseSuggestion.Entry resultEntry = buildResultEntry(suggestion, spare, checkerResult.cutoffScore); response.addTerm(resultEntry); @@ -144,6 +147,11 @@ public Suggestion> innerExecute(String name, P return response; } + private static TokenStream tokenStream(Analyzer analyzer, BytesRef query, CharsRefBuilder spare, String field) throws IOException { + spare.copyUTF8Bytes(query); + return analyzer.tokenStream(field, new CharArrayReader(spare.chars(), 0, spare.length())); + } + private static PhraseSuggestion.Entry buildResultEntry(SuggestionContext suggestion, CharsRefBuilder spare, double cutoffScore) { spare.copyUTF8Bytes(suggestion.getText()); return new PhraseSuggestion.Entry(new Text(spare.toString()), 0, spare.length(), cutoffScore); diff --git a/server/src/test/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellCheckerTests.java b/server/src/test/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellCheckerTests.java index d819d880c86d3..94c5bf329eb74 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellCheckerTests.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/phrase/NoisyChannelSpellCheckerTests.java @@ -21,6 +21,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.LowerCaseFilter; import org.apache.lucene.analysis.TokenFilter; +import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.core.WhitespaceAnalyzer; import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper; @@ -34,6 +35,7 @@ import org.apache.lucene.document.Field; import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.MultiTerms; @@ -42,14 +44,18 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CharsRefBuilder; import org.elasticsearch.search.suggest.phrase.NoisyChannelSpellChecker.Result; import org.elasticsearch.test.ESTestCase; +import java.io.CharArrayReader; import java.io.IOException; import java.io.StringReader; import java.util.HashMap; import java.util.Map; +import static org.elasticsearch.search.suggest.phrase.NoisyChannelSpellChecker.DEFAULT_TOKEN_LIMIT; +import static org.elasticsearch.search.suggest.phrase.NoisyChannelSpellChecker.REAL_WORD_LIKELIHOOD; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -113,12 +119,12 @@ protected TokenStreamComponents createComponents(String fieldName) { WordScorer wordScorer = new LaplaceScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.95d, new BytesRef(" "), 0.5f); - NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(); + NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(REAL_WORD_LIKELIHOOD, true, DEFAULT_TOKEN_LIMIT); DirectSpellChecker spellchecker = new DirectSpellChecker(); spellchecker.setMinQueryLength(1); DirectCandidateGenerator generator = new DirectCandidateGenerator(spellchecker, "body", SuggestMode.SUGGEST_MORE_POPULAR, ir, 0.95, 5); - Result result = suggester.getCorrections(wrapper, new BytesRef("american ame"), generator, 1, 1, + Result result = getCorrections(suggester, wrapper, new BytesRef("american ame"), generator, 1, 1, ir, "body", wordScorer, 1, 2); Correction[] corrections = result.corrections; assertThat(corrections.length, equalTo(1)); @@ -126,7 +132,7 @@ protected TokenStreamComponents createComponents(String fieldName) { assertThat(corrections[0].join(space, preTag, postTag).utf8ToString(), equalTo("american ace")); assertThat(result.cutoffScore, greaterThan(0d)); - result = suggester.getCorrections(wrapper, new BytesRef("american ame"), generator, 1, 1, + result = getCorrections(suggester, wrapper, new BytesRef("american ame"), generator, 1, 1, ir, "body", wordScorer, 0, 1); corrections = result.corrections; assertThat(corrections.length, equalTo(1)); @@ -134,10 +140,10 @@ protected TokenStreamComponents createComponents(String fieldName) { assertThat(corrections[0].join(space, preTag, postTag).utf8ToString(), equalTo("american ame")); assertThat(result.cutoffScore, equalTo(Double.MIN_VALUE)); - suggester = new NoisyChannelSpellChecker(0.85); + suggester = new NoisyChannelSpellChecker(0.85, true, DEFAULT_TOKEN_LIMIT); wordScorer = new LaplaceScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.85d, new BytesRef(" "), 0.5f); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, ir, "body", wordScorer, 0, 2).corrections; assertThat(corrections.length, equalTo(4)); assertThat(corrections[0].join(space).utf8ToString(), equalTo("xorr the god jewel")); @@ -149,7 +155,7 @@ protected TokenStreamComponents createComponents(String fieldName) { assertThat(corrections[2].join(space, preTag, postTag).utf8ToString(), equalTo("xorn the god jewel")); assertThat(corrections[3].join(space, preTag, postTag).utf8ToString(), equalTo("xorr the got jewel")); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(4)); assertThat(corrections[0].join(space).utf8ToString(), equalTo("xorr the god jewel")); @@ -158,10 +164,10 @@ protected TokenStreamComponents createComponents(String fieldName) { assertThat(corrections[3].join(space).utf8ToString(), equalTo("xorr the got jewel")); // Test some of the highlighting corner cases - suggester = new NoisyChannelSpellChecker(0.85); + suggester = new NoisyChannelSpellChecker(0.85, true, DEFAULT_TOKEN_LIMIT); wordScorer = new LaplaceScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.85d, new BytesRef(" "), 0.5f); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor teh Got-Jewel"), generator, 4f, 4, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor teh Got-Jewel"), generator, 4f, 4, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(4)); assertThat(corrections[0].join(space).utf8ToString(), equalTo("xorr the god jewel")); @@ -195,17 +201,17 @@ protected TokenStreamComponents createComponents(String fieldName) { spellchecker.setAccuracy(0.0f); spellchecker.setMinPrefix(1); spellchecker.setMinQueryLength(1); - suggester = new NoisyChannelSpellChecker(0.85); + suggester = new NoisyChannelSpellChecker(0.85, true, DEFAULT_TOKEN_LIMIT); wordScorer = new LaplaceScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.85d, new BytesRef(" "), 0.5f); - corrections = suggester.getCorrections(analyzer, new BytesRef("captian usa"), generator, 2, 4, + corrections = getCorrections(suggester, analyzer, new BytesRef("captian usa"), generator, 2, 4, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections[0].join(space).utf8ToString(), equalTo("captain america")); assertThat(corrections[0].join(space, preTag, postTag).utf8ToString(), equalTo("captain america")); generator = new DirectCandidateGenerator(spellchecker, "body", SuggestMode.SUGGEST_MORE_POPULAR, ir, 0.85, 10, null, analyzer, MultiTerms.getTerms(ir, "body")); - corrections = suggester.getCorrections(analyzer, new BytesRef("captian usw"), generator, 2, 4, + corrections = getCorrections(suggester, analyzer, new BytesRef("captian usw"), generator, 2, 4, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("captain america")); assertThat(corrections[0].join(space, preTag, postTag).utf8ToString(), equalTo("captain america")); @@ -213,7 +219,7 @@ protected TokenStreamComponents createComponents(String fieldName) { // Make sure that user supplied text is not marked as highlighted in the presence of a synonym filter generator = new DirectCandidateGenerator(spellchecker, "body", SuggestMode.SUGGEST_MORE_POPULAR, ir, 0.85, 10, null, analyzer, MultiTerms.getTerms(ir, "body")); - corrections = suggester.getCorrections(analyzer, new BytesRef("captain usw"), generator, 2, 4, ir, + corrections = getCorrections(suggester, analyzer, new BytesRef("captain usw"), generator, 2, 4, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("captain america")); assertThat(corrections[0].join(space, preTag, postTag).utf8ToString(), equalTo("captain america")); @@ -282,7 +288,7 @@ protected TokenStreamComponents createComponents(String fieldName) { DirectoryReader ir = DirectoryReader.open(writer); LaplaceScorer wordScorer = new LaplaceScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.95d, new BytesRef(" "), 0.5f); - NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(); + NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(REAL_WORD_LIKELIHOOD, true, DEFAULT_TOKEN_LIMIT); DirectSpellChecker spellchecker = new DirectSpellChecker(); spellchecker.setMinQueryLength(1); DirectCandidateGenerator forward = new DirectCandidateGenerator(spellchecker, "body", SuggestMode.SUGGEST_ALWAYS, ir, @@ -291,27 +297,27 @@ protected TokenStreamComponents createComponents(String fieldName) { 0.95, 10, wrapper, wrapper, MultiTerms.getTerms(ir, "body_reverse")); CandidateGenerator generator = new MultiCandidateGeneratorWrapper(10, forward, reverse); - Correction[] corrections = suggester.getCorrections(wrapper, new BytesRef("american cae"), generator, 1, 1, + Correction[] corrections = getCorrections(suggester, wrapper, new BytesRef("american cae"), generator, 1, 1, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("american ace")); generator = new MultiCandidateGeneratorWrapper(5, forward, reverse); - corrections = suggester.getCorrections(wrapper, new BytesRef("american ame"), generator, 1, 1, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("american ame"), generator, 1, 1, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("american ace")); - corrections = suggester.getCorrections(wrapper, new BytesRef("american cae"), forward, 1, 1, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("american cae"), forward, 1, 1, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(0)); // only use forward with constant prefix - corrections = suggester.getCorrections(wrapper, new BytesRef("america cae"), generator, 2, 1, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("america cae"), generator, 2, 1, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("american ace")); - corrections = suggester.getCorrections(wrapper, new BytesRef("Zorr the Got-Jewel"), generator, 0.5f, 4, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("Zorr the Got-Jewel"), generator, 0.5f, 4, ir, "body", wordScorer, 0, 2).corrections; assertThat(corrections.length, equalTo(4)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); @@ -319,18 +325,18 @@ protected TokenStreamComponents createComponents(String fieldName) { assertThat(corrections[2].join(new BytesRef(" ")).utf8ToString(), equalTo("four the god jewel")); - corrections = suggester.getCorrections(wrapper, new BytesRef("Zorr the Got-Jewel"), generator, 0.5f, 1, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("Zorr the Got-Jewel"), generator, 0.5f, 1, ir, "body", wordScorer, 1.5f, 2).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 1, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 1, ir, "body", wordScorer, 1.5f, 2).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); // Test a special case where one of the suggest term is unchanged by the postFilter, 'II' here is unchanged by the reverse analyzer. - corrections = suggester.getCorrections(wrapper, new BytesRef("Quazar II"), generator, 1, 1, ir, + corrections = getCorrections(suggester, wrapper, new BytesRef("Quazar II"), generator, 1, 1, ir, "body", wordScorer, 1, 2).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("quasar ii")); @@ -391,24 +397,24 @@ protected TokenStreamComponents createComponents(String fieldName) { WordScorer wordScorer = new LinearInterpolatingScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.85d, new BytesRef(" "), 0.5, 0.4, 0.1); - NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(); + NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(REAL_WORD_LIKELIHOOD, true, DEFAULT_TOKEN_LIMIT); DirectSpellChecker spellchecker = new DirectSpellChecker(); spellchecker.setMinQueryLength(1); DirectCandidateGenerator generator = new DirectCandidateGenerator(spellchecker, "body", SuggestMode.SUGGEST_MORE_POPULAR, ir, 0.95, 5); - Correction[] corrections = suggester.getCorrections(wrapper, new BytesRef("american ame"), generator, 1, 1, + Correction[] corrections = getCorrections(suggester, wrapper, new BytesRef("american ame"), generator, 1, 1, ir, "body", wordScorer, 1, 3).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("american ace")); - corrections = suggester.getCorrections(wrapper, new BytesRef("american ame"), generator, 1, 1, + corrections = getCorrections(suggester, wrapper, new BytesRef("american ame"), generator, 1, 1, ir, "body", wordScorer, 1, 1).corrections; assertThat(corrections.length, equalTo(0)); // assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("american ape")); wordScorer = new LinearInterpolatingScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.85d, new BytesRef(" "), 0.5, 0.4, 0.1); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, ir, "body", wordScorer, 0, 3).corrections; assertThat(corrections.length, equalTo(4)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); @@ -419,7 +425,7 @@ protected TokenStreamComponents createComponents(String fieldName) { - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 4, ir, "body", wordScorer, 1, 3).corrections; assertThat(corrections.length, equalTo(4)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); @@ -428,7 +434,7 @@ protected TokenStreamComponents createComponents(String fieldName) { assertThat(corrections[3].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the got jewel")); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 1, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 1, ir, "body", wordScorer, 100, 3).corrections; assertThat(corrections.length, equalTo(1)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); @@ -456,23 +462,23 @@ protected TokenStreamComponents createComponents(String fieldName) { spellchecker.setAccuracy(0.0f); spellchecker.setMinPrefix(1); spellchecker.setMinQueryLength(1); - suggester = new NoisyChannelSpellChecker(0.95); + suggester = new NoisyChannelSpellChecker(0.95, true, DEFAULT_TOKEN_LIMIT); wordScorer = new LinearInterpolatingScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.95d, new BytesRef(" "), 0.5, 0.4, 0.1); - corrections = suggester.getCorrections(analyzer, new BytesRef("captian usa"), generator, 2, 4, + corrections = getCorrections(suggester, analyzer, new BytesRef("captian usa"), generator, 2, 4, ir, "body", wordScorer, 1, 3).corrections; assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("captain america")); generator = new DirectCandidateGenerator(spellchecker, "body", SuggestMode.SUGGEST_MORE_POPULAR, ir, 0.95, 10, null, analyzer, MultiTerms.getTerms(ir, "body")); - corrections = suggester.getCorrections(analyzer, new BytesRef("captian usw"), generator, 2, 4, + corrections = getCorrections(suggester, analyzer, new BytesRef("captian usw"), generator, 2, 4, ir, "body", wordScorer, 1, 3).corrections; assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("captain america")); wordScorer = new StupidBackoffScorer(ir, MultiTerms.getTerms(ir, "body_ngram"), "body_ngram", 0.85d, new BytesRef(" "), 0.4); - corrections = suggester.getCorrections(wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 2, + corrections = getCorrections(suggester, wrapper, new BytesRef("Xor the Got-Jewel"), generator, 0.5f, 2, ir, "body", wordScorer, 0, 3).corrections; assertThat(corrections.length, equalTo(2)); assertThat(corrections[0].join(new BytesRef(" ")).utf8ToString(), equalTo("xorr the god jewel")); @@ -494,11 +500,11 @@ public void testFewDocsEgdeCase() throws Exception { try (DirectoryReader ir = DirectoryReader.open(dir)) { WordScorer wordScorer = new StupidBackoffScorer(ir, MultiTerms.getTerms(ir, "field"), "field", 0.95d, new BytesRef(" "), 0.4f); - NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(); + NoisyChannelSpellChecker suggester = new NoisyChannelSpellChecker(REAL_WORD_LIKELIHOOD, true, DEFAULT_TOKEN_LIMIT); DirectSpellChecker spellchecker = new DirectSpellChecker(); DirectCandidateGenerator generator = new DirectCandidateGenerator(spellchecker, "field", SuggestMode.SUGGEST_MORE_POPULAR, ir, 0.95, 5); - Result result = suggester.getCorrections(new StandardAnalyzer(), new BytesRef("valeu"), generator, 1, 1, + Result result = getCorrections(suggester, new StandardAnalyzer(), new BytesRef("valeu"), generator, 1, 1, ir, "field", wordScorer, 1, 2); assertThat(result.corrections.length, equalTo(1)); assertThat(result.corrections[0].join(space).utf8ToString(), equalTo("value")); @@ -506,4 +512,13 @@ public void testFewDocsEgdeCase() throws Exception { } } + private Result getCorrections(NoisyChannelSpellChecker checker, Analyzer analyzer, BytesRef query, CandidateGenerator generator, + float maxErrors, int numCorrections, IndexReader reader, String analysisField, WordScorer scorer, float confidence, + int gramSize) throws IOException { + CharsRefBuilder spare = new CharsRefBuilder(); + spare.copyUTF8Bytes(query); + TokenStream tokenStream = analyzer.tokenStream(analysisField, new CharArrayReader(spare.chars(), 0, spare.length())); + return checker.getCorrections(tokenStream, generator, maxErrors, numCorrections, scorer, confidence, gramSize); + } + } From fd5ef6bf44c4beb80393fdf59feef2a71f2139c9 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 17 Apr 2019 10:29:02 +0200 Subject: [PATCH 070/260] Add Repository Consistency Assertion to SnapshotResiliencyTests (#40857) * Add Repository Consistency Assertion to SnapshotResiliencyTests * Add some quick validation on not leaving behind any dangling metadata or dangling indices to the snapshot resiliency tests * Added todo about expanding this assertion further --- .../blobstore/BlobStoreRepository.java | 4 +- .../snapshots/SnapshotResiliencyTests.java | 77 ++++++++++++++++++- 2 files changed, 77 insertions(+), 4 deletions(-) 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 9351c5bf84e87..cb674d9c521be 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -130,9 +130,9 @@ * |- index-N - list of all snapshot ids and the indices belonging to each snapshot, N is the generation of the file * |- index.latest - contains the numeric value of the latest generation of the index file (i.e. N from above) * |- incompatible-snapshots - list of all snapshot ids that are no longer compatible with the current version of the cluster - * |- snap-20131010 - JSON serialized Snapshot for snapshot "20131010" + * |- snap-20131010.dat - JSON serialized Snapshot for snapshot "20131010" * |- meta-20131010.dat - JSON serialized MetaData for snapshot "20131010" (includes only global metadata) - * |- snap-20131011 - JSON serialized Snapshot for snapshot "20131011" + * |- snap-20131011.dat - JSON serialized Snapshot for snapshot "20131011" * |- meta-20131011.dat - JSON serialized MetaData for snapshot "20131011" * ..... * |- indices/ - data for all indices diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 208dabff531b2..71c44d1e702b9 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -105,6 +105,8 @@ import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.ClusterSettings; @@ -115,7 +117,11 @@ import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.TestEnvironment; @@ -140,8 +146,10 @@ import org.elasticsearch.ingest.IngestService; import org.elasticsearch.node.ResponseCollectorService; import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchService; @@ -160,6 +168,8 @@ import org.junit.Before; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; import java.nio.file.Path; import java.util.Collection; import java.util.Collections; @@ -206,8 +216,12 @@ public void createServices() { } @After - public void stopServices() { - testClusterNodes.nodes.values().forEach(TestClusterNode::stop); + public void verifyReposThenStopServices() throws IOException { + try { + assertNoStaleRepositoryData(); + } finally { + testClusterNodes.nodes.values().forEach(TestClusterNode::stop); + } } public void testSuccessfulSnapshotAndRestore() { @@ -502,6 +516,65 @@ public void run() { assertThat(snapshotIds, either(hasSize(1)).or(hasSize(0))); } + /** + * Assert that there are no unreferenced indices or unreferenced root-level metadata blobs in any repository. + * TODO: Expand the logic here to also check for unreferenced segment blobs and shard level metadata + */ + private void assertNoStaleRepositoryData() throws IOException { + final Path repoPath = tempDir.resolve("repo").toAbsolutePath(); + final List repos; + try (Stream reposDir = Files.list(repoPath)) { + repos = reposDir.filter(s -> s.getFileName().toString().startsWith("extra") == false).collect(Collectors.toList()); + } + for (Path repoRoot : repos) { + final Path latestIndexGenBlob = repoRoot.resolve("index.latest"); + assertTrue("Could not find index.latest blob for repo at [" + repoRoot + ']', Files.exists(latestIndexGenBlob)); + final long latestGen = ByteBuffer.wrap(Files.readAllBytes(latestIndexGenBlob)).getLong(0); + assertIndexGenerations(repoRoot, latestGen); + final RepositoryData repositoryData; + try (XContentParser parser = + XContentHelper.createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, + new BytesArray(Files.readAllBytes(repoRoot.resolve("index-" + latestGen))), XContentType.JSON)) { + repositoryData = RepositoryData.snapshotsFromXContent(parser, latestGen); + } + assertIndexUUIDs(repoRoot, repositoryData); + assertSnapshotUUIDs(repoRoot, repositoryData); + } + } + + private static void assertIndexGenerations(Path repoRoot, long latestGen) throws IOException { + try (Stream repoRootBlobs = Files.list(repoRoot)) { + final long[] indexGenerations = repoRootBlobs.filter(p -> p.getFileName().toString().startsWith("index-")) + .map(p -> p.getFileName().toString().replace("index-", "")) + .mapToLong(Long::parseLong).sorted().toArray(); + assertEquals(latestGen, indexGenerations[indexGenerations.length - 1]); + assertTrue(indexGenerations.length <= 2); + } + } + + private static void assertIndexUUIDs(Path repoRoot, RepositoryData repositoryData) throws IOException { + final List expectedIndexUUIDs = + repositoryData.getIndices().values().stream().map(IndexId::getId).collect(Collectors.toList()); + try (Stream indexRoots = Files.list(repoRoot.resolve("indices"))) { + final List foundIndexUUIDs = indexRoots.filter(s -> s.getFileName().toString().startsWith("extra") == false) + .map(p -> p.getFileName().toString()).collect(Collectors.toList()); + assertThat(foundIndexUUIDs, containsInAnyOrder(expectedIndexUUIDs.toArray(Strings.EMPTY_ARRAY))); + } + } + + private static void assertSnapshotUUIDs(Path repoRoot, RepositoryData repositoryData) throws IOException { + final List expectedSnapshotUUIDs = + repositoryData.getSnapshotIds().stream().map(SnapshotId::getUUID).collect(Collectors.toList()); + for (String prefix : new String[]{"snap-", "meta-"}) { + try (Stream repoRootBlobs = Files.list(repoRoot)) { + final Collection foundSnapshotUUIDs = repoRootBlobs.filter(p -> p.getFileName().toString().startsWith(prefix)) + .map(p -> p.getFileName().toString().replace(prefix, "").replace(".dat", "")) + .collect(Collectors.toSet()); + assertThat(foundSnapshotUUIDs, containsInAnyOrder(expectedSnapshotUUIDs.toArray(Strings.EMPTY_ARRAY))); + } + } + } + private void clearDisruptionsAndAwaitSync() { testClusterNodes.clearNetworkDisruptions(); runUntil(() -> { From 16f9df79c8aabbb394a86e08690c25f68c093121 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 17 Apr 2019 06:59:29 -0400 Subject: [PATCH 071/260] Always check for archiving broken index settings (#41209) Today we check if an index has broken settings when checking if an index needs to be upgraded. However, it can be the case that an index setting became broken even if an index is already upgraded to the current version if the user removed a plugin (or downgraded from the default distribution to the non-default distribution) while on the same version of Elasticsearch. In this case, some registered settings would go missing and the index would now be broken. Yet, we miss this check and instead of archiving the settings, the index becomes unassigned due to the missing settings. This commit addresses this by checking for broken settings whether or not the index is upgraded. --- .../metadata/MetaDataIndexUpgradeService.java | 6 +- .../admin/indices/create/CreateIndexIT.java | 69 ------------------- .../MetaDataIndexUpgradeServiceTests.java | 40 +++++++---- 3 files changed, 33 insertions(+), 82 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index 6bc9104000fed..d3520da670289 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -88,7 +88,11 @@ public MetaDataIndexUpgradeService(Settings settings, NamedXContentRegistry xCon public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData, Version minimumIndexCompatibilityVersion) { // Throws an exception if there are too-old segments: if (isUpgraded(indexMetaData)) { - return indexMetaData; + /* + * We still need to check for broken index settings since it might be that a user removed a plugin that registers a setting + * needed by this index. + */ + return archiveBrokenIndexSettings(indexMetaData); } checkSupportedVersion(indexMetaData, minimumIndexCompatibilityVersion); IndexMetaData newMetaData = indexMetaData; diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java b/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java index f23dbaa8ea413..27e3ffefd6351 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java @@ -19,8 +19,6 @@ package org.elasticsearch.action.admin.indices.create; -import com.carrotsearch.hppc.cursors.ObjectCursor; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; @@ -33,28 +31,18 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.gateway.MetaStateService; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.query.RangeQueryBuilder; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; -import org.elasticsearch.test.InternalTestCluster; import java.util.HashMap; -import java.util.HashSet; -import java.util.Set; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; @@ -63,12 +51,8 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBlocked; import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasToString; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.core.IsNull.notNullValue; @@ -396,57 +380,4 @@ public void testIndexNameInResponse() { assertEquals("Should have index name in response", "foo", response.index()); } - public void testIndexWithUnknownSetting() throws Exception { - final int replicas = internalCluster().numDataNodes() - 1; - final Settings settings = Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", replicas).build(); - client().admin().indices().prepareCreate("test").setSettings(settings).get(); - ensureGreen("test"); - final ClusterState state = client().admin().cluster().prepareState().get().getState(); - - final Set dataOrMasterNodeNames = new HashSet<>(); - for (final ObjectCursor node : state.nodes().getMasterAndDataNodes().values()) { - assertTrue(dataOrMasterNodeNames.add(node.value.getName())); - } - - final IndexMetaData metaData = state.getMetaData().index("test"); - internalCluster().fullRestart(new InternalTestCluster.RestartCallback() { - @Override - public Settings onNodeStopped(String nodeName) throws Exception { - if (dataOrMasterNodeNames.contains(nodeName)) { - final MetaStateService metaStateService = internalCluster().getInstance(MetaStateService.class, nodeName); - final IndexMetaData brokenMetaData = - IndexMetaData - .builder(metaData) - .settings(Settings.builder().put(metaData.getSettings()).put("index.foo", true)) - .build(); - // so evil - metaStateService.writeIndexAndUpdateManifest("broken metadata", brokenMetaData); - } - return super.onNodeStopped(nodeName); - } - }); - - // check that the cluster does not keep reallocating shards - assertBusy(() -> { - final RoutingTable routingTable = client().admin().cluster().prepareState().get().getState().routingTable(); - final IndexRoutingTable indexRoutingTable = routingTable.index("test"); - assertNotNull(indexRoutingTable); - for (IndexShardRoutingTable shardRoutingTable : indexRoutingTable) { - assertTrue(shardRoutingTable.primaryShard().unassigned()); - assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_NO, - shardRoutingTable.primaryShard().unassignedInfo().getLastAllocationStatus()); - assertThat(shardRoutingTable.primaryShard().unassignedInfo().getNumFailedAllocations(), greaterThan(0)); - } - }, 60, TimeUnit.SECONDS); - client().admin().indices().prepareClose("test").get(); - - // try to open the index - final ElasticsearchException e = - expectThrows(ElasticsearchException.class, () -> client().admin().indices().prepareOpen("test").get()); - assertThat(e, hasToString(containsString("Failed to verify index " + metaData.getIndex()))); - assertNotNull(e.getCause()); - assertThat(e.getCause(), instanceOf(IllegalArgumentException.class)); - assertThat(e, hasToString(containsString("unknown setting [index.foo]"))); - } - } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java index 827680cca1b71..4c1ba0ff77e34 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeServiceTests.java @@ -28,12 +28,12 @@ import java.util.Collections; +import static org.hamcrest.Matchers.equalTo; + public class MetaDataIndexUpgradeServiceTests extends ESTestCase { public void testArchiveBrokenIndexSettings() { - MetaDataIndexUpgradeService service = new MetaDataIndexUpgradeService(Settings.EMPTY, xContentRegistry(), - new MapperRegistry(Collections.emptyMap(), Collections.emptyMap(), MapperPlugin.NOOP_FIELD_FILTER), - IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, Collections.emptyList()); + MetaDataIndexUpgradeService service = getMetaDataIndexUpgradeService(); IndexMetaData src = newIndexMeta("foo", Settings.EMPTY); IndexMetaData indexMetaData = service.archiveBrokenIndexSettings(src); assertSame(indexMetaData, src); @@ -58,10 +58,20 @@ public void testArchiveBrokenIndexSettings() { assertSame(indexMetaData, src); } + public void testAlreadyUpgradedIndexArchivesBrokenIndexSettings() { + final MetaDataIndexUpgradeService service = getMetaDataIndexUpgradeService(); + final IndexMetaData initial = newIndexMeta( + "foo", + Settings.builder().put(IndexMetaData.SETTING_VERSION_UPGRADED, Version.CURRENT).put("index.refresh_interval", "-200").build()); + assertTrue(service.isUpgraded(initial)); + final IndexMetaData after = service.upgradeIndexMetaData(initial, Version.CURRENT.minimumIndexCompatibilityVersion()); + // the index does not need to be upgraded, but checking that it does should archive any broken settings + assertThat(after.getSettings().get("archived.index.refresh_interval"), equalTo("-200")); + assertNull(after.getSettings().get("index.refresh_interval")); + } + public void testUpgrade() { - MetaDataIndexUpgradeService service = new MetaDataIndexUpgradeService(Settings.EMPTY, xContentRegistry(), - new MapperRegistry(Collections.emptyMap(), Collections.emptyMap(), MapperPlugin.NOOP_FIELD_FILTER), - IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, Collections.emptyList()); + MetaDataIndexUpgradeService service = getMetaDataIndexUpgradeService(); IndexMetaData src = newIndexMeta("foo", Settings.builder().put("index.refresh_interval", "-200").build()); assertFalse(service.isUpgraded(src)); src = service.upgradeIndexMetaData(src, Version.CURRENT.minimumIndexCompatibilityVersion()); @@ -72,9 +82,7 @@ public void testUpgrade() { } public void testIsUpgraded() { - MetaDataIndexUpgradeService service = new MetaDataIndexUpgradeService(Settings.EMPTY, xContentRegistry(), - new MapperRegistry(Collections.emptyMap(), Collections.emptyMap(), MapperPlugin.NOOP_FIELD_FILTER), - IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, Collections.emptyList()); + MetaDataIndexUpgradeService service = getMetaDataIndexUpgradeService(); IndexMetaData src = newIndexMeta("foo", Settings.builder().put("index.refresh_interval", "-200").build()); assertFalse(service.isUpgraded(src)); Version version = VersionUtils.randomVersionBetween(random(), VersionUtils.getFirstVersion(), VersionUtils.getPreviousVersion()); @@ -85,9 +93,7 @@ public void testIsUpgraded() { } public void testFailUpgrade() { - MetaDataIndexUpgradeService service = new MetaDataIndexUpgradeService(Settings.EMPTY, xContentRegistry(), - new MapperRegistry(Collections.emptyMap(), Collections.emptyMap(), MapperPlugin.NOOP_FIELD_FILTER), - IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, Collections.emptyList()); + MetaDataIndexUpgradeService service = getMetaDataIndexUpgradeService(); Version minCompat = Version.CURRENT.minimumIndexCompatibilityVersion(); Version indexUpgraded = VersionUtils.randomVersionBetween(random(), minCompat, @@ -144,6 +150,15 @@ public void testPluginUpgradeFailure() { assertEquals(message, "Cannot upgrade index foo"); } + private MetaDataIndexUpgradeService getMetaDataIndexUpgradeService() { + return new MetaDataIndexUpgradeService( + Settings.EMPTY, + xContentRegistry(), + new MapperRegistry(Collections.emptyMap(), Collections.emptyMap(), MapperPlugin.NOOP_FIELD_FILTER), + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + Collections.emptyList()); + } + public static IndexMetaData newIndexMeta(String name, Settings indexSettings) { Settings build = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) @@ -155,4 +170,5 @@ public static IndexMetaData newIndexMeta(String name, Settings indexSettings) { .build(); return IndexMetaData.builder(name).settings(build).build(); } + } From bdc0c9ca4739833a46681bc9c404560303391c9a Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 17 Apr 2019 13:48:36 +0200 Subject: [PATCH 072/260] ProfileScorer should propagate `setMinCompetitiveScore`. (#40958) Currently enabling profiling disables top-hits optimizations, which is unfortunate: it would be nice to be able to notice the difference in method counts and timings depending on whether total hit counts are requested. --- docs/reference/search/profile.asciidoc | 24 +++-- .../search/profile/query/ProfileScorer.java | 14 ++- .../search/profile/query/QueryTimingType.java | 3 +- .../profile/query/ProfileScorerTests.java | 92 +++++++++++++++++++ 4 files changed, 125 insertions(+), 8 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/search/profile/query/ProfileScorerTests.java diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 6ac5c6bcaf044..9a6779defab39 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -82,7 +82,9 @@ This will yield the following result: "compute_max_score": 0, "compute_max_score_count": 0, "shallow_advance": 0, - "shallow_advance_count": 0 + "shallow_advance_count": 0, + "set_min_competitive_score": 0, + "set_min_competitive_score_count": 0 }, "children": [ { @@ -105,7 +107,9 @@ This will yield the following result: "compute_max_score": 0, "compute_max_score_count": 0, "shallow_advance": 0, - "shallow_advance_count": 0 + "shallow_advance_count": 0, + "set_min_competitive_score": 0, + "set_min_competitive_score_count": 0 } }, { @@ -128,7 +132,9 @@ This will yield the following result: "compute_max_score": 0, "compute_max_score_count": 0, "shallow_advance": 0, - "shallow_advance_count": 0 + "shallow_advance_count": 0, + "set_min_competitive_score": 0, + "set_min_competitive_score_count": 0 } } ] @@ -311,7 +317,9 @@ The `breakdown` component lists detailed timing statistics about low-level Lucen "compute_max_score": 0, "compute_max_score_count": 0, "shallow_advance": 0, - "shallow_advance_count": 0 + "shallow_advance_count": 0, + "set_min_competitive_score": 0, + "set_min_competitive_score_count": 0 } -------------------------------------------------- // TESTRESPONSE[s/^/{\n"took": $body.took,\n"timed_out": $body.timed_out,\n"_shards": $body._shards,\n"hits": $body.hits,\n"profile": {\n"shards": [ {\n"id": "$body.$_path",\n"searches": [{\n"query": [{\n"type": "BooleanQuery",\n"description": "message:some message:number",\n"time_in_nanos": $body.$_path,/] @@ -575,7 +583,9 @@ And the response: "compute_max_score": 0, "compute_max_score_count": 0, "shallow_advance": 0, - "shallow_advance_count": 0 + "shallow_advance_count": 0, + "set_min_competitive_score": 0, + "set_min_competitive_score_count": 0 } }, { @@ -598,7 +608,9 @@ And the response: "compute_max_score": 0, "compute_max_score_count": 0, "shallow_advance": 0, - "shallow_advance_count": 0 + "shallow_advance_count": 0, + "set_min_competitive_score": 0, + "set_min_competitive_score_count": 0 } } ], diff --git a/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java b/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java index b2354667889b6..e5155fc2c63ca 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java +++ b/server/src/main/java/org/elasticsearch/search/profile/query/ProfileScorer.java @@ -37,7 +37,8 @@ final class ProfileScorer extends Scorer { private final Scorer scorer; private ProfileWeight profileWeight; - private final Timer scoreTimer, nextDocTimer, advanceTimer, matchTimer, shallowAdvanceTimer, computeMaxScoreTimer; + private final Timer scoreTimer, nextDocTimer, advanceTimer, matchTimer, shallowAdvanceTimer, computeMaxScoreTimer, + setMinCompetitiveScoreTimer; ProfileScorer(ProfileWeight w, Scorer scorer, QueryProfileBreakdown profile) throws IOException { super(w); @@ -49,6 +50,7 @@ final class ProfileScorer extends Scorer { matchTimer = profile.getTimer(QueryTimingType.MATCH); shallowAdvanceTimer = profile.getTimer(QueryTimingType.SHALLOW_ADVANCE); computeMaxScoreTimer = profile.getTimer(QueryTimingType.COMPUTE_MAX_SCORE); + setMinCompetitiveScoreTimer = profile.getTimer(QueryTimingType.SET_MIN_COMPETITIVE_SCORE); } @Override @@ -189,4 +191,14 @@ public float getMaxScore(int upTo) throws IOException { computeMaxScoreTimer.stop(); } } + + @Override + public void setMinCompetitiveScore(float minScore) throws IOException { + setMinCompetitiveScoreTimer.start(); + try { + scorer.setMinCompetitiveScore(minScore); + } finally { + setMinCompetitiveScoreTimer.stop(); + } + } } diff --git a/server/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java b/server/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java index 146bd8f07bcd1..aecc41d8a23b5 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java +++ b/server/src/main/java/org/elasticsearch/search/profile/query/QueryTimingType.java @@ -29,7 +29,8 @@ public enum QueryTimingType { MATCH, SCORE, SHALLOW_ADVANCE, - COMPUTE_MAX_SCORE; + COMPUTE_MAX_SCORE, + SET_MIN_COMPETITIVE_SCORE; @Override public String toString() { diff --git a/server/src/test/java/org/elasticsearch/search/profile/query/ProfileScorerTests.java b/server/src/test/java/org/elasticsearch/search/profile/query/ProfileScorerTests.java new file mode 100644 index 0000000000000..fd72bdfa6de2d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/profile/query/ProfileScorerTests.java @@ -0,0 +1,92 @@ +/* + * 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.profile.query; + +import org.apache.lucene.index.MultiReader; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +public class ProfileScorerTests extends ESTestCase { + + private static class FakeScorer extends Scorer { + + public float maxScore, minCompetitiveScore; + + protected FakeScorer(Weight weight) { + super(weight); + } + + @Override + public DocIdSetIterator iterator() { + throw new UnsupportedOperationException(); + } + + @Override + public float getMaxScore(int upTo) throws IOException { + return maxScore; + } + + @Override + public float score() throws IOException { + return 1f; + } + + @Override + public int docID() { + throw new UnsupportedOperationException(); + } + + @Override + public void setMinCompetitiveScore(float minScore) { + this.minCompetitiveScore = minScore; + } + } + + public void testPropagateMinCompetitiveScore() throws IOException { + Query query = new MatchAllDocsQuery(); + Weight weight = query.createWeight(new IndexSearcher(new MultiReader()), ScoreMode.TOP_SCORES, 1f); + FakeScorer fakeScorer = new FakeScorer(weight); + QueryProfileBreakdown profile = new QueryProfileBreakdown(); + ProfileWeight profileWeight = new ProfileWeight(query, weight, profile); + ProfileScorer profileScorer = new ProfileScorer(profileWeight, fakeScorer, profile); + profileScorer.setMinCompetitiveScore(0.42f); + assertEquals(0.42f, fakeScorer.minCompetitiveScore, 0f); + } + + public void testPropagateMaxScore() throws IOException { + Query query = new MatchAllDocsQuery(); + Weight weight = query.createWeight(new IndexSearcher(new MultiReader()), ScoreMode.TOP_SCORES, 1f); + FakeScorer fakeScorer = new FakeScorer(weight); + QueryProfileBreakdown profile = new QueryProfileBreakdown(); + ProfileWeight profileWeight = new ProfileWeight(query, weight, profile); + ProfileScorer profileScorer = new ProfileScorer(profileWeight, fakeScorer, profile); + profileScorer.setMinCompetitiveScore(0.42f); + fakeScorer.maxScore = 42f; + assertEquals(42f, profileScorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0f); + } +} From d706b40c638b3e29357617982e92c7b68391afa8 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 17 Apr 2019 13:49:04 +0200 Subject: [PATCH 073/260] Clean up Node#close. (#39317) `Node#close` is pretty hard to rely on today: - it might swallow exceptions - it waits for 10 seconds for threads to terminate but doesn't signal anything if threads are still not terminated after 10 seconds This commit makes `IOException`s propagated and splits `Node#close` into `Node#close` and `Node#awaitClose` so that the decision what to do if a node takes too long to close can be done on top of `Node#close`. It also adds synchronization to lifecycle transitions to make them atomic. I don't think it is a source of problems today, but it makes things easier to reason about. --- .../elasticsearch/bootstrap/Bootstrap.java | 14 +++ .../component/AbstractLifecycleComponent.java | 88 ++++++++--------- .../common/component/Lifecycle.java | 25 +++-- .../common/util/concurrent/ThreadContext.java | 13 ++- .../elasticsearch/indices/IndicesService.java | 16 ++++ .../java/org/elasticsearch/node/Node.java | 51 ++++++---- .../org/elasticsearch/node/NodeService.java | 9 ++ .../elasticsearch/transport/TcpTransport.java | 1 + .../transport/TransportKeepAlive.java | 6 +- .../indices/IndicesServiceCloseTests.java | 8 ++ .../org/elasticsearch/node/NodeTests.java | 94 +++++++++++++++++++ .../search/SearchServiceTests.java | 23 ++--- .../test/ESSingleNodeTestCase.java | 8 +- .../test/InternalTestCluster.java | 7 ++ 14 files changed, 278 insertions(+), 85 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java b/server/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java index e615dcf8aced0..a4a2696fdad73 100644 --- a/server/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java +++ b/server/src/main/java/org/elasticsearch/bootstrap/Bootstrap.java @@ -59,6 +59,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; /** * Internal startup code. @@ -183,8 +184,15 @@ public void run() { IOUtils.close(node, spawner); LoggerContext context = (LoggerContext) LogManager.getContext(false); Configurator.shutdown(context); + if (node != null && node.awaitClose(10, TimeUnit.SECONDS) == false) { + throw new IllegalStateException("Node didn't stop within 10 seconds. " + + "Any outstanding requests or tasks might get killed."); + } } catch (IOException ex) { throw new ElasticsearchException("failed to stop node", ex); + } catch (InterruptedException e) { + LogManager.getLogger(Bootstrap.class).warn("Thread got interrupted while waiting for the node to shutdown."); + Thread.currentThread().interrupt(); } } }); @@ -267,6 +275,12 @@ private void start() throws NodeValidationException { static void stop() throws IOException { try { IOUtils.close(INSTANCE.node, INSTANCE.spawner); + if (INSTANCE.node != null && INSTANCE.node.awaitClose(10, TimeUnit.SECONDS) == false) { + throw new IllegalStateException("Node didn't stop within 10 seconds. Any outstanding requests or tasks might get killed."); + } + } catch (InterruptedException e) { + LogManager.getLogger(Bootstrap.class).warn("Thread got interrupted while waiting for the node to shutdown."); + Thread.currentThread().interrupt(); } finally { INSTANCE.keepAliveLatch.countDown(); } diff --git a/server/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java b/server/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java index 772d2d89cf5be..a7f72c63091ab 100644 --- a/server/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java +++ b/server/src/main/java/org/elasticsearch/common/component/AbstractLifecycleComponent.java @@ -19,15 +19,12 @@ package org.elasticsearch.common.component; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - import java.io.IOException; +import java.io.UncheckedIOException; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; public abstract class AbstractLifecycleComponent implements LifecycleComponent { - private static final Logger logger = LogManager.getLogger(AbstractLifecycleComponent.class); protected final Lifecycle lifecycle = new Lifecycle(); @@ -52,16 +49,18 @@ public void removeLifecycleListener(LifecycleListener listener) { @Override public void start() { - if (!lifecycle.canMoveToStarted()) { - return; - } - for (LifecycleListener listener : listeners) { - listener.beforeStart(); - } - doStart(); - lifecycle.moveToStarted(); - for (LifecycleListener listener : listeners) { - listener.afterStart(); + synchronized (lifecycle) { + if (!lifecycle.canMoveToStarted()) { + return; + } + for (LifecycleListener listener : listeners) { + listener.beforeStart(); + } + doStart(); + lifecycle.moveToStarted(); + for (LifecycleListener listener : listeners) { + listener.afterStart(); + } } } @@ -69,16 +68,18 @@ public void start() { @Override public void stop() { - if (!lifecycle.canMoveToStopped()) { - return; - } - for (LifecycleListener listener : listeners) { - listener.beforeStop(); - } - lifecycle.moveToStopped(); - doStop(); - for (LifecycleListener listener : listeners) { - listener.afterStop(); + synchronized (lifecycle) { + if (!lifecycle.canMoveToStopped()) { + return; + } + for (LifecycleListener listener : listeners) { + listener.beforeStop(); + } + lifecycle.moveToStopped(); + doStop(); + for (LifecycleListener listener : listeners) { + listener.afterStop(); + } } } @@ -86,25 +87,26 @@ public void stop() { @Override public void close() { - if (lifecycle.started()) { - stop(); - } - if (!lifecycle.canMoveToClosed()) { - return; - } - for (LifecycleListener listener : listeners) { - listener.beforeClose(); - } - lifecycle.moveToClosed(); - try { - doClose(); - } catch (IOException e) { - // TODO: we need to separate out closing (ie shutting down) services, vs releasing runtime transient - // structures. Shutting down services should use IOUtils.close - logger.warn("failed to close " + getClass().getName(), e); - } - for (LifecycleListener listener : listeners) { - listener.afterClose(); + synchronized (lifecycle) { + if (lifecycle.started()) { + stop(); + } + if (!lifecycle.canMoveToClosed()) { + return; + } + for (LifecycleListener listener : listeners) { + listener.beforeClose(); + } + lifecycle.moveToClosed(); + try { + doClose(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + for (LifecycleListener listener : listeners) { + listener.afterClose(); + } + } } } diff --git a/server/src/main/java/org/elasticsearch/common/component/Lifecycle.java b/server/src/main/java/org/elasticsearch/common/component/Lifecycle.java index e71c9b0389995..82042ab2b7dcf 100644 --- a/server/src/main/java/org/elasticsearch/common/component/Lifecycle.java +++ b/server/src/main/java/org/elasticsearch/common/component/Lifecycle.java @@ -39,15 +39,22 @@ * } * *

+ * NOTE: The Lifecycle class is thread-safe. It is also possible to prevent concurrent state transitions + * by locking on the Lifecycle object itself. This is typically useful when chaining multiple transitions. + *

* Note, closed is only allowed to be called when stopped, so make sure to stop the component first. - * Here is how the logic can be applied: + * Here is how the logic can be applied. A lock of the {@code lifecycleState} object is taken so that + * another thread cannot move the state from {@code STOPPED} to {@code STARTED} before it has moved to + * {@code CLOSED}. *

  * public void close() {
- *  if (lifecycleState.started()) {
- *      stop();
- *  }
- *  if (!lifecycleState.moveToClosed()) {
- *      return;
+ *  synchronized (lifecycleState) {
+ *      if (lifecycleState.started()) {
+ *          stop();
+ *      }
+ *      if (!lifecycleState.moveToClosed()) {
+ *          return;
+ *      }
  *  }
  *  // perform close logic here
  * }
@@ -116,7 +123,7 @@ public boolean canMoveToStarted() throws IllegalStateException {
     }
 
 
-    public boolean moveToStarted() throws IllegalStateException {
+    public synchronized boolean moveToStarted() throws IllegalStateException {
         State localState = this.state;
         if (localState == State.INITIALIZED || localState == State.STOPPED) {
             state = State.STARTED;
@@ -145,7 +152,7 @@ public boolean canMoveToStopped() throws IllegalStateException {
         throw new IllegalStateException("Can't move to stopped with unknown state");
     }
 
-    public boolean moveToStopped() throws IllegalStateException {
+    public synchronized boolean moveToStopped() throws IllegalStateException {
         State localState = state;
         if (localState == State.STARTED) {
             state = State.STOPPED;
@@ -171,7 +178,7 @@ public boolean canMoveToClosed() throws IllegalStateException {
         return true;
     }
 
-    public boolean moveToClosed() throws IllegalStateException {
+    public synchronized boolean moveToClosed() throws IllegalStateException {
         State localState = state;
         if (localState == State.CLOSED) {
             return false;
diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java
index 01e2384377728..80a9a30032e17 100644
--- a/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java
+++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java
@@ -131,7 +131,18 @@ public void close() {
     public StoredContext stashContext() {
         final ThreadContextStruct context = threadLocal.get();
         threadLocal.set(null);
-        return () -> threadLocal.set(context);
+        return () -> {
+            // If the node and thus the threadLocal get closed while this task
+            // is still executing, we don't want this runnable to fail with an
+            // uncaught exception
+            try {
+                threadLocal.set(context);
+            } catch (IllegalStateException e) {
+                if (isClosed() == false) {
+                    throw e;
+                }
+            }
+        };
     }
 
     /**
diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java
index 0eeca2ae2c9dd..d2f7f38762b19 100644
--- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java
+++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java
@@ -23,6 +23,7 @@
 import org.apache.logging.log4j.Logger;
 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;
@@ -200,6 +201,7 @@ public class IndicesService extends AbstractLifecycleComponent
     private final Collection>> engineFactoryProviders;
     private final Map> indexStoreFactories;
     final AbstractRefCounted indicesRefCount; // pkg-private for testing
+    private final CountDownLatch closeLatch = new CountDownLatch(1);
 
     @Override
     protected void doStart() {
@@ -273,6 +275,8 @@ protected void closeInternal() {
                             indicesQueryCache);
                 } catch (IOException e) {
                     throw new UncheckedIOException(e);
+                } finally {
+                    closeLatch.countDown();
                 }
             }
         };
@@ -311,6 +315,18 @@ protected void doClose() throws IOException {
         indicesRefCount.decRef();
     }
 
+    /**
+     * Wait for this {@link IndicesService} to be effectively closed. When this returns {@code true}, all shards and shard stores
+     * are closed and all shard {@link CacheHelper#addClosedListener(org.apache.lucene.index.IndexReader.ClosedListener) closed
+     * listeners} have run. However some {@link IndexEventListener#onStoreClosed(ShardId) shard closed listeners} might not have
+     * run.
+     * @returns true if all shards closed within the given timeout, false otherwise
+     * @throws InterruptedException if the current thread got interrupted while waiting for shards to close
+     */
+    public boolean awaitClose(long timeout, TimeUnit timeUnit) throws InterruptedException {
+        return closeLatch.await(timeout, timeUnit);
+    }
+
     /**
      * Returns the node stats indices stats. The {@code includePrevious} flag controls
      * if old shards stats will be aggregated as well (only for relevant stats, such as
diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java
index 1963d0c8b40c6..4e8b81aea2e7b 100644
--- a/server/src/main/java/org/elasticsearch/node/Node.java
+++ b/server/src/main/java/org/elasticsearch/node/Node.java
@@ -783,11 +783,13 @@ private Node stop() {
     // In this case the process will be terminated even if the first call to close() has not finished yet.
     @Override
     public synchronized void close() throws IOException {
-        if (lifecycle.started()) {
-            stop();
-        }
-        if (!lifecycle.moveToClosed()) {
-            return;
+        synchronized (lifecycle) {
+            if (lifecycle.started()) {
+                stop();
+            }
+            if (!lifecycle.moveToClosed()) {
+                return;
+            }
         }
 
         logger.info("closing ...");
@@ -835,21 +837,12 @@ public synchronized void close() throws IOException {
         toClose.add(injector.getInstance(ScriptService.class));
 
         toClose.add(() -> stopWatch.stop().start("thread_pool"));
-        // TODO this should really use ThreadPool.terminate()
         toClose.add(() -> injector.getInstance(ThreadPool.class).shutdown());
-        toClose.add(() -> {
-            try {
-                injector.getInstance(ThreadPool.class).awaitTermination(10, TimeUnit.SECONDS);
-            } catch (InterruptedException e) {
-                // ignore
-            }
-        });
-
-        toClose.add(() -> stopWatch.stop().start("thread_pool_force_shutdown"));
-        toClose.add(() -> injector.getInstance(ThreadPool.class).shutdownNow());
+        // Don't call shutdownNow here, it might break ongoing operations on Lucene indices.
+        // See https://issues.apache.org/jira/browse/LUCENE-7248. We call shutdownNow in
+        // awaitClose if the node doesn't finish closing within the specified time.
         toClose.add(() -> stopWatch.stop());
 
-
         toClose.add(injector.getInstance(NodeEnvironment.class));
         toClose.add(injector.getInstance(PageCacheRecycler.class));
 
@@ -860,6 +853,30 @@ public synchronized void close() throws IOException {
         logger.info("closed");
     }
 
+    /**
+     * Wait for this node to be effectively closed.
+     */
+    // synchronized to prevent running concurrently with close()
+    public synchronized boolean awaitClose(long timeout, TimeUnit timeUnit) throws InterruptedException {
+        if (lifecycle.closed() == false) {
+            // We don't want to shutdown the threadpool or interrupt threads on a node that is not
+            // closed yet.
+            throw new IllegalStateException("Call close() first");
+        }
+
+
+        ThreadPool threadPool = injector.getInstance(ThreadPool.class);
+        final boolean terminated = ThreadPool.terminate(threadPool, timeout, timeUnit);
+        if (terminated) {
+            // All threads terminated successfully. Because search, recovery and all other operations
+            // that run on shards run in the threadpool, indices should be effectively closed by now.
+            if (nodeService.awaitClose(0, TimeUnit.MILLISECONDS) == false) {
+                throw new IllegalStateException("Some shards are still open after the threadpool terminated. " +
+                        "Something is leaking index readers or store references.");
+            }
+        }
+        return terminated;
+    }
 
     /**
      * Returns {@code true} if the node is closed.
diff --git a/server/src/main/java/org/elasticsearch/node/NodeService.java b/server/src/main/java/org/elasticsearch/node/NodeService.java
index 0567641b8a5d6..3f71a21966c76 100644
--- a/server/src/main/java/org/elasticsearch/node/NodeService.java
+++ b/server/src/main/java/org/elasticsearch/node/NodeService.java
@@ -43,6 +43,7 @@
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 public class NodeService implements Closeable {
     private final Settings settings;
@@ -135,4 +136,12 @@ public void close() throws IOException {
         IOUtils.close(indicesService);
     }
 
+    /**
+     * Wait for the node to be effectively closed.
+     * @see IndicesService#awaitClose(long, TimeUnit)
+     */
+    public boolean awaitClose(long timeout, TimeUnit timeUnit) throws InterruptedException {
+        return indicesService.awaitClose(timeout, timeUnit);
+    }
+
 }
diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
index eb61af8d2a38c..42d613016352a 100644
--- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
+++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java
@@ -360,6 +360,7 @@ private InetSocketAddress bindToPort(final String name, final InetAddress hostAd
         final AtomicReference boundSocket = new AtomicReference<>();
         closeLock.writeLock().lock();
         try {
+            // No need for locking here since Lifecycle objects can't move from STARTED to INITIALIZED
             if (lifecycle.initialized() == false && lifecycle.started() == false) {
                 throw new IllegalStateException("transport has been stopped");
             }
diff --git a/server/src/main/java/org/elasticsearch/transport/TransportKeepAlive.java b/server/src/main/java/org/elasticsearch/transport/TransportKeepAlive.java
index 571ced1c118f9..fc7ebe4b9644e 100644
--- a/server/src/main/java/org/elasticsearch/transport/TransportKeepAlive.java
+++ b/server/src/main/java/org/elasticsearch/transport/TransportKeepAlive.java
@@ -136,8 +136,10 @@ public void onFailure(Exception e) {
 
     @Override
     public void close() {
-        lifecycle.moveToStopped();
-        lifecycle.moveToClosed();
+        synchronized (lifecycle) {
+            lifecycle.moveToStopped();
+            lifecycle.moveToClosed();
+        }
     }
 
     private class ScheduledPing extends AbstractLifecycleRunnable {
diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java
index 15b45330530d3..e22253be7fcfc 100644
--- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java
+++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java
@@ -49,6 +49,7 @@
 
 import java.nio.file.Path;
 import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
 import java.util.Collections;
 
 import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
@@ -95,8 +96,10 @@ public void testCloseEmptyIndicesService() throws Exception {
         Node node = startNode();
         IndicesService indicesService = node.injector().getInstance(IndicesService.class);
         assertEquals(1, indicesService.indicesRefCount.refCount());
+        assertFalse(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
         node.close();
         assertEquals(0, indicesService.indicesRefCount.refCount());
+        assertTrue(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
     }
 
     public void testCloseNonEmptyIndicesService() throws Exception {
@@ -108,9 +111,11 @@ public void testCloseNonEmptyIndicesService() throws Exception {
                 .setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)));
 
         assertEquals(2, indicesService.indicesRefCount.refCount());
+        assertFalse(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
 
         node.close();
         assertEquals(0, indicesService.indicesRefCount.refCount());
+        assertTrue(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
     }
 
     public void testCloseWithIncedRefStore() throws Exception {
@@ -126,12 +131,15 @@ public void testCloseWithIncedRefStore() throws Exception {
         IndexService indexService = indicesService.iterator().next();
         IndexShard shard = indexService.getShard(0);
         shard.store().incRef();
+        assertFalse(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
 
         node.close();
         assertEquals(1, indicesService.indicesRefCount.refCount());
+        assertFalse(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
 
         shard.store().decRef();
         assertEquals(0, indicesService.indicesRefCount.refCount());
+        assertTrue(indicesService.awaitClose(0, TimeUnit.MILLISECONDS));
     }
 
     public void testCloseWhileOngoingRequest() throws Exception {
diff --git a/server/src/test/java/org/elasticsearch/node/NodeTests.java b/server/src/test/java/org/elasticsearch/node/NodeTests.java
index 288817d5c7708..6f0419421b868 100644
--- a/server/src/test/java/org/elasticsearch/node/NodeTests.java
+++ b/server/src/test/java/org/elasticsearch/node/NodeTests.java
@@ -26,18 +26,30 @@
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.env.Environment;
+import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.engine.Engine.Searcher;
+import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.InternalTestCluster;
 import org.elasticsearch.test.MockHttpTransport;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.hamcrest.Matchers;
 
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
+import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+
 @LuceneTestCase.SuppressFileSystems(value = "ExtrasFS")
 public class NodeTests extends ESTestCase {
 
@@ -136,5 +148,87 @@ private static Settings.Builder baseSettings() {
                 .put(Node.NODE_DATA_SETTING.getKey(), true);
     }
 
+    public void testCloseOnOutstandingTask() throws Exception {
+        Node node = new MockNode(baseSettings().build(), basePlugins());
+        node.start();
+        ThreadPool threadpool = node.injector().getInstance(ThreadPool.class);
+        AtomicBoolean shouldRun = new AtomicBoolean(true);
+        threadpool.executor(ThreadPool.Names.SEARCH).execute(() -> {
+            while (shouldRun.get());
+        });
+        node.close();
+        shouldRun.set(false);
+        assertTrue(node.awaitClose(1, TimeUnit.DAYS));
+    }
+
+    public void testAwaitCloseTimeoutsOnNonInterruptibleTask() throws Exception {
+        Node node = new MockNode(baseSettings().build(), basePlugins());
+        node.start();
+        ThreadPool threadpool = node.injector().getInstance(ThreadPool.class);
+        AtomicBoolean shouldRun = new AtomicBoolean(true);
+        threadpool.executor(ThreadPool.Names.SEARCH).execute(() -> {
+            while (shouldRun.get());
+        });
+        node.close();
+        assertFalse(node.awaitClose(0, TimeUnit.MILLISECONDS));
+        shouldRun.set(false);
+    }
+
+    public void testCloseOnInterruptibleTask() throws Exception {
+        Node node = new MockNode(baseSettings().build(), basePlugins());
+        node.start();
+        ThreadPool threadpool = node.injector().getInstance(ThreadPool.class);
+        CountDownLatch latch = new CountDownLatch(1);
+        final CountDownLatch finishLatch = new CountDownLatch(1);
+        final AtomicBoolean interrupted = new AtomicBoolean(false);
+        threadpool.executor(ThreadPool.Names.SEARCH).execute(() -> {
+            try {
+                latch.await();
+            } catch (InterruptedException e) {
+                interrupted.set(true);
+                Thread.currentThread().interrupt();
+            } finally {
+                finishLatch.countDown();
+            }
+        });
+        node.close();
+        // close should not interrput ongoing tasks
+        assertFalse(interrupted.get());
+        // but awaitClose should
+        node.awaitClose(0, TimeUnit.SECONDS);
+        finishLatch.await();
+        assertTrue(interrupted.get());
+    }
+
+    public void testCloseOnLeakedIndexReaderReference() throws Exception {
+        Node node = new MockNode(baseSettings().build(), basePlugins());
+        node.start();
+        IndicesService indicesService = node.injector().getInstance(IndicesService.class);
+        assertAcked(node.client().admin().indices().prepareCreate("test")
+                .setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)));
+        IndexService indexService = indicesService.iterator().next();
+        IndexShard shard = indexService.getShard(0);
+        Searcher searcher = shard.acquireSearcher("test");
+        node.close();
+
+        IllegalStateException e = expectThrows(IllegalStateException.class, () -> node.awaitClose(1, TimeUnit.DAYS));
+        searcher.close();
+        assertThat(e.getMessage(), Matchers.containsString("Something is leaking index readers or store references"));
+    }
+
+    public void testCloseOnLeakedStoreReference() throws Exception {
+        Node node = new MockNode(baseSettings().build(), basePlugins());
+        node.start();
+        IndicesService indicesService = node.injector().getInstance(IndicesService.class);
+        assertAcked(node.client().admin().indices().prepareCreate("test")
+                .setSettings(Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)));
+        IndexService indexService = indicesService.iterator().next();
+        IndexShard shard = indexService.getShard(0);
+        shard.store().incRef();
+        node.close();
 
+        IllegalStateException e = expectThrows(IllegalStateException.class, () -> node.awaitClose(1, TimeUnit.DAYS));
+        shard.store().decRef();
+        assertThat(e.getMessage(), Matchers.containsString("Something is leaking index readers or store references"));
+    }
 }
diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java
index 641f1a1c19be4..90957c2779e8f 100644
--- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java
+++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java
@@ -652,16 +652,17 @@ public void testCreateSearchContext() throws IOException {
         searchRequest.allowPartialSearchResults(randomBoolean());
         ShardSearchTransportRequest request = new ShardSearchTransportRequest(OriginalIndices.NONE, searchRequest, shardId,
             indexService.numberOfShards(), AliasFilter.EMPTY, 1f, nowInMillis, clusterAlias, Strings.EMPTY_ARRAY);
-        DefaultSearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()));
-        SearchShardTarget searchShardTarget = searchContext.shardTarget();
-        QueryShardContext queryShardContext = searchContext.getQueryShardContext();
-        String expectedIndexName = clusterAlias == null ? index : clusterAlias + ":" + index;
-        assertEquals(expectedIndexName, queryShardContext.getFullyQualifiedIndex().getName());
-        assertEquals(expectedIndexName, searchShardTarget.getFullyQualifiedIndexName());
-        assertEquals(clusterAlias, searchShardTarget.getClusterAlias());
-        assertEquals(shardId, searchShardTarget.getShardId());
-        assertSame(searchShardTarget, searchContext.dfsResult().getSearchShardTarget());
-        assertSame(searchShardTarget, searchContext.queryResult().getSearchShardTarget());
-        assertSame(searchShardTarget, searchContext.fetchResult().getSearchShardTarget());
+        try (DefaultSearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) {
+            SearchShardTarget searchShardTarget = searchContext.shardTarget();
+            QueryShardContext queryShardContext = searchContext.getQueryShardContext();
+            String expectedIndexName = clusterAlias == null ? index : clusterAlias + ":" + index;
+            assertEquals(expectedIndexName, queryShardContext.getFullyQualifiedIndex().getName());
+            assertEquals(expectedIndexName, searchShardTarget.getFullyQualifiedIndexName());
+            assertEquals(clusterAlias, searchShardTarget.getClusterAlias());
+            assertEquals(shardId, searchShardTarget.getShardId());
+            assertSame(searchShardTarget, searchContext.dfsResult().getSearchShardTarget());
+            assertSame(searchShardTarget, searchContext.queryResult().getSearchShardTarget());
+            assertSame(searchShardTarget, searchContext.fetchResult().getSearchShardTarget());
+        }
     }
 }
diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java
index 4d8f9fed51b24..621f303c98388 100644
--- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java
@@ -60,6 +60,7 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.concurrent.TimeUnit;
 
 import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
 import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING;
@@ -99,10 +100,13 @@ protected void startNode(long seed) throws Exception {
             ).get();
     }
 
-    private static void stopNode() throws IOException {
+    private static void stopNode() throws IOException, InterruptedException {
         Node node = NODE;
         NODE = null;
         IOUtils.close(node);
+        if (node != null && node.awaitClose(10, TimeUnit.SECONDS) == false) {
+            throw new AssertionError("Node couldn't close within 10 seconds.");
+        }
     }
 
     @Override
@@ -144,7 +148,7 @@ public static void setUpClass() throws Exception {
     }
 
     @AfterClass
-    public static void tearDownClass() throws IOException {
+    public static void tearDownClass() throws Exception {
         stopNode();
     }
 
diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
index 86f012fbd6583..8de9989fd6652 100644
--- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
+++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java
@@ -993,6 +993,13 @@ public void close() throws IOException {
                 closed.set(true);
                 markNodeDataDirsAsPendingForWipe(node);
                 node.close();
+                try {
+                    if (node.awaitClose(10, TimeUnit.SECONDS) == false) {
+                        throw new IOException("Node didn't close within 10 seconds.");
+                    }
+                } catch (InterruptedException e) {
+                    throw new AssertionError("Interruption while waiting for the node to close", e);
+                }
             }
         }
 

From 4bd8e7b9f49ed091a53ce2717d65dc0ecb77b8d8 Mon Sep 17 00:00:00 2001
From: David Kyle 
Date: Wed, 17 Apr 2019 14:56:30 +0100
Subject: [PATCH 074/260] [ML-DataFrame] Refactorings and tidying  (#41248)

Remove unnecessary generic params from SingleGroupSource
and unused code from the HLRC
---
 .../transforms/pivot/SingleGroupSource.java   | 29 ++-----------------
 .../transforms/DataFrameTransformConfig.java  |  2 +-
 .../pivot/DateHistogramGroupSource.java       |  4 +--
 .../transforms/pivot/GroupConfig.java         | 14 ++++-----
 .../pivot/HistogramGroupSource.java           |  4 +--
 .../transforms/pivot/PivotConfig.java         |  2 +-
 .../transforms/pivot/SingleGroupSource.java   |  8 ++---
 .../transforms/pivot/TermsGroupSource.java    |  4 +--
 .../transforms/pivot/GroupConfigTests.java    |  6 ++--
 ...TransportGetDataFrameTransformsAction.java |  4 +--
 ...nsportPreviewDataFrameTransformAction.java |  3 +-
 .../checkpoint/CheckpointException.java       | 13 ++-------
 .../transforms/DataFrameTransformTask.java    |  4 +--
 .../transforms/pivot/SchemaUtil.java          | 14 +++++----
 14 files changed, 39 insertions(+), 72 deletions(-)

diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/SingleGroupSource.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/SingleGroupSource.java
index 8168d8850e700..b1234277d8c4e 100644
--- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/SingleGroupSource.java
+++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/SingleGroupSource.java
@@ -30,32 +30,9 @@ public abstract class SingleGroupSource implements ToXContentObject {
     protected static final ParseField FIELD = new ParseField("field");
 
     public enum Type {
-        TERMS(0),
-        HISTOGRAM(1),
-        DATE_HISTOGRAM(2);
-
-        private final byte id;
-
-        Type(int id) {
-            this.id = (byte) id;
-        }
-
-        public byte getId() {
-            return id;
-        }
-
-        public static Type fromId(byte id) {
-            switch (id) {
-                case 0:
-                    return TERMS;
-                case 1:
-                    return HISTOGRAM;
-                case 2:
-                    return DATE_HISTOGRAM;
-                default:
-                    throw new IllegalArgumentException("unknown type");
-            }
-        }
+        TERMS,
+        HISTOGRAM,
+        DATE_HISTOGRAM;
 
         public String value() {
             return name().toLowerCase(Locale.ROOT);
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java
index a8e14faf2f0c4..89eab5605ca90 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java
@@ -215,7 +215,7 @@ public String toString() {
     }
 
     public static DataFrameTransformConfig fromXContent(final XContentParser parser, @Nullable final String optionalTransformId,
-            boolean lenient) throws IOException {
+            boolean lenient) {
 
         return lenient ? LENIENT_PARSER.apply(parser, optionalTransformId) : STRICT_PARSER.apply(parser, optionalTransformId);
     }
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/DateHistogramGroupSource.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/DateHistogramGroupSource.java
index a60a7ef98e954..f4bf094235ae4 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/DateHistogramGroupSource.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/DateHistogramGroupSource.java
@@ -20,7 +20,7 @@
 import java.util.Objects;
 
 
-public class DateHistogramGroupSource extends SingleGroupSource {
+public class DateHistogramGroupSource extends SingleGroupSource {
 
     private static final String NAME = "data_frame_date_histogram_group";
     private static final ParseField TIME_ZONE = new ParseField("time_zone");
@@ -51,7 +51,7 @@ private static ConstructingObjectParser createPa
             return new DateHistogramGroupSource(field);
         });
 
-        declareValuesSourceFields(parser, null);
+        declareValuesSourceFields(parser);
 
         parser.declareField((histogram, interval) -> {
             if (interval instanceof Long) {
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java
index de394fa3f199f..e5ba14c381a4a 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java
@@ -40,9 +40,9 @@ public class GroupConfig implements Writeable, ToXContentObject {
     private static final Logger logger = LogManager.getLogger(GroupConfig.class);
 
     private final Map source;
-    private final Map> groups;
+    private final Map groups;
 
-    public GroupConfig(final Map source, final Map> groups) {
+    public GroupConfig(final Map source, final Map groups) {
         this.source = ExceptionsHelper.requireNonNull(source, DataFrameField.GROUP_BY.getPreferredName());
         this.groups = groups;
     }
@@ -64,7 +64,7 @@ public GroupConfig(StreamInput in) throws IOException {
         });
     }
 
-    public Map > getGroups() {
+    public Map  getGroups() {
         return groups;
     }
 
@@ -109,7 +109,7 @@ public int hashCode() {
     public static GroupConfig fromXContent(final XContentParser parser, boolean lenient) throws IOException {
         NamedXContentRegistry registry = parser.getXContentRegistry();
         Map source = parser.mapOrdered();
-        Map> groups = null;
+        Map groups = null;
 
         if (source.isEmpty()) {
             if (lenient) {
@@ -133,9 +133,9 @@ public static GroupConfig fromXContent(final XContentParser parser, boolean leni
         return new GroupConfig(source, groups);
     }
 
-    private static Map> parseGroupConfig(final XContentParser parser,
+    private static Map parseGroupConfig(final XContentParser parser,
             boolean lenient) throws IOException {
-        LinkedHashMap> groups = new LinkedHashMap<>();
+        LinkedHashMap groups = new LinkedHashMap<>();
 
         // be parsing friendly, whether the token needs to be advanced or not (similar to what ObjectParser does)
         XContentParser.Token token;
@@ -158,7 +158,7 @@ private static Map> parseGroupConfig(final XContent
 
             token = parser.nextToken();
             ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
-            SingleGroupSource groupSource;
+            SingleGroupSource groupSource;
             switch (groupType) {
             case TERMS:
                 groupSource = TermsGroupSource.fromXContent(parser, lenient);
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/HistogramGroupSource.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/HistogramGroupSource.java
index 95e1068e0f4fe..737590a0cc197 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/HistogramGroupSource.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/HistogramGroupSource.java
@@ -17,7 +17,7 @@
 
 import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
 
-public class HistogramGroupSource extends SingleGroupSource {
+public class HistogramGroupSource extends SingleGroupSource {
 
     static final ParseField INTERVAL = new ParseField("interval");
     private static final String NAME = "data_frame_histogram_group";
@@ -44,7 +44,7 @@ private static ConstructingObjectParser createParser
             double interval = (double) args[1];
             return new HistogramGroupSource(field, interval);
         });
-        declareValuesSourceFields(parser, null);
+        declareValuesSourceFields(parser);
         parser.declareDouble(optionalConstructorArg(), INTERVAL);
         return parser;
     }
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java
index 4eef08bd9c075..993ba78482aed 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java
@@ -91,7 +91,7 @@ public void toCompositeAggXContent(XContentBuilder builder, Params params) throw
         builder.field(CompositeAggregationBuilder.SOURCES_FIELD_NAME.getPreferredName());
         builder.startArray();
 
-        for (Entry> groupBy : groups.getGroups().entrySet()) {
+        for (Entry groupBy : groups.getGroups().entrySet()) {
             builder.startObject();
             builder.startObject(groupBy.getKey());
             builder.field(groupBy.getValue().getType().value(), groupBy.getValue());
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java
index 8ba5caddb6f5f..0cdef0e4c3a96 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java
@@ -13,7 +13,6 @@
 import org.elasticsearch.common.xcontent.AbstractObjectParser;
 import org.elasticsearch.common.xcontent.ToXContentObject;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.search.aggregations.support.ValueType;
 
 import java.io.IOException;
 import java.util.Locale;
@@ -24,7 +23,7 @@
 /*
  * Base class for a single source for group_by
  */
-public abstract class SingleGroupSource> implements Writeable, ToXContentObject {
+public abstract class SingleGroupSource implements Writeable, ToXContentObject {
 
     public enum Type {
         TERMS(0),
@@ -64,8 +63,7 @@ public String value() {
     // TODO: add script
     protected final String field;
 
-    static , T> void declareValuesSourceFields(AbstractObjectParser parser,
-            ValueType targetValueType) {
+    static  void declareValuesSourceFields(AbstractObjectParser parser) {
         // either script or field
         parser.declareString(optionalConstructorArg(), FIELD);
     }
@@ -109,7 +107,7 @@ public boolean equals(Object other) {
             return false;
         }
 
-        final SingleGroupSource that = (SingleGroupSource) other;
+        final SingleGroupSource that = (SingleGroupSource) other;
 
         return Objects.equals(this.field, that.field);
     }
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/TermsGroupSource.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/TermsGroupSource.java
index 8c18e43be078a..d4585a611b367 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/TermsGroupSource.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/TermsGroupSource.java
@@ -15,7 +15,7 @@
 /*
  * A terms aggregation source for group_by
  */
-public class TermsGroupSource extends SingleGroupSource {
+public class TermsGroupSource extends SingleGroupSource {
     private static final String NAME = "data_frame_terms_group";
 
     private static final ConstructingObjectParser STRICT_PARSER = createParser(false);
@@ -27,7 +27,7 @@ private static ConstructingObjectParser createParser(boo
             return new TermsGroupSource(field);
         });
 
-        SingleGroupSource.declareValuesSourceFields(parser, null);
+        SingleGroupSource.declareValuesSourceFields(parser);
         return parser;
     }
 
diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfigTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfigTests.java
index e503d887cf3cb..f7b9552584221 100644
--- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfigTests.java
+++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfigTests.java
@@ -29,14 +29,14 @@ public class GroupConfigTests extends AbstractSerializingTestCase {
 
     public static GroupConfig randomGroupConfig() {
         Map source = new LinkedHashMap<>();
-        Map> groups = new LinkedHashMap<>();
+        Map groups = new LinkedHashMap<>();
 
         // ensure that the unlikely does not happen: 2 group_by's share the same name
         Set names = new HashSet<>();
         for (int i = 0; i < randomIntBetween(1, 20); ++i) {
             String targetFieldName = randomAlphaOfLengthBetween(1, 20);
             if (names.add(targetFieldName)) {
-                SingleGroupSource groupBy;
+                SingleGroupSource groupBy;
                 Type type = randomFrom(SingleGroupSource.Type.values());
                 switch (type) {
                 case TERMS:
@@ -88,7 +88,7 @@ public void testEmptyGroupBy() throws IOException {
         }
     }
 
-    private static Map getSource(SingleGroupSource groupSource) {
+    private static Map getSource(SingleGroupSource groupSource) {
         try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
             XContentBuilder content = groupSource.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS);
             return XContentHelper.convertToMap(BytesReference.bytes(content), true, XContentType.JSON).v2();
diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsAction.java
index bbdd8a6dee8d3..00bc15b1db6c5 100644
--- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsAction.java
+++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsAction.java
@@ -28,8 +28,6 @@
 import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
 import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex;
 
-import java.io.IOException;
-
 import static org.elasticsearch.xpack.core.dataframe.DataFrameField.INDEX_DOC_TYPE;
 
 
@@ -62,7 +60,7 @@ protected String[] getIndices() {
     }
 
     @Override
-    protected DataFrameTransformConfig parse(XContentParser parser) throws IOException {
+    protected DataFrameTransformConfig parse(XContentParser parser) {
         return DataFrameTransformConfig.fromXContent(parser, null, true);
     }
 
diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java
index b65830f72e7ca..b5642310df369 100644
--- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java
+++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java
@@ -85,9 +85,8 @@ private void getPreview(Pivot pivot, ActionListener>> l
                             DataFrameIndexerTransformStats stats = DataFrameIndexerTransformStats.withDefaultTransformId();
                             // remove all internal fields
                             List> results = pivot.extractResults(agg, deducedMappings, stats)
-                                    .map(record -> {
+                                    .peek(record -> {
                                         record.keySet().removeIf(k -> k.startsWith("_"));
-                                        return record;
                                     }).collect(Collectors.toList());
                             listener.onResponse(results);
                         },
diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/CheckpointException.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/CheckpointException.java
index 0a0a50761f021..f8405d3705767 100644
--- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/CheckpointException.java
+++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/checkpoint/CheckpointException.java
@@ -7,20 +7,13 @@
 package org.elasticsearch.xpack.dataframe.checkpoint;
 
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.common.io.stream.StreamInput;
 
-import java.io.IOException;
-
-public class CheckpointException extends ElasticsearchException {
-    public CheckpointException(String msg, Object... params) {
+class CheckpointException extends ElasticsearchException {
+    CheckpointException(String msg, Object... params) {
         super(msg, null, params);
     }
 
-    public CheckpointException(String msg, Throwable cause, Object... params) {
+    CheckpointException(String msg, Throwable cause, Object... params) {
         super(msg, cause, params);
     }
-
-    public CheckpointException(StreamInput in) throws IOException {
-        super(in);
-    }
 }
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 4088863a89500..b8ceb2e7bd497 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
@@ -240,7 +240,7 @@ public synchronized void stop(ActionListener resolveMappings(Map aggregati
             String sourceMapping = sourceFieldName == null ? null : sourceMappings.get(sourceFieldName);
             String destinationMapping = Aggregations.resolveTargetMapping(aggregationName, sourceMapping);
 
-            logger.debug(
-                    "Deduced mapping for: [" + targetFieldName + "], agg type [" + aggregationName + "] to [" + destinationMapping + "]");
+            logger.debug("Deduced mapping for: [{}], agg type [{}] to [{}]",
+                    targetFieldName, aggregationName, destinationMapping);
+
             if (Aggregations.isDynamicMapping(destinationMapping)) {
-                logger.info("Dynamic target mapping set for field ["+ targetFieldName +"] and aggregation [" + aggregationName +"]");
+                logger.debug("Dynamic target mapping set for field [{}] and aggregation [{}]", targetFieldName, aggregationName);
             } else if (destinationMapping != null) {
                 targetMapping.put(targetFieldName, destinationMapping);
             } else {
@@ -146,8 +147,7 @@ private static Map resolveMappings(Map aggregati
 
         fieldNamesForGrouping.forEach((targetFieldName, sourceFieldName) -> {
             String destinationMapping = sourceMappings.get(sourceFieldName);
-            logger.debug(
-                    "Deduced mapping for: [" + targetFieldName + "] to [" + destinationMapping + "]");
+            logger.debug("Deduced mapping for: [{}] to [{}]", targetFieldName, destinationMapping);
             if (destinationMapping != null) {
                 targetMapping.put(targetFieldName, destinationMapping);
             } else {
@@ -187,7 +187,9 @@ private static Map extractFieldMappings(Map map = (Map) typeMap;
                             if (map.containsKey("type")) {
                                 String type = map.get("type").toString();
-                                logger.debug("Extracted type for [" + fieldName + "] : [" + type + "] from index [" + indexName +"]");
+                                if (logger.isTraceEnabled()) {
+                                    logger.trace("Extracted type for [" + fieldName + "] : [" + type + "] from index [" + indexName + "]");
+                                }
                                 // TODO: overwrites types, requires resolve if
                                 // types are mixed
                                 extractedTypes.put(fieldName, type);

From 05c1476d3a565a7b13a6c1e6a680836c475ab6c4 Mon Sep 17 00:00:00 2001
From: Iana Bondarska 
Date: Wed, 17 Apr 2019 17:01:46 +0200
Subject: [PATCH 075/260] [ML] Exclude analysis fields with core field names
 from anomaly results (#41093)

Added "_index", "_type", "_id" to list of reserved fields.

Closes #39406
---
 .../core/ml/job/results/ReservedFieldNames.java   | 15 ++++++++++++---
 .../persistence/ElasticsearchMappingsTests.java   |  9 +++++++++
 .../ml/job/results/ReservedFieldNamesTests.java   |  6 +++++-
 3 files changed, 26 insertions(+), 4 deletions(-)

diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ReservedFieldNames.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ReservedFieldNames.java
index 333b87b0c294f..51bdc5ce594ad 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ReservedFieldNames.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ReservedFieldNames.java
@@ -5,6 +5,7 @@
  */
 package org.elasticsearch.xpack.core.ml.job.results;
 
+import org.elasticsearch.index.get.GetResult;
 import org.elasticsearch.xpack.core.ml.datafeed.ChunkingConfig;
 import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig;
 import org.elasticsearch.xpack.core.ml.datafeed.DelayedDataCheckConfig;
@@ -171,8 +172,12 @@ public final class ReservedFieldNames {
 
             Result.RESULT_TYPE.getPreferredName(),
             Result.TIMESTAMP.getPreferredName(),
-            Result.IS_INTERIM.getPreferredName()
-    };
+            Result.IS_INTERIM.getPreferredName(),
+
+            GetResult._ID,
+            GetResult._INDEX,
+            GetResult._TYPE
+   };
 
     /**
      * This array should be updated to contain all the field names that appear
@@ -256,7 +261,11 @@ public final class ReservedFieldNames {
             ChunkingConfig.MODE_FIELD.getPreferredName(),
             ChunkingConfig.TIME_SPAN_FIELD.getPreferredName(),
 
-            ElasticsearchMappings.CONFIG_TYPE
+            ElasticsearchMappings.CONFIG_TYPE,
+
+            GetResult._ID,
+            GetResult._INDEX,
+            GetResult._TYPE
     };
 
     /**
diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/job/persistence/ElasticsearchMappingsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/job/persistence/ElasticsearchMappingsTests.java
index f5461a1abf3f8..42e328e3591e0 100644
--- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/job/persistence/ElasticsearchMappingsTests.java
+++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/job/persistence/ElasticsearchMappingsTests.java
@@ -19,6 +19,7 @@
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.index.get.GetResult;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.test.VersionUtils;
 import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig;
@@ -63,6 +64,12 @@ public class ElasticsearchMappingsTests extends ESTestCase {
             ElasticsearchMappings.WHITESPACE
     );
 
+    private static List INTERNAL_FIELDS = Arrays.asList(
+            GetResult._ID,
+            GetResult._INDEX,
+            GetResult._TYPE
+    );
+
     public void testResultsMapppingReservedFields() throws Exception {
         Set overridden = new HashSet<>(KEYWORDS);
 
@@ -76,6 +83,7 @@ public void testResultsMapppingReservedFields() throws Exception {
 
         Set expected = collectResultsDocFieldNames();
         expected.removeAll(overridden);
+        expected.addAll(INTERNAL_FIELDS);
 
         compareFields(expected, ReservedFieldNames.RESERVED_RESULT_FIELD_NAMES);
     }
@@ -91,6 +99,7 @@ public void testConfigMapppingReservedFields() throws Exception {
 
         Set expected = collectConfigDocFieldNames();
         expected.removeAll(overridden);
+        expected.addAll(INTERNAL_FIELDS);
 
         compareFields(expected, ReservedFieldNames.RESERVED_CONFIG_FIELD_NAMES);
     }
diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/results/ReservedFieldNamesTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/results/ReservedFieldNamesTests.java
index a08b53fba3c0d..d594404e9c284 100644
--- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/results/ReservedFieldNamesTests.java
+++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/results/ReservedFieldNamesTests.java
@@ -5,6 +5,7 @@
  */
 package org.elasticsearch.xpack.ml.job.results;
 
+import org.elasticsearch.index.get.GetResult;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.core.ml.job.results.AnomalyRecord;
 import org.elasticsearch.xpack.core.ml.job.results.ReservedFieldNames;
@@ -16,5 +17,8 @@ public void testIsValidFieldName() {
         assertTrue(ReservedFieldNames.isValidFieldName("host.actual"));
         assertFalse(ReservedFieldNames.isValidFieldName("actual.host"));
         assertFalse(ReservedFieldNames.isValidFieldName(AnomalyRecord.BUCKET_SPAN.getPreferredName()));
+        assertFalse(ReservedFieldNames.isValidFieldName(GetResult._INDEX));
+        assertFalse(ReservedFieldNames.isValidFieldName(GetResult._TYPE));
+        assertFalse(ReservedFieldNames.isValidFieldName(GetResult._ID));
     }
-}
\ No newline at end of file
+}

From 48fea49c2cf2f860a320018e9fce4dd9ed920fb7 Mon Sep 17 00:00:00 2001
From: David Turner 
Date: Wed, 17 Apr 2019 16:36:04 +0100
Subject: [PATCH 076/260] Assert the stability of custom search preferences
 (#41150)

Today the `?preference=custom_string_value` search preference will only change
its choice of a shard copy if something changes the `IndexShardRoutingTable`
for that specific shard. Users can use this behaviour to route searches to a
consistent set of shard copies, which means they can reliably hit copies with
hot caches, and use the other copies only for redundancy in case of failure.
However we do not assert this property anywhere, so we might break it in
future.

This commit adds a test that shows that searches are routed consistently even
if other indices are created/rebalanced/deleted.

Relates https://discuss.elastic.co/t/176598, #41115, #26791
---
 .../search/preference/SearchPreferenceIT.java | 71 +++++++++++++++++--
 1 file changed, 64 insertions(+), 7 deletions(-)

diff --git a/server/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java b/server/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java
index 366975071ce43..23c29ce9f4674 100644
--- a/server/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java
+++ b/server/src/test/java/org/elasticsearch/search/preference/SearchPreferenceIT.java
@@ -25,10 +25,13 @@
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.health.ClusterHealthStatus;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.routing.OperationRouting;
+import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.node.Node;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.test.ESIntegTestCase;
 
@@ -42,10 +45,10 @@
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasToString;
-import static org.hamcrest.Matchers.not;
 import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.hasToString;
+import static org.hamcrest.Matchers.not;
 
 @ESIntegTestCase.ClusterScope(minNumDataNodes = 2)
 public class SearchPreferenceIT extends ESIntegTestCase {
@@ -57,7 +60,7 @@ public Settings nodeSettings(int nodeOrdinal) {
     }
 
     // see #2896
-    public void testStopOneNodePreferenceWithRedState() throws InterruptedException, IOException {
+    public void testStopOneNodePreferenceWithRedState() throws IOException {
         assertAcked(prepareCreate("test").setSettings(Settings.builder().put("index.number_of_shards", cluster().numDataNodes()+2)
                 .put("index.number_of_replicas", 0)));
         ensureGreen();
@@ -87,7 +90,7 @@ public void testStopOneNodePreferenceWithRedState() throws InterruptedException,
         assertThat("_only_local", searchResponse.getFailedShards(), greaterThanOrEqualTo(0));
     }
 
-    public void testNoPreferenceRandom() throws Exception {
+    public void testNoPreferenceRandom() {
         assertAcked(prepareCreate("test").setSettings(
                 //this test needs at least a replica to make sure two consecutive searches go to two different copies of the same data
                 Settings.builder().put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, between(1, maximumNumberOfReplicas()))
@@ -106,7 +109,7 @@ public void testNoPreferenceRandom() throws Exception {
         assertThat(firstNodeId, not(equalTo(secondNodeId)));
     }
 
-    public void testSimplePreference() throws Exception {
+    public void testSimplePreference() {
         client().admin().indices().prepareCreate("test").setSettings("{\"number_of_replicas\": 1}", XContentType.JSON).get();
         ensureGreen();
 
@@ -123,7 +126,7 @@ public void testSimplePreference() throws Exception {
         assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L));
     }
 
-    public void testThatSpecifyingNonExistingNodesReturnsUsefulError() throws Exception {
+    public void testThatSpecifyingNonExistingNodesReturnsUsefulError() {
         createIndex("test");
         ensureGreen();
 
@@ -135,7 +138,7 @@ public void testThatSpecifyingNonExistingNodesReturnsUsefulError() throws Except
         }
     }
 
-    public void testNodesOnlyRandom() throws Exception {
+    public void testNodesOnlyRandom() {
         assertAcked(prepareCreate("test").setSettings(
             //this test needs at least a replica to make sure two consecutive searches go to two different copies of the same data
             Settings.builder().put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, between(1, maximumNumberOfReplicas()))));
@@ -193,4 +196,58 @@ private void assertSearchOnRandomNodes(SearchRequestBuilder request) {
         }
         assertThat(hitNodes.size(), greaterThan(1));
     }
+
+    public void testCustomPreferenceUnaffectedByOtherShardMovements() {
+
+        /*
+         * Custom preferences can be used to encourage searches to go to a consistent set of shard copies, meaning that other copies' data
+         * is rarely touched and can be dropped from the filesystem cache. This works best if the set of shards searched doesn't change
+         * unnecessarily, so this test verifies a consistent routing even as other shards are created/relocated/removed.
+         */
+
+        assertAcked(prepareCreate("test").setSettings(Settings.builder().put(indexSettings())
+            .put(SETTING_NUMBER_OF_REPLICAS, between(1, maximumNumberOfReplicas()))
+            .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE)));
+        ensureGreen();
+        client().prepareIndex("test", "_doc").setSource("field1", "value1").get();
+        refresh();
+
+        final String customPreference = randomAlphaOfLength(10);
+
+        final String nodeId = client().prepareSearch("test").setQuery(matchAllQuery()).setPreference(customPreference)
+            .get().getHits().getAt(0).getShard().getNodeId();
+
+        assertSearchesSpecificNode("test", customPreference, nodeId);
+
+        final int replicasInNewIndex = between(1, maximumNumberOfReplicas());
+        assertAcked(prepareCreate("test2").setSettings(
+            Settings.builder().put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, replicasInNewIndex)));
+        ensureGreen();
+
+        assertSearchesSpecificNode("test", customPreference, nodeId);
+
+        assertAcked(client().admin().indices().prepareUpdateSettings("test2").setSettings(Settings.builder()
+            .put(SETTING_NUMBER_OF_REPLICAS, replicasInNewIndex - 1)));
+
+        assertSearchesSpecificNode("test", customPreference, nodeId);
+
+        assertAcked(client().admin().indices().prepareUpdateSettings("test2").setSettings(Settings.builder()
+            .put(SETTING_NUMBER_OF_REPLICAS, 0)
+            .put(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_PREFIX + "._name",
+                internalCluster().getDataNodeInstance(Node.class).settings().get(Node.NODE_NAME_SETTING.getKey()))));
+
+        ensureGreen();
+
+        assertSearchesSpecificNode("test", customPreference, nodeId);
+
+        assertAcked(client().admin().indices().prepareDelete("test2"));
+
+        assertSearchesSpecificNode("test", customPreference, nodeId);
+    }
+
+    private static void assertSearchesSpecificNode(String index, String customPreference, String nodeId) {
+        final SearchResponse searchResponse = client().prepareSearch(index).setQuery(matchAllQuery()).setPreference(customPreference).get();
+        assertThat(searchResponse.getHits().getHits().length, equalTo(1));
+        assertThat(searchResponse.getHits().getAt(0).getShard().getNodeId(), equalTo(nodeId));
+    }
 }

From f3ac4e6cb396f0111e91ab9819623b9be144c55e Mon Sep 17 00:00:00 2001
From: Gordon Brown 
Date: Wed, 17 Apr 2019 10:06:52 -0600
Subject: [PATCH 077/260] Extract template management from Watcher (#41169)

This commit extracts the template management from Watcher into an
abstract class, so that templates and lifecycle policies can be managed
in the same way across multiple plugins. This will be useful for SLM, as
well as potentially ILM and any other plugins which need to manage index
templates.
---
 .../core/template/IndexTemplateConfig.java    |  62 +++++
 .../core/template/IndexTemplateRegistry.java  | 229 +++++++++++++++
 .../core/template/LifecyclePolicyConfig.java  |  44 +++
 .../support/WatcherIndexTemplateRegistry.java | 263 ++++--------------
 .../WatcherIndexTemplateRegistryTests.java    |  15 +-
 5 files changed, 400 insertions(+), 213 deletions(-)
 create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateConfig.java
 create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java
 create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/LifecyclePolicyConfig.java

diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateConfig.java
new file mode 100644
index 0000000000000..5eb219f11a7a0
--- /dev/null
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateConfig.java
@@ -0,0 +1,62 @@
+/*
+ * 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.core.template;
+
+import java.nio.charset.StandardCharsets;
+import java.util.regex.Pattern;
+
+/**
+ * Describes an index template to be loaded from a resource file for use with an {@link IndexTemplateRegistry}.
+ */
+public class IndexTemplateConfig {
+
+    private final String templateName;
+    private final String fileName;
+    private final String version;
+    private final String versionProperty;
+
+    /**
+     * Describes a template to be loaded from a resource file. Includes handling for substituting a version property into the template.
+     *
+     * The {@code versionProperty} parameter will be used to substitute the value of {@code version} into the template. For example,
+     * this template:
+     * {@code {"myTemplateVersion": "${my.version.property}"}}
+     * With {@code version = "42"; versionProperty = "my.version.property"} will result in {@code {"myTemplateVersion": "42"}}.
+     *
+     * @param templateName The name that will be used for the index template. Literal, include the version in this string if
+     *                     it should be used.
+     * @param fileName The filename the template should be loaded from. Literal, should include leading {@literal /} and
+     *                 extension if necessary.
+     * @param version The version of the template. Substituted for {@code versionProperty} as described above.
+     * @param versionProperty The property that will be replaced with the {@code version} string as described above.
+     */
+    public IndexTemplateConfig(String templateName, String fileName, String version, String versionProperty) {
+        this.templateName = templateName;
+        this.fileName = fileName;
+        this.version = version;
+        this.versionProperty = versionProperty;
+    }
+
+    public String getFileName() {
+        return fileName;
+    }
+
+    public String getTemplateName() {
+        return templateName;
+    }
+
+    /**
+     * Loads the template from disk as a UTF-8 byte array.
+     * @return The template as a UTF-8 byte array.
+     */
+    public byte[] loadBytes() {
+        String template = TemplateUtils.loadTemplate(fileName, version,
+            Pattern.quote("${" + versionProperty + "}"));
+        assert template != null && template.length() > 0;
+        return template.getBytes(StandardCharsets.UTF_8);
+    }
+}
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java
new file mode 100644
index 0000000000000..d0a086bd649f0
--- /dev/null
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java
@@ -0,0 +1,229 @@
+/*
+ * 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.core.template;
+
+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.admin.indices.template.put.PutIndexTemplateRequest;
+import org.elasticsearch.action.support.master.AcknowledgedResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.cluster.ClusterChangedEvent;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.ClusterStateListener;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.gateway.GatewayService;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.xpack.core.XPackClient;
+import org.elasticsearch.xpack.core.XPackSettings;
+import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata;
+import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
+import org.elasticsearch.xpack.core.indexlifecycle.action.PutLifecycleAction;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin;
+
+/**
+ * Abstracts the logic of managing versioned index templates and lifecycle policies for plugins that require such things.
+ */
+public abstract class IndexTemplateRegistry implements ClusterStateListener {
+    private static final Logger logger = LogManager.getLogger(IndexTemplateRegistry.class);
+
+    protected final Settings settings;
+    protected final Client client;
+    protected final ThreadPool threadPool;
+    protected final NamedXContentRegistry xContentRegistry;
+    protected final ConcurrentMap templateCreationsInProgress = new ConcurrentHashMap<>();
+    protected final ConcurrentMap policyCreationsInProgress = new ConcurrentHashMap<>();
+
+    public IndexTemplateRegistry(Settings nodeSettings, ClusterService clusterService, ThreadPool threadPool, Client client,
+                                 NamedXContentRegistry xContentRegistry) {
+        this.settings = nodeSettings;
+        this.client = client;
+        this.threadPool = threadPool;
+        this.xContentRegistry = xContentRegistry;
+        clusterService.addListener(this);
+    }
+
+    /**
+     * Retrieves return a list of {@link IndexTemplateConfig} that represents
+     * the index templates that should be installed and managed.
+     * @return The configurations for the templates that should be installed.
+     */
+    protected abstract List getTemplateConfigs();
+
+    /**
+     * Retrieves a list of {@link LifecyclePolicyConfig} that represents the ILM
+     * policies that should be installed and managed. Only called if ILM is enabled.
+     * @return The configurations for the lifecycle policies that should be installed.
+     */
+    protected abstract List getPolicyConfigs();
+
+    /**
+     * Retrieves an identifier that is used to identify which plugin is asking for this.
+     * @return A string ID for the plugin managing these templates.
+     */
+    protected abstract String getOrigin();
+
+    /**
+     * Called when creation of an index template fails.
+     * @param config The template config that failed to be created.
+     * @param e The exception that caused the failure.
+     */
+    protected void onPutTemplateFailure(IndexTemplateConfig config, Exception e) {
+        logger.error(new ParameterizedMessage("error adding index template [{}] from [{}] for [{}]",
+            config.getTemplateName(), config.getFileName(), getOrigin()), e);
+    }
+
+    /**
+     * Called when creation of a lifecycle policy fails.
+     * @param policy The lifecycle policy that failed to be created.
+     * @param e The exception that caused the failure.
+     */
+    protected void onPutPolicyFailure(LifecyclePolicy policy, Exception e) {
+        logger.error(new ParameterizedMessage("error adding lifecycle policy [{}] for [{}]",
+            policy.getName(), getOrigin()), e);
+    }
+
+    @Override
+    public void clusterChanged(ClusterChangedEvent event) {
+        ClusterState state = event.state();
+        if (state.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) {
+            // wait until the gateway has recovered from disk, otherwise we think may not have the index templates,
+            // while they actually do exist
+            return;
+        }
+
+        // no master node, exit immediately
+        DiscoveryNode masterNode = event.state().getNodes().getMasterNode();
+        if (masterNode == null) {
+            return;
+        }
+
+        // if this node is newer than the master node, we probably need to add the template, which might be newer than the
+        // template the master node has, so we need potentially add new templates despite being not the master node
+        DiscoveryNode localNode = event.state().getNodes().getLocalNode();
+        boolean localNodeVersionAfterMaster = localNode.getVersion().after(masterNode.getVersion());
+
+        if (event.localNodeMaster() || localNodeVersionAfterMaster) {
+            addTemplatesIfMissing(state);
+            addIndexLifecyclePoliciesIfMissing(state);
+        }
+    }
+
+    private void addTemplatesIfMissing(ClusterState state) {
+        final List indexTemplates = getTemplateConfigs();
+        for (IndexTemplateConfig template : indexTemplates) {
+            final String templateName = template.getTemplateName();
+            final AtomicBoolean creationCheck = templateCreationsInProgress.computeIfAbsent(templateName, key -> new AtomicBoolean(false));
+            if (creationCheck.compareAndSet(false, true)) {
+                if (!state.metaData().getTemplates().containsKey(templateName)) {
+                    logger.debug("adding index template [{}] for [{}], because it doesn't exist", templateName, getOrigin());
+                    putTemplate(template, creationCheck);
+                } else {
+                    creationCheck.set(false);
+                    logger.trace("not adding index template [{}] for [{}], because it already exists", templateName, getOrigin());
+                }
+            }
+        }
+    }
+
+    private void putTemplate(final IndexTemplateConfig config, final AtomicBoolean creationCheck) {
+        final Executor executor = threadPool.generic();
+        executor.execute(() -> {
+            final String templateName = config.getTemplateName();
+
+            PutIndexTemplateRequest request = new PutIndexTemplateRequest(templateName).source(config.loadBytes(), XContentType.JSON);
+            request.masterNodeTimeout(TimeValue.timeValueMinutes(1));
+            executeAsyncWithOrigin(client.threadPool().getThreadContext(), getOrigin(), request,
+                new ActionListener() {
+                    @Override
+                    public void onResponse(AcknowledgedResponse response) {
+                        creationCheck.set(false);
+                        if (response.isAcknowledged() == false) {
+                            logger.error("error adding index template [{}] for [{}], request was not acknowledged",
+                                templateName, getOrigin());
+                        }
+                    }
+
+                    @Override
+                    public void onFailure(Exception e) {
+                        creationCheck.set(false);
+                        onPutTemplateFailure(config, e);
+                    }
+                }, client.admin().indices()::putTemplate);
+        });
+    }
+
+    private void addIndexLifecyclePoliciesIfMissing(ClusterState state) {
+        boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(settings);
+
+        if (ilmSupported) {
+            Optional maybeMeta = Optional.ofNullable(state.metaData().custom(IndexLifecycleMetadata.TYPE));
+            List policies = getPolicyConfigs().stream()
+                .map(policyConfig -> policyConfig.load(xContentRegistry))
+                .collect(Collectors.toList());
+
+            for (LifecyclePolicy policy : policies) {
+                final AtomicBoolean creationCheck = policyCreationsInProgress.computeIfAbsent(policy.getName(),
+                    key -> new AtomicBoolean(false));
+                if (creationCheck.compareAndSet(false, true)) {
+                    final boolean policyNeedsToBeCreated = maybeMeta
+                        .flatMap(ilmMeta -> Optional.ofNullable(ilmMeta.getPolicies().get(policy.getName())))
+                        .isPresent() == false;
+                    if (policyNeedsToBeCreated) {
+                        logger.debug("adding lifecycle policy [{}] for [{}], because it doesn't exist", policy.getName(), getOrigin());
+                        putPolicy(policy, creationCheck);
+                    } else {
+                        logger.trace("not adding lifecycle policy [{}] for [{}], because it already exists",
+                            policy.getName(), getOrigin());
+                        creationCheck.set(false);
+                    }
+                }
+            }
+        }
+    }
+
+    private void putPolicy(final LifecyclePolicy policy, final AtomicBoolean creationCheck) {
+        final Executor executor = threadPool.generic();
+        executor.execute(() -> {
+            PutLifecycleAction.Request request = new PutLifecycleAction.Request(policy);
+            request.masterNodeTimeout(TimeValue.timeValueMinutes(1));
+            executeAsyncWithOrigin(client.threadPool().getThreadContext(), getOrigin(), request,
+                new ActionListener() {
+                    @Override
+                    public void onResponse(PutLifecycleAction.Response response) {
+                        creationCheck.set(false);
+                        if (response.isAcknowledged() == false) {
+                            logger.error("error adding lifecycle policy [{}] for [{}], request was not acknowledged",
+                                policy.getName(), getOrigin());
+                        }
+                    }
+
+                    @Override
+                    public void onFailure(Exception e) {
+                        creationCheck.set(false);
+                        onPutPolicyFailure(policy, e);
+                    }
+                }, (req, listener) -> new XPackClient(client).ilmClient().putLifecyclePolicy(req, listener));
+        });
+    }
+
+}
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/LifecyclePolicyConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/LifecyclePolicyConfig.java
new file mode 100644
index 0000000000000..c27b262f9f131
--- /dev/null
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/LifecyclePolicyConfig.java
@@ -0,0 +1,44 @@
+/*
+ * 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.core.template;
+
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
+import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicyUtils;
+
+/**
+ * Describes an index lifecycle policy to be loaded from a resource file for use with an {@link IndexTemplateRegistry}.
+ */
+public class LifecyclePolicyConfig {
+
+    private final String policyName;
+    private final String fileName;
+
+    /**
+     * Describes a lifecycle policy definition to be loaded from a resource file.
+     *
+     * @param policyName The name that will be used for the policy.
+     * @param fileName The filename the policy definition should be loaded from. Literal, should include leading {@literal /} and
+     *                 extension if necessary.
+     */
+    public LifecyclePolicyConfig(String policyName, String fileName) {
+        this.policyName = policyName;
+        this.fileName = fileName;
+    }
+
+    public String getPolicyName() {
+        return policyName;
+    }
+
+    public String getFileName() {
+        return fileName;
+    }
+
+    public LifecyclePolicy load(NamedXContentRegistry xContentRegistry) {
+        return LifecyclePolicyUtils.loadPolicy(policyName, fileName, xContentRegistry);
+    }
+}
diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java
index 4ebcc5a8f4173..9f5027f7a0f7a 100644
--- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java
+++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistry.java
@@ -5,240 +5,83 @@
  */
 package org.elasticsearch.xpack.watcher.support;
 
-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.admin.indices.template.put.PutIndexTemplateRequest;
-import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.Client;
-import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.ClusterStateListener;
-import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
-import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.gateway.GatewayService;
 import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.xpack.core.XPackClient;
 import org.elasticsearch.xpack.core.XPackSettings;
-import org.elasticsearch.xpack.core.indexlifecycle.IndexLifecycleMetadata;
-import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicy;
-import org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicyUtils;
-import org.elasticsearch.xpack.core.indexlifecycle.action.PutLifecycleAction;
-import org.elasticsearch.xpack.core.template.TemplateUtils;
+import org.elasticsearch.xpack.core.template.IndexTemplateConfig;
+import org.elasticsearch.xpack.core.template.IndexTemplateRegistry;
+import org.elasticsearch.xpack.core.template.LifecyclePolicyConfig;
 import org.elasticsearch.xpack.core.watcher.support.WatcherIndexTemplateRegistryField;
 
-import java.nio.charset.StandardCharsets;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executor;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.regex.Pattern;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
 
 import static org.elasticsearch.xpack.core.ClientHelper.WATCHER_ORIGIN;
-import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin;
 
-public class WatcherIndexTemplateRegistry implements ClusterStateListener {
-
-    public static final TemplateConfig TEMPLATE_CONFIG_TRIGGERED_WATCHES = new TemplateConfig(
-            WatcherIndexTemplateRegistryField.TRIGGERED_TEMPLATE_NAME, "triggered-watches");
-    public static final TemplateConfig TEMPLATE_CONFIG_WATCH_HISTORY = new TemplateConfig(
-        WatcherIndexTemplateRegistryField.HISTORY_TEMPLATE_NAME, "watch-history");
-    public static final TemplateConfig TEMPLATE_CONFIG_WATCH_HISTORY_NO_ILM = new TemplateConfig(
-        WatcherIndexTemplateRegistryField.HISTORY_TEMPLATE_NAME_NO_ILM, "watch-history-no-ilm");
-    public static final TemplateConfig TEMPLATE_CONFIG_WATCHES = new TemplateConfig(
-            WatcherIndexTemplateRegistryField.WATCHES_TEMPLATE_NAME, "watches");
-    public static final TemplateConfig[] TEMPLATE_CONFIGS = new TemplateConfig[]{
-        TEMPLATE_CONFIG_TRIGGERED_WATCHES, TEMPLATE_CONFIG_WATCH_HISTORY, TEMPLATE_CONFIG_WATCHES
-    };
-    public static final TemplateConfig[] TEMPLATE_CONFIGS_NO_ILM = new TemplateConfig[]{
-        TEMPLATE_CONFIG_TRIGGERED_WATCHES, TEMPLATE_CONFIG_WATCH_HISTORY_NO_ILM, TEMPLATE_CONFIG_WATCHES
-    };
-
-    public static final PolicyConfig POLICY_WATCH_HISTORY = new PolicyConfig("watch-history-ilm-policy", "/watch-history-ilm-policy.json");
-
-    private static final Logger logger = LogManager.getLogger(WatcherIndexTemplateRegistry.class);
-
-    private final Settings nodeSettings;
-    private final Client client;
-    private final ThreadPool threadPool;
-    private final NamedXContentRegistry xContentRegistry;
-    private final ConcurrentMap templateCreationsInProgress = new ConcurrentHashMap<>();
-    private final AtomicBoolean historyPolicyCreationInProgress = new AtomicBoolean();
-
-    public WatcherIndexTemplateRegistry(Settings nodeSettings, ClusterService clusterService,
-                                        ThreadPool threadPool, Client client,
+public class WatcherIndexTemplateRegistry extends IndexTemplateRegistry {
+
+    public static final String WATCHER_TEMPLATE_VERSION_VARIABLE = "xpack.watcher.template.version";
+    public static final IndexTemplateConfig TEMPLATE_CONFIG_TRIGGERED_WATCHES = new IndexTemplateConfig(
+        WatcherIndexTemplateRegistryField.TRIGGERED_TEMPLATE_NAME,
+        "/triggered-watches.json",
+        WatcherIndexTemplateRegistryField.INDEX_TEMPLATE_VERSION,
+        WATCHER_TEMPLATE_VERSION_VARIABLE);
+    public static final IndexTemplateConfig TEMPLATE_CONFIG_WATCH_HISTORY = new IndexTemplateConfig(
+        WatcherIndexTemplateRegistryField.HISTORY_TEMPLATE_NAME,
+        "/watch-history.json",
+        WatcherIndexTemplateRegistryField.INDEX_TEMPLATE_VERSION,
+        WATCHER_TEMPLATE_VERSION_VARIABLE);
+    public static final IndexTemplateConfig TEMPLATE_CONFIG_WATCH_HISTORY_NO_ILM = new IndexTemplateConfig(
+        WatcherIndexTemplateRegistryField.HISTORY_TEMPLATE_NAME_NO_ILM,
+        "/watch-history-no-ilm.json",
+        WatcherIndexTemplateRegistryField.INDEX_TEMPLATE_VERSION,
+        WATCHER_TEMPLATE_VERSION_VARIABLE);
+    public static final IndexTemplateConfig TEMPLATE_CONFIG_WATCHES = new IndexTemplateConfig(
+        WatcherIndexTemplateRegistryField.WATCHES_TEMPLATE_NAME,
+        "/watches.json",
+        WatcherIndexTemplateRegistryField.INDEX_TEMPLATE_VERSION,
+        WATCHER_TEMPLATE_VERSION_VARIABLE);
+
+    public static final LifecyclePolicyConfig POLICY_WATCH_HISTORY = new LifecyclePolicyConfig("watch-history-ilm-policy",
+        "/watch-history-ilm-policy.json");
+
+    private final List templatesToUse;
+
+    public WatcherIndexTemplateRegistry(Settings nodeSettings, ClusterService clusterService, ThreadPool threadPool, Client client,
                                         NamedXContentRegistry xContentRegistry) {
-        this.nodeSettings = nodeSettings;
-        this.client = client;
-        this.threadPool = threadPool;
-        this.xContentRegistry = xContentRegistry;
-        clusterService.addListener(this);
+        super(nodeSettings, clusterService, threadPool, client, xContentRegistry);
+        boolean ilmEnabled = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(settings);
+        templatesToUse = Arrays.asList(
+            ilmEnabled ? TEMPLATE_CONFIG_WATCH_HISTORY : TEMPLATE_CONFIG_WATCH_HISTORY_NO_ILM,
+            TEMPLATE_CONFIG_TRIGGERED_WATCHES,
+            TEMPLATE_CONFIG_WATCHES
+        );
     }
 
     @Override
-    public void clusterChanged(ClusterChangedEvent event) {
-        ClusterState state = event.state();
-        if (state.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) {
-            // wait until the gateway has recovered from disk, otherwise we think may not have the index templates,
-            // while they actually do exist
-            return;
-        }
-
-        // no master node, exit immediately
-        DiscoveryNode masterNode = event.state().getNodes().getMasterNode();
-        if (masterNode == null) {
-            return;
-        }
-
-        // if this node is newer than the master node, we probably need to add the history template, which might be newer than the
-        // history template the master node has, so we need potentially add new templates despite being not the master node
-        DiscoveryNode localNode = event.state().getNodes().getLocalNode();
-        boolean localNodeVersionAfterMaster = localNode.getVersion().after(masterNode.getVersion());
-
-        if (event.localNodeMaster() || localNodeVersionAfterMaster) {
-            addTemplatesIfMissing(state);
-            addIndexLifecyclePolicyIfMissing(state);
-        }
-    }
-
-    private void addTemplatesIfMissing(ClusterState state) {
-        boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(this.nodeSettings);
-        final TemplateConfig[] indexTemplates = ilmSupported ? TEMPLATE_CONFIGS : TEMPLATE_CONFIGS_NO_ILM;
-        for (TemplateConfig template : indexTemplates) {
-            final String templateName = template.getTemplateName();
-            final AtomicBoolean creationCheck = templateCreationsInProgress.computeIfAbsent(templateName, key -> new AtomicBoolean(false));
-            if (creationCheck.compareAndSet(false, true)) {
-                if (!state.metaData().getTemplates().containsKey(templateName)) {
-                    logger.debug("adding index template [{}], because it doesn't exist", templateName);
-                    putTemplate(template, creationCheck);
-                } else {
-                    creationCheck.set(false);
-                    logger.trace("not adding index template [{}], because it already exists", templateName);
-                }
-            }
-        }
-    }
-
-    private void putTemplate(final TemplateConfig config, final AtomicBoolean creationCheck) {
-        final Executor executor = threadPool.generic();
-        executor.execute(() -> {
-            final String templateName = config.getTemplateName();
-
-            PutIndexTemplateRequest request = new PutIndexTemplateRequest(templateName).source(config.load(), XContentType.JSON);
-            request.masterNodeTimeout(TimeValue.timeValueMinutes(1));
-            executeAsyncWithOrigin(client.threadPool().getThreadContext(), WATCHER_ORIGIN, request,
-                    new ActionListener() {
-                        @Override
-                        public void onResponse(AcknowledgedResponse response) {
-                            creationCheck.set(false);
-                            if (response.isAcknowledged() == false) {
-                                logger.error("Error adding watcher template [{}], request was not acknowledged", templateName);
-                            }
-                        }
-
-                        @Override
-                        public void onFailure(Exception e) {
-                            creationCheck.set(false);
-                            logger.error(new ParameterizedMessage("Error adding watcher template [{}]", templateName), e);
-                        }
-                    }, client.admin().indices()::putTemplate);
-        });
-    }
-
-    // Package visible for testing
-    LifecyclePolicy loadWatcherHistoryPolicy() {
-        return LifecyclePolicyUtils.loadPolicy(POLICY_WATCH_HISTORY.policyName, POLICY_WATCH_HISTORY.fileName, xContentRegistry);
+    protected List getTemplateConfigs() {
+        return templatesToUse;
     }
 
-    private void addIndexLifecyclePolicyIfMissing(ClusterState state) {
-        boolean ilmSupported = XPackSettings.INDEX_LIFECYCLE_ENABLED.get(this.nodeSettings);
-        if (ilmSupported && historyPolicyCreationInProgress.compareAndSet(false, true)) {
-            final LifecyclePolicy policyOnDisk = loadWatcherHistoryPolicy();
-
-            Optional maybeMeta = Optional.ofNullable(state.metaData().custom(IndexLifecycleMetadata.TYPE));
-            final boolean needsUpdating = maybeMeta
-                .flatMap(ilmMeta -> Optional.ofNullable(ilmMeta.getPolicies().get(policyOnDisk.getName())))
-                .isPresent() == false; // If there is no policy then one needs to be put;
-
-            if (needsUpdating) {
-                putPolicy(policyOnDisk, historyPolicyCreationInProgress);
-            } else {
-                historyPolicyCreationInProgress.set(false);
-            }
-        }
+    @Override
+    protected List getPolicyConfigs() {
+        return Collections.singletonList(POLICY_WATCH_HISTORY);
     }
 
-    private void putPolicy(final LifecyclePolicy policy, final AtomicBoolean creationCheck) {
-        final Executor executor = threadPool.generic();
-        executor.execute(() -> {
-            PutLifecycleAction.Request request = new PutLifecycleAction.Request(policy);
-            request.masterNodeTimeout(TimeValue.timeValueMinutes(1));
-            executeAsyncWithOrigin(client.threadPool().getThreadContext(), WATCHER_ORIGIN, request,
-                new ActionListener() {
-                    @Override
-                    public void onResponse(PutLifecycleAction.Response response) {
-                        creationCheck.set(false);
-                        if (response.isAcknowledged() == false) {
-                            logger.error("error adding watcher index lifecycle policy [{}], request was not acknowledged",
-                                policy.getName());
-                        }
-                    }
-
-                    @Override
-                    public void onFailure(Exception e) {
-                        creationCheck.set(false);
-                        logger.error(new ParameterizedMessage("error adding watcher index lifecycle policy [{}]",
-                            policy.getName()), e);
-                    }
-                }, (req, listener) -> new XPackClient(client).ilmClient().putLifecyclePolicy(req, listener));
-        });
+    @Override
+    protected String getOrigin() {
+        return WATCHER_ORIGIN;
     }
 
     public static boolean validate(ClusterState state) {
         return state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.HISTORY_TEMPLATE_NAME) &&
-                state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.TRIGGERED_TEMPLATE_NAME) &&
-                state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.WATCHES_TEMPLATE_NAME);
+            state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.TRIGGERED_TEMPLATE_NAME) &&
+            state.getMetaData().getTemplates().containsKey(WatcherIndexTemplateRegistryField.WATCHES_TEMPLATE_NAME);
     }
 
-    public static class TemplateConfig {
-
-        private final String templateName;
-        private String fileName;
-
-        TemplateConfig(String templateName, String fileName) {
-            this.templateName = templateName;
-            this.fileName = fileName;
-        }
-
-        public String getFileName() {
-            return fileName;
-        }
-
-        public String getTemplateName() {
-            return templateName;
-        }
-
-        public byte[] load() {
-            String template = TemplateUtils.loadTemplate("/" + fileName + ".json", WatcherIndexTemplateRegistryField.INDEX_TEMPLATE_VERSION,
-                Pattern.quote("${xpack.watcher.template.version}"));
-            assert template != null && template.length() > 0;
-            return template.getBytes(StandardCharsets.UTF_8);
-        }
-    }
-    public static class PolicyConfig {
-
-        private final String policyName;
-        private String fileName;
-
-        PolicyConfig(String templateName, String fileName) {
-            this.policyName = templateName;
-            this.fileName = fileName;
-        }
-    }
 }
diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java
index bd55e75795382..a96c04ab7cd99 100644
--- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java
+++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/support/WatcherIndexTemplateRegistryTests.java
@@ -59,6 +59,7 @@
 import static org.elasticsearch.mock.orig.Mockito.verify;
 import static org.elasticsearch.mock.orig.Mockito.when;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.is;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyObject;
@@ -164,7 +165,11 @@ public void testPolicyAlreadyExists() {
         DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build();
 
         Map policyMap = new HashMap<>();
-        LifecyclePolicy policy = registry.loadWatcherHistoryPolicy();
+        List policies = registry.getPolicyConfigs().stream()
+            .map(policyConfig -> policyConfig.load(xContentRegistry))
+            .collect(Collectors.toList());
+        assertThat(policies, hasSize(1));
+        LifecyclePolicy policy = policies.get(0);
         policyMap.put(policy.getName(), policy);
         ClusterChangedEvent event = createClusterChangedEvent(Collections.emptyList(), policyMap, nodes);
         registry.clusterChanged(event);
@@ -183,13 +188,17 @@ public void testNoPolicyButILMDisabled() {
         verify(client, times(0)).execute(eq(PutLifecycleAction.INSTANCE), anyObject(), anyObject());
     }
 
-    public void testPolicyAlreadyExistsButDiffers() throws IOException  {
+    public void testPolicyAlreadyExistsButDiffers() throws IOException {
         DiscoveryNode node = new DiscoveryNode("node", ESTestCase.buildNewFakeTransportAddress(), Version.CURRENT);
         DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build();
 
         Map policyMap = new HashMap<>();
         String policyStr = "{\"phases\":{\"delete\":{\"min_age\":\"1m\",\"actions\":{\"delete\":{}}}}}";
-        LifecyclePolicy policy = registry.loadWatcherHistoryPolicy();
+        List policies = registry.getPolicyConfigs().stream()
+            .map(policyConfig -> policyConfig.load(xContentRegistry))
+            .collect(Collectors.toList());
+        assertThat(policies, hasSize(1));
+        LifecyclePolicy policy = policies.get(0);
         try (XContentParser parser = XContentType.JSON.xContent()
             .createParser(xContentRegistry, LoggingDeprecationHandler.THROW_UNSUPPORTED_OPERATION, policyStr)) {
             LifecyclePolicy different = LifecyclePolicy.parse(parser, policy.getName());

From c31a5b1138588846091252574b8a89c6570f8a6e Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Christoph=20B=C3=BCscher?= 
Date: Wed, 17 Apr 2019 18:44:07 +0200
Subject: [PATCH 078/260] Fix error applying `ignore_malformed` to boolean
 values (#41261)

The `ignore_malformed` option currently works on numeric fields only when the
bad value isn't a string value but not if it is a boolean. In this case we get a
parsing error from the xContent parser which we need to catch in addition to the
field mapper.

Closes #11498
---
 .../index/mapper/NumberFieldMapper.java       |  6 +-
 .../index/mapper/NumberFieldMapperTests.java  | 86 ++++++++++++-------
 2 files changed, 59 insertions(+), 33 deletions(-)

diff --git a/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java
index 06e12ca8b5e4c..927bce5d9d6dd 100644
--- a/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java
+++ b/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java
@@ -19,6 +19,8 @@
 
 package org.elasticsearch.index.mapper;
 
+import com.fasterxml.jackson.core.JsonParseException;
+
 import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatPoint;
@@ -1042,8 +1044,8 @@ protected void parseCreateField(ParseContext context, List field
         } else {
             try {
                 numericValue = fieldType().type.parse(parser, coerce.value());
-            } catch (IllegalArgumentException e) {
-                if (ignoreMalformed.value()) {
+            } catch (IllegalArgumentException | JsonParseException e) {
+                if (ignoreMalformed.value() && parser.currentToken().isValue()) {
                     context.addIgnoredField(fieldType.name());
                     return;
                 } else {
diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java
index b4b9242daa456..77953c0903fd2 100644
--- a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java
+++ b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java
@@ -20,11 +20,14 @@
 package org.elasticsearch.index.mapper;
 
 import com.carrotsearch.randomizedtesting.annotations.Timeout;
+
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexableField;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressedXContent;
+import org.elasticsearch.common.xcontent.ToXContentObject;
+import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.index.mapper.NumberFieldMapper.NumberType;
@@ -37,6 +40,7 @@
 import java.util.HashSet;
 import java.util.List;
 
+import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 import static org.hamcrest.Matchers.containsString;
 
 public class NumberFieldMapperTests extends AbstractNumericFieldMapperTestCase {
@@ -218,45 +222,65 @@ protected void doTestDecimalCoerce(String type) throws IOException {
 
     public void testIgnoreMalformed() throws Exception {
         for (String type : TYPES) {
-            doTestIgnoreMalformed(type);
-        }
-    }
+            for (Object malformedValue : new Object[] { "a", Boolean.FALSE }) {
+                String mapping = Strings.toString(jsonBuilder().startObject().startObject("type").startObject("properties")
+                        .startObject("field").field("type", type).endObject().endObject().endObject().endObject());
 
-    private void doTestIgnoreMalformed(String type) throws IOException {
-        String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
-                .startObject("properties").startObject("field").field("type", type).endObject().endObject()
-                .endObject().endObject());
+                DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
 
-        DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
+                assertEquals(mapping, mapper.mappingSource().toString());
 
-        assertEquals(mapping, mapper.mappingSource().toString());
+                ThrowingRunnable runnable = () -> mapper.parse(new SourceToParse("test", "type", "1",
+                        BytesReference.bytes(jsonBuilder().startObject().field("field", malformedValue).endObject()), XContentType.JSON));
+                MapperParsingException e = expectThrows(MapperParsingException.class, runnable);
+                if (malformedValue instanceof String) {
+                    assertThat(e.getCause().getMessage(), containsString("For input string: \"a\""));
+                } else {
+                    assertThat(e.getCause().getMessage(), containsString("Current token"));
+                    assertThat(e.getCause().getMessage(), containsString("not numeric, can not use numeric value accessors"));
+                }
 
-        ThrowingRunnable runnable = () -> mapper.parse(new SourceToParse("test", "type", "1", BytesReference
-                .bytes(XContentFactory.jsonBuilder()
-                        .startObject()
-                        .field("field", "a")
-                        .endObject()),
-                XContentType.JSON));
-        MapperParsingException e = expectThrows(MapperParsingException.class, runnable);
-
-        assertThat(e.getCause().getMessage(), containsString("For input string: \"a\""));
+                mapping = Strings.toString(jsonBuilder().startObject().startObject("type").startObject("properties").startObject("field")
+                        .field("type", type).field("ignore_malformed", true).endObject().endObject().endObject().endObject());
 
-        mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
-                .startObject("properties").startObject("field").field("type", type).field("ignore_malformed", true).endObject().endObject()
-                .endObject().endObject());
+                DocumentMapper mapper2 = parser.parse("type", new CompressedXContent(mapping));
 
-        DocumentMapper mapper2 = parser.parse("type", new CompressedXContent(mapping));
+                ParsedDocument doc = mapper2.parse(new SourceToParse("test", "type", "1",
+                        BytesReference.bytes(jsonBuilder().startObject().field("field", malformedValue).endObject()), XContentType.JSON));
 
-        ParsedDocument doc = mapper2.parse(new SourceToParse("test", "type", "1", BytesReference
-                .bytes(XContentFactory.jsonBuilder()
-                        .startObject()
-                        .field("field", "a")
-                        .endObject()),
-                XContentType.JSON));
+                IndexableField[] fields = doc.rootDoc().getFields("field");
+                assertEquals(0, fields.length);
+                assertArrayEquals(new String[] { "field" }, doc.rootDoc().getValues("_ignored"));
+            }
+        }
+    }
 
-        IndexableField[] fields = doc.rootDoc().getFields("field");
-        assertEquals(0, fields.length);
-        assertArrayEquals(new String[] { "field" }, doc.rootDoc().getValues("_ignored"));
+    /**
+     * Test that in case the malformed value is an xContent object we throw error regardless of `ignore_malformed`
+     */
+    public void testIgnoreMalformedWithObject() throws Exception {
+        for (String type : TYPES) {
+            Object malformedValue = new ToXContentObject() {
+                @Override
+                public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+                    return builder.startObject().field("foo", "bar").endObject();
+                }
+            };
+            for (Boolean ignoreMalformed : new Boolean[] { true, false }) {
+                String mapping = Strings.toString(
+                        jsonBuilder().startObject().startObject("type").startObject("properties").startObject("field").field("type", type)
+                                .field("ignore_malformed", ignoreMalformed).endObject().endObject().endObject().endObject());
+                DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
+                assertEquals(mapping, mapper.mappingSource().toString());
+
+                MapperParsingException e = expectThrows(MapperParsingException.class,
+                        () -> mapper.parse(new SourceToParse("test", "type", "1",
+                                BytesReference.bytes(jsonBuilder().startObject().field("field", malformedValue).endObject()),
+                                XContentType.JSON)));
+                assertThat(e.getCause().getMessage(), containsString("Current token"));
+                assertThat(e.getCause().getMessage(), containsString("not numeric, can not use numeric value accessors"));
+            }
+        }
     }
 
     public void testRejectNorms() throws IOException {

From 1f51f20b3b007759588cc2c673168bad564de52f Mon Sep 17 00:00:00 2001
From: Zachary Tong 
Date: Wed, 17 Apr 2019 13:33:51 -0400
Subject: [PATCH 079/260] [Rollup] Validate timezones based on rules not string
 comparision (#36237)

The date_histogram internally converts obsolete timezones (such as
"Canada/Mountain") into their modern equivalent ("America/Edmonton").
But rollup just stored the TZ as provided by the user.

When checking the TZ for query validation we used a string comparison,
which would fail due to the date_histo's upgrading behavior.

Instead, we should convert both to a TimeZone object and check if their
rules are compatible.
---
 .../elasticsearch/common/time/DateUtils.java  | 121 +++++++
 .../rollup/job/DateHistogramGroupConfig.java  |  10 +-
 .../RollupDataExtractorFactory.java           |   4 +-
 .../rollup/RollupJobIdentifierUtils.java      |   9 +-
 .../xpack/rollup/RollupRequestTranslator.java |  52 +--
 .../action/TransportPutRollupJobAction.java   |  15 +
 .../action/TransportRollupSearchAction.java   |   4 +-
 .../xpack/rollup/job/RollupIndexer.java       |   9 -
 .../rollup/RollupJobIdentifierUtilTests.java  |  46 +++
 .../rollup/RollupRequestTranslationTests.java | 114 +------
 .../action/PutJobStateMachineTests.java       |  22 ++
 .../rollup/action/SearchActionTests.java      |   4 +-
 .../xpack/rollup/config/ConfigTests.java      |   5 +
 .../xpack/rollup/job/IndexerUtilsTests.java   |  84 +++++
 .../test/rollup/rollup_search.yml             | 304 ++++++++++++++++++
 15 files changed, 645 insertions(+), 158 deletions(-)

diff --git a/server/src/main/java/org/elasticsearch/common/time/DateUtils.java b/server/src/main/java/org/elasticsearch/common/time/DateUtils.java
index 910934a8f5069..e6bf6a65105b7 100644
--- a/server/src/main/java/org/elasticsearch/common/time/DateUtils.java
+++ b/server/src/main/java/org/elasticsearch/common/time/DateUtils.java
@@ -66,6 +66,127 @@ public static DateTimeZone zoneIdToDateTimeZone(ZoneId zoneId) {
         DEPRECATED_SHORT_TZ_IDS = tzs.keySet();
     }
 
+    // Map of deprecated timezones and their recommended new counterpart
+    public static final Map DEPRECATED_LONG_TIMEZONES;
+    static {
+        Map tzs = new HashMap<>();
+        tzs.put("Africa/Asmera","Africa/Nairobi");
+        tzs.put("Africa/Timbuktu","Africa/Abidjan");
+        tzs.put("America/Argentina/ComodRivadavia","America/Argentina/Catamarca");
+        tzs.put("America/Atka","America/Adak");
+        tzs.put("America/Buenos_Aires","America/Argentina/Buenos_Aires");
+        tzs.put("America/Catamarca","America/Argentina/Catamarca");
+        tzs.put("America/Coral_Harbour","America/Atikokan");
+        tzs.put("America/Cordoba","America/Argentina/Cordoba");
+        tzs.put("America/Ensenada","America/Tijuana");
+        tzs.put("America/Fort_Wayne","America/Indiana/Indianapolis");
+        tzs.put("America/Indianapolis","America/Indiana/Indianapolis");
+        tzs.put("America/Jujuy","America/Argentina/Jujuy");
+        tzs.put("America/Knox_IN","America/Indiana/Knox");
+        tzs.put("America/Louisville","America/Kentucky/Louisville");
+        tzs.put("America/Mendoza","America/Argentina/Mendoza");
+        tzs.put("America/Montreal","America/Toronto");
+        tzs.put("America/Porto_Acre","America/Rio_Branco");
+        tzs.put("America/Rosario","America/Argentina/Cordoba");
+        tzs.put("America/Santa_Isabel","America/Tijuana");
+        tzs.put("America/Shiprock","America/Denver");
+        tzs.put("America/Virgin","America/Port_of_Spain");
+        tzs.put("Antarctica/South_Pole","Pacific/Auckland");
+        tzs.put("Asia/Ashkhabad","Asia/Ashgabat");
+        tzs.put("Asia/Calcutta","Asia/Kolkata");
+        tzs.put("Asia/Chongqing","Asia/Shanghai");
+        tzs.put("Asia/Chungking","Asia/Shanghai");
+        tzs.put("Asia/Dacca","Asia/Dhaka");
+        tzs.put("Asia/Harbin","Asia/Shanghai");
+        tzs.put("Asia/Kashgar","Asia/Urumqi");
+        tzs.put("Asia/Katmandu","Asia/Kathmandu");
+        tzs.put("Asia/Macao","Asia/Macau");
+        tzs.put("Asia/Rangoon","Asia/Yangon");
+        tzs.put("Asia/Saigon","Asia/Ho_Chi_Minh");
+        tzs.put("Asia/Tel_Aviv","Asia/Jerusalem");
+        tzs.put("Asia/Thimbu","Asia/Thimphu");
+        tzs.put("Asia/Ujung_Pandang","Asia/Makassar");
+        tzs.put("Asia/Ulan_Bator","Asia/Ulaanbaatar");
+        tzs.put("Atlantic/Faeroe","Atlantic/Faroe");
+        tzs.put("Atlantic/Jan_Mayen","Europe/Oslo");
+        tzs.put("Australia/ACT","Australia/Sydney");
+        tzs.put("Australia/Canberra","Australia/Sydney");
+        tzs.put("Australia/LHI","Australia/Lord_Howe");
+        tzs.put("Australia/NSW","Australia/Sydney");
+        tzs.put("Australia/North","Australia/Darwin");
+        tzs.put("Australia/Queensland","Australia/Brisbane");
+        tzs.put("Australia/South","Australia/Adelaide");
+        tzs.put("Australia/Tasmania","Australia/Hobart");
+        tzs.put("Australia/Victoria","Australia/Melbourne");
+        tzs.put("Australia/West","Australia/Perth");
+        tzs.put("Australia/Yancowinna","Australia/Broken_Hill");
+        tzs.put("Brazil/Acre","America/Rio_Branco");
+        tzs.put("Brazil/DeNoronha","America/Noronha");
+        tzs.put("Brazil/East","America/Sao_Paulo");
+        tzs.put("Brazil/West","America/Manaus");
+        tzs.put("Canada/Atlantic","America/Halifax");
+        tzs.put("Canada/Central","America/Winnipeg");
+        tzs.put("Canada/East-Saskatchewan","America/Regina");
+        tzs.put("Canada/Eastern","America/Toronto");
+        tzs.put("Canada/Mountain","America/Edmonton");
+        tzs.put("Canada/Newfoundland","America/St_Johns");
+        tzs.put("Canada/Pacific","America/Vancouver");
+        tzs.put("Canada/Yukon","America/Whitehorse");
+        tzs.put("Chile/Continental","America/Santiago");
+        tzs.put("Chile/EasterIsland","Pacific/Easter");
+        tzs.put("Cuba","America/Havana");
+        tzs.put("Egypt","Africa/Cairo");
+        tzs.put("Eire","Europe/Dublin");
+        tzs.put("Europe/Belfast","Europe/London");
+        tzs.put("Europe/Tiraspol","Europe/Chisinau");
+        tzs.put("GB","Europe/London");
+        tzs.put("GB-Eire","Europe/London");
+        tzs.put("Greenwich","Etc/GMT");
+        tzs.put("Hongkong","Asia/Hong_Kong");
+        tzs.put("Iceland","Atlantic/Reykjavik");
+        tzs.put("Iran","Asia/Tehran");
+        tzs.put("Israel","Asia/Jerusalem");
+        tzs.put("Jamaica","America/Jamaica");
+        tzs.put("Japan","Asia/Tokyo");
+        tzs.put("Kwajalein","Pacific/Kwajalein");
+        tzs.put("Libya","Africa/Tripoli");
+        tzs.put("Mexico/BajaNorte","America/Tijuana");
+        tzs.put("Mexico/BajaSur","America/Mazatlan");
+        tzs.put("Mexico/General","America/Mexico_City");
+        tzs.put("NZ","Pacific/Auckland");
+        tzs.put("NZ-CHAT","Pacific/Chatham");
+        tzs.put("Navajo","America/Denver");
+        tzs.put("PRC","Asia/Shanghai");
+        tzs.put("Pacific/Johnston","Pacific/Honolulu");
+        tzs.put("Pacific/Ponape","Pacific/Pohnpei");
+        tzs.put("Pacific/Samoa","Pacific/Pago_Pago");
+        tzs.put("Pacific/Truk","Pacific/Chuuk");
+        tzs.put("Pacific/Yap","Pacific/Chuuk");
+        tzs.put("Poland","Europe/Warsaw");
+        tzs.put("Portugal","Europe/Lisbon");
+        tzs.put("ROC","Asia/Taipei");
+        tzs.put("ROK","Asia/Seoul");
+        tzs.put("Singapore","Asia/Singapore");
+        tzs.put("Turkey","Europe/Istanbul");
+        tzs.put("UCT","Etc/UCT");
+        tzs.put("US/Alaska","America/Anchorage");
+        tzs.put("US/Aleutian","America/Adak");
+        tzs.put("US/Arizona","America/Phoenix");
+        tzs.put("US/Central","America/Chicago");
+        tzs.put("US/East-Indiana","America/Indiana/Indianapolis");
+        tzs.put("US/Eastern","America/New_York");
+        tzs.put("US/Hawaii","Pacific/Honolulu");
+        tzs.put("US/Indiana-Starke","America/Indiana/Knox");
+        tzs.put("US/Michigan","America/Detroit");
+        tzs.put("US/Mountain","America/Denver");
+        tzs.put("US/Pacific","America/Los_Angeles");
+        tzs.put("US/Samoa","Pacific/Pago_Pago");
+        tzs.put("Universal","Etc/UTC");
+        tzs.put("W-SU","Europe/Moscow");
+        tzs.put("Zulu","Etc/UTC");
+        DEPRECATED_LONG_TIMEZONES = Collections.unmodifiableMap(tzs);
+    }
+
     public static ZoneId dateTimeZoneToZoneId(DateTimeZone timeZone) {
         if (timeZone == null) {
             return null;
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/DateHistogramGroupConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/DateHistogramGroupConfig.java
index f4fee8acc3d1f..c9fe0c644a86b 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/DateHistogramGroupConfig.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/DateHistogramGroupConfig.java
@@ -24,6 +24,7 @@
 
 import java.io.IOException;
 import java.time.ZoneId;
+import java.time.ZoneOffset;
 import java.util.Map;
 import java.util.Objects;
 
@@ -52,7 +53,8 @@ public class DateHistogramGroupConfig implements Writeable, ToXContentObject {
     private static final String FIELD = "field";
     public static final String TIME_ZONE = "time_zone";
     public static final String DELAY = "delay";
-    private static final String DEFAULT_TIMEZONE = "UTC";
+    public static final String DEFAULT_TIMEZONE = "UTC";
+    public static final ZoneId DEFAULT_ZONEID_TIMEZONE = ZoneOffset.UTC;
     private static final ConstructingObjectParser PARSER;
     static {
         PARSER = new ConstructingObjectParser<>(NAME, a ->
@@ -210,12 +212,12 @@ public boolean equals(final Object other) {
         return Objects.equals(interval, that.interval)
             && Objects.equals(field, that.field)
             && Objects.equals(delay, that.delay)
-            && Objects.equals(timeZone, that.timeZone);
+            && ZoneId.of(timeZone, ZoneId.SHORT_IDS).getRules().equals(ZoneId.of(that.timeZone, ZoneId.SHORT_IDS).getRules());
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(interval, field, delay, timeZone);
+        return Objects.hash(interval, field, delay, ZoneId.of(timeZone));
     }
 
     @Override
@@ -235,7 +237,7 @@ private static Rounding createRounding(final String expr, final String timeZone)
         } else {
             rounding = new Rounding.Builder(TimeValue.parseTimeValue(expr, "createRounding"));
         }
-        rounding.timeZone(ZoneId.of(timeZone));
+        rounding.timeZone(ZoneId.of(timeZone, ZoneId.SHORT_IDS));
         return rounding.build();
     }
 }
diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/RollupDataExtractorFactory.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/RollupDataExtractorFactory.java
index 4971ad838799d..8264d3e15fd59 100644
--- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/RollupDataExtractorFactory.java
+++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/RollupDataExtractorFactory.java
@@ -23,6 +23,8 @@
 import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
 import org.elasticsearch.xpack.ml.datafeed.extractor.DataExtractorFactory;
 
+import java.time.ZoneId;
+import java.time.ZoneOffset;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -124,7 +126,7 @@ private static boolean validInterval(long datafeedInterval, ParsedRollupCaps rol
         if (rollupJobGroupConfig.hasDatehistogram() == false) {
             return false;
         }
-        if ("UTC".equalsIgnoreCase(rollupJobGroupConfig.getTimezone()) == false) {
+        if (ZoneId.of(rollupJobGroupConfig.getTimezone()).getRules().equals(ZoneOffset.UTC.getRules()) == false) {
             return false;
         }
         try {
diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupJobIdentifierUtils.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupJobIdentifierUtils.java
index 59141d2a83aeb..95b5069edcf88 100644
--- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupJobIdentifierUtils.java
+++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupJobIdentifierUtils.java
@@ -17,6 +17,7 @@
 import org.elasticsearch.xpack.core.rollup.action.RollupJobCaps;
 import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
 
+import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -96,11 +97,13 @@ private static void checkDateHisto(DateHistogramAggregationBuilder source, List<
                     if (agg.get(RollupField.AGG).equals(DateHistogramAggregationBuilder.NAME)) {
                         DateHistogramInterval interval = new DateHistogramInterval((String)agg.get(RollupField.INTERVAL));
 
-                        String thisTimezone  = (String)agg.get(DateHistogramGroupConfig.TIME_ZONE);
-                        String sourceTimeZone = source.timeZone() == null ? "UTC" : source.timeZone().toString();
+                        ZoneId thisTimezone = ZoneId.of(((String) agg.get(DateHistogramGroupConfig.TIME_ZONE)), ZoneId.SHORT_IDS);
+                        ZoneId sourceTimeZone = source.timeZone() == null
+                            ? DateHistogramGroupConfig.DEFAULT_ZONEID_TIMEZONE
+                            : ZoneId.of(source.timeZone().toString(), ZoneId.SHORT_IDS);
 
                         // Ensure we are working on the same timezone
-                        if (thisTimezone.equalsIgnoreCase(sourceTimeZone) == false) {
+                        if (thisTimezone.getRules().equals(sourceTimeZone.getRules()) == false) {
                             continue;
                         }
                         if (source.dateHistogramInterval() != null) {
diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupRequestTranslator.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupRequestTranslator.java
index 4546268119884..7cf8f8d1293e1 100644
--- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupRequestTranslator.java
+++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/RollupRequestTranslator.java
@@ -11,8 +11,6 @@
 import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.TermQueryBuilder;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
 import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
@@ -22,8 +20,8 @@
 import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
 import org.elasticsearch.xpack.core.rollup.RollupField;
 import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig;
-import org.joda.time.DateTimeZone;
 
+import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -47,7 +45,7 @@
  * }
* * - * The only publicly "consumable" API is {@link #translateAggregation(AggregationBuilder, List, NamedWriteableRegistry)}. + * The only publicly "consumable" API is {@link #translateAggregation(AggregationBuilder, NamedWriteableRegistry)}. */ public class RollupRequestTranslator { @@ -116,26 +114,22 @@ public class RollupRequestTranslator { * relevant method below. * * @param source The source aggregation to translate into rollup-enabled version - * @param filterConditions A list used to track any filter conditions that sub-aggs may - * require. * @param registry Registry containing the various aggregations so that we can easily * deserialize into a stream for cloning * @return Returns the fully translated aggregation tree. Note that it returns a list instead * of a single AggBuilder, since some aggregations (e.g. avg) may result in two * translated aggs (sum + count) */ - public static List translateAggregation(AggregationBuilder source, - List filterConditions, - NamedWriteableRegistry registry) { + public static List translateAggregation(AggregationBuilder source, NamedWriteableRegistry registry) { if (source.getWriteableName().equals(DateHistogramAggregationBuilder.NAME)) { - return translateDateHistogram((DateHistogramAggregationBuilder) source, filterConditions, registry); + return translateDateHistogram((DateHistogramAggregationBuilder) source, registry); } else if (source.getWriteableName().equals(HistogramAggregationBuilder.NAME)) { - return translateHistogram((HistogramAggregationBuilder) source, filterConditions, registry); + return translateHistogram((HistogramAggregationBuilder) source, registry); } else if (RollupField.SUPPORTED_METRICS.contains(source.getWriteableName())) { return translateVSLeaf((ValuesSourceAggregationBuilder.LeafOnly)source, registry); } else if (source.getWriteableName().equals(TermsAggregationBuilder.NAME)) { - return translateTerms((TermsAggregationBuilder)source, filterConditions, registry); + return translateTerms((TermsAggregationBuilder)source, registry); } else { throw new IllegalArgumentException("Unable to translate aggregation tree into Rollup. Aggregation [" + source.getName() + "] is of type [" + source.getClass().getSimpleName() + "] which is " + @@ -195,22 +189,13 @@ public static List translateAggregation(AggregationBuilder s *
  • Field: `{timestamp field}.date_histogram._count`
  • * * - *
  • Add a filter condition:
  • - *
  • - *
      - *
    • Query type: TermQuery
    • - *
    • Field: `{timestamp_field}.date_histogram.interval`
    • - *
    • Value: `{source interval}`
    • - *
    - *
  • * * */ private static List translateDateHistogram(DateHistogramAggregationBuilder source, - List filterConditions, NamedWriteableRegistry registry) { - return translateVSAggBuilder(source, filterConditions, registry, () -> { + return translateVSAggBuilder(source, registry, () -> { DateHistogramAggregationBuilder rolledDateHisto = new DateHistogramAggregationBuilder(source.getName()); @@ -220,13 +205,9 @@ private static List translateDateHistogram(DateHistogramAggr rolledDateHisto.interval(source.interval()); } - String timezone = source.timeZone() == null ? DateTimeZone.UTC.toString() : source.timeZone().toString(); - filterConditions.add(new TermQueryBuilder(RollupField.formatFieldName(source, - DateHistogramGroupConfig.TIME_ZONE), timezone)); + ZoneId timeZone = source.timeZone() == null ? DateHistogramGroupConfig.DEFAULT_ZONEID_TIMEZONE : source.timeZone(); + rolledDateHisto.timeZone(timeZone); - if (source.timeZone() != null) { - rolledDateHisto.timeZone(source.timeZone()); - } rolledDateHisto.offset(source.offset()); if (source.extendedBounds() != null) { rolledDateHisto.extendedBounds(source.extendedBounds()); @@ -248,14 +229,13 @@ private static List translateDateHistogram(DateHistogramAggr * Notably, it adds a Sum metric to calculate the doc_count in each bucket. * * Conventions are identical to a date_histogram (excepting date-specific details), so see - * {@link #translateDateHistogram(DateHistogramAggregationBuilder, List, NamedWriteableRegistry)} for + * {@link #translateDateHistogram(DateHistogramAggregationBuilder, NamedWriteableRegistry)} for * a complete list of conventions, examples, etc */ private static List translateHistogram(HistogramAggregationBuilder source, - List filterConditions, NamedWriteableRegistry registry) { - return translateVSAggBuilder(source, filterConditions, registry, () -> { + return translateVSAggBuilder(source, registry, () -> { HistogramAggregationBuilder rolledHisto = new HistogramAggregationBuilder(source.getName()); @@ -328,10 +308,9 @@ private static List translateHistogram(HistogramAggregationB * */ private static List translateTerms(TermsAggregationBuilder source, - List filterConditions, NamedWriteableRegistry registry) { - return translateVSAggBuilder(source, filterConditions, registry, () -> { + return translateVSAggBuilder(source, registry, () -> { TermsAggregationBuilder rolledTerms = new TermsAggregationBuilder(source.getName(), source.valueType()); rolledTerms.field(RollupField.formatFieldName(source, RollupField.VALUE)); @@ -359,8 +338,6 @@ private static List translateTerms(TermsAggregationBuilder s * ValueSourceBuilder. This method is called by all the agg-specific methods (e.g. translateDateHistogram()) * * @param source The source aggregation that we wish to translate - * @param filterConditions A list of existing filter conditions, in case we need to add some - * for this particular agg * @param registry Named registry for serializing leaf metrics. Not actually used by this method, * but is passed downwards for leaf usage * @param factory A factory closure that generates a new shallow clone of the `source`. E.g. if `source` is @@ -371,15 +348,14 @@ private static List translateTerms(TermsAggregationBuilder s * @return the translated multi-bucket ValueSourceAggBuilder */ private static List - translateVSAggBuilder(ValuesSourceAggregationBuilder source, List filterConditions, - NamedWriteableRegistry registry, Supplier factory) { + translateVSAggBuilder(ValuesSourceAggregationBuilder source, NamedWriteableRegistry registry, Supplier factory) { T rolled = factory.get(); // Translate all subaggs and add to the newly translated agg // NOTE: using for loop instead of stream because compiler explodes with a bug :/ for (AggregationBuilder subAgg : source.getSubAggregations()) { - List translated = translateAggregation(subAgg, filterConditions, registry); + List translated = translateAggregation(subAgg, registry); for (AggregationBuilder t : translated) { rolled.subAggregation(t); } 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 cb04f5554b437..db6c1c5ddeaa7 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 @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.rollup.action; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; @@ -32,6 +33,8 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.license.LicenseUtils; @@ -57,6 +60,8 @@ public class TransportPutRollupJobAction extends TransportMasterNodeAction filteredHeaders = threadPool.getThreadContext().getHeaders().entrySet().stream() diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportRollupSearchAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportRollupSearchAction.java index e85a92c061366..414a0d08ef35a 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportRollupSearchAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportRollupSearchAction.java @@ -173,10 +173,12 @@ static MultiSearchRequest createMSearchRequest(SearchRequest request, NamedWrite for (AggregationBuilder agg : sourceAgg.getAggregatorFactories()) { + // TODO this filter agg is now redundant given we filter on job ID + // in the query and the translator doesn't add any clauses anymore List filterConditions = new ArrayList<>(5); // Translate the agg tree, and collect any potential filtering clauses - List translatedAgg = RollupRequestTranslator.translateAggregation(agg, filterConditions, registry); + List translatedAgg = RollupRequestTranslator.translateAggregation(agg, registry); BoolQueryBuilder boolQuery = new BoolQueryBuilder(); filterConditions.forEach(boolQuery::must); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index e051e912c482b..daa888562e94f 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -41,7 +41,6 @@ import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.core.rollup.job.TermsGroupConfig; -import org.joda.time.DateTimeZone; import java.time.ZoneId; import java.util.ArrayList; @@ -311,13 +310,5 @@ static List createAggregationBuilders(final List caps = singletonSet(cap); + + DateHistogramAggregationBuilder builder = new DateHistogramAggregationBuilder("foo").field("foo") + .dateHistogramInterval(job.getGroupConfig().getDateHistogram().getInterval()) + .timeZone(ZoneId.of("Canada/Mountain")); + + Set bestCaps = RollupJobIdentifierUtils.findBestJobs(builder, caps); + assertThat(bestCaps.size(), equalTo(1)); + + builder = new DateHistogramAggregationBuilder("foo").field("foo") + .dateHistogramInterval(job.getGroupConfig().getDateHistogram().getInterval()) + .timeZone(ZoneId.of("America/Edmonton")); + + bestCaps = RollupJobIdentifierUtils.findBestJobs(builder, caps); + assertThat(bestCaps.size(), equalTo(1)); + + // now the reverse, job has "new" timezone + + dateHisto = new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h"), null, "America/Edmonton"); + group = new GroupConfig(dateHisto); + job = new RollupJobConfig("foo", "index", "rollup", "*/5 * * * * ?", 10, group, emptyList(), null); + cap = new RollupJobCaps(job); + caps = singletonSet(cap); + + builder = new DateHistogramAggregationBuilder("foo").field("foo") + .dateHistogramInterval(job.getGroupConfig().getDateHistogram().getInterval()) + .timeZone(ZoneId.of("Canada/Mountain")); + + bestCaps = RollupJobIdentifierUtils.findBestJobs(builder, caps); + assertThat(bestCaps.size(), equalTo(1)); + + builder = new DateHistogramAggregationBuilder("foo").field("foo") + .dateHistogramInterval(job.getGroupConfig().getDateHistogram().getInterval()) + .timeZone(ZoneId.of("America/Edmonton")); + + bestCaps = RollupJobIdentifierUtils.findBestJobs(builder, caps); + assertThat(bestCaps.size(), equalTo(1)); + } + private static long getMillis(RollupJobCaps cap) { for (RollupJobCaps.RollupFieldCaps fieldCaps : cap.getFieldCaps().values()) { for (Map agg : fieldCaps.getAggs()) { diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupRequestTranslationTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupRequestTranslationTests.java index f691d10d20dc7..db58115489d2a 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupRequestTranslationTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/RollupRequestTranslationTests.java @@ -9,8 +9,6 @@ import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; @@ -33,7 +31,6 @@ import java.io.IOException; import java.time.ZoneId; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -65,9 +62,8 @@ public void testBasicDateHisto() { .extendedBounds(new ExtendedBounds(0L, 1000L)) .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")) .subAggregation(new AvgAggregationBuilder("the_avg").field("avg_field")); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(histo, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(histo, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), Matchers.instanceOf(DateHistogramAggregationBuilder.class)); DateHistogramAggregationBuilder translatedHisto = (DateHistogramAggregationBuilder)translated.get(0); @@ -93,22 +89,6 @@ public void testBasicDateHisto() { assertThat(subAggs.get("test_histo._count"), Matchers.instanceOf(SumAggregationBuilder.class)); assertThat(((SumAggregationBuilder)subAggs.get("test_histo._count")).field(), equalTo("foo.date_histogram._count")); - - assertThat(filterConditions.size(), equalTo(1)); - for (QueryBuilder q : filterConditions) { - if (q instanceof TermQueryBuilder) { - switch (((TermQueryBuilder) q).fieldName()) { - case "foo.date_histogram.time_zone": - assertThat(((TermQueryBuilder) q).value(), equalTo("UTC")); - break; - default: - fail("Unexpected Term Query in filter conditions: [" + ((TermQueryBuilder) q).fieldName() + "]"); - break; - } - } else { - fail("Unexpected query builder in filter conditions"); - } - } } public void testFormattedDateHisto() { @@ -118,9 +98,8 @@ public void testFormattedDateHisto() { .extendedBounds(new ExtendedBounds(0L, 1000L)) .format("yyyy-MM-dd") .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(histo, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(histo, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), Matchers.instanceOf(DateHistogramAggregationBuilder.class)); DateHistogramAggregationBuilder translatedHisto = (DateHistogramAggregationBuilder)translated.get(0); @@ -133,7 +112,6 @@ public void testFormattedDateHisto() { public void testSimpleMetric() { int i = ESTestCase.randomIntBetween(0, 2); List translated = new ArrayList<>(); - List filterConditions = new ArrayList<>(); Class clazz = null; String fieldName = null; @@ -141,17 +119,17 @@ public void testSimpleMetric() { if (i == 0) { translated = translateAggregation(new MaxAggregationBuilder("test_metric") - .field("foo"), filterConditions, namedWriteableRegistry); + .field("foo"), namedWriteableRegistry); clazz = MaxAggregationBuilder.class; fieldName = "foo.max.value"; } else if (i == 1) { translated = translateAggregation(new MinAggregationBuilder("test_metric") - .field("foo"), filterConditions, namedWriteableRegistry); + .field("foo"), namedWriteableRegistry); clazz = MinAggregationBuilder.class; fieldName = "foo.min.value"; } else if (i == 2) { translated = translateAggregation(new SumAggregationBuilder("test_metric") - .field("foo"), filterConditions, namedWriteableRegistry); + .field("foo"), namedWriteableRegistry); clazz = SumAggregationBuilder.class; fieldName = "foo.sum.value"; } @@ -160,14 +138,12 @@ public void testSimpleMetric() { assertThat(translated.get(0), Matchers.instanceOf(clazz)); assertThat((translated.get(0)).getName(), equalTo("test_metric")); assertThat(((ValuesSourceAggregationBuilder)translated.get(0)).field(), equalTo(fieldName)); - - assertThat(filterConditions.size(), equalTo(0)); } public void testUnsupportedMetric() { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> translateAggregation(new StatsAggregationBuilder("test_metric") - .field("foo"), Collections.emptyList(), namedWriteableRegistry)); + .field("foo"), namedWriteableRegistry)); assertThat(e.getMessage(), equalTo("Unable to translate aggregation tree into Rollup. Aggregation [test_metric] is of type " + "[StatsAggregationBuilder] which is currently unsupported.")); } @@ -178,9 +154,8 @@ public void testDateHistoIntervalWithMinMax() { .field("foo") .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")) .subAggregation(new AvgAggregationBuilder("the_avg").field("avg_field")); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(histo, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(histo, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), instanceOf(DateHistogramAggregationBuilder.class)); DateHistogramAggregationBuilder translatedHisto = (DateHistogramAggregationBuilder)translated.get(0); @@ -206,20 +181,6 @@ public void testDateHistoIntervalWithMinMax() { assertThat(subAggs.get("test_histo._count"), instanceOf(SumAggregationBuilder.class)); assertThat(((SumAggregationBuilder)subAggs.get("test_histo._count")).field(), equalTo("foo.date_histogram._count")); - - assertThat(filterConditions.size(), equalTo(1)); - - for (QueryBuilder q : filterConditions) { - if (q instanceof TermQueryBuilder) { - if (((TermQueryBuilder) q).fieldName().equals("foo.date_histogram.time_zone")) { - assertThat(((TermQueryBuilder) q).value(), equalTo("UTC")); - } else { - fail("Unexpected Term Query in filter conditions: [" + ((TermQueryBuilder) q).fieldName() + "]"); - } - } else { - fail("Unexpected query builder in filter conditions"); - } - } } public void testDateHistoLongIntervalWithMinMax() { @@ -228,9 +189,8 @@ public void testDateHistoLongIntervalWithMinMax() { .field("foo") .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")) .subAggregation(new AvgAggregationBuilder("the_avg").field("avg_field")); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(histo, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(histo, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), instanceOf(DateHistogramAggregationBuilder.class)); DateHistogramAggregationBuilder translatedHisto = (DateHistogramAggregationBuilder)translated.get(0); @@ -256,20 +216,6 @@ public void testDateHistoLongIntervalWithMinMax() { assertThat(subAggs.get("test_histo._count"), instanceOf(SumAggregationBuilder.class)); assertThat(((SumAggregationBuilder)subAggs.get("test_histo._count")).field(), equalTo("foo.date_histogram._count")); - - assertThat(filterConditions.size(), equalTo(1)); - - for (QueryBuilder q : filterConditions) { - if (q instanceof TermQueryBuilder) { - if (((TermQueryBuilder) q).fieldName().equals("foo.date_histogram.time_zone")) { - assertThat(((TermQueryBuilder) q).value(), equalTo("UTC")); - } else { - fail("Unexpected Term Query in filter conditions: [" + ((TermQueryBuilder) q).fieldName() + "]"); - } - } else { - fail("Unexpected query builder in filter conditions"); - } - } } public void testDateHistoWithTimezone() { @@ -278,9 +224,8 @@ public void testDateHistoWithTimezone() { histo.interval(86400000) .field("foo") .timeZone(timeZone); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(histo, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(histo, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), instanceOf(DateHistogramAggregationBuilder.class)); DateHistogramAggregationBuilder translatedHisto = (DateHistogramAggregationBuilder)translated.get(0); @@ -288,25 +233,11 @@ public void testDateHistoWithTimezone() { assertThat(translatedHisto.interval(), equalTo(86400000L)); assertThat(translatedHisto.field(), equalTo("foo.date_histogram.timestamp")); assertThat(translatedHisto.timeZone(), equalTo(timeZone)); - assertThat(filterConditions.size(), equalTo(1)); - - for (QueryBuilder q : filterConditions) { - if (q instanceof TermQueryBuilder) { - if (((TermQueryBuilder) q).fieldName().equals("foo.date_histogram.time_zone")) { - assertThat(((TermQueryBuilder) q).value(), equalTo(timeZone.toString())); - } else { - fail("Unexpected Term Query in filter conditions: [" + ((TermQueryBuilder) q).fieldName() + "]"); - } - } else { - fail("Unexpected query builder in filter conditions"); - } - } } public void testAvgMetric() { - List filterConditions = new ArrayList<>(); List translated = translateAggregation(new AvgAggregationBuilder("test_metric") - .field("foo"), filterConditions, namedWriteableRegistry); + .field("foo"), namedWriteableRegistry); assertThat(translated.size(), equalTo(2)); Map metrics = translated.stream() @@ -319,8 +250,6 @@ public void testAvgMetric() { assertThat(metrics.get("test_metric._count"), Matchers.instanceOf(SumAggregationBuilder.class)); assertThat(((SumAggregationBuilder)metrics.get("test_metric._count")).field(), equalTo("foo.avg._count")); - - assertThat(filterConditions.size(), equalTo(0)); } public void testStringTerms() throws IOException { @@ -329,9 +258,8 @@ public void testStringTerms() throws IOException { terms.field("foo") .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")) .subAggregation(new AvgAggregationBuilder("the_avg").field("avg_field")); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(terms, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(terms, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), Matchers.instanceOf(TermsAggregationBuilder.class)); TermsAggregationBuilder translatedHisto = (TermsAggregationBuilder)translated.get(0); @@ -356,8 +284,6 @@ public void testStringTerms() throws IOException { assertThat(subAggs.get("test_string_terms._count"), Matchers.instanceOf(SumAggregationBuilder.class)); assertThat(((SumAggregationBuilder)subAggs.get("test_string_terms._count")).field(), equalTo("foo.terms._count")); - - assertThat(filterConditions.size(), equalTo(0)); } public void testBasicHisto() { @@ -368,9 +294,8 @@ public void testBasicHisto() { .extendedBounds(0.0, 1000.0) .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")) .subAggregation(new AvgAggregationBuilder("the_avg").field("avg_field")); - List filterConditions = new ArrayList<>(); - List translated = translateAggregation(histo, filterConditions, namedWriteableRegistry); + List translated = translateAggregation(histo, namedWriteableRegistry); assertThat(translated.size(), equalTo(1)); assertThat(translated.get(0), Matchers.instanceOf(HistogramAggregationBuilder.class)); HistogramAggregationBuilder translatedHisto = (HistogramAggregationBuilder)translated.get(0); @@ -396,18 +321,6 @@ public void testBasicHisto() { assertThat(((SumAggregationBuilder)subAggs.get("test_histo._count")).field(), equalTo("foo.histogram._count")); - assertThat(filterConditions.size(), equalTo(0)); - for (QueryBuilder q : filterConditions) { - if (q instanceof TermQueryBuilder) { - switch (((TermQueryBuilder) q).fieldName()) { - default: - fail("Unexpected Term Query in filter conditions: [" + ((TermQueryBuilder) q).fieldName() + "]"); - break; - } - } else { - fail("Unexpected query builder in filter conditions"); - } - } } public void testUnsupportedAgg() { @@ -415,10 +328,9 @@ public void testUnsupportedAgg() { geo.field("foo") .subAggregation(new MaxAggregationBuilder("the_max").field("max_field")) .subAggregation(new AvgAggregationBuilder("the_avg").field("avg_field")); - List filterConditions = new ArrayList<>(); Exception e = expectThrows(RuntimeException.class, - () -> translateAggregation(geo, filterConditions, namedWriteableRegistry)); + () -> translateAggregation(geo, namedWriteableRegistry)); assertThat(e.getMessage(), equalTo("Unable to translate aggregation tree into Rollup. Aggregation [test_geo] is of type " + "[GeoDistanceAggregationBuilder] which is currently unsupported.")); } diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/PutJobStateMachineTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/PutJobStateMachineTests.java index 3d346456ea98d..3f49609953ea9 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/PutJobStateMachineTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/PutJobStateMachineTests.java @@ -23,9 +23,13 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.persistent.PersistentTasksService; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.rollup.ConfigTestHelpers; import org.elasticsearch.xpack.core.rollup.RollupField; +import org.elasticsearch.xpack.core.rollup.action.PutRollupJobAction; +import org.elasticsearch.xpack.core.rollup.job.DateHistogramGroupConfig; +import org.elasticsearch.xpack.core.rollup.job.GroupConfig; import org.elasticsearch.xpack.core.rollup.job.RollupJob; import org.elasticsearch.xpack.core.rollup.job.RollupJobConfig; import org.elasticsearch.xpack.rollup.Rollup; @@ -424,4 +428,22 @@ public void testStartTask() { verify(tasksService).sendStartRequest(eq(job.getConfig().getId()), eq(RollupField.TASK_NAME), eq(job), any()); verify(tasksService).waitForPersistentTaskCondition(eq(job.getConfig().getId()), any(), any(), any()); } + + public void testDeprecatedTimeZone() { + GroupConfig groupConfig = new GroupConfig(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h"), null, "Japan")); + RollupJobConfig config = new RollupJobConfig("foo", randomAlphaOfLength(5), "rollup", ConfigTestHelpers.randomCron(), + 100, groupConfig, Collections.emptyList(), null); + PutRollupJobAction.Request request = new PutRollupJobAction.Request(config); + TransportPutRollupJobAction.checkForDeprecatedTZ(request); + assertWarnings("Creating Rollup job [foo] with timezone [Japan], but [Japan] has been deprecated by the IANA. " + + "Use [Asia/Tokyo] instead."); + } + + public void testTimeZone() { + GroupConfig groupConfig = new GroupConfig(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h"), null, "EST")); + RollupJobConfig config = new RollupJobConfig("foo", randomAlphaOfLength(5), "rollup", ConfigTestHelpers.randomCron(), + 100, groupConfig, Collections.emptyList(), null); + PutRollupJobAction.Request request = new PutRollupJobAction.Request(config); + TransportPutRollupJobAction.checkForDeprecatedTZ(request); + } } diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/SearchActionTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/SearchActionTests.java index 5a851d17e5eaf..a795edca83ed3 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/SearchActionTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/action/SearchActionTests.java @@ -118,7 +118,7 @@ public void testBadQuery() { assertThat(e.getMessage(), equalTo("Unsupported Query in search request: [match_phrase]")); } - public void testRange() { + public void testRangeTimezoneUTC() { final GroupConfig groupConfig = new GroupConfig(new DateHistogramGroupConfig("foo", new DateHistogramInterval("1h"))); final RollupJobConfig config = new RollupJobConfig("foo", "index", "rollup", "*/5 * * * * ?", 10, groupConfig, emptyList(), null); RollupJobCaps cap = new RollupJobCaps(config); @@ -127,6 +127,7 @@ public void testRange() { QueryBuilder rewritten = TransportRollupSearchAction.rewriteQuery(new RangeQueryBuilder("foo").gt(1).timeZone("UTC"), caps); assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); assertThat(((RangeQueryBuilder)rewritten).fieldName(), equalTo("foo.date_histogram.timestamp")); + assertThat(((RangeQueryBuilder)rewritten).timeZone(), equalTo("UTC")); } public void testRangeNullTimeZone() { @@ -138,6 +139,7 @@ public void testRangeNullTimeZone() { QueryBuilder rewritten = TransportRollupSearchAction.rewriteQuery(new RangeQueryBuilder("foo").gt(1), caps); assertThat(rewritten, instanceOf(RangeQueryBuilder.class)); assertThat(((RangeQueryBuilder)rewritten).fieldName(), equalTo("foo.date_histogram.timestamp")); + assertNull(((RangeQueryBuilder)rewritten).timeZone()); } public void testRangeDifferentTZ() { diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/config/ConfigTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/config/ConfigTests.java index 9f8796f4c9589..2f0612a65d2dc 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/config/ConfigTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/config/ConfigTests.java @@ -90,6 +90,11 @@ public void testUnknownTimeZone() { assertThat(e.getMessage(), equalTo("Unknown time-zone ID: FOO")); } + public void testObsoleteTimeZone() { + DateHistogramGroupConfig config = new DateHistogramGroupConfig("foo", DateHistogramInterval.HOUR, null, "Canada/Mountain"); + assertThat(config.getTimeZone(), equalTo("Canada/Mountain")); + } + public void testEmptyHistoField() { Exception e = expectThrows(IllegalArgumentException.class, () -> new HistogramGroupConfig(1L, (String[]) null)); assertThat(e.getMessage(), equalTo("Fields must have at least one value")); diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index cbf85e84b16c3..38b90328a8743 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -47,6 +47,7 @@ import org.mockito.stubbing.Answer; import java.io.IOException; +import java.time.ZoneId; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -561,6 +562,89 @@ public void testMissingBuckets() throws IOException { } } + public void testTimezone() throws IOException { + String indexName = randomAlphaOfLengthBetween(1, 10); + RollupIndexerJobStats stats = new RollupIndexerJobStats(0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + + String timestampField = "the_histo"; + String valueField = "the_avg"; + + Directory directory = newDirectory(); + RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory); + + { + Document document = new Document(); + long timestamp = 1443659400000L; // 2015-10-01T00:30:00Z + document.add(new SortedNumericDocValuesField(timestampField, timestamp)); + document.add(new LongPoint(timestampField, timestamp)); + document.add(new SortedNumericDocValuesField(valueField, randomIntBetween(1, 100))); + indexWriter.addDocument(document); + } + { + Document document = new Document(); + long timestamp = 1443663000000L; // 2015-10-01T01:30:00Z + document.add(new SortedNumericDocValuesField(timestampField, timestamp)); + document.add(new LongPoint(timestampField, timestamp)); + document.add(new SortedNumericDocValuesField(valueField, randomIntBetween(1, 100))); + indexWriter.addDocument(document); + } + indexWriter.close(); + + IndexReader indexReader = DirectoryReader.open(directory); + IndexSearcher indexSearcher = newIndexSearcher(indexReader); + + DateFieldMapper.Builder builder = new DateFieldMapper.Builder(timestampField); + DateFieldMapper.DateFieldType timestampFieldType = builder.fieldType(); + timestampFieldType.setHasDocValues(true); + timestampFieldType.setName(timestampField); + + MappedFieldType valueFieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + valueFieldType.setName(valueField); + valueFieldType.setHasDocValues(true); + valueFieldType.setName(valueField); + + // Setup the composite agg + DateHistogramValuesSourceBuilder dateHisto + = new DateHistogramValuesSourceBuilder("the_histo." + DateHistogramAggregationBuilder.NAME) + .field(timestampField) + .dateHistogramInterval(new DateHistogramInterval("1d")) + .timeZone(ZoneId.of("-01:00", ZoneId.SHORT_IDS)); // adds a timezone so that we aren't on default UTC + + CompositeAggregationBuilder compositeBuilder = new CompositeAggregationBuilder(RollupIndexer.AGGREGATION_NAME, + singletonList(dateHisto)); + + MetricConfig metricConfig = new MetricConfig(valueField, singletonList("max")); + List metricAgg = createAggregationBuilders(singletonList(metricConfig)); + metricAgg.forEach(compositeBuilder::subAggregation); + + Aggregator aggregator = createAggregator(compositeBuilder, indexSearcher, timestampFieldType, valueFieldType); + aggregator.preCollection(); + indexSearcher.search(new MatchAllDocsQuery(), aggregator); + aggregator.postCollection(); + CompositeAggregation composite = (CompositeAggregation) aggregator.buildAggregation(0L); + indexReader.close(); + directory.close(); + + final GroupConfig groupConfig = randomGroupConfig(random()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + + assertThat(docs.size(), equalTo(2)); + + Map map = docs.get(0).sourceAsMap(); + assertNotNull(map.get(valueField + "." + MaxAggregationBuilder.NAME + "." + RollupField.VALUE)); + assertThat(map.get("the_histo." + DateHistogramAggregationBuilder.NAME + "." + RollupField.COUNT_FIELD), equalTo(1)); + assertThat(map.get("the_histo." + DateHistogramAggregationBuilder.NAME + "." + RollupField.TIMESTAMP), + equalTo(1443574800000L)); // 2015-09-30T00:00:00.000-01:00 + + map = docs.get(1).sourceAsMap(); + assertNotNull(map.get(valueField + "." + MaxAggregationBuilder.NAME + "." + RollupField.VALUE)); + assertThat(map.get("the_histo." + DateHistogramAggregationBuilder.NAME + "." + RollupField.COUNT_FIELD), equalTo(1)); + assertThat(map.get("the_histo." + DateHistogramAggregationBuilder.NAME + "." + RollupField.TIMESTAMP), + equalTo(1443661200000L)); // 2015-10-01T00:00:00.000-01:00 + + + } + interface Mock { List getBuckets(); } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml index a7765dfc15fe3..be9c9f4a41e1c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/rollup_search.yml @@ -881,6 +881,308 @@ setup: interval: "1h" time_zone: "UTC" +--- +"Obsolete Timezone": + - skip: + version: " - 7.0.99" + reason: "IANA TZ deprecations in 7.1" + features: "warnings" + - do: + indices.create: + index: tz + body: + mappings: + properties: + timestamp: + type: date + partition: + type: keyword + price: + type: integer + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + warnings: + - "Creating Rollup job [tz] with timezone [Canada/Mountain], but [Canada/Mountain] has been deprecated by the IANA. Use [America/Edmonton] instead." + rollup.put_job: + id: tz + body: > + { + "index_pattern": "tz", + "rollup_index": "tz_rollup", + "cron": "*/30 * * * * ?", + "page_size" :10, + "groups" : { + "date_histogram": { + "field": "timestamp", + "interval": "5m", + "time_zone": "Canada/Mountain" + }, + "terms": { + "fields": ["partition"] + } + }, + "metrics": [ + { + "field": "price", + "metrics": ["max"] + } + ] + } + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + bulk: + refresh: true + body: + - index: + _index: "tz_rollup" + _type: "_doc" + - timestamp.date_histogram.timestamp: 1531221000000 + timestamp.date_histogram.interval: "5m" + timestamp.date_histogram.time_zone: "America/Edmonton" + timestamp.date_histogram._count: 1 + partition.terms.value: "a" + partition.terms._count: 1 + price.max.value: 1 + "_rollup.id": "tz" + "_rollup.version": 2 + + - index: + _index: "tz_rollup" + _type: "_doc" + - timestamp.date_histogram.timestamp: 1531221300000 + timestamp.date_histogram.interval: "5m" + timestamp.date_histogram.time_zone: "America/Edmonton" + timestamp.date_histogram._count: 2 + partition.terms.value: "b" + partition.terms._count: 2 + price.max.value: 2 + "_rollup.id": "tz" + "_rollup.version": 2 + + - index: + _index: "tz_rollup" + _type: "_doc" + - timestamp.date_histogram.timestamp: 1531221600000 + timestamp.date_histogram.interval: "5m" + timestamp.date_histogram.time_zone: "America/Edmonton" + timestamp.date_histogram._count: 10 + partition.terms.value: "a" + partition.terms._count: 10 + price.max.value: 3 + "_rollup.id": "tz" + "_rollup.version": 2 + + - do: + rollup.rollup_search: + index: "tz_rollup" + body: + size: 0 + aggs: + histo: + date_histogram: + field: "timestamp" + interval: "5m" + time_zone: "America/Edmonton" + aggs: + the_max: + max: + field: "price" + + - length: { aggregations.histo.buckets: 3 } + - match: { aggregations.histo.buckets.0.key_as_string: "2018-07-10T05:10:00.000-06:00" } + - match: { aggregations.histo.buckets.0.doc_count: 1 } + - match: { aggregations.histo.buckets.0.the_max.value: 1 } + - match: { aggregations.histo.buckets.1.key_as_string: "2018-07-10T05:15:00.000-06:00" } + - match: { aggregations.histo.buckets.1.doc_count: 2 } + - match: { aggregations.histo.buckets.1.the_max.value: 2 } + - match: { aggregations.histo.buckets.2.key_as_string: "2018-07-10T05:20:00.000-06:00" } + - match: { aggregations.histo.buckets.2.doc_count: 10 } + - match: { aggregations.histo.buckets.2.the_max.value: 3 } + + - do: + rollup.rollup_search: + index: "tz_rollup" + body: + size: 0 + aggs: + histo: + date_histogram: + field: "timestamp" + interval: "5m" + time_zone: "Canada/Mountain" + aggs: + the_max: + max: + field: "price" + + - length: { aggregations.histo.buckets: 3 } + - match: { aggregations.histo.buckets.0.key_as_string: "2018-07-10T05:10:00.000-06:00" } + - match: { aggregations.histo.buckets.0.doc_count: 1 } + - match: { aggregations.histo.buckets.0.the_max.value: 1 } + - match: { aggregations.histo.buckets.1.key_as_string: "2018-07-10T05:15:00.000-06:00" } + - match: { aggregations.histo.buckets.1.doc_count: 2 } + - match: { aggregations.histo.buckets.1.the_max.value: 2 } + - match: { aggregations.histo.buckets.2.key_as_string: "2018-07-10T05:20:00.000-06:00" } + - match: { aggregations.histo.buckets.2.doc_count: 10 } + - match: { aggregations.histo.buckets.2.the_max.value: 3 } + +--- +"Obsolete BWC Timezone": + - skip: + version: " - 7.0.99" + reason: "IANA TZ deprecations in 7.1" + - do: + indices.create: + index: tz_rollup + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 + mappings: + properties: + partition.terms.value: + type: keyword + partition.terms._count: + type: long + timestamp.date_histogram.time_zone: + type: keyword + timestamp.date_histogram.interval: + type: keyword + timestamp.date_histogram.timestamp: + type: date + timestamp.date_histogram._count: + type: long + price.max.value: + type: double + _rollup.id: + type: keyword + _rollup.version: + type: long + _meta: + _rollup: + sensor: + cron: "* * * * * ?" + rollup_index: "tz_rollup" + index_pattern: "tz" + timeout: "20s" + page_size: 1000 + groups: + date_histogram: + field: "timestamp" + interval: "5m" + time_zone: "Canada/Mountain" + terms: + fields: + - "partition" + id: tz + metrics: + - field: "price" + metrics: + - max + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + bulk: + refresh: true + body: + - index: + _index: "tz_rollup" + _type: "_doc" + - timestamp.date_histogram.timestamp: 1531221000000 + timestamp.date_histogram.interval: "5m" + timestamp.date_histogram.time_zone: "Canada/Mountain" + timestamp.date_histogram._count: 1 + partition.terms.value: "a" + partition.terms._count: 1 + price.max.value: 1 + "_rollup.id": "tz" + "_rollup.version": 2 + + - index: + _index: "tz_rollup" + _type: "_doc" + - timestamp.date_histogram.timestamp: 1531221300000 + timestamp.date_histogram.interval: "5m" + timestamp.date_histogram.time_zone: "Canada/Mountain" + timestamp.date_histogram._count: 2 + partition.terms.value: "b" + partition.terms._count: 2 + price.max.value: 2 + "_rollup.id": "tz" + "_rollup.version": 2 + + - index: + _index: "tz_rollup" + _type: "_doc" + - timestamp.date_histogram.timestamp: 1531221600000 + timestamp.date_histogram.interval: "5m" + timestamp.date_histogram.time_zone: "Canada/Mountain" + timestamp.date_histogram._count: 10 + partition.terms.value: "a" + partition.terms._count: 10 + price.max.value: 3 + "_rollup.id": "tz" + "_rollup.version": 2 + + - do: + rollup.rollup_search: + index: "tz_rollup" + body: + size: 0 + aggs: + histo: + date_histogram: + field: "timestamp" + interval: "5m" + time_zone: "America/Edmonton" + aggs: + the_max: + max: + field: "price" + + - length: { aggregations.histo.buckets: 3 } + - match: { aggregations.histo.buckets.0.key_as_string: "2018-07-10T05:10:00.000-06:00" } + - match: { aggregations.histo.buckets.0.doc_count: 1 } + - match: { aggregations.histo.buckets.0.the_max.value: 1 } + - match: { aggregations.histo.buckets.1.key_as_string: "2018-07-10T05:15:00.000-06:00" } + - match: { aggregations.histo.buckets.1.doc_count: 2 } + - match: { aggregations.histo.buckets.1.the_max.value: 2 } + - match: { aggregations.histo.buckets.2.key_as_string: "2018-07-10T05:20:00.000-06:00" } + - match: { aggregations.histo.buckets.2.doc_count: 10 } + - match: { aggregations.histo.buckets.2.the_max.value: 3 } + + + - do: + rollup.rollup_search: + index: "tz_rollup" + body: + size: 0 + aggs: + histo: + date_histogram: + field: "timestamp" + interval: "5m" + time_zone: "Canada/Mountain" + aggs: + the_max: + max: + field: "price" + + - length: { aggregations.histo.buckets: 3 } + - match: { aggregations.histo.buckets.0.key_as_string: "2018-07-10T05:10:00.000-06:00" } + - match: { aggregations.histo.buckets.0.doc_count: 1 } + - match: { aggregations.histo.buckets.0.the_max.value: 1 } + - match: { aggregations.histo.buckets.1.key_as_string: "2018-07-10T05:15:00.000-06:00" } + - match: { aggregations.histo.buckets.1.doc_count: 2 } + - match: { aggregations.histo.buckets.1.the_max.value: 2 } + - match: { aggregations.histo.buckets.2.key_as_string: "2018-07-10T05:20:00.000-06:00" } + - match: { aggregations.histo.buckets.2.doc_count: 10 } + - match: { aggregations.histo.buckets.2.the_max.value: 3 } + --- "Search with typed_keys": @@ -914,3 +1216,5 @@ setup: - match: { aggregations.date_histogram#histo.buckets.3.key_as_string: "2017-01-01T08:00:00.000Z" } - match: { aggregations.date_histogram#histo.buckets.3.doc_count: 20 } - match: { aggregations.date_histogram#histo.buckets.3.max#the_max.value: 4 } + + From 2931e9ca456870a9b5a49489e8eb1aab5ad32d22 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 17 Apr 2019 20:28:39 +0200 Subject: [PATCH 080/260] Handle Bulk Requests on Write Threadpool (#40866) * Bulk requests can be thousands of items large and take more than O(10ms) time to handle => we should not handle them on the transport threadpool to not block select loops * relates #39128 * relates #39658 --- .../action/bulk/TransportBulkAction.java | 5 +++-- .../action/support/HandledTransportAction.java | 11 ++++++++--- .../action/bulk/BulkProcessorRetryIT.java | 12 +++++++++--- ...ortBulkActionIndicesThatCannotBeCreatedTests.java | 9 ++++++++- .../action/bulk/TransportBulkActionIngestTests.java | 10 +++++++++- 5 files changed, 37 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 0249aff74363b..eae849407666e 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -117,7 +117,7 @@ public TransportBulkAction(ThreadPool threadPool, TransportService transportServ TransportShardBulkAction shardBulkAction, NodeClient client, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, AutoCreateIndex autoCreateIndex, LongSupplier relativeTimeProvider) { - super(BulkAction.NAME, transportService, actionFilters, (Supplier) BulkRequest::new); + super(BulkAction.NAME, transportService, actionFilters, (Supplier) BulkRequest::new, ThreadPool.Names.WRITE); Objects.requireNonNull(relativeTimeProvider); this.threadPool = threadPool; this.clusterService = clusterService; @@ -258,7 +258,8 @@ protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener executeBulk(task, bulkRequest, startTime, listener, responses, indicesThatCannotBeCreated)); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java b/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java index c0bc0af839967..ca10583ce248a 100644 --- a/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.action.support; -import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.Writeable; @@ -57,6 +56,13 @@ protected HandledTransportAction(String actionName, boolean canTripCircuitBreake new TransportHandler()); } + protected HandledTransportAction(String actionName, TransportService transportService, ActionFilters actionFilters, + Supplier request, String executor) { + super(actionName, actionFilters, transportService.getTaskManager()); + transportService.registerRequestHandler(actionName, request, executor, false, true, + new TransportHandler()); + } + protected HandledTransportAction(String actionName, boolean canTripCircuitBreaker, TransportService transportService, ActionFilters actionFilters, Writeable.Reader requestReader) { @@ -73,9 +79,8 @@ protected HandledTransportAction(String actionName, boolean canTripCircuitBreake class TransportHandler implements TransportRequestHandler { @Override - public final void messageReceived(final Request request, final TransportChannel channel, Task task) throws Exception { + public final void messageReceived(final Request request, final TransportChannel channel, Task task) { // We already got the task created on the network layer - no need to create it again on the transport layer - Logger logger = HandledTransportAction.this.logger; execute(task, request, new ChannelActionListener<>(channel, actionName, request)); } } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index 515c539a884fc..e4b6fff9fc353 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -25,6 +25,7 @@ import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.transport.RemoteTransportException; import java.util.Collections; import java.util.Iterator; @@ -133,9 +134,14 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) } } } else { - Throwable t = (Throwable) response; - // we're not expecting any other errors - throw new AssertionError("Unexpected failure", t); + if (response instanceof RemoteTransportException + && ((RemoteTransportException) response).status() == RestStatus.TOO_MANY_REQUESTS && rejectedExecutionExpected) { + // ignored, we exceeded the write queue size with dispatching the initial bulk request + } else { + Throwable t = (Throwable) response; + // we're not expecting any other errors + throw new AssertionError("Unexpected failure", t); + } } } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java index 10014c6fb3f56..f213b523fbfaf 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java @@ -30,20 +30,24 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.VersionType; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.util.Arrays; import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.function.Function; import static java.util.Collections.emptySet; import static java.util.Collections.singleton; +import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -102,7 +106,10 @@ private void indicesThatCannotBeCreatedTestCase(Set expected, ClusterState state = mock(ClusterState.class); when(state.getMetaData()).thenReturn(MetaData.EMPTY_META_DATA); when(clusterService.state()).thenReturn(state); - TransportBulkAction action = new TransportBulkAction(null, mock(TransportService.class), clusterService, + final ThreadPool threadPool = mock(ThreadPool.class); + final ExecutorService direct = EsExecutors.newDirectExecutorService(); + when(threadPool.executor(anyString())).thenReturn(direct); + TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService, null, null, null, mock(ActionFilters.class), null, null) { @Override void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener listener, diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java index b3ecc59076759..b570ec8f781a6 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -45,11 +45,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.ingest.IngestService; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import org.junit.Before; @@ -61,6 +63,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -68,6 +71,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -92,6 +96,7 @@ public class TransportBulkActionIngestTests extends ESTestCase { TransportService transportService; ClusterService clusterService; IngestService ingestService; + ThreadPool threadPool; /** Arguments to callbacks we want to capture, but which require generics, so we must use @Captor */ @Captor @@ -126,7 +131,7 @@ class TestTransportBulkAction extends TransportBulkAction { boolean indexCreated = true; // If set to false, will be set to true by call to createIndex TestTransportBulkAction() { - super(null, transportService, clusterService, ingestService, + super(threadPool, transportService, clusterService, ingestService, null, null, new ActionFilters(Collections.emptySet()), null, new AutoCreateIndex( SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), @@ -163,6 +168,9 @@ class TestSingleItemBulkWriteAction extends TransportSingleItemBulkWriteAction Date: Wed, 17 Apr 2019 19:54:55 +0100 Subject: [PATCH 081/260] Assert TransportReplicationActions acquire permits (#41271) Today we do not distinguish "no operations in flight" from "operations are blocked", since both return `0` from `IndexShard#getActiveOperationsCount()`. We therefore cannot assert that every `TransportReplicationAction` performs its actions under permit(s). This commit fixes this by returning `IndexShard#OPERATIONS_BLOCKED` if operations are blocked, allowing these two cases to be distinguished. --- ...TransportVerifyShardBeforeCloseAction.java | 4 ++-- .../TransportReplicationAction.java | 2 ++ .../elasticsearch/index/shard/IndexShard.java | 17 ++++++++++---- .../shard/IndexShardOperationPermits.java | 11 +++------- .../indices/flush/SyncedFlushService.java | 3 ++- ...portVerifyShardBeforeCloseActionTests.java | 8 +++---- ...TransportResyncReplicationActionTests.java | 7 +++++- .../TransportReplicationActionTests.java | 22 ++++++++++++------- ...ReplicationAllPermitsAcquisitionTests.java | 7 +++--- .../index/shard/IndexShardIT.java | 14 +++++++++++- .../IndexShardOperationPermitsTests.java | 4 ++-- .../index/shard/IndexShardTests.java | 17 +++++++++++--- 12 files changed, 79 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index 7d691717de1f2..22a0777f7bffb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -101,8 +101,8 @@ protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) { final ShardId shardId = indexShard.shardId(); - if (indexShard.getActiveOperationsCount() != 0) { - throw new IllegalStateException("On-going operations in progress while checking index shard " + shardId + " before closing"); + if (indexShard.getActiveOperationsCount() != IndexShard.OPERATIONS_BLOCKED) { + throw new IllegalStateException("Index shard " + shardId + " is not blocking all operations during closing"); } final ClusterBlocks clusterBlocks = clusterService.state().blocks(); diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 0c464d27e1957..6edaa95033997 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -513,6 +513,7 @@ private final class AsyncReplicaAction extends AbstractRunnable implements Actio @Override public void onResponse(Releasable releasable) { try { + assert replica.getActiveOperationsCount() != 0 : "must perform shard operation under a permit"; final ReplicaResult replicaResult = shardOperationOnReplica(replicaRequest.getRequest(), replica); releasable.close(); // release shard operation lock before responding to caller final TransportReplicationAction.ReplicaResponse response = @@ -912,6 +913,7 @@ public void perform(Request request, ActionListener consumer indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { forceRefreshes.close(); // no shard operation permits are being held here, move state from started to relocated - assert indexShardOperationPermits.getActiveOperationsCount() == 0 : + assert indexShardOperationPermits.getActiveOperationsCount() == OPERATIONS_BLOCKED : "in-flight operations in progress while moving shard state to relocated"; /* * We should not invoke the runnable under the mutex as the expected implementation is to handoff the primary context via a @@ -1553,7 +1553,8 @@ private void ensureWriteAllowed(Engine.Operation.Origin origin) throws IllegalIn assert assertReplicationTarget(); } else { assert origin == Engine.Operation.Origin.LOCAL_RESET; - assert getActiveOperationsCount() == 0 : "Ongoing writes [" + getActiveOperations() + "]"; + assert getActiveOperationsCount() == OPERATIONS_BLOCKED + : "locally resetting without blocking operations, active operations are [" + getActiveOperations() + "]"; } if (writeAllowedStates.contains(state) == false) { throw new IllegalIndexShardStateException(shardId, state, "operation only allowed when shard state is one of " + @@ -2740,8 +2741,15 @@ private boolean requirePrimaryTermUpdate(final long opPrimaryTerm, final boolean return (opPrimaryTerm > pendingPrimaryTerm) || (allPermits && opPrimaryTerm > getOperationPrimaryTerm()); } + public static final int OPERATIONS_BLOCKED = -1; + + /** + * Obtain the active operation count, or {@link IndexShard#OPERATIONS_BLOCKED} if all permits are held (even if there are + * outstanding operations in flight). + * + * @return the active operation count, or {@link IndexShard#OPERATIONS_BLOCKED} when all permits are held. + */ public int getActiveOperationsCount() { - // refCount is incremented on successful acquire and decremented on close return indexShardOperationPermits.getActiveOperationsCount(); } @@ -3069,7 +3077,8 @@ public ParsedDocument newNoopTombstoneDoc(String reason) { * Rollback the current engine to the safe commit, then replay local translog up to the global checkpoint. */ void resetEngineToGlobalCheckpoint() throws IOException { - assert getActiveOperationsCount() == 0 : "Ongoing writes [" + getActiveOperations() + "]"; + assert getActiveOperationsCount() == OPERATIONS_BLOCKED + : "resetting engine without blocking operations; active operations are [" + getActiveOperations() + ']'; sync(); // persist the global checkpoint to disk final SeqNoStats seqNoStats = seqNoStats(); final TranslogStats translogStats = translogStats(); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java index fe7a5392a080d..672e69743d481 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShardOperationPermits.java @@ -293,19 +293,14 @@ private Releasable acquire(Object debugInfo, StackTraceElement[] stackTrace) thr } /** - * Obtain the active operation count, or zero if all permits are held (even if there are outstanding operations in flight). + * Obtain the active operation count, or {@link IndexShard#OPERATIONS_BLOCKED} if all permits are held. * - * @return the active operation count, or zero when all permits are held + * @return the active operation count, or {@link IndexShard#OPERATIONS_BLOCKED} when all permits are held. */ int getActiveOperationsCount() { int availablePermits = semaphore.availablePermits(); if (availablePermits == 0) { - /* - * This occurs when either doBlockOperations is holding all the permits or there are outstanding operations in flight and the - * remainder of the permits are held by doBlockOperations. We do not distinguish between these two cases and simply say that - * the active operations count is zero. - */ - return 0; + return IndexShard.OPERATIONS_BLOCKED; // This occurs when blockOperations() has acquired all the permits. } else { return TOTAL_PERMITS - availablePermits; } diff --git a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java index 0423559aaf5a5..79a2d6c3c0a91 100644 --- a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java +++ b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java @@ -538,7 +538,7 @@ private InFlightOpsResponse performInFlightOps(InFlightOpsRequest request) { throw new IllegalStateException("[" + request.shardId() +"] expected a primary shard"); } int opCount = indexShard.getActiveOperationsCount(); - return new InFlightOpsResponse(opCount); + return new InFlightOpsResponse(opCount == IndexShard.OPERATIONS_BLOCKED ? 0 : opCount); } public static final class PreShardSyncedFlushRequest extends TransportRequest { @@ -781,6 +781,7 @@ static final class InFlightOpsResponse extends TransportResponse { } InFlightOpsResponse(int opCount) { + assert opCount >= 0 : opCount; this.opCount = opCount; } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 75f70ad02df64..d7974ed1c6365 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -100,7 +100,7 @@ public void setUp() throws Exception { super.setUp(); indexShard = mock(IndexShard.class); - when(indexShard.getActiveOperationsCount()).thenReturn(0); + when(indexShard.getActiveOperationsCount()).thenReturn(IndexShard.OPERATIONS_BLOCKED); final ShardId shardId = new ShardId("index", "_na_", randomIntBetween(0, 3)); when(indexShard.shardId()).thenReturn(shardId); @@ -165,12 +165,12 @@ public void testShardIsFlushed() throws Throwable { assertThat(flushRequest.getValue().force(), is(true)); } - public void testOperationFailsWithOnGoingOps() { - when(indexShard.getActiveOperationsCount()).thenReturn(randomIntBetween(1, 10)); + public void testOperationFailsWhenNotBlocked() { + when(indexShard.getActiveOperationsCount()).thenReturn(randomIntBetween(0, 10)); IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica); assertThat(exception.getMessage(), - equalTo("On-going operations in progress while checking index shard " + indexShard.shardId() + " before closing")); + equalTo("Index shard " + indexShard.shardId() + " is not blocking all operations during closing")); verify(indexShard, times(0)).flush(any(FlushRequest.class)); } diff --git a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java index ed3663ed18d2c..167518f4fc9db 100644 --- a/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/resync/TransportResyncReplicationActionTests.java @@ -58,6 +58,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import static java.util.Collections.emptyList; import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; @@ -118,13 +119,17 @@ public void testResyncDoesNotBlockOnPrimaryAction() throws Exception { final String allocationId = primaryShardRouting.allocationId().getId(); final long primaryTerm = indexMetaData.primaryTerm(shardId.id()); + final AtomicInteger acquiredPermits = new AtomicInteger(); final IndexShard indexShard = mock(IndexShard.class); when(indexShard.shardId()).thenReturn(shardId); when(indexShard.routingEntry()).thenReturn(primaryShardRouting); when(indexShard.getPendingPrimaryTerm()).thenReturn(primaryTerm); + when(indexShard.getOperationPrimaryTerm()).thenReturn(primaryTerm); + when(indexShard.getActiveOperationsCount()).then(i -> acquiredPermits.get()); doAnswer(invocation -> { ActionListener callback = (ActionListener) invocation.getArguments()[0]; - callback.onResponse(() -> logger.trace("released")); + acquiredPermits.incrementAndGet(); + callback.onResponse(acquiredPermits::decrementAndGet); return null; }).when(indexShard).acquirePrimaryOperationPermit(any(ActionListener.class), anyString(), anyObject()); when(indexShard.getReplicationGroup()).thenReturn( diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index ccb23a9111a4e..12cc9097b652c 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -87,6 +87,7 @@ import org.hamcrest.Matcher; import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -678,16 +679,17 @@ public void testPrimaryReference() { }; TestAction.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable); final Request request = new Request(NO_SHARD_ID); - primary.perform(request, ActionTestUtils.assertNoFailureListener(r -> { - final ElasticsearchException exception = new ElasticsearchException("testing"); - primary.failShard("test", exception); + shard.runUnderPrimaryPermit(() -> + primary.perform(request, ActionTestUtils.assertNoFailureListener(r -> { + final ElasticsearchException exception = new ElasticsearchException("testing"); + primary.failShard("test", exception); - verify(shard).failShard("test", exception); + verify(shard).failShard("test", exception); - primary.close(); + primary.close(); - assertTrue(closed.get()); - })); + assertTrue(closed.get()); + })), Assert::assertNotNull, null, null); } public void testReplicaProxy() throws InterruptedException, ExecutionException { @@ -775,10 +777,12 @@ public void testSeqNoIsSetOnPrimary() { inSyncIds, shardRoutingTable.getAllAllocationIds())); doAnswer(invocation -> { + count.incrementAndGet(); //noinspection unchecked - ((ActionListener)invocation.getArguments()[0]).onResponse(() -> {}); + ((ActionListener)invocation.getArguments()[0]).onResponse(count::decrementAndGet); return null; }).when(shard).acquirePrimaryOperationPermit(any(), anyString(), anyObject()); + when(shard.getActiveOperationsCount()).thenAnswer(i -> count.get()); final IndexService indexService = mock(IndexService.class); when(indexService.getShard(shard.shardId().id())).thenReturn(shard); @@ -1286,6 +1290,8 @@ private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService return null; }).when(indexShard) .acquireReplicaOperationPermit(anyLong(), anyLong(), anyLong(), any(ActionListener.class), anyString(), anyObject()); + when(indexShard.getActiveOperationsCount()).thenAnswer(i -> count.get()); + when(indexShard.routingEntry()).thenAnswer(invocationOnMock -> { final ClusterState state = clusterService.state(); final RoutingNode node = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java index 8463d66e98e71..28373347b19ca 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java @@ -316,7 +316,8 @@ private void assertBlockIsPresentForDelayedOp() { allPermitsAction.new AsyncPrimaryAction(primaryRequest, allPermitFuture, null) { @Override void runWithPrimaryShardReference(final TransportReplicationAction.PrimaryShardReference reference) { - assertEquals("All permits must be acquired", 0, reference.indexShard.getActiveOperationsCount()); + assertEquals("All permits must be acquired", + IndexShard.OPERATIONS_BLOCKED, reference.indexShard.getActiveOperationsCount()); assertSame(primary, reference.indexShard); final ClusterState clusterState = clusterService.state(); @@ -549,13 +550,13 @@ protected void acquireReplicaOperationPermit(IndexShard shard, Request request, @Override protected void shardOperationOnPrimary(Request shardRequest, IndexShard shard, ActionListener> listener) { - assertEquals("All permits must be acquired", 0, shard.getActiveOperationsCount()); + assertEquals("All permits must be acquired", IndexShard.OPERATIONS_BLOCKED, shard.getActiveOperationsCount()); super.shardOperationOnPrimary(shardRequest, shard, listener); } @Override protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard shard) throws Exception { - assertEquals("All permits must be acquired", 0, shard.getActiveOperationsCount()); + assertEquals("All permits must be acquired", IndexShard.OPERATIONS_BLOCKED, shard.getActiveOperationsCount()); return super.shardOperationOnReplica(shardRequest, shard); } } 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 8e2403aedc26c..d0bc06ab1cc7f 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -84,6 +84,7 @@ import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPoolStats; +import org.junit.Assert; import java.io.IOException; import java.io.UncheckedIOException; @@ -878,7 +879,18 @@ public void testInvalidateIndicesRequestCacheWhenRollbackEngine() throws Excepti shard.refresh("test"); assertThat(client().search(countRequest).actionGet().getHits().getTotalHits().value, equalTo(numDocs)); assertThat(shard.getLocalCheckpoint(), equalTo(shard.seqNoStats().getMaxSeqNo())); - shard.resetEngineToGlobalCheckpoint(); + + final CountDownLatch engineResetLatch = new CountDownLatch(1); + shard.acquireAllPrimaryOperationsPermits(ActionListener.wrap(r -> { + try { + shard.resetEngineToGlobalCheckpoint(); + } finally { + r.close(); + engineResetLatch.countDown(); + } + }, Assert::assertNotNull), TimeValue.timeValueMinutes(1L)); + engineResetLatch.await(); + final long moreDocs = between(10, 20); for (int i = 0; i < moreDocs; i++) { client().prepareIndex("test", "_doc", Long.toString(i + numDocs)).setSource("{}", XContentType.JSON).get(); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java index a785c2c4d8224..416e717099023 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardOperationPermitsTests.java @@ -523,8 +523,8 @@ public void testActiveOperationsCount() throws ExecutionException, InterruptedEx future2.get().close(); assertThat(permits.getActiveOperationsCount(), equalTo(0)); - try (Releasable releasable = blockAndWait()) { - assertThat(permits.getActiveOperationsCount(), equalTo(0)); + try (Releasable ignored = blockAndWait()) { + assertThat(permits.getActiveOperationsCount(), equalTo(IndexShard.OPERATIONS_BLOCKED)); } PlainActionFuture future3 = new PlainActionFuture<>(); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 0c7a1033eedb0..725a6a0bcbec4 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -126,6 +126,7 @@ import org.elasticsearch.test.FieldMaskingReader; import org.elasticsearch.test.VersionUtils; import org.elasticsearch.threadpool.ThreadPool; +import org.junit.Assert; import java.io.IOException; import java.nio.charset.Charset; @@ -707,7 +708,7 @@ public void onResponse(final Releasable releasable) { if (singlePermit) { assertThat(indexShard.getActiveOperationsCount(), greaterThan(0)); } else { - assertThat(indexShard.getActiveOperationsCount(), equalTo(0)); + assertThat(indexShard.getActiveOperationsCount(), equalTo(IndexShard.OPERATIONS_BLOCKED)); } releasable.close(); super.onResponse(releasable); @@ -757,7 +758,7 @@ public void onResponse(final Releasable releasable) { indexShard.acquireAllPrimaryOperationsPermits(futureAllPermits, TimeValue.timeValueSeconds(30L)); allPermitsAcquired.await(); assertTrue(blocked.get()); - assertEquals(0, indexShard.getActiveOperationsCount()); + assertEquals(IndexShard.OPERATIONS_BLOCKED, indexShard.getActiveOperationsCount()); assertTrue("Expected no results, operations are blocked", results.asList().isEmpty()); futures.forEach(future -> assertFalse(future.isDone())); @@ -3660,7 +3661,17 @@ public void testResetEngine() throws Exception { }); thread.start(); latch.await(); - shard.resetEngineToGlobalCheckpoint(); + + final CountDownLatch engineResetLatch = new CountDownLatch(1); + shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), globalCheckpoint, 0L, ActionListener.wrap(r -> { + try { + shard.resetEngineToGlobalCheckpoint(); + } finally { + r.close(); + engineResetLatch.countDown(); + } + }, Assert::assertNotNull), TimeValue.timeValueMinutes(1L)); + engineResetLatch.await(); assertThat(getShardDocUIDs(shard), equalTo(docBelowGlobalCheckpoint)); assertThat(shard.seqNoStats().getMaxSeqNo(), equalTo(globalCheckpoint)); assertThat(shard.translogStats().estimatedNumberOfOperations(), equalTo(translogStats.estimatedNumberOfOperations())); From 29bca0060400be5b2b00aeee9b08af0ccfbf9285 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Wed, 17 Apr 2019 15:44:51 -0400 Subject: [PATCH 082/260] Purge Rollup doc ID upgrade code from 8.0 (#41227) This is no longer needed in 8.x, because all jobs moving from 6.x to 7.x will be forced to upgrade their rollup ID if they haven't already. So by time we get to 8.x, all jobs will be on the new scheme. This removes the old CRC generator and all the flags and state checking to manage it. We do need to keep the serialization check since a mixed cluster will have 7.x nodes sending/receiving the flag, so that is just hardcoded until 9.0 when we can remove it. --- .../client/rollup/GetRollupJobResponse.java | 26 +-- .../org/elasticsearch/client/RollupIT.java | 1 - .../rollup/GetRollupJobResponseTests.java | 6 +- docs/reference/rollup/apis/get-job.asciidoc | 9 +- .../core/rollup/job/RollupJobStatus.java | 44 ++--- .../job/JobWrapperSerializingTests.java | 2 +- .../core/rollup/job/RollupJobStatusTests.java | 2 +- .../xpack/rollup/job/IndexerUtils.java | 13 +- .../xpack/rollup/job/RollupIDGenerator.java | 186 ++++++------------ .../xpack/rollup/job/RollupIndexer.java | 25 +-- .../xpack/rollup/job/RollupJobTask.java | 42 +--- .../xpack/rollup/job/IndexerUtilsTests.java | 69 ++----- .../job/RollupIndexerIndexingTests.java | 32 ++- .../rollup/job/RollupIndexerStateTests.java | 26 +-- .../xpack/rollup/job/RollupJobTaskTests.java | 51 ++--- .../rest-api-spec/test/rollup/delete_job.yml | 3 - .../rest-api-spec/test/rollup/get_jobs.yml | 3 - .../rest-api-spec/test/rollup/put_job.yml | 1 - .../elasticsearch/multi_node/RollupIT.java | 2 +- .../upgrades/RollupIDUpgradeIT.java | 10 - 20 files changed, 157 insertions(+), 396 deletions(-) delete mode 100644 x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RollupIDUpgradeIT.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/GetRollupJobResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/GetRollupJobResponse.java index 125bd6dab6c21..e63daf5949002 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/GetRollupJobResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/rollup/GetRollupJobResponse.java @@ -48,7 +48,7 @@ public class GetRollupJobResponse { static final ParseField STATE = new ParseField("job_state"); static final ParseField CURRENT_POSITION = new ParseField("current_position"); static final ParseField ROLLUPS_INDEXED = new ParseField("rollups_indexed"); - static final ParseField UPGRADED_DOC_ID = new ParseField("upgraded_doc_id"); + private static final ParseField UPGRADED_DOC_ID = new ParseField("upgraded_doc_id"); private List jobs; @@ -207,12 +207,10 @@ public static class RollupIndexerJobStats extends IndexerJobStats { public static class RollupJobStatus { private final IndexerState state; private final Map currentPosition; - private final boolean upgradedDocumentId; - RollupJobStatus(IndexerState state, Map position, boolean upgradedDocumentId) { + RollupJobStatus(IndexerState state, Map position) { this.state = state; this.currentPosition = position; - this.upgradedDocumentId = upgradedDocumentId; } /** @@ -227,13 +225,6 @@ public IndexerState getState() { public Map getCurrentPosition() { return currentPosition; } - /** - * Flag holds the state of the ID scheme, e.g. if it has been upgraded - * to the concatenation scheme. - */ - public boolean getUpgradedDocumentId() { - return upgradedDocumentId; - } private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( STATUS.getPreferredName(), @@ -242,8 +233,7 @@ public boolean getUpgradedDocumentId() { IndexerState state = (IndexerState) args[0]; @SuppressWarnings("unchecked") // We're careful of the contents Map currentPosition = (Map) args[1]; - Boolean upgradedDocumentId = (Boolean) args[2]; - return new RollupJobStatus(state, currentPosition, upgradedDocumentId == null ? false : upgradedDocumentId); + return new RollupJobStatus(state, currentPosition); }); static { PARSER.declareField(constructorArg(), p -> IndexerState.fromString(p.text()), STATE, ObjectParser.ValueType.STRING); @@ -257,7 +247,7 @@ public boolean getUpgradedDocumentId() { throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); }, CURRENT_POSITION, ObjectParser.ValueType.VALUE_OBJECT_ARRAY); - // Optional to accommodate old versions of state + // Optional to accommodate old versions of state, not used PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), UPGRADED_DOC_ID); } @@ -267,20 +257,18 @@ public boolean equals(Object other) { if (other == null || getClass() != other.getClass()) return false; RollupJobStatus that = (RollupJobStatus) other; return Objects.equals(state, that.state) - && Objects.equals(currentPosition, that.currentPosition) - && upgradedDocumentId == that.upgradedDocumentId; + && Objects.equals(currentPosition, that.currentPosition); } @Override public int hashCode() { - return Objects.hash(state, currentPosition, upgradedDocumentId); + return Objects.hash(state, currentPosition); } @Override public final String toString() { return "{stats=" + state - + ", currentPosition=" + currentPosition - + ", upgradedDocumentId=" + upgradedDocumentId + "}"; + + ", currentPosition=" + currentPosition + "}"; } } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java index d876ce6ed5fb3..6d6e702500d6c 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java @@ -233,7 +233,6 @@ public void testPutStartAndGetRollupJob() throws Exception { assertEquals(1, job.getStats().getOutputDocuments()); assertThat(job.getStatus().getState(), either(equalTo(IndexerState.STARTED)).or(equalTo(IndexerState.INDEXING))); assertThat(job.getStatus().getCurrentPosition(), hasKey("date.date_histogram")); - assertEquals(true, job.getStatus().getUpgradedDocumentId()); // stop the job StopRollupJobRequest stopRequest = new StopRollupJobRequest(id); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupJobResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupJobResponseTests.java index a063294cae6d7..9fc2d9706ce57 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupJobResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/rollup/GetRollupJobResponseTests.java @@ -74,10 +74,7 @@ private RollupJobStatus randomStatus() { while (currentPosition.size() < positions) { currentPosition.put(randomAlphaOfLength(2), randomAlphaOfLength(2)); } - return new RollupJobStatus( - randomFrom(IndexerState.values()), - currentPosition, - randomBoolean()); + return new RollupJobStatus(randomFrom(IndexerState.values()), currentPosition); } private void toXContent(GetRollupJobResponse response, XContentBuilder builder) throws IOException { @@ -108,7 +105,6 @@ public void toXContent(RollupJobStatus status, XContentBuilder builder, ToXConte if (status.getCurrentPosition() != null) { builder.field(GetRollupJobResponse.CURRENT_POSITION.getPreferredName(), status.getCurrentPosition()); } - builder.field(GetRollupJobResponse.UPGRADED_DOC_ID.getPreferredName(), status.getUpgradedDocumentId()); builder.endObject(); } diff --git a/docs/reference/rollup/apis/get-job.asciidoc b/docs/reference/rollup/apis/get-job.asciidoc index ff4d62fb8002c..a40c5fd8996c5 100644 --- a/docs/reference/rollup/apis/get-job.asciidoc +++ b/docs/reference/rollup/apis/get-job.asciidoc @@ -94,8 +94,7 @@ Which will yield the following response: "page_size" : 1000 }, "status" : { - "job_state" : "stopped", - "upgraded_doc_id": true + "job_state" : "stopped" }, "stats" : { "pages_processed" : 0, @@ -220,8 +219,7 @@ Which will yield the following response: "page_size" : 1000 }, "status" : { - "job_state" : "stopped", - "upgraded_doc_id": true + "job_state" : "stopped" }, "stats" : { "pages_processed" : 0, @@ -275,8 +273,7 @@ Which will yield the following response: "page_size" : 1000 }, "status" : { - "job_state" : "stopped", - "upgraded_doc_id": true + "job_state" : "stopped" }, "stats" : { "pages_processed" : 0, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java index d95efcef08b82..eaf6c0d04910b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatus.java @@ -41,19 +41,13 @@ public class RollupJobStatus implements Task.Status, PersistentTaskState { @Nullable private final TreeMap currentPosition; - // Flag holds the state of the ID scheme, e.g. if it has been upgraded to the - // concatenation scheme. See #32372 for more details - private boolean upgradedDocumentID; - private static final ParseField STATE = new ParseField("job_state"); private static final ParseField CURRENT_POSITION = new ParseField("current_position"); - private static final ParseField UPGRADED_DOC_ID = new ParseField("upgraded_doc_id"); + private static final ParseField UPGRADED_DOC_ID = new ParseField("upgraded_doc_id"); // This can be removed in 9.0 public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, - args -> new RollupJobStatus((IndexerState) args[0], - (HashMap) args[1], - (Boolean)args[2])); + args -> new RollupJobStatus((IndexerState) args[0], (HashMap) args[1])); static { PARSER.declareField(constructorArg(), p -> { @@ -72,26 +66,23 @@ public class RollupJobStatus implements Task.Status, PersistentTaskState { throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); }, CURRENT_POSITION, ObjectParser.ValueType.VALUE_OBJECT_ARRAY); - // Optional to accommodate old versions of state + // Optional to accommodate old versions of state, not used in ctor PARSER.declareBoolean(ConstructingObjectParser.optionalConstructorArg(), UPGRADED_DOC_ID); } - public RollupJobStatus(IndexerState state, @Nullable Map position, - @Nullable Boolean upgradedDocumentID) { + public RollupJobStatus(IndexerState state, @Nullable Map position) { this.state = state; this.currentPosition = position == null ? null : new TreeMap<>(position); - this.upgradedDocumentID = upgradedDocumentID != null ? upgradedDocumentID : false; //default to false if missing } public RollupJobStatus(StreamInput in) throws IOException { state = IndexerState.fromStream(in); currentPosition = in.readBoolean() ? new TreeMap<>(in.readMap()) : null; - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - upgradedDocumentID = in.readBoolean(); - } else { - // If we're getting this job from a pre-6.4.0 node, - // it is using the old ID scheme - upgradedDocumentID = false; + if (in.getVersion().before(Version.V_8_0_0)) { + // 7.x nodes serialize `upgradedDocumentID` flag. We don't need it anymore, but + // we need to pull it off the stream + // This can go away completely in 9.0 + in.readBoolean(); } } @@ -103,10 +94,6 @@ public Map getPosition() { return currentPosition; } - public boolean isUpgradedDocumentID() { - return upgradedDocumentID; - } - public static RollupJobStatus fromXContent(XContentParser parser) { try { return PARSER.parse(parser, null); @@ -122,7 +109,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (currentPosition != null) { builder.field(CURRENT_POSITION.getPreferredName(), currentPosition); } - builder.field(UPGRADED_DOC_ID.getPreferredName(), upgradedDocumentID); builder.endObject(); return builder; } @@ -139,8 +125,11 @@ public void writeTo(StreamOutput out) throws IOException { if (currentPosition != null) { out.writeMap(currentPosition); } - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeBoolean(upgradedDocumentID); + if (out.getVersion().before(Version.V_8_0_0)) { + // 7.x nodes expect a boolean `upgradedDocumentID` flag. We don't have it anymore, + // but we need to tell them we are upgraded in case there is a mixed cluster + // This can go away completely in 9.0 + out.writeBoolean(true); } } @@ -157,12 +146,11 @@ public boolean equals(Object other) { RollupJobStatus that = (RollupJobStatus) other; return Objects.equals(this.state, that.state) - && Objects.equals(this.currentPosition, that.currentPosition) - && Objects.equals(this.upgradedDocumentID, that.upgradedDocumentID); + && Objects.equals(this.currentPosition, that.currentPosition); } @Override public int hashCode() { - return Objects.hash(state, currentPosition, upgradedDocumentID); + return Objects.hash(state, currentPosition); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java index cca60bbf33eec..70105b48533bd 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/JobWrapperSerializingTests.java @@ -44,6 +44,6 @@ protected GetRollupJobsAction.JobWrapper createTestInstance() { new RollupIndexerJobStats(randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong()), - new RollupJobStatus(state, Collections.emptyMap(), randomBoolean())); + new RollupJobStatus(state, Collections.emptyMap())); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java index f46bda788bf5b..90a919b61045e 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/rollup/job/RollupJobStatusTests.java @@ -34,7 +34,7 @@ private Map randomPosition() { @Override protected RollupJobStatus createTestInstance() { - return new RollupJobStatus(randomFrom(IndexerState.values()), randomPosition(), randomBoolean()); + return new RollupJobStatus(randomFrom(IndexerState.values()), randomPosition()); } @Override diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java index 94d64b17de8f3..d8dbc0ceade7a 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/IndexerUtils.java @@ -43,11 +43,10 @@ class IndexerUtils { * @param stats The stats accumulator for this job's task * @param groupConfig The grouping configuration for the job * @param jobId The ID for the job - * @param isUpgradedDocID `true` if this job is using the new ID scheme * @return A list of rolled documents derived from the response */ static List processBuckets(CompositeAggregation agg, String rollupIndex, RollupIndexerJobStats stats, - GroupConfig groupConfig, String jobId, boolean isUpgradedDocID) { + GroupConfig groupConfig, String jobId) { logger.debug("Buckets: [" + agg.getBuckets().size() + "][" + jobId + "]"); return agg.getBuckets().stream().map(b ->{ @@ -58,20 +57,14 @@ static List processBuckets(CompositeAggregation agg, String rollup TreeMap keys = new TreeMap<>(b.getKey()); List metrics = b.getAggregations().asList(); - RollupIDGenerator idGenerator; - if (isUpgradedDocID) { - idGenerator = new RollupIDGenerator.Murmur3(jobId); - } else { - idGenerator = new RollupIDGenerator.CRC(); - } + RollupIDGenerator idGenerator = new RollupIDGenerator(jobId); Map doc = new HashMap<>(keys.size() + metrics.size()); processKeys(keys, doc, b.getDocCount(), groupConfig, idGenerator); idGenerator.add(jobId); processMetrics(metrics, doc); - doc.put(RollupField.ROLLUP_META + "." + RollupField.VERSION_FIELD, - isUpgradedDocID ? Rollup.CURRENT_ROLLUP_VERSION : Rollup.ROLLUP_VERSION_V1); + doc.put(RollupField.ROLLUP_META + "." + RollupField.VERSION_FIELD, Rollup.CURRENT_ROLLUP_VERSION ); doc.put(RollupField.ROLLUP_META + "." + RollupField.ID.getPreferredName(), jobId); IndexRequest request = new IndexRequest(rollupIndex, RollupField.TYPE_NAME, idGenerator.getID()); diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIDGenerator.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIDGenerator.java index 3f9fc73873914..761ddcf757631 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIDGenerator.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIDGenerator.java @@ -12,7 +12,6 @@ import java.nio.charset.StandardCharsets; import java.util.Base64; -import java.util.zip.CRC32; /** * The ID Generator creates a deterministic document ID to be used for rollup docs. @@ -25,154 +24,85 @@ * * NOTE: this class is not thread safe; there is no synchronization on the "generated" * flag and it is unsafe to use from multiple threads. + * + * A 128 bit Murmur3 hash of all the keys concatenated together, base64-encoded, then + * prepended with the job ID and a `$` delimiter + * + * Null values are hashed as a (hopefully) unique string `__NULL_PLACEHOLDER__830f1de2__` */ -public abstract class RollupIDGenerator { - public abstract void add(Integer v); - public abstract void add(Long v); - public abstract void add(Double v); - public abstract void add(String v); - public abstract void addNull(); - public abstract String getID(); - +public class RollupIDGenerator { private boolean generated = false; - - final boolean check(Object v) { - if (generated) { - throw new RuntimeException("Cannot update ID as it has already been generated."); - } - if (v == null) { - addNull(); - return false; - } - return true; + private static final long SEED = 19; + private static final BytesRef DELIM = new BytesRef("$"); + private static final BytesRef NULL_PLACEHOLDER = new BytesRef("__NULL_PLACEHOLDER__830f1de2__"); + private final BytesRefBuilder id = new BytesRefBuilder(); + private final String jobId; + + RollupIDGenerator(String jobId) { + this.jobId = jobId; } - final void setFlag() { - if (generated) { - throw new RuntimeException("Cannot generate ID as it has already been generated."); + public void add(Integer v) { + if (check(v)) { + update(Numbers.intToBytes(v)); } - generated = true; } - /** - * The "old" style ID used in Rollup V1. A rolling 32 bit CRC. - * - * Null values are hashed as (int)19. - */ - @Deprecated - public static class CRC extends RollupIDGenerator { - private final CRC32 crc = new CRC32(); - - @Override - public void add(Integer v) { - if (check(v)) { - crc.update(v); - } - } - - @Override - public void add(Long v) { - if (check(v)) { - crc.update(Numbers.longToBytes(v), 0, 8); - } - } - - @Override - public void add(Double v) { - if (check(v)) { - crc.update(Numbers.doubleToBytes(v), 0, 8); - } - } - - @Override - public void add(String v) { - if (check(v)) { - byte[] vs = (v).getBytes(StandardCharsets.UTF_8); - crc.update(vs, 0, vs.length); - } - } - - @Override - public void addNull() { - // Old ID scheme used (int)19 as the null placeholder. - // Not great but we're stuck with it :( - crc.update(19); - } - - @Override - public String getID() { - setFlag(); - return String.valueOf(crc.getValue()); + public void add(Long v) { + if (check(v)) { + update(Numbers.longToBytes(v)); } } - /** - * The "new" style ID, used in Rollup V2. A 128 bit Murmur3 hash of - * all the keys concatenated together, base64-encoded, then prepended - * with the job ID and a `$` delimiter - * - * Null values are hashed as a (hopefully) unique string `__NULL_PLACEHOLDER__830f1de2__` - */ - public static class Murmur3 extends RollupIDGenerator { - private static final long SEED = 19; - private static final BytesRef DELIM = new BytesRef("$"); - private static final BytesRef NULL_PLACEHOLDER = new BytesRef("__NULL_PLACEHOLDER__830f1de2__"); - private final BytesRefBuilder id = new BytesRefBuilder(); - private final String jobId; - - Murmur3(String jobId) { - this.jobId = jobId; + public void add(Double v) { + if (check(v)) { + update(Numbers.doubleToBytes(v)); } + } - @Override - public void add(Integer v) { - if (check(v)) { - update(Numbers.intToBytes(v)); - } + public void add(String v) { + if (check(v)) { + update((v).getBytes(StandardCharsets.UTF_8)); } + } - @Override - public void add(Long v) { - if (check(v)) { - update(Numbers.longToBytes(v)); - } - } + void addNull() { + // New ID scheme uses a (hopefully) unique placeholder for null + update(NULL_PLACEHOLDER.bytes); + } - @Override - public void add(Double v) { - if (check(v)) { - update(Numbers.doubleToBytes(v)); - } - } + private void update(byte[] v) { + id.append(v, 0, v.length); + id.append(DELIM); + } - @Override - public void add(String v) { - if (check(v)) { - update((v).getBytes(StandardCharsets.UTF_8)); - } + private boolean check(Object v) { + if (generated) { + throw new RuntimeException("Cannot update ID as it has already been generated."); } - - @Override - public void addNull() { - // New ID scheme uses a (hopefully) unique placeholder for null - update(NULL_PLACEHOLDER.bytes); + if (v == null) { + addNull(); + return false; } + return true; + } - private void update(byte[] v) { - id.append(v, 0, v.length); - id.append(DELIM); + private void setFlag() { + if (generated) { + throw new RuntimeException("Cannot generate ID as it has already been generated."); } + generated = true; + } - @Override - public String getID() { - setFlag(); - MurmurHash3.Hash128 hasher - = MurmurHash3.hash128(id.bytes(), 0, id.length(), SEED, new MurmurHash3.Hash128()); - byte[] hashedBytes = new byte[16]; - System.arraycopy(Numbers.longToBytes(hasher.h1), 0, hashedBytes, 0, 8); - System.arraycopy(Numbers.longToBytes(hasher.h2), 0, hashedBytes, 8, 8); - return jobId + "$" + Base64.getUrlEncoder().withoutPadding().encodeToString(hashedBytes); + public String getID() { + setFlag(); + MurmurHash3.Hash128 hasher + = MurmurHash3.hash128(id.bytes(), 0, id.length(), SEED, new MurmurHash3.Hash128()); + byte[] hashedBytes = new byte[16]; + System.arraycopy(Numbers.longToBytes(hasher.h1), 0, hashedBytes, 0, 8); + System.arraycopy(Numbers.longToBytes(hasher.h2), 0, hashedBytes, 8, 8); + return jobId + "$" + Base64.getUrlEncoder().withoutPadding().encodeToString(hashedBytes); - } } + } diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java index daa888562e94f..f99d5ec5993b5 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupIndexer.java @@ -49,7 +49,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.xpack.core.rollup.RollupField.formatFieldName; @@ -61,7 +60,6 @@ public abstract class RollupIndexer extends AsyncTwoPhaseIndexer initialState, Map initialPosition, - AtomicBoolean upgradedDocumentID) { - this(executor, job, initialState, initialPosition, upgradedDocumentID, new RollupIndexerJobStats()); + RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition) { + this(executor, job, initialState, initialPosition, new RollupIndexerJobStats()); } /** @@ -84,22 +80,13 @@ public abstract class RollupIndexer extends AsyncTwoPhaseIndexer initialState, Map initialPosition, - AtomicBoolean upgradedDocumentID, RollupIndexerJobStats jobStats) { + RollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, + Map initialPosition, RollupIndexerJobStats jobStats) { super(executor, initialState, initialPosition, jobStats); this.job = job; this.compositeBuilder = createCompositeBuilder(job.getConfig()); - this.upgradedDocumentID = upgradedDocumentID; - } - - /** - * Returns if this job has upgraded it's ID scheme yet or not - */ - public boolean isUpgradedDocumentID() { - return upgradedDocumentID.get(); } @Override @@ -127,8 +114,6 @@ protected void onStart(long now, ActionListener listener) { @Override protected SearchRequest buildSearchRequest() { - // Indexer is single-threaded, and only place that the ID scheme can get upgraded is doSaveState(), so - // we can pass down the boolean value rather than the atomic here final Map position = getPosition(); SearchSourceBuilder searchSource = new SearchSourceBuilder() .size(0) @@ -145,7 +130,7 @@ protected IterationResult> doProcess(SearchResponse searchRe return new IterationResult<>( IndexerUtils.processBuckets(response, job.getConfig().getRollupIndex(), getStats(), - job.getConfig().getGroupConfig(), job.getConfig().getId(), upgradedDocumentID.get()), + job.getConfig().getGroupConfig(), job.getConfig().getId()), response.afterKey(), response.getBuckets().isEmpty()); } 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 fecda3a2ce24b..038c69365c839 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 @@ -36,7 +36,6 @@ import org.elasticsearch.xpack.rollup.Rollup; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; /** @@ -98,9 +97,8 @@ protected class ClientRollupPageManager extends RollupIndexer { private final RollupJob job; ClientRollupPageManager(RollupJob job, IndexerState initialState, Map initialPosition, - Client client, AtomicBoolean upgradedDocumentID) { - super(threadPool.executor(ThreadPool.Names.GENERIC), job, new AtomicReference<>(initialState), - initialPosition, upgradedDocumentID); + Client client) { + super(threadPool.executor(ThreadPool.Names.GENERIC), job, new AtomicReference<>(initialState), initialPosition); this.client = client; this.job = job; } @@ -124,16 +122,9 @@ protected void doSaveState(IndexerState indexerState, Map positi next.run(); } else { // Otherwise, attempt to persist our state - // Upgrade to the new ID scheme while we are at it - boolean oldState = upgradedDocumentID.getAndSet(true); - final RollupJobStatus state = new RollupJobStatus(indexerState, getPosition(), upgradedDocumentID.get()); + final RollupJobStatus state = new RollupJobStatus(indexerState, getPosition()); logger.debug("Updating persistent state of job [" + job.getConfig().getId() + "] to [" + indexerState.toString() + "]"); - updatePersistentTaskState(state, ActionListener.wrap(task -> next.run(), exc -> { - // We failed to update the persistent task for some reason, - // set our flag back to what it was before - upgradedDocumentID.set(oldState); - next.run(); - })); + updatePersistentTaskState(state, ActionListener.wrap(task -> next.run(), exc -> next.run())); } } @@ -158,7 +149,6 @@ protected void onAbort() { private final SchedulerEngine schedulerEngine; private final ThreadPool threadPool; private final RollupIndexer indexer; - private AtomicBoolean upgradedDocumentID; RollupJobTask(long id, String type, String action, TaskId parentTask, RollupJob job, RollupJobStatus state, Client client, SchedulerEngine schedulerEngine, ThreadPool threadPool, Map headers) { @@ -167,9 +157,6 @@ protected void onAbort() { this.schedulerEngine = schedulerEngine; this.threadPool = threadPool; - // We can assume the new ID scheme only for new jobs - this.upgradedDocumentID = new AtomicBoolean(true); - // If status is not null, we are resuming rather than starting fresh. Map initialPosition = null; IndexerState initialState = IndexerState.STOPPED; @@ -196,22 +183,14 @@ protected void onAbort() { } initialPosition = state.getPosition(); - // Since we have state, we are resuming a job/checkpoint. Although we are resuming - // from something that was checkpointed, we can't guarantee it was the _final_ checkpoint - // before the job ended (e.g. it could have been STOPPING, still indexing and killed, leaving - // us with an interval of time partially indexed). - // - // To be safe, if we are resuming any job, use it's ID upgrade status. It will only - // be true if it actually finished a full checkpoint. - this.upgradedDocumentID.set(state.isUpgradedDocumentID()); } this.indexer = new ClientRollupPageManager(job, initialState, initialPosition, - new ParentTaskAssigningClient(client, new TaskId(getPersistentTaskId())), upgradedDocumentID); + new ParentTaskAssigningClient(client, new TaskId(getPersistentTaskId()))); } @Override public Status getStatus() { - return new RollupJobStatus(indexer.getState(), indexer.getPosition(), upgradedDocumentID.get()); + return new RollupJobStatus(indexer.getState(), indexer.getPosition()); } /** @@ -256,7 +235,7 @@ public synchronized void start(ActionListener lis } - final RollupJobStatus state = new RollupJobStatus(IndexerState.STARTED, indexer.getPosition(), upgradedDocumentID.get()); + final RollupJobStatus state = new RollupJobStatus(IndexerState.STARTED, indexer.getPosition()); logger.debug("Updating state for rollup job [" + job.getConfig().getId() + "] to [" + state.getIndexerState() + "][" + state.getPosition() + "]"); updatePersistentTaskState(state, @@ -300,11 +279,10 @@ public synchronized void stop(ActionListener liste case STOPPING: // update the persistent state to STOPPED. There are two scenarios and both are safe: // 1. we persist STOPPED now, indexer continues a bit then sees the flag and checkpoints another - // STOPPED with the more recent position. That will also upgrade the ID scheme + // STOPPED with the more recent position. // 2. we persist STOPPED now, indexer continues a bit but then dies. When/if we resume we'll pick up - // at last checkpoint, overwrite some docs and eventually checkpoint. At that time we'll also - // upgrade the ID scheme - RollupJobStatus state = new RollupJobStatus(IndexerState.STOPPED, indexer.getPosition(), upgradedDocumentID.get()); + // at last checkpoint, overwrite some docs and eventually checkpoint. + RollupJobStatus state = new RollupJobStatus(IndexerState.STOPPED, indexer.getPosition()); updatePersistentTaskState(state, ActionListener.wrap( (task) -> { diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index 38b90328a8743..d0c4df2244773 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -32,8 +32,8 @@ import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; import org.elasticsearch.xpack.core.rollup.RollupField; @@ -119,7 +119,7 @@ public void testMissingFields() throws IOException { directory.close(); final GroupConfig groupConfig = randomGroupConfig(random()); - List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo"); assertThat(docs.size(), equalTo(numDocs)); for (IndexRequest doc : docs) { @@ -187,7 +187,7 @@ public void testCorrectFields() throws IOException { directory.close(); final GroupConfig groupConfig = randomGroupConfig(random()); - List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo"); assertThat(docs.size(), equalTo(numDocs)); for (IndexRequest doc : docs) { @@ -243,7 +243,7 @@ public void testNumericTerms() throws IOException { directory.close(); final GroupConfig groupConfig = randomGroupConfig(random()); - List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo"); assertThat(docs.size(), equalTo(numDocs)); for (IndexRequest doc : docs) { @@ -310,7 +310,7 @@ public void testEmptyCounts() throws IOException { directory.close(); final GroupConfig groupConfig = randomGroupConfig(random()); - List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo"); assertThat(docs.size(), equalTo(numDocs)); for (IndexRequest doc : docs) { @@ -321,53 +321,7 @@ public void testEmptyCounts() throws IOException { } } - public void testKeyOrderingOldID() { - CompositeAggregation composite = mock(CompositeAggregation.class); - - when(composite.getBuckets()).thenAnswer((Answer>) invocationOnMock -> { - List foos = new ArrayList<>(); - - CompositeAggregation.Bucket bucket = mock(CompositeAggregation.Bucket.class); - LinkedHashMap keys = new LinkedHashMap<>(3); - keys.put("foo.date_histogram", 123L); - keys.put("bar.terms", "baz"); - keys.put("abc.histogram", 1.9); - keys = shuffleMap(keys, Collections.emptySet()); - when(bucket.getKey()).thenReturn(keys); - - List list = new ArrayList<>(3); - InternalNumericMetricsAggregation.SingleValue mockAgg = mock(InternalNumericMetricsAggregation.SingleValue.class); - when(mockAgg.getName()).thenReturn("123"); - list.add(mockAgg); - - InternalNumericMetricsAggregation.SingleValue mockAgg2 = mock(InternalNumericMetricsAggregation.SingleValue.class); - when(mockAgg2.getName()).thenReturn("abc"); - list.add(mockAgg2); - - InternalNumericMetricsAggregation.SingleValue mockAgg3 = mock(InternalNumericMetricsAggregation.SingleValue.class); - when(mockAgg3.getName()).thenReturn("yay"); - list.add(mockAgg3); - - Collections.shuffle(list, random()); - - Aggregations aggs = new Aggregations(list); - when(bucket.getAggregations()).thenReturn(aggs); - when(bucket.getDocCount()).thenReturn(1L); - - foos.add(bucket); - - return foos; - }); - - // The content of the config don't actually matter for this test - // because the test is just looking at agg keys - GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(123L, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo", false); - assertThat(docs.size(), equalTo(1)); - assertThat(docs.get(0).id(), equalTo("1237859798")); - } - - public void testKeyOrderingNewID() { + public void testKeyOrdering() { CompositeAggregation composite = mock(CompositeAggregation.class); when(composite.getBuckets()).thenAnswer((Answer>) invocationOnMock -> { @@ -406,7 +360,7 @@ public void testKeyOrderingNewID() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(1L, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo", true); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo"); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("foo$c9LcrFqeFW92uN_Z7sv1hA")); } @@ -414,7 +368,7 @@ public void testKeyOrderingNewID() { /* A test to make sure very long keys don't break the hash */ - public void testKeyOrderingNewIDLong() { + public void testKeyOrderingLong() { CompositeAggregation composite = mock(CompositeAggregation.class); when(composite.getBuckets()).thenAnswer((Answer>) invocationOnMock -> { @@ -456,7 +410,7 @@ public void testKeyOrderingNewIDLong() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), new HistogramGroupConfig(1, "abc"), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo", true); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo"); assertThat(docs.size(), equalTo(1)); assertThat(docs.get(0).id(), equalTo("foo$VAFKZpyaEqYRPLyic57_qw")); } @@ -483,8 +437,7 @@ public void testNullKeys() { }); GroupConfig groupConfig = new GroupConfig(randomDateHistogramGroupConfig(random()), randomHistogramGroupConfig(random()), null); - List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), - groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, "foo", new RollupIndexerJobStats(), groupConfig, "foo"); assertThat(docs.size(), equalTo(1)); assertFalse(Strings.isNullOrEmpty(docs.get(0).id())); } @@ -548,7 +501,7 @@ public void testMissingBuckets() throws IOException { directory.close(); final GroupConfig groupConfig = randomGroupConfig(random()); - List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo"); assertThat(docs.size(), equalTo(6)); for (IndexRequest doc : docs) { diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java index 743d1d94e6040..0528494ebaa5f 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java @@ -72,7 +72,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -83,7 +82,6 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase { private QueryShardContext queryShardContext; private IndexSettings settings; - private final boolean newIDScheme = randomBoolean(); @Before private void setup() { @@ -113,7 +111,7 @@ public void testSimpleDateHisto() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", 3, "the_histo.date_histogram.interval", "1ms", "the_histo.date_histogram._count", 2, @@ -126,7 +124,7 @@ public void testSimpleDateHisto() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", 7, "the_histo.date_histogram.interval", "1ms", "the_histo.date_histogram._count", 1, @@ -171,7 +169,7 @@ public void testDateHistoAndMetrics() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T03:00:00.000Z"), "the_histo.date_histogram.interval", "1h", "the_histo.date_histogram._count", 3, @@ -189,7 +187,7 @@ public void testDateHistoAndMetrics() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T04:00:00.000Z"), "the_histo.date_histogram.interval", "1h", "the_histo.date_histogram._count", 3, @@ -207,7 +205,7 @@ public void testDateHistoAndMetrics() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T05:00:00.000Z"), "the_histo.date_histogram.interval", "1h", "the_histo.date_histogram._count", 4, @@ -225,7 +223,7 @@ public void testDateHistoAndMetrics() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T06:00:00.000Z"), "the_histo.date_histogram.interval", "1h", "the_histo.date_histogram._count", 3, @@ -243,7 +241,7 @@ public void testDateHistoAndMetrics() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T07:00:00.000Z"), "the_histo.date_histogram.interval", "1h", "the_histo.date_histogram._count", 3, @@ -289,7 +287,7 @@ public void testSimpleDateHistoWithDelay() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", rounding.round(now - TimeValue.timeValueHours(5).getMillis()), "the_histo.date_histogram.interval", "1m", "the_histo.date_histogram._count", 2, @@ -302,7 +300,7 @@ public void testSimpleDateHistoWithDelay() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", rounding.round(now - TimeValue.timeValueMinutes(75).getMillis()), "the_histo.date_histogram.interval", "1m", "the_histo.date_histogram._count", 2, @@ -315,7 +313,7 @@ public void testSimpleDateHistoWithDelay() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", rounding.round(now - TimeValue.timeValueMinutes(61).getMillis()), "the_histo.date_histogram.interval", "1m", "the_histo.date_histogram._count", 1, @@ -354,7 +352,7 @@ public void testSimpleDateHistoWithTimeZone() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T03:00:00.000Z"), "the_histo.date_histogram.interval", "1d", "the_histo.date_histogram._count", 2, @@ -373,7 +371,7 @@ public void testSimpleDateHistoWithTimeZone() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-03-31T03:00:00.000Z"), "the_histo.date_histogram.interval", "1d", "the_histo.date_histogram._count", 2, @@ -386,7 +384,7 @@ public void testSimpleDateHistoWithTimeZone() throws Exception { assertThat(request.type(), equalTo("_doc")); assertThat(request.sourceAsMap(), equalTo( asMap( - "_rollup.version", newIDScheme ? 2 : 1, + "_rollup.version", 2, "the_histo.date_histogram.timestamp", asLong("2015-04-01T03:00:00.000Z"), "the_histo.date_histogram.interval", "1d", "the_histo.date_histogram._count", 5, @@ -425,7 +423,7 @@ public void testRandomizedDateHisto() throws Exception { Map source = ((IndexRequest) request).sourceAsMap(); - assertThat(source.get("_rollup.version"), equalTo(newIDScheme ? 2 : 1)); + assertThat(source.get("_rollup.version"), equalTo(2)); assertThat(source.get("ts.date_histogram.interval"), equalTo(timeInterval.toString())); assertNotNull(source.get("the_avg.avg._count")); assertNotNull(source.get("the_avg.avg.value")); @@ -574,7 +572,7 @@ class SyncRollupIndexer extends RollupIndexer { SyncRollupIndexer(Executor executor, RollupJob job, IndexSearcher searcher, MappedFieldType[] fieldTypes, MappedFieldType timestampField) { - super(executor, job, new AtomicReference<>(IndexerState.STARTED), null, new AtomicBoolean(newIDScheme)); + super(executor, job, new AtomicReference<>(IndexerState.STARTED), null); this.searcher = searcher; this.fieldTypes = fieldTypes; this.timestampField = timestampField; diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java index e6264b02bca16..db03ed0fb66ee 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java @@ -53,21 +53,15 @@ public class RollupIndexerStateTests extends ESTestCase { private static class EmptyRollupIndexer extends RollupIndexer { EmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, - Map initialPosition, boolean upgraded, RollupIndexerJobStats stats) { - super(executor, job, initialState, initialPosition, new AtomicBoolean(upgraded), stats); - } - - EmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, - Map initialPosition, boolean upgraded) { - super(executor, job, initialState, initialPosition, new AtomicBoolean(upgraded)); + Map initialPosition, RollupIndexerJobStats stats) { + super(executor, job, initialState, initialPosition, stats); } EmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition) { - this(executor, job, initialState, initialPosition, randomBoolean()); + super(executor, job, initialState, initialPosition); } - @Override protected void doNextSearch(SearchRequest request, ActionListener nextPhase) { // TODO Should use InternalComposite constructor but it is package protected in core. @@ -140,19 +134,14 @@ protected void onFinish(ActionListener listener) { private static class DelayedEmptyRollupIndexer extends EmptyRollupIndexer { protected CountDownLatch latch; - DelayedEmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, - Map initialPosition, boolean upgraded) { - super(executor, job, initialState, initialPosition, upgraded); - } - DelayedEmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition) { - super(executor, job, initialState, initialPosition, randomBoolean()); + super(executor, job, initialState, initialPosition); } DelayedEmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition, RollupIndexerJobStats stats) { - super(executor, job, initialState, initialPosition, randomBoolean(), stats); + super(executor, job, initialState, initialPosition, stats); } private CountDownLatch newLatch() { @@ -180,7 +169,7 @@ private static class NonEmptyRollupIndexer extends RollupIndexer { NonEmptyRollupIndexer(Executor executor, RollupJob job, AtomicReference initialState, Map initialPosition, Function searchFunction, Function bulkFunction, Consumer failureConsumer) { - super(executor, job, initialState, initialPosition, new AtomicBoolean(randomBoolean())); + super(executor, job, initialState, initialPosition); this.searchFunction = searchFunction; this.bulkFunction = bulkFunction; this.failureConsumer = failureConsumer; @@ -245,8 +234,7 @@ public void testStarted() throws Exception { AtomicReference state = new AtomicReference<>(IndexerState.STOPPED); final ExecutorService executor = Executors.newFixedThreadPool(1); try { - RollupIndexer indexer = new EmptyRollupIndexer(executor, job, state, null, true); - assertTrue(indexer.isUpgradedDocumentID()); + RollupIndexer indexer = new EmptyRollupIndexer(executor, job, state, null); indexer.start(); assertThat(indexer.getState(), equalTo(IndexerState.STARTED)); assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis())); 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 59073e763c259..85962a5cfdbcf 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 @@ -63,7 +63,7 @@ public static void stopThreadPool() { public void testInitialStatusStopped() { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -76,7 +76,7 @@ public void testInitialStatusStopped() { public void testInitialStatusAborting() { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.ABORTING, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.ABORTING, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -89,7 +89,7 @@ public void testInitialStatusAborting() { public void testInitialStatusStopping() { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPING, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPING, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -102,7 +102,7 @@ public void testInitialStatusStopping() { public void testInitialStatusStarted() { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STARTED, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STARTED, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -115,7 +115,7 @@ public void testInitialStatusStarted() { public void testInitialStatusIndexingOldID() { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.INDEXING, Collections.singletonMap("foo", "bar"), false); + RollupJobStatus status = new RollupJobStatus(IndexerState.INDEXING, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -124,12 +124,11 @@ public void testInitialStatusIndexingOldID() { assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); - assertFalse(((RollupJobStatus) task.getStatus()).isUpgradedDocumentID()); } public void testInitialStatusIndexingNewID() { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.INDEXING, Collections.singletonMap("foo", "bar"), true); + RollupJobStatus status = new RollupJobStatus(IndexerState.INDEXING, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -138,7 +137,6 @@ public void testInitialStatusIndexingNewID() { assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); assertThat(((RollupJobStatus)task.getStatus()).getPosition().size(), equalTo(1)); assertTrue(((RollupJobStatus)task.getStatus()).getPosition().containsKey("foo")); - assertTrue(((RollupJobStatus) task.getStatus()).isUpgradedDocumentID()); } public void testNoInitialStatus() { @@ -150,12 +148,11 @@ public void testNoInitialStatus() { null, client, schedulerEngine, pool, Collections.emptyMap()); assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); - assertTrue(((RollupJobStatus) task.getStatus()).isUpgradedDocumentID()); } public void testStartWhenStarted() throws InterruptedException { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STARTED, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STARTED, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -263,7 +260,7 @@ public void onFailure(Exception e) { public void testStartWhenStopped() throws InterruptedException { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); @@ -301,7 +298,7 @@ public void onFailure(Exception e) { public void testTriggerUnrelated() throws InterruptedException { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, Collections.singletonMap("foo", "bar"), randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, Collections.singletonMap("foo", "bar")); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); @@ -426,10 +423,6 @@ public void updatePersistentTaskState(PersistentTaskState taskState, listener.onResponse(new PersistentTasksCustomMetaData.PersistentTask<>("foo", RollupField.TASK_NAME, job, 1, new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } else if (counterValue == 1) { - // When we get here, doSaveState() was just invoked so we will have - // have upgraded IDs - RollupJobStatus s = (RollupJobStatus)this.getStatus(); - assertTrue(s.isUpgradedDocumentID()); finished.set(true); } @@ -438,11 +431,11 @@ public void updatePersistentTaskState(PersistentTaskState taskState, assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); - task.start(new ActionListener() { + task.start(new ActionListener<>() { @Override public void onResponse(StartRollupJobAction.Response response) { assertTrue(response.isStarted()); - assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); + assertThat(((RollupJobStatus) task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); started.set(true); } @@ -514,10 +507,6 @@ public void updatePersistentTaskState(PersistentTaskState taskState, listener.onResponse(new PersistentTasksCustomMetaData.PersistentTask<>("foo", RollupField.TASK_NAME, job, 1, new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } else if (counterValue == 1) { - // When we get here, doSaveState() was just invoked so we will have - // have upgraded IDs - RollupJobStatus s = (RollupJobStatus)this.getStatus(); - assertTrue(s.isUpgradedDocumentID()); finished.set(true); } @@ -526,11 +515,11 @@ public void updatePersistentTaskState(PersistentTaskState taskState, assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); - task.start(new ActionListener() { + task.start(new ActionListener<>() { @Override public void onResponse(StartRollupJobAction.Response response) { assertTrue(response.isStarted()); - assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); + assertThat(((RollupJobStatus) task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); started.set(true); } @@ -590,7 +579,7 @@ public void testSaveStateChangesIDScheme() throws InterruptedException { }).when(client).execute(anyObject(), anyObject(), anyObject()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null, false); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null); RollupJobTask task = new RollupJobTask(1, "type", "action", new TaskId("node", 123), job, status, client, schedulerEngine, pool, Collections.emptyMap()) { @Override @@ -603,10 +592,6 @@ public void updatePersistentTaskState(PersistentTaskState taskState, listener.onResponse(new PersistentTasksCustomMetaData.PersistentTask<>("foo", RollupField.TASK_NAME, job, 1, new PersistentTasksCustomMetaData.Assignment("foo", "foo"))); } else if (counterValue == 1) { - // When we get here, doSaveState() was just invoked so we will have - // have upgraded IDs - RollupJobStatus s = (RollupJobStatus)this.getStatus(); - assertTrue(s.isUpgradedDocumentID()); finished.set(true); } @@ -615,11 +600,11 @@ public void updatePersistentTaskState(PersistentTaskState taskState, assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STOPPED)); assertNull(((RollupJobStatus)task.getStatus()).getPosition()); - task.start(new ActionListener() { + task.start(new ActionListener<>() { @Override public void onResponse(StartRollupJobAction.Response response) { assertTrue(response.isStarted()); - assertThat(((RollupJobStatus)task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); + assertThat(((RollupJobStatus) task.getStatus()).getIndexerState(), equalTo(IndexerState.STARTED)); started.set(true); } @@ -642,7 +627,7 @@ public void onFailure(Exception e) { public void testStopWhenStopped() throws InterruptedException { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null, randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); @@ -749,7 +734,7 @@ public void onFailure(Exception e) { public void testStopWhenAborting() throws InterruptedException { RollupJob job = new RollupJob(ConfigTestHelpers.randomRollupJobConfig(random()), Collections.emptyMap()); - RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null, randomBoolean()); + RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null); Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); SchedulerEngine schedulerEngine = new SchedulerEngine(SETTINGS, Clock.systemUTC()); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml index 1710e51c32bdc..c9902514cc339 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/delete_job.yml @@ -77,7 +77,6 @@ setup: search_total: 0 status: job_state: "stopped" - upgraded_doc_id: true - do: rollup.delete_job: @@ -130,7 +129,6 @@ setup: search_total: 0 status: job_state: "stopped" - upgraded_doc_id: true - do: rollup.delete_job: @@ -183,7 +181,6 @@ setup: search_total: 0 status: job_state: "stopped" - upgraded_doc_id: true - do: rollup.start_job: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml index cd00a6f717b02..629bad5479c19 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_jobs.yml @@ -78,7 +78,6 @@ setup: search_total: 0 status: job_state: "stopped" - upgraded_doc_id: true --- "Test get with no jobs": @@ -189,7 +188,6 @@ setup: trigger_count: 0 status: job_state: "stopped" - upgraded_doc_id: true - config: id: "bar" index_pattern: "bar" @@ -221,6 +219,5 @@ setup: search_total: 0 status: job_state: "stopped" - upgraded_doc_id: true diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml index 7983778108bd0..220d8992a7c0c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/put_job.yml @@ -78,7 +78,6 @@ setup: search_total: 0 status: job_state: "stopped" - upgraded_doc_id: true --- "Test put_job with existing name": diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java index a5579ad0aa5d8..5aa38ea7f7d66 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java @@ -14,13 +14,13 @@ import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.ObjectPath; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.rest.ESRestTestCase; -import org.elasticsearch.common.xcontent.ObjectPath; import java.io.IOException; import java.time.Instant; diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RollupIDUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RollupIDUpgradeIT.java deleted file mode 100644 index d986f79eb396f..0000000000000 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RollupIDUpgradeIT.java +++ /dev/null @@ -1,10 +0,0 @@ -/* - * 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.upgrades; - -public class RollupIDUpgradeIT extends AbstractUpgradeTestCase { - -} From 7d29e37a21fe6a3ad8424ccc302bf25ccb864521 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Wed, 17 Apr 2019 15:58:13 -0400 Subject: [PATCH 083/260] Fix bad compilation in test Method signature changed --- .../org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java index d0c4df2244773..85401aa187d8a 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/IndexerUtilsTests.java @@ -579,7 +579,7 @@ public void testTimezone() throws IOException { directory.close(); final GroupConfig groupConfig = randomGroupConfig(random()); - List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo", randomBoolean()); + List docs = IndexerUtils.processBuckets(composite, indexName, stats, groupConfig, "foo"); assertThat(docs.size(), equalTo(2)); From 8fb43ab99468ee86d5af9ee6c8817fcf8b16f479 Mon Sep 17 00:00:00 2001 From: Bob Blank <39975390+BobBlank12@users.noreply.github.com> Date: Wed, 17 Apr 2019 16:06:03 -0500 Subject: [PATCH 084/260] Fixing missing link on session_token (#41314) session_token was missing a "secure link" --- docs/plugins/repository-s3.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/plugins/repository-s3.asciidoc b/docs/plugins/repository-s3.asciidoc index 084d67f236472..e2e489bb93eea 100644 --- a/docs/plugins/repository-s3.asciidoc +++ b/docs/plugins/repository-s3.asciidoc @@ -94,10 +94,10 @@ settings belong in the `elasticsearch.yml` file. An S3 secret key. The `access_key` setting must also be specified. -`session_token`:: +`session_token` ({ref}/secure-settings.html[Secure]):: An S3 session token. The `access_key` and `secret_key` settings must also be - specified. (Secure) + specified. `endpoint`:: From 78e7f4a94c4a633100c86c02d073f7f6a26444b5 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Thu, 18 Apr 2019 08:51:10 +0300 Subject: [PATCH 085/260] Clean up clusters between tests (#41187) This PR adds additional cleanup when stopping the node. The data dir is excepted because it gets reused in some tests. Without this cleanup the number of working dir copies could grew to exhaust all available disk space. --- .../org/elasticsearch/gradle/test/ClusterFormationTasks.groovy | 2 ++ 1 file changed, 2 insertions(+) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 274a705ab8add..1b5eb26f9d322 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -936,6 +936,8 @@ class ClusterFormationTasks { } doLast { project.delete(node.pidFile) + // Large tests can exhaust disk space, clean up on stop, but leave the data dir as some tests reuse it + project.delete(project.fileTree(node.baseDir).minus(project.fileTree(node.dataDir))) } } } From 9aa2a8f5a5bcf7c92f52d9acc779057f24ccce30 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Thu, 18 Apr 2019 09:25:30 +0300 Subject: [PATCH 086/260] Mute failing test Tracking #41326 --- .../org/elasticsearch/snapshots/SnapshotResiliencyTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 71c44d1e702b9..b2e7562286caa 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -378,6 +378,7 @@ public void testSnapshotWithNodeDisconnects() { assertThat(snapshotIds, hasSize(1)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41326") public void testConcurrentSnapshotCreateAndDelete() { setupTestCluster(randomFrom(1, 3, 5), randomIntBetween(2, 10)); @@ -427,6 +428,7 @@ public void testConcurrentSnapshotCreateAndDelete() { * Simulates concurrent restarts of data and master nodes as well as relocating a primary shard, while starting and subsequently * deleting a snapshot. */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41326") public void testSnapshotPrimaryRelocations() { final int masterNodeCount = randomFrom(1, 3, 5); setupTestCluster(masterNodeCount, randomIntBetween(2, 10)); From d29fa1ab9899ee6dc5f27b1369a01219487fd139 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 18 Apr 2019 08:42:06 +0200 Subject: [PATCH 087/260] Mute BulkProcessorRetryIT#testBulkRejectionLoadWithBackoff (#41325) * For #41324 --- .../test/java/org/elasticsearch/client/BulkProcessorRetryIT.java | 1 + .../java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java | 1 + 2 files changed, 2 insertions(+) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java index e3b280225cb7b..19e2de22e54ef 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java @@ -56,6 +56,7 @@ public void testBulkRejectionLoadWithoutBackoff() throws Exception { executeBulkRejectionLoad(BackoffPolicy.noBackoff(), rejectedExecutionExpected); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41324") public void testBulkRejectionLoadWithBackoff() throws Throwable { boolean rejectedExecutionExpected = false; executeBulkRejectionLoad(BackoffPolicy.exponentialBackoff(), rejectedExecutionExpected); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index e4b6fff9fc353..054e213d5abe9 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -64,6 +64,7 @@ public void testBulkRejectionLoadWithoutBackoff() throws Throwable { executeBulkRejectionLoad(BackoffPolicy.noBackoff(), rejectedExecutionExpected); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41324") public void testBulkRejectionLoadWithBackoff() throws Throwable { boolean rejectedExecutionExpected = false; executeBulkRejectionLoad(BackoffPolicy.exponentialBackoff(), rejectedExecutionExpected); From 1350cdc37e7efc2c61f7aa758ca3d29789d6a999 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Thu, 18 Apr 2019 09:50:49 +0300 Subject: [PATCH 088/260] Disable composePull only if it exists (#41306) The task will not be created when docker is not available. --- distribution/docker/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/docker/build.gradle b/distribution/docker/build.gradle index 71fc62673dca9..00fceb9df6a14 100644 --- a/distribution/docker/build.gradle +++ b/distribution/docker/build.gradle @@ -142,4 +142,4 @@ assemble.dependsOn "buildDockerImage" // We build the images used in compose locally, but the pull command insists on using a repository // thus we must disable it to prevent it from doing so. // Everything will still be pulled since we will build the local images on a pull -composePull.enabled = false +tasks.matching { name == "composePull" }.all { enabled = false } From b8396c0a8e275c133d204aa0f9a04be1a1230d37 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Apr 2019 11:18:25 +0200 Subject: [PATCH 089/260] Disable CcrRetentionLeaseIT#testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes. Relates #39331. --- .../java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index bfa142a64e789..6f97a6ba5867b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -266,6 +266,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { } + @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/39331") public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws Exception { final String leaderIndex = "leader"; final int numberOfShards = randomIntBetween(1, 3); From acc57d0ef21961508f47f8f1cf58af8869eb99b5 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Apr 2019 11:30:52 +0200 Subject: [PATCH 090/260] Revert "Disable CcrRetentionLeaseIT#testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes." This reverts commit b8396c0a8e275c133d204aa0f9a04be1a1230d37. --- .../java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 6f97a6ba5867b..bfa142a64e789 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -266,7 +266,6 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { } - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/39331") public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws Exception { final String leaderIndex = "leader"; final int numberOfShards = randomIntBetween(1, 3); From a0c4eabe8dbf902d8401463e2789f0aa0b3f8b45 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 18 Apr 2019 14:29:20 +0200 Subject: [PATCH 091/260] Simplify Some Code in Snapshotting (#41316) * Just a few obvious simplifications and dead code removals --- .../snapshots/IndexShardSnapshotStatus.java | 10 ++++----- .../BlobStoreIndexShardSnapshot.java | 17 +++----------- .../BlobStoreIndexShardSnapshots.java | 22 +++++-------------- .../snapshots/blobstore/SnapshotFiles.java | 2 +- .../repositories/RepositoriesService.java | 4 ++-- .../repositories/RepositoryData.java | 21 ++++-------------- .../repositories/VerificationFailure.java | 20 +---------------- .../blobstore/BlobStoreFormat.java | 3 +-- .../blobstore/BlobStoreRepository.java | 15 +++++-------- .../blobstore/ChecksumBlobStoreFormat.java | 2 +- .../snapshots/SnapshotShardsService.java | 2 +- .../snapshots/blobstore/FileInfoTests.java | 3 +-- .../repositories/RepositoryDataTests.java | 7 +++--- 13 files changed, 33 insertions(+), 95 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java index bfc3faae9344f..a558051c58d7e 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java @@ -71,8 +71,7 @@ public enum Stage { private IndexShardSnapshotStatus(final Stage stage, final long startTime, final long totalTime, final int incrementalFileCount, final int totalFileCount, final int processedFileCount, - final long incrementalSize, final long totalSize, final long processedSize, - final long indexVersion, final String failure) { + final long incrementalSize, final long totalSize, final long processedSize, final String failure) { this.stage = new AtomicReference<>(Objects.requireNonNull(stage)); this.startTime = startTime; this.totalTime = totalTime; @@ -82,7 +81,6 @@ private IndexShardSnapshotStatus(final Stage stage, final long startTime, final this.totalSize = totalSize; this.processedSize = processedSize; this.incrementalSize = incrementalSize; - this.indexVersion = indexVersion; this.failure = failure; } @@ -161,7 +159,7 @@ public synchronized IndexShardSnapshotStatus.Copy asCopy() { } public static IndexShardSnapshotStatus newInitializing() { - return new IndexShardSnapshotStatus(Stage.INIT, 0L, 0L, 0, 0, 0, 0, 0, 0, 0, null); + return new IndexShardSnapshotStatus(Stage.INIT, 0L, 0L, 0, 0, 0, 0, 0, 0, null); } public static IndexShardSnapshotStatus newFailed(final String failure) { @@ -169,7 +167,7 @@ public static IndexShardSnapshotStatus newFailed(final String failure) { if (failure == null) { throw new IllegalArgumentException("A failure description is required for a failed IndexShardSnapshotStatus"); } - return new IndexShardSnapshotStatus(Stage.FAILURE, 0L, 0L, 0, 0, 0, 0, 0, 0, 0, failure); + return new IndexShardSnapshotStatus(Stage.FAILURE, 0L, 0L, 0, 0, 0, 0, 0, 0, failure); } public static IndexShardSnapshotStatus newDone(final long startTime, final long totalTime, @@ -177,7 +175,7 @@ public static IndexShardSnapshotStatus newDone(final long startTime, final long final long incrementalSize, final long size) { // The snapshot is done which means the number of processed files is the same as total return new IndexShardSnapshotStatus(Stage.DONE, startTime, totalTime, incrementalFileCount, fileCount, incrementalFileCount, - incrementalSize, size, incrementalSize, 0, null); + incrementalSize, size, incrementalSize, null); } /** diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java index 297c12744cc26..d3435bf8cbf37 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -247,9 +246,8 @@ public boolean hasUnknownChecksum() { * * @param file file info * @param builder XContent builder - * @param params parameters */ - public static void toXContent(FileInfo file, XContentBuilder builder, ToXContent.Params params) throws IOException { + public static void toXContent(FileInfo file, XContentBuilder builder) throws IOException { builder.startObject(); builder.field(NAME, file.name); builder.field(PHYSICAL_NAME, file.metadata.name()); @@ -382,22 +380,13 @@ public BlobStoreIndexShardSnapshot(String snapshot, long indexVersion, List= 0; this.snapshot = snapshot; this.indexVersion = indexVersion; - this.indexFiles = Collections.unmodifiableList(new ArrayList<>(indexFiles)); + this.indexFiles = List.copyOf(indexFiles); this.startTime = startTime; this.time = time; this.incrementalFileCount = incrementalFileCount; this.incrementalSize = incrementalSize; } - /** - * Returns index version - * - * @return index version - */ - public long indexVersion() { - return indexVersion; - } - /** * Returns snapshot id * @@ -493,7 +482,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(INCREMENTAL_SIZE, incrementalSize); builder.startArray(FILES); for (FileInfo fileInfo : indexFiles) { - FileInfo.toXContent(fileInfo, builder, params); + FileInfo.toXContent(fileInfo, builder); } builder.endArray(); return builder; diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java index 34b07932e48ff..c9957a0483e76 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java @@ -49,7 +49,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable, To private final Map> physicalFiles; public BlobStoreIndexShardSnapshots(List shardSnapshots) { - this.shardSnapshots = Collections.unmodifiableList(new ArrayList<>(shardSnapshots)); + this.shardSnapshots = List.copyOf(shardSnapshots); // Map between blob names and file info Map newFiles = new HashMap<>(); // Map between original physical names and file info @@ -65,17 +65,12 @@ public BlobStoreIndexShardSnapshots(List shardSnapshots) { // the first loop de-duplicates fileInfo objects that were loaded from different snapshots but refer to // the same blob for (FileInfo fileInfo : snapshot.indexFiles()) { - List physicalFileList = physicalFiles.get(fileInfo.physicalName()); - if (physicalFileList == null) { - physicalFileList = new ArrayList<>(); - physicalFiles.put(fileInfo.physicalName(), physicalFileList); - } - physicalFileList.add(newFiles.get(fileInfo.name())); + physicalFiles.computeIfAbsent(fileInfo.physicalName(), k -> new ArrayList<>()).add(newFiles.get(fileInfo.name())); } } Map> mapBuilder = new HashMap<>(); for (Map.Entry> entry : physicalFiles.entrySet()) { - mapBuilder.put(entry.getKey(), Collections.unmodifiableList(new ArrayList<>(entry.getValue()))); + mapBuilder.put(entry.getKey(), List.copyOf(entry.getValue())); } this.physicalFiles = unmodifiableMap(mapBuilder); this.files = unmodifiableMap(newFiles); @@ -87,17 +82,12 @@ private BlobStoreIndexShardSnapshots(Map files, List> physicalFiles = new HashMap<>(); for (SnapshotFiles snapshot : shardSnapshots) { for (FileInfo fileInfo : snapshot.indexFiles()) { - List physicalFileList = physicalFiles.get(fileInfo.physicalName()); - if (physicalFileList == null) { - physicalFileList = new ArrayList<>(); - physicalFiles.put(fileInfo.physicalName(), physicalFileList); - } - physicalFileList.add(files.get(fileInfo.name())); + physicalFiles.computeIfAbsent(fileInfo.physicalName(), k -> new ArrayList<>()).add(files.get(fileInfo.name())); } } Map> mapBuilder = new HashMap<>(); for (Map.Entry> entry : physicalFiles.entrySet()) { - mapBuilder.put(entry.getKey(), Collections.unmodifiableList(new ArrayList<>(entry.getValue()))); + mapBuilder.put(entry.getKey(), List.copyOf(entry.getValue())); } this.physicalFiles = unmodifiableMap(mapBuilder); } @@ -203,7 +193,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws // First we list all blobs with their file infos: builder.startArray(Fields.FILES); for (Map.Entry entry : files.entrySet()) { - FileInfo.toXContent(entry.getValue(), builder, params); + FileInfo.toXContent(entry.getValue(), builder); } builder.endArray(); // Then we list all snapshots with list of all blobs that are used by the snapshot diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java index 17e54878fed7a..6466ecbaecbe4 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java @@ -66,7 +66,7 @@ public boolean containPhysicalIndexFile(String physicalName) { * @param physicalName the original file name * @return information about this file */ - public FileInfo findPhysicalIndexFile(String physicalName) { + private FileInfo findPhysicalIndexFile(String physicalName) { if (physicalFiles == null) { Map files = new HashMap<>(); for(FileInfo fileInfo : indexFiles) { diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index e141d0d6014e7..bb20b2085d456 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -119,7 +119,7 @@ public void registerRepository(final PutRepositoryRequest request, final ActionL } clusterService.submitStateUpdateTask("put_repository [" + request.name() + "]", - new AckedClusterStateUpdateTask(request, registrationListener) { + new AckedClusterStateUpdateTask<>(request, registrationListener) { @Override protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { return new ClusterStateUpdateResponse(acknowledged); @@ -186,7 +186,7 @@ public boolean mustAck(DiscoveryNode discoveryNode) { */ public void unregisterRepository(final DeleteRepositoryRequest request, final ActionListener listener) { clusterService.submitStateUpdateTask("delete_repository [" + request.name() + "]", - new AckedClusterStateUpdateTask(request, listener) { + new AckedClusterStateUpdateTask<>(request, listener) { @Override protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { return new ClusterStateUpdateResponse(acknowledged); diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java index a97cf4bb419a2..589e0432c032d 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java @@ -169,18 +169,7 @@ public RepositoryData addSnapshot(final SnapshotId snapshotId, newSnapshotStates.put(snapshotId.getUUID(), snapshotState); Map> allIndexSnapshots = new HashMap<>(indexSnapshots); for (final IndexId indexId : snapshottedIndices) { - if (allIndexSnapshots.containsKey(indexId)) { - Set ids = allIndexSnapshots.get(indexId); - if (ids == null) { - ids = new LinkedHashSet<>(); - allIndexSnapshots.put(indexId, ids); - } - ids.add(snapshotId); - } else { - Set ids = new LinkedHashSet<>(); - ids.add(snapshotId); - allIndexSnapshots.put(indexId, ids); - } + allIndexSnapshots.computeIfAbsent(indexId, k -> new LinkedHashSet<>()).add(snapshotId); } return new RepositoryData(genId, snapshots, newSnapshotStates, allIndexSnapshots, incompatibleSnapshotIds); } @@ -308,7 +297,7 @@ public List resolveNewIndices(final List indicesToResolve) { * Writes the snapshots metadata and the related indices metadata to x-content, omitting the * incompatible snapshots. */ - public XContentBuilder snapshotsToXContent(final XContentBuilder builder, final ToXContent.Params params) throws IOException { + public XContentBuilder snapshotsToXContent(final XContentBuilder builder) throws IOException { builder.startObject(); // write the snapshots list builder.startArray(SNAPSHOTS); @@ -453,18 +442,16 @@ public static RepositoryData snapshotsFromXContent(final XContentParser parser, /** * Writes the incompatible snapshot ids to x-content. */ - public XContentBuilder incompatibleSnapshotsToXContent(final XContentBuilder builder, final ToXContent.Params params) - throws IOException { + public void incompatibleSnapshotsToXContent(final XContentBuilder builder) throws IOException { builder.startObject(); // write the incompatible snapshots list builder.startArray(INCOMPATIBLE_SNAPSHOTS); for (final SnapshotId snapshot : getIncompatibleSnapshotIds()) { - snapshot.toXContent(builder, params); + snapshot.toXContent(builder, ToXContent.EMPTY_PARAMS); } builder.endArray(); builder.endObject(); - return builder; } /** diff --git a/server/src/main/java/org/elasticsearch/repositories/VerificationFailure.java b/server/src/main/java/org/elasticsearch/repositories/VerificationFailure.java index 14eb27ec70ede..c260bfa1d20ba 100644 --- a/server/src/main/java/org/elasticsearch/repositories/VerificationFailure.java +++ b/server/src/main/java/org/elasticsearch/repositories/VerificationFailure.java @@ -31,23 +31,11 @@ public class VerificationFailure implements Streamable { private Exception cause; - VerificationFailure() { - - } - - public VerificationFailure(String nodeId, Exception cause) { + VerificationFailure(String nodeId, Exception cause) { this.nodeId = nodeId; this.cause = cause; } - public String nodeId() { - return nodeId; - } - - public Throwable cause() { - return cause; - } - @Override public void readFrom(StreamInput in) throws IOException { nodeId = in.readOptionalString(); @@ -60,12 +48,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeException(cause); } - public static VerificationFailure readNode(StreamInput in) throws IOException { - VerificationFailure failure = new VerificationFailure(); - failure.readFrom(in); - return failure; - } - @Override public String toString() { return "[" + nodeId + ", '" + cause + "']"; diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java index dc9f8092e3fc0..617fc6998a62a 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java @@ -111,8 +111,7 @@ public String blobName(String name) { protected T read(BytesReference bytes) throws IOException { try (XContentParser parser = XContentHelper .createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, bytes)) { - T obj = reader.apply(parser); - return obj; + return reader.apply(parser); } } 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 cb674d9c521be..41e2355a21f1d 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -65,7 +65,6 @@ import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; @@ -476,7 +475,7 @@ private void deleteIndices(List indices, SnapshotId snapshotId, ActionL } final ActionListener groupedListener = new GroupedActionListener<>(ActionListener.map(listener, v -> null), indices.size()); for (IndexId indexId: indices) { - threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable(groupedListener) { + threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable<>(groupedListener) { @Override protected void doRun() { @@ -516,9 +515,6 @@ private void deleteIndexMetaDataBlobIgnoringErrors(SnapshotId snapshotId, IndexI } } - /** - * {@inheritDoc} - */ @Override public SnapshotInfo finalizeSnapshot(final SnapshotId snapshotId, final List indices, @@ -718,7 +714,7 @@ protected void writeIndexGen(final RepositoryData repositoryData, final long rep try (BytesStreamOutput bStream = new BytesStreamOutput()) { try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) { XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream); - repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS); + repositoryData.snapshotsToXContent(builder); builder.close(); } snapshotsBytes = bStream.bytes(); @@ -752,10 +748,9 @@ void writeIncompatibleSnapshots(RepositoryData repositoryData) throws IOExceptio assert isReadOnly() == false; // can not write to a read only repository final BytesReference bytes; try (BytesStreamOutput bStream = new BytesStreamOutput()) { - try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) { - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream); - repositoryData.incompatibleSnapshotsToXContent(builder, ToXContent.EMPTY_PARAMS); - builder.close(); + try (StreamOutput stream = new OutputStreamStreamOutput(bStream); + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream)) { + repositoryData.incompatibleSnapshotsToXContent(builder); } bytes = bStream.bytes(); } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index ca6ec74dc2ce2..d4d009b8ad63e 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -163,7 +163,7 @@ private void writeTo(final T obj, final String blobName, final CheckedConsumer() { diff --git a/server/src/test/java/org/elasticsearch/index/snapshots/blobstore/FileInfoTests.java b/server/src/test/java/org/elasticsearch/index/snapshots/blobstore/FileInfoTests.java index 626b2b0e0e2b8..419730c81a3b7 100644 --- a/server/src/test/java/org/elasticsearch/index/snapshots/blobstore/FileInfoTests.java +++ b/server/src/test/java/org/elasticsearch/index/snapshots/blobstore/FileInfoTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; @@ -56,7 +55,7 @@ public void testToFromXContent() throws IOException { ByteSizeValue size = new ByteSizeValue(Math.abs(randomLong())); BlobStoreIndexShardSnapshot.FileInfo info = new BlobStoreIndexShardSnapshot.FileInfo("_foobar", meta, size); XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON).prettyPrint(); - BlobStoreIndexShardSnapshot.FileInfo.toXContent(info, builder, ToXContent.EMPTY_PARAMS); + BlobStoreIndexShardSnapshot.FileInfo.toXContent(info, builder); byte[] xcontent = BytesReference.toBytes(BytesReference.bytes(shuffleXContent(builder))); final BlobStoreIndexShardSnapshot.FileInfo parsedInfo; diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoryDataTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoryDataTests.java index 1d37490e2ff5f..d17094bf4b778 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoryDataTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoryDataTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -61,7 +60,7 @@ public void testEqualsAndHashCode() { public void testXContent() throws IOException { RepositoryData repositoryData = generateRandomRepoData(); XContentBuilder builder = JsonXContent.contentBuilder(); - repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS); + repositoryData.snapshotsToXContent(builder); try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { long gen = (long) randomIntBetween(0, 500); RepositoryData fromXContent = RepositoryData.snapshotsFromXContent(parser, gen); @@ -166,7 +165,7 @@ public void testIndexThatReferencesAnUnknownSnapshot() throws IOException { final RepositoryData repositoryData = generateRandomRepoData(); XContentBuilder builder = XContentBuilder.builder(xContent); - repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS); + repositoryData.snapshotsToXContent(builder); RepositoryData parsedRepositoryData; try (XContentParser xParser = createParser(builder)) { parsedRepositoryData = RepositoryData.snapshotsFromXContent(xParser, repositoryData.getGenId()); @@ -197,7 +196,7 @@ public void testIndexThatReferencesAnUnknownSnapshot() throws IOException { indexSnapshots, new ArrayList<>(parsedRepositoryData.getIncompatibleSnapshotIds())); final XContentBuilder corruptedBuilder = XContentBuilder.builder(xContent); - corruptedRepositoryData.snapshotsToXContent(corruptedBuilder, ToXContent.EMPTY_PARAMS); + corruptedRepositoryData.snapshotsToXContent(corruptedBuilder); try (XContentParser xParser = createParser(corruptedBuilder)) { ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () -> From b1652ed26e5c88d47267ca252cb58351360aadbd Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Apr 2019 14:35:13 +0200 Subject: [PATCH 092/260] Fix javadoc tag. (#41329) s/returns/return/ --- .../src/main/java/org/elasticsearch/indices/IndicesService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index d2f7f38762b19..bb3ffde6bc3b0 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -320,7 +320,7 @@ protected void doClose() throws IOException { * are closed and all shard {@link CacheHelper#addClosedListener(org.apache.lucene.index.IndexReader.ClosedListener) closed * listeners} have run. However some {@link IndexEventListener#onStoreClosed(ShardId) shard closed listeners} might not have * run. - * @returns true if all shards closed within the given timeout, false otherwise + * @return true if all shards closed within the given timeout, false otherwise * @throws InterruptedException if the current thread got interrupted while waiting for shards to close */ public boolean awaitClose(long timeout, TimeUnit timeUnit) throws InterruptedException { From 12d62740885d0efe5cab3f09f011cf0c98dd1743 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 18 Apr 2019 09:17:22 -0400 Subject: [PATCH 093/260] Docs: Fix deprecation warning in Asciidoctor Fix a deprecation warning that wasn't rendering correctly in asciidoctor. This one needed to be explicitly marked as an inline macro because it is on its own line and it needed to have its text escaped because it contained a `,`. It also was missing explanitory text for what the setting was. --- docs/plugins/repository-gcs.asciidoc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/plugins/repository-gcs.asciidoc b/docs/plugins/repository-gcs.asciidoc index 52a4cc9d5f3cf..9ad458614b71d 100644 --- a/docs/plugins/repository-gcs.asciidoc +++ b/docs/plugins/repository-gcs.asciidoc @@ -245,7 +245,8 @@ include::repository-shared-settings.asciidoc[] `application_name`:: - deprecated[7.0.0, This setting is now defined in the <>] + deprecated:[6.3.0, "This setting is now defined in the <>."] + Name used by the client when it uses the Google Cloud Storage service. [[repository-gcs-bucket-permission]] ===== Recommended Bucket Permission From 8d03825ddd13ce2fc9577afc1c3d6c22cc90d136 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 18 Apr 2019 16:49:07 +0200 Subject: [PATCH 094/260] Fix SnapshotResiliencyTest Repo Consistency Check (#41332) * Fix SnapshotResiliencyTest Repo Consistency Check * Due to the random creation of an empty `extra0` file by the Lucene mockFS we see broken tests because we use the existence of an index folder in assertions and the index deletion doesn't go through if there are extra files in an index folder * Fixed by removing the `extra0` file and resulting empty directory trees before asserting repo consistency * Closes #41326 --- .../snapshots/SnapshotResiliencyTests.java | 59 +++++++++++++++++-- 1 file changed, 54 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index b2e7562286caa..2661f958fc2a2 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -167,10 +167,16 @@ import org.junit.After; import org.junit.Before; +import java.io.FileNotFoundException; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.DirectoryNotEmptyException; +import java.nio.file.FileVisitResult; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -378,7 +384,6 @@ public void testSnapshotWithNodeDisconnects() { assertThat(snapshotIds, hasSize(1)); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41326") public void testConcurrentSnapshotCreateAndDelete() { setupTestCluster(randomFrom(1, 3, 5), randomIntBetween(2, 10)); @@ -525,10 +530,11 @@ public void run() { private void assertNoStaleRepositoryData() throws IOException { final Path repoPath = tempDir.resolve("repo").toAbsolutePath(); final List repos; - try (Stream reposDir = Files.list(repoPath)) { + try (Stream reposDir = repoFilesByPrefix(repoPath)) { repos = reposDir.filter(s -> s.getFileName().toString().startsWith("extra") == false).collect(Collectors.toList()); } for (Path repoRoot : repos) { + cleanupEmptyTrees(repoRoot); final Path latestIndexGenBlob = repoRoot.resolve("index.latest"); assertTrue("Could not find index.latest blob for repo at [" + repoRoot + ']', Files.exists(latestIndexGenBlob)); final long latestGen = ByteBuffer.wrap(Files.readAllBytes(latestIndexGenBlob)).getLong(0); @@ -544,8 +550,37 @@ private void assertNoStaleRepositoryData() throws IOException { } } + // Lucene's mock file system randomly generates empty `extra0` files that break the deletion of blob-store directories. + // We clean those up here before checking a blob-store for stale files. + private void cleanupEmptyTrees(Path repoPath) { + try { + Files.walkFileTree(repoPath, new SimpleFileVisitor<>() { + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + if (file.getFileName().toString().startsWith("extra")) { + Files.delete(file); + } + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + try { + Files.delete(dir); + } catch (DirectoryNotEmptyException e) { + // We're only interested in deleting empty trees here, just ignore directories with content + } + return FileVisitResult.CONTINUE; + } + }); + } catch (IOException e) { + throw new AssertionError(e); + } + } + private static void assertIndexGenerations(Path repoRoot, long latestGen) throws IOException { - try (Stream repoRootBlobs = Files.list(repoRoot)) { + try (Stream repoRootBlobs = repoFilesByPrefix(repoRoot)) { final long[] indexGenerations = repoRootBlobs.filter(p -> p.getFileName().toString().startsWith("index-")) .map(p -> p.getFileName().toString().replace("index-", "")) .mapToLong(Long::parseLong).sorted().toArray(); @@ -557,7 +592,7 @@ private static void assertIndexGenerations(Path repoRoot, long latestGen) throws private static void assertIndexUUIDs(Path repoRoot, RepositoryData repositoryData) throws IOException { final List expectedIndexUUIDs = repositoryData.getIndices().values().stream().map(IndexId::getId).collect(Collectors.toList()); - try (Stream indexRoots = Files.list(repoRoot.resolve("indices"))) { + try (Stream indexRoots = repoFilesByPrefix(repoRoot.resolve("indices"))) { final List foundIndexUUIDs = indexRoots.filter(s -> s.getFileName().toString().startsWith("extra") == false) .map(p -> p.getFileName().toString()).collect(Collectors.toList()); assertThat(foundIndexUUIDs, containsInAnyOrder(expectedIndexUUIDs.toArray(Strings.EMPTY_ARRAY))); @@ -568,7 +603,7 @@ private static void assertSnapshotUUIDs(Path repoRoot, RepositoryData repository final List expectedSnapshotUUIDs = repositoryData.getSnapshotIds().stream().map(SnapshotId::getUUID).collect(Collectors.toList()); for (String prefix : new String[]{"snap-", "meta-"}) { - try (Stream repoRootBlobs = Files.list(repoRoot)) { + try (Stream repoRootBlobs = repoFilesByPrefix(repoRoot)) { final Collection foundSnapshotUUIDs = repoRootBlobs.filter(p -> p.getFileName().toString().startsWith(prefix)) .map(p -> p.getFileName().toString().replace(prefix, "").replace(".dat", "")) .collect(Collectors.toSet()); @@ -577,6 +612,20 @@ private static void assertSnapshotUUIDs(Path repoRoot, RepositoryData repository } } + /** + * List contents of a blob path and return an empty stream if the path doesn't exist. + * @param prefix Path to find children for + * @return stream of child paths + * @throws IOException on failure + */ + private static Stream repoFilesByPrefix(Path prefix) throws IOException { + try { + return Files.list(prefix); + } catch (FileNotFoundException | NoSuchFileException e) { + return Stream.empty(); + } + } + private void clearDisruptionsAndAwaitSync() { testClusterNodes.clearNetworkDisruptions(); runUntil(() -> { From 9e863311836cdbeb4f8e771099863f4058015624 Mon Sep 17 00:00:00 2001 From: Mark Vieira Date: Thu, 18 Apr 2019 08:09:35 -0700 Subject: [PATCH 095/260] Fix issue with subproject test task dependencies (#41321) --- .../elasticsearch/gradle/precommit/TestingConventionsTasks.java | 2 +- x-pack/plugin/sql/qa/security/build.gradle | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java index b2228f5c1b132..95ad323ceda83 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java @@ -65,7 +65,7 @@ public class TestingConventionsTasks extends DefaultTask { public TestingConventionsTasks() { setDescription("Tests various testing conventions"); // Run only after everything is compiled - Boilerplate.getJavaSourceSets(getProject()).all(sourceSet -> dependsOn(sourceSet.getClassesTaskName())); + Boilerplate.getJavaSourceSets(getProject()).all(sourceSet -> dependsOn(sourceSet.getOutput().getClassesDirs())); naming = getProject().container(TestingConventionRule.class); } diff --git a/x-pack/plugin/sql/qa/security/build.gradle b/x-pack/plugin/sql/qa/security/build.gradle index 525acaf99d694..a0e6e82ed4d67 100644 --- a/x-pack/plugin/sql/qa/security/build.gradle +++ b/x-pack/plugin/sql/qa/security/build.gradle @@ -15,6 +15,7 @@ subprojects { test { mainProject.sourceSets.test.output.classesDirs.each { dir -> output.addClassesDir { dir } + output.builtBy(mainProject.tasks.testClasses) } runtimeClasspath += mainProject.sourceSets.test.output } From 6472ad4f95d8470c582c93054aa7002eb7839290 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 18 Apr 2019 17:33:51 +0200 Subject: [PATCH 096/260] Fix Broken Index Shard Snapshot File Preventing Snapshot Creation (#41310) * The problem here is that if we run into a corrupted index-N file, instead of generating a new index-(N+1) file, we instead set the newest index generation to -1 and thus tried to create `index-0` * If `index-0` is corrupt, this prevents us from ever creating a new snapshot using the broken shard, because we are unable to create `index-0` since it already exists * Fixed by still using the index generation for naming the next index file, even if it was a broken index file * Added test that makes sure restoring as well as snapshotting on top of the broken shard index file work as expected * closes #41304 --- .../BlobStoreIndexShardSnapshots.java | 4 +- .../blobstore/BlobStoreRepository.java | 21 ++-- .../SharedClusterSnapshotRestoreIT.java | 102 ++++++++++++++++++ 3 files changed, 115 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java index c9957a0483e76..c1fde6d8b4e3b 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java @@ -37,10 +37,10 @@ import static java.util.Collections.unmodifiableMap; /** - * Contains information about all snapshot for the given shard in repository + * Contains information about all snapshots for the given shard in repository *

    * This class is used to find files that were already snapshotted and clear out files that no longer referenced by any - * snapshots + * snapshots. */ public class BlobStoreIndexShardSnapshots implements Iterable, ToXContentFragment { 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 41e2355a21f1d..b7ca6224841e1 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -126,17 +126,18 @@ *

      * {@code
      *   STORE_ROOT
    - *   |- index-N           - list of all snapshot ids and the indices belonging to each snapshot, N is the generation of the file
    + *   |- index-N           - JSON serialized {@link RepositoryData} containing a list of all snapshot ids and the indices belonging to
    + *   |                      each snapshot, N is the generation of the file
      *   |- index.latest      - contains the numeric value of the latest generation of the index file (i.e. N from above)
      *   |- incompatible-snapshots - list of all snapshot ids that are no longer compatible with the current version of the cluster
    - *   |- snap-20131010.dat - JSON serialized Snapshot for snapshot "20131010"
    - *   |- meta-20131010.dat - JSON serialized MetaData for snapshot "20131010" (includes only global metadata)
    - *   |- snap-20131011.dat - JSON serialized Snapshot for snapshot "20131011"
    - *   |- meta-20131011.dat - JSON serialized MetaData for snapshot "20131011"
    + *   |- snap-20131010.dat - SMILE serialized {@link SnapshotInfo} for snapshot "20131010"
    + *   |- meta-20131010.dat - SMILE serialized {@link MetaData} for snapshot "20131010" (includes only global metadata)
    + *   |- snap-20131011.dat - SMILE serialized {@link SnapshotInfo} for snapshot "20131011"
    + *   |- meta-20131011.dat - SMILE serialized {@link MetaData} for snapshot "20131011"
      *   .....
      *   |- indices/ - data for all indices
      *      |- Ac1342-B_x/ - data for index "foo" which was assigned the unique id of Ac1342-B_x in the repository
    - *      |  |- meta-20131010.dat - JSON Serialized IndexMetaData for index "foo"
    + *      |  |- meta-20131010.dat - JSON Serialized {@link IndexMetaData} for index "foo"
      *      |  |- 0/ - data for shard "0" of index "foo"
      *      |  |  |- __1                      \  (files with numeric names were created by older ES versions)
      *      |  |  |- __2                      |
    @@ -144,9 +145,9 @@
      *      |  |  |- __1gbJy18wS_2kv1qI7FgKuQ |
      *      |  |  |- __R8JvZAHlSMyMXyZc2SS8Zg /
      *      |  |  .....
    - *      |  |  |- snap-20131010.dat - JSON serialized BlobStoreIndexShardSnapshot for snapshot "20131010"
    - *      |  |  |- snap-20131011.dat - JSON serialized BlobStoreIndexShardSnapshot for snapshot "20131011"
    - *      |  |  |- list-123 - JSON serialized BlobStoreIndexShardSnapshot for snapshot "20131011"
    + *      |  |  |- snap-20131010.dat - SMILE serialized {@link BlobStoreIndexShardSnapshot} for snapshot "20131010"
    + *      |  |  |- snap-20131011.dat - SMILE serialized {@link BlobStoreIndexShardSnapshot} for snapshot "20131011"
    + *      |  |  |- index-123 - SMILE serialized {@link BlobStoreIndexShardSnapshots} for the shard
      *      |  |
      *      |  |- 1/ - data for shard "1" of index "foo"
      *      |  |  |- __1
    @@ -1086,7 +1087,7 @@ protected Tuple buildBlobStoreIndexShardS
                         logger.warn(() -> new ParameterizedMessage("failed to read commit point [{}]", name), e);
                     }
                 }
    -            return new Tuple<>(new BlobStoreIndexShardSnapshots(snapshots), -1);
    +            return new Tuple<>(new BlobStoreIndexShardSnapshots(snapshots), latest);
             }
         }
     
    diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java
    index ffdbaea36f2df..e41898727d7e2 100644
    --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java
    +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java
    @@ -2961,6 +2961,108 @@ public void testRestoreSnapshotWithCorruptedIndexMetadata() throws Exception {
             assertAcked(client().admin().cluster().prepareDeleteSnapshot("test-repo", snapshotInfo.snapshotId().getName()).get());
         }
     
    +    /**
    +     * Tests that a shard snapshot with a corrupted shard index file can still be used for restore and incremental snapshots.
    +     */
    +    public void testSnapshotWithCorruptedShardIndexFile() throws Exception {
    +        final Client client = client();
    +        final Path repo = randomRepoPath();
    +        final String indexName = "test-idx";
    +        final int nDocs = randomIntBetween(1, 10);
    +
    +        logger.info("-->  creating index [{}] with [{}] documents in it", indexName, nDocs);
    +        assertAcked(prepareCreate(indexName).setSettings(Settings.builder()
    +            .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 0)));
    +
    +        final IndexRequestBuilder[] documents = new IndexRequestBuilder[nDocs];
    +        for (int j = 0; j < nDocs; j++) {
    +            documents[j] = client.prepareIndex(indexName, "_doc").setSource("foo", "bar");
    +        }
    +        indexRandom(true, documents);
    +        flushAndRefresh();
    +
    +        logger.info("-->  creating repository");
    +        assertAcked(client().admin().cluster().preparePutRepository("test-repo")
    +            .setType("fs")
    +            .setSettings(Settings.builder()
    +                .put("location", repo)));
    +
    +        final String snapshot1 = "test-snap-1";
    +        logger.info("-->  creating snapshot [{}]", snapshot1);
    +        final SnapshotInfo snapshotInfo = client().admin().cluster().prepareCreateSnapshot("test-repo", snapshot1)
    +            .setWaitForCompletion(true)
    +            .get()
    +            .getSnapshotInfo();
    +        assertThat(snapshotInfo.failedShards(), equalTo(0));
    +        assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards()));
    +        assertThat(snapshotInfo.indices(), hasSize(1));
    +
    +        RepositoriesService service = internalCluster().getInstance(RepositoriesService.class, internalCluster().getMasterName());
    +        Repository repository = service.repository("test-repo");
    +
    +        final RepositoryData repositoryData = getRepositoryData(repository);
    +        final Map indexIds = repositoryData.getIndices();
    +        assertThat(indexIds.size(), equalTo(1));
    +
    +        final IndexId corruptedIndex = indexIds.get(indexName);
    +        final Path shardIndexFile = repo.resolve("indices")
    +            .resolve(corruptedIndex.getId()).resolve("0")
    +            .resolve("index-0");
    +
    +        logger.info("-->  truncating shard index file [{}]", shardIndexFile);
    +        try (SeekableByteChannel outChan = Files.newByteChannel(shardIndexFile, StandardOpenOption.WRITE)) {
    +            outChan.truncate(randomInt(10));
    +        }
    +
    +        logger.info("-->  verifying snapshot state for [{}]", snapshot1);
    +        List snapshotInfos = client().admin().cluster().prepareGetSnapshots("test-repo").get().getSnapshots();
    +        assertThat(snapshotInfos.size(), equalTo(1));
    +        assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));
    +        assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo(snapshot1));
    +
    +        logger.info("-->  deleting index [{}]", indexName);
    +        assertAcked(client().admin().indices().prepareDelete(indexName));
    +
    +        logger.info("-->  restoring snapshot [{}]", snapshot1);
    +        client().admin().cluster().prepareRestoreSnapshot("test-repo", snapshot1)
    +            .setRestoreGlobalState(randomBoolean())
    +            .setWaitForCompletion(true)
    +            .get();
    +        ensureGreen();
    +
    +        assertHitCount(client().prepareSearch(indexName).setSize(0).get(), nDocs);
    +
    +        logger.info("-->  indexing [{}] more documents into [{}]", nDocs, indexName);
    +        for (int j = 0; j < nDocs; j++) {
    +            documents[j] = client.prepareIndex(indexName, "_doc").setSource("foo2", "bar2");
    +        }
    +        indexRandom(true, documents);
    +
    +        final String snapshot2 = "test-snap-2";
    +        logger.info("-->  creating snapshot [{}]", snapshot2);
    +        final SnapshotInfo snapshotInfo2 = client().admin().cluster().prepareCreateSnapshot("test-repo", snapshot2)
    +            .setWaitForCompletion(true)
    +            .get()
    +            .getSnapshotInfo();
    +        assertThat(snapshotInfo2.state(), equalTo(SnapshotState.SUCCESS));
    +        assertThat(snapshotInfo2.failedShards(), equalTo(0));
    +        assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo.totalShards()));
    +        assertThat(snapshotInfo2.indices(), hasSize(1));
    +
    +        logger.info("-->  deleting index [{}]", indexName);
    +        assertAcked(client().admin().indices().prepareDelete(indexName));
    +
    +        logger.info("-->  restoring snapshot [{}]", snapshot2);
    +        client().admin().cluster().prepareRestoreSnapshot("test-repo", snapshot2)
    +            .setRestoreGlobalState(randomBoolean())
    +            .setWaitForCompletion(true)
    +            .get();
    +
    +        ensureGreen();
    +
    +        assertHitCount(client().prepareSearch(indexName).setSize(0).get(), 2 * nDocs);
    +    }
    +
         public void testCannotCreateSnapshotsWithSameName() throws Exception {
             final String repositoryName = "test-repo";
             final String snapshotName = "test-snap";
    
    From e58cb90d024222fa96aef33c9c9ed3ac7aa5ce44 Mon Sep 17 00:00:00 2001
    From: Armin Braun 
    Date: Thu, 18 Apr 2019 18:49:34 +0200
    Subject: [PATCH 097/260] Fix BulkProcessor Retry ITs (#41338)
    
    * The test fails for the retry backoff enabled case because the retry handler in the bulk processor hasn't been adjusted to account for #40866 which now might lead to an outright rejection of the request instead of its items individually
       * Fixed by adding retry functionality to the top level request as well
    * Also fixed the duplicate test for the HLRC that wasn't handling the non-backoff case yet the same way the non-client IT did
    * closes #41324
    ---
     .../client/BulkProcessorRetryIT.java              | 13 +++++++++----
     .../java/org/elasticsearch/action/bulk/Retry.java | 15 ++++++++++-----
     .../action/bulk/BulkProcessorRetryIT.java         |  1 -
     3 files changed, 19 insertions(+), 10 deletions(-)
    
    diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java
    index 19e2de22e54ef..c18c4363897e7 100644
    --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java
    +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java
    @@ -29,6 +29,7 @@
     import org.elasticsearch.common.unit.TimeValue;
     import org.elasticsearch.common.xcontent.XContentType;
     import org.elasticsearch.rest.RestStatus;
    +import org.elasticsearch.transport.RemoteTransportException;
     
     import java.util.Collections;
     import java.util.Iterator;
    @@ -56,7 +57,6 @@ public void testBulkRejectionLoadWithoutBackoff() throws Exception {
             executeBulkRejectionLoad(BackoffPolicy.noBackoff(), rejectedExecutionExpected);
         }
     
    -    @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41324")
         public void testBulkRejectionLoadWithBackoff() throws Throwable {
             boolean rejectedExecutionExpected = false;
             executeBulkRejectionLoad(BackoffPolicy.exponentialBackoff(), rejectedExecutionExpected);
    @@ -122,9 +122,14 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure)
                         }
                     }
                 } else {
    -                Throwable t = (Throwable) response;
    -                // we're not expecting any other errors
    -                throw new AssertionError("Unexpected failure", t);
    +                if (response instanceof RemoteTransportException
    +                    && ((RemoteTransportException) response).status() == RestStatus.TOO_MANY_REQUESTS && rejectedExecutionExpected) {
    +                    // ignored, we exceeded the write queue size with dispatching the initial bulk request
    +                } else {
    +                    Throwable t = (Throwable) response;
    +                    // we're not expecting any other errors
    +                    throw new AssertionError("Unexpected failure", t);
    +                }
                 }
             }
     
    diff --git a/server/src/main/java/org/elasticsearch/action/bulk/Retry.java b/server/src/main/java/org/elasticsearch/action/bulk/Retry.java
    index a3c6e27eaf838..e2f90049759ec 100644
    --- a/server/src/main/java/org/elasticsearch/action/bulk/Retry.java
    +++ b/server/src/main/java/org/elasticsearch/action/bulk/Retry.java
    @@ -26,6 +26,7 @@
     import org.elasticsearch.rest.RestStatus;
     import org.elasticsearch.threadpool.Scheduler;
     import org.elasticsearch.threadpool.ThreadPool;
    +import org.elasticsearch.transport.RemoteTransportException;
     
     import java.util.ArrayList;
     import java.util.Iterator;
    @@ -118,11 +119,15 @@ public void onResponse(BulkResponse bulkItemResponses) {
     
             @Override
             public void onFailure(Exception e) {
    -            try {
    -                listener.onFailure(e);
    -            } finally {
    -                if (retryCancellable != null) {
    -                    retryCancellable.cancel();
    +            if (e instanceof RemoteTransportException && ((RemoteTransportException) e).status() == RETRY_STATUS && backoff.hasNext()) {
    +                retry(currentBulkRequest);
    +            } else {
    +                try {
    +                    listener.onFailure(e);
    +                } finally {
    +                    if (retryCancellable != null) {
    +                        retryCancellable.cancel();
    +                    }
                     }
                 }
             }
    diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java
    index 054e213d5abe9..e4b6fff9fc353 100644
    --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java
    +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java
    @@ -64,7 +64,6 @@ public void testBulkRejectionLoadWithoutBackoff() throws Throwable {
             executeBulkRejectionLoad(BackoffPolicy.noBackoff(), rejectedExecutionExpected);
         }
     
    -    @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41324")
         public void testBulkRejectionLoadWithBackoff() throws Throwable {
             boolean rejectedExecutionExpected = false;
             executeBulkRejectionLoad(BackoffPolicy.exponentialBackoff(), rejectedExecutionExpected);
    
    From 6f642196cbab90079c610097befc794746170df1 Mon Sep 17 00:00:00 2001
    From: Andrei Stefan 
    Date: Thu, 18 Apr 2019 20:09:59 +0300
    Subject: [PATCH 098/260] SQL: fix *SecurityIT tests by covering edge case
     scenarios when audit file rolls over at midnight (#41328)
    
    * Handle the scenario where assertLogs() is not called from a test method
    but the audit rolling file rolls over.
    * Use a local boolean variable instead of the static one to account for
    assertBusy() code block possibly being called multiple times and having
    different execution paths.
    ---
     .../sql/qa/security/SqlSecurityTestCase.java      | 15 ++++++++++++++-
     1 file changed, 14 insertions(+), 1 deletion(-)
    
    diff --git a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java
    index f4aadbdf7cdfc..313d0cdb5cf7f 100644
    --- a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java
    +++ b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java
    @@ -188,6 +188,16 @@ public void setInitialAuditLogOffset() {
                 } catch (IOException e) {
                     throw new RuntimeException(e);
                 }
    +            
    +            // The log file can roll over without being caught by assertLogs() method: in those tests where exceptions are being handled
    +            // and no audit logs being read (and, thus, assertLogs() is not called) - for example testNoMonitorMain() method: there are no
    +            // calls to auditLogs(), and the method could run while the audit file is rolled over.
    +            // If this happens, next call to auditLogs() will make the tests read from the rolled over file using the main audit file
    +            // offset, which will most likely not going to work since the offset will happen somewhere in the middle of a json line.
    +            if (auditFileRolledOver == false && Files.exists(ROLLED_OVER_AUDIT_LOG_FILE)) {
    +                // once the audit file rolled over, it will stay like this
    +                auditFileRolledOver = true;
    +            }
                 return null;
             });
         }
    @@ -568,6 +578,9 @@ public void assertLogs() throws Exception {
                 assertFalse("Previous test had an audit-related failure. All subsequent audit related assertions are bogus because we can't "
                         + "guarantee that we fully cleaned up after the last test.", auditFailure);
                 try {
    +                // use a second variable since the `assertBusy()` block can be executed multiple times and the
    +                // static auditFileRolledOver value can change and mess up subsequent calls of this code block
    +                boolean localAuditFileRolledOver = auditFileRolledOver;
                     assertBusy(() -> {
                         SecurityManager sm = System.getSecurityManager();
                         if (sm != null) {
    @@ -579,7 +592,7 @@ public void assertLogs() throws Exception {
                             try {
                                 // the audit log file rolled over during the test
                                 // and we need to consume the rest of the rolled over file plus the new audit log file
    -                            if (auditFileRolledOver == false && Files.exists(ROLLED_OVER_AUDIT_LOG_FILE)) {
    +                            if (localAuditFileRolledOver == false && Files.exists(ROLLED_OVER_AUDIT_LOG_FILE)) {
                                     // once the audit file rolled over, it will stay like this
                                     auditFileRolledOver = true;
                                     // the order in the array matters, as the readers will be used in that order
    
    From 55db0e267ff5b205f23aa003eb35013b3de17173 Mon Sep 17 00:00:00 2001
    From: Simon Willnauer 
    Date: Thu, 18 Apr 2019 19:14:23 +0200
    Subject: [PATCH 099/260] Mark searcher as accessed in acquireSearcher (#41335)
    
    This fixes an issue where every N seconds a slow search request is triggered
    since the searcher access time is not set unless the shard is idle. This change
    moves to a more pro-active approach setting the searcher as accessed all the time.
    ---
     .../main/java/org/elasticsearch/index/shard/IndexShard.java | 5 ++---
     .../java/org/elasticsearch/index/shard/IndexShardTests.java | 6 ++++++
     2 files changed, 8 insertions(+), 3 deletions(-)
    
    diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java
    index efae251fb9344..c5dc2d024aaa8 100644
    --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java
    +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java
    @@ -1215,6 +1215,7 @@ private void markSearcherAccessed() {
     
         private Engine.Searcher acquireSearcher(String source, Engine.SearcherScope scope) {
             readAllowed();
    +        markSearcherAccessed();
             final Engine engine = getEngine();
             final Engine.Searcher searcher = engine.acquireSearcher(source, scope);
             assert ElasticsearchDirectoryReader.unwrap(searcher.getDirectoryReader())
    @@ -2983,9 +2984,7 @@ private void setRefreshPending(Engine engine) {
          *                 true if the listener was registered to wait for a refresh.
          */
         public final void awaitShardSearchActive(Consumer listener) {
    -        if (isSearchIdle()) {
    -            markSearcherAccessed(); // move the shard into non-search idle
    -        }
    +        markSearcherAccessed(); // move the shard into non-search idle
             final Translog.Location location = pendingRefreshLocation.get();
             if (location != null) {
                 addRefreshListener(location, (b) -> {
    diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
    index 725a6a0bcbec4..183cd5519269b 100644
    --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
    @@ -3234,6 +3234,12 @@ public void testIsSearchIdle() throws Exception {
                 // now loop until we are fast enough... shouldn't take long
                 primary.awaitShardSearchActive(aBoolean -> {});
             } while (primary.isSearchIdle());
    +
    +        assertBusy(() -> assertTrue(primary.isSearchIdle()));
    +        do {
    +            // now loop until we are fast enough... shouldn't take long
    +            primary.acquireSearcher("test").close();
    +        } while (primary.isSearchIdle());
             closeShards(primary);
         }
     
    
    From 8b0a74f11c217b480a3e45d9806470f707a86d7f Mon Sep 17 00:00:00 2001
    From: Nhat Nguyen 
    Date: Thu, 18 Apr 2019 14:19:31 -0400
    Subject: [PATCH 100/260] Clean up outdated skip statements in yaml tests
     (#41165)
    
    These skip statements become no-ops in 8.0 for we don't support
    a mixed cluster between 6.x and 8.0.
    
    Relates #41164
    ---
     .../test/analysis-common/20_analyzers.yml     |  3 --
     .../test/analysis-common/30_tokenizers.yml    |  3 --
     .../test/analysis-common/40_token_filters.yml |  4 --
     .../test/indices.analyze/10_analyze.yml       |  3 +-
     .../test/lang_mustache/30_search_template.yml |  4 --
     .../50_multi_search_template.yml              |  4 --
     .../rest-api-spec/test/painless/30_search.yml |  4 --
     .../test/painless/70_mov_fn_agg.yml           |  3 --
     .../test/painless/80_script_score.yml         |  5 ---
     .../painless/90_interval_query_filter.yml     |  4 --
     .../test/rank_feature/10_basic.yml            |  4 --
     .../test/rank_features/10_basic.yml           |  4 --
     .../resources/rest-api-spec/test/10_basic.yml |  5 ---
     .../rest-api-spec/test/rank_eval/10_basic.yml | 15 -------
     .../rest-api-spec/test/rank_eval/20_dcg.yml   |  5 ---
     .../test/rank_eval/30_failures.yml            |  5 ---
     .../test/rank_eval/40_rank_eval_templated.yml |  5 ---
     .../test/delete_by_query/10_basic.yml         |  4 --
     .../test/update_by_query/10_basic.yml         |  3 --
     .../test/analysis_icu/10_basic.yml            |  1 -
     .../test/mapper_annotatedtext/10_basic.yml    |  8 ----
     .../test/multi_cluster/20_info.yml            |  4 --
     .../test/ingest/20_combine_processors.yml     |  5 ---
     .../rest-api-spec/test/bulk/60_deprecated.yml |  5 ---
     .../rest-api-spec/test/bulk/80_cas.yml        |  5 ---
     .../rest-api-spec/test/index/30_cas.yml       |  5 ---
     .../test/index/70_mix_typeless_typeful.yml    | 10 -----
     .../test/indices.clear_cache/10_basic.yml     |  4 --
     .../20_mix_typeless_typeful.yml               | 20 ---------
     .../test/indices.get_alias/30_wildcards.yml   | 18 --------
     .../test/indices.get_mapping/10_basic.yml     |  3 --
     .../indices.get_mapping/30_missing_index.yml  |  3 --
     .../50_wildcard_expansion.yml                 |  6 ---
     .../test/indices.get_settings/30_defaults.yml |  3 --
     .../test/indices.get_template/10_basic.yml    |  3 --
     .../test/indices.put_mapping/10_basic.yml     | 11 -----
     .../20_mix_typeless_typeful.yml               |  6 ---
     .../indices.put_mapping/all_path_options.yml  |  3 --
     .../30_max_size_condition.yml                 |  4 --
     .../test/indices.rollover/40_mapping.yml      |  8 ----
     .../test/indices.split/10_basic.yml           | 10 -----
     .../test/indices.split/20_source_mapping.yml  |  4 --
     .../test/indices.split/30_copy_settings.yml   |  4 --
     .../test/scroll/20_keep_alive.yml             |  3 --
     .../200_top_hits_metric.yml                   |  4 --
     .../test/search.aggregation/230_composite.yml | 39 -----------------
     .../search.aggregation/240_max_buckets.yml    |  4 --
     .../test/search.aggregation/250_moving_fn.yml |  5 ---
     .../search.aggregation/260_weighted_avg.yml   |  3 --
     .../270_median_absolute_deviation_metric.yml  |  3 --
     .../30_max_analyzed_offset.yml                | 16 ++-----
     .../test/search/170_terms_query.yml           |  3 --
     .../test/search/190_index_prefix_search.yml   | 10 -----
     .../test/search/200_ignore_malformed.yml      |  3 --
     .../test/search/200_index_phrase_search.yml   |  4 +-
     .../test/search/210_rescore_explain.yml       |  3 --
     .../50_completion_with_multi_fields.yml       | 20 ---------
     .../test/update/35_if_seq_no.yml              |  5 ---
     .../rest-api-spec/test/graph/10_basic.yml     |  3 --
     .../test/indices.freeze/10_basic.yml          | 10 -----
     .../test/rollup/get_rollup_index_caps.yml     |  4 --
     .../rest-api-spec/test/rollup/stop_job.yml    |  8 ----
     .../rest-api-spec/test/sql/translate.yml      |  4 --
     .../80_put_get_watch_with_passwords.yml       |  4 --
     .../test/watcher/stats/10_basic.yml           |  9 ----
     .../test/rankeval/10_rankeval.yml             |  5 ---
     .../test/old_cluster/30_ml_jobs_crud.yml      | 42 -------------------
     .../test/mustache/30_search_input.yml         |  7 ----
     .../rest-api-spec/test/painless/10_basic.yml  |  4 --
     69 files changed, 6 insertions(+), 459 deletions(-)
    
    diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml
    index fe5b997974ad1..15de6fe664786 100644
    --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml
    +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml
    @@ -69,9 +69,6 @@
     
     ---
     "standard_html_strip":
    -    - skip:
    -        version: " - 6.99.99"
    -        reason:  only starting from version 7.x this throws an error
         - do:
             catch: /\[standard_html_strip\] analyzer is not supported for new indices, use a custom analyzer using \[standard\] tokenizer and \[html_strip\] char_filter, plus \[lowercase\] filter/
             indices.analyze:
    diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/30_tokenizers.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/30_tokenizers.yml
    index 4fe5162e68743..26c699f447986 100644
    --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/30_tokenizers.yml
    +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/30_tokenizers.yml
    @@ -141,9 +141,6 @@
     
     ---
     "ngram_exception":
    -    - skip:
    -        version: " - 6.99.99"
    -        reason: only starting from version 7.x this throws an error
         - do:
             catch: /The difference between max_gram and min_gram in NGram Tokenizer must be less than or equal to[:] \[1\] but was \[2\]\. This limit can be set by changing the \[index.max_ngram_diff\] index level setting\./
             indices.analyze:
    diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml
    index d6fe6b9a980e4..3ce4b6bc04ed3 100644
    --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml
    +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/40_token_filters.yml
    @@ -1137,10 +1137,6 @@
     
     ---
     "delimited_payload_filter_error":
    -    - skip:
    -        version: " - 6.99.99"
    -        reason:  using delimited_payload_filter throws error from 7.0 on
    -
         - do:
             catch: /\[delimited_payload_filter\] is not supported for new indices, use \[delimited_payload\] instead/
             indices.create:
    diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/indices.analyze/10_analyze.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/indices.analyze/10_analyze.yml
    index 56bbed7044e14..baa32662b577f 100644
    --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/indices.analyze/10_analyze.yml
    +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/indices.analyze/10_analyze.yml
    @@ -17,8 +17,7 @@
     ---
     "htmlStrip_deprecated":
         - skip:
    -        version: " - 6.2.99"
    -        reason: deprecated in 6.3
    +        reason: deprecated
             features: "warnings"
     
         - do:
    diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml
    index bc039a5ba7c50..c0cc784250b90 100644
    --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml
    +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/30_search_template.yml
    @@ -128,10 +128,6 @@
     
     ---
     "Test with new response format":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: hits.total is returned as an object in 7.0.0
    -
       - do:
           index:
             index:  test
    diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/50_multi_search_template.yml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/50_multi_search_template.yml
    index a5072d529b9b5..f339731ad130f 100644
    --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/50_multi_search_template.yml
    +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/50_multi_search_template.yml
    @@ -166,10 +166,6 @@ setup:
     
     ---
     "Test with rest_total_hits_as_int":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: hits.total is returned as an object in 7.0.0
    -
       - do:
           put_script:
             id: stored_template_1
    diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/30_search.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/30_search.yml
    index 0ce1e369cb7c5..a006fde630716 100644
    --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/30_search.yml
    +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/30_search.yml
    @@ -452,10 +452,6 @@
     ---
     
     "Exception on negative score":
    -    - skip:
    -        version: " - 6.99.99"
    -        reason: "check on negative scores was added from 7.0.0 on"
    -
         - do:
             index:
                 index: test
    diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_mov_fn_agg.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_mov_fn_agg.yml
    index c2fb38611a30d..a19738732472d 100644
    --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_mov_fn_agg.yml
    +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/70_mov_fn_agg.yml
    @@ -1,9 +1,6 @@
     # Sanity integration test to make sure the custom context and whitelist work for moving_fn pipeline agg
     #
     setup:
    -  - skip:
    -        version: " - 6.3.99"
    -        reason:  "moving_fn added in 6.4.0"
       - do:
           indices.create:
             index: test
    diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml
    index cf55810058d92..05b094df94891 100644
    --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml
    +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/80_script_score.yml
    @@ -1,10 +1,5 @@
     # Integration tests for ScriptScoreQuery using Painless
     
    -setup:
    -- skip:
    -    version: " - 6.99.99"
    -    reason: "script score query was introduced in 7.0.0"
    -
     ---
     "Math functions":
         - do:
    diff --git a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/90_interval_query_filter.yml b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/90_interval_query_filter.yml
    index 0a6cf993e2a2e..7db3eb8b9b2aa 100644
    --- a/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/90_interval_query_filter.yml
    +++ b/modules/lang-painless/src/test/resources/rest-api-spec/test/painless/90_interval_query_filter.yml
    @@ -1,8 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  "Implemented in 7.0"
    -
       - do:
           indices.create:
             index:  test
    diff --git a/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_feature/10_basic.yml b/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_feature/10_basic.yml
    index 8a874d30591f6..6fea35eb21f4e 100644
    --- a/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_feature/10_basic.yml
    +++ b/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_feature/10_basic.yml
    @@ -1,8 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: "The rank feature field/query was introduced in 7.0.0"
    -
       - do:
           indices.create:
               index: test
    diff --git a/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_features/10_basic.yml b/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_features/10_basic.yml
    index f524bd93bb600..d4d5d2a360406 100644
    --- a/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_features/10_basic.yml
    +++ b/modules/mapper-extras/src/test/resources/rest-api-spec/test/rank_features/10_basic.yml
    @@ -1,8 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: "The rank_features field was introduced in 7.0.0"
    -
       - do:
           indices.create:
               index: test
    diff --git a/modules/percolator/src/test/resources/rest-api-spec/test/10_basic.yml b/modules/percolator/src/test/resources/rest-api-spec/test/10_basic.yml
    index 08d344687adc7..4d01600f76e43 100644
    --- a/modules/percolator/src/test/resources/rest-api-spec/test/10_basic.yml
    +++ b/modules/percolator/src/test/resources/rest-api-spec/test/10_basic.yml
    @@ -1,10 +1,5 @@
     ---
     "Test percolator basics via rest":
    -
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: types are required in requests before 7.0.0
    -
       - do:
           indices.create:
             index: queries_index
    diff --git a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/10_basic.yml b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/10_basic.yml
    index b9001cb782a80..14279bbb5270f 100644
    --- a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/10_basic.yml
    +++ b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/10_basic.yml
    @@ -40,11 +40,6 @@ setup:
     
     ---
     "Response format":
    -
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: response format was updated in 6.3
    -
       - do:
           rank_eval:
             index: foo,
    @@ -120,11 +115,6 @@ setup:
     
     ---
     "Mean Reciprocal Rank":
    -
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: response format was updated in 6.3
    -
       - do:
           rank_eval:
             body: {
    @@ -159,11 +149,6 @@ setup:
     
     ---
     "Expected Reciprocal Rank":
    -
    -  - skip:
    -      version: " - 6.3.99"
    -      reason: ERR was introduced in 6.4
    -
       - do:
           rank_eval:
             body: {
    diff --git a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/20_dcg.yml b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/20_dcg.yml
    index 90094baabb9db..82005efcebe18 100644
    --- a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/20_dcg.yml
    +++ b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/20_dcg.yml
    @@ -1,10 +1,5 @@
     ---
     "Response format":
    -
    -  - skip:
    -      version: " - 6.1.99"
    -      reason: the ranking evaluation feature is available since 6.2
    -
       - do:
           index:
             index:   foo
    diff --git a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/30_failures.yml b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/30_failures.yml
    index b9f55ed12ad7e..c88a769b8687b 100644
    --- a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/30_failures.yml
    +++ b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/30_failures.yml
    @@ -1,10 +1,5 @@
     ---
     "Response format":
    -
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: response format was updated in 6.3
    -
       - do:
           index:
             index:   foo
    diff --git a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/40_rank_eval_templated.yml b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/40_rank_eval_templated.yml
    index 57d5aa5642ef6..08897e17ef900 100644
    --- a/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/40_rank_eval_templated.yml
    +++ b/modules/rank-eval/src/test/resources/rest-api-spec/test/rank_eval/40_rank_eval_templated.yml
    @@ -48,11 +48,6 @@ setup:
     
     ---
     "Basic rank-eval request with template":
    -
    -  - skip:
    -      version: " - 6.1.99"
    -      reason: the ranking evaluation feature is available since 6.2
    -
       - do:
           rank_eval:
             body: {
    diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml b/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml
    index d11f160bcf571..ded91a095f611 100644
    --- a/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml
    +++ b/modules/reindex/src/test/resources/rest-api-spec/test/delete_by_query/10_basic.yml
    @@ -145,10 +145,6 @@
     
     ---
     "Response for version conflict (seq no powered)":
    -  - skip:
    -      version: " - 6.6.99"
    -      reason: reindex moved to rely on sequence numbers for concurrency control
    -
       - do:
           indices.create:
             index: test
    diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml
    index 08c8465c40960..40119f22abb5b 100644
    --- a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml
    +++ b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/10_basic.yml
    @@ -116,9 +116,6 @@
     
     ---
     "Response for version conflict (seq no powered)":
    -  - skip:
    -      version: " - 6.6.99"
    -      reason: reindex moved to rely on sequence numbers for concurrency control
       - do:
           indices.create:
             index: test
    diff --git a/plugins/analysis-icu/src/test/resources/rest-api-spec/test/analysis_icu/10_basic.yml b/plugins/analysis-icu/src/test/resources/rest-api-spec/test/analysis_icu/10_basic.yml
    index 5cdfcde72b020..da95501c05d21 100644
    --- a/plugins/analysis-icu/src/test/resources/rest-api-spec/test/analysis_icu/10_basic.yml
    +++ b/plugins/analysis-icu/src/test/resources/rest-api-spec/test/analysis_icu/10_basic.yml
    @@ -107,7 +107,6 @@
     ---
     "Normalization with deprecated unicodeSetFilter":
         - skip:
    -        version: " - 6.99.99"
             reason:  unicodeSetFilter deprecated in 7.0.0, replaced by unicode_set_filter
             features: "warnings"
     
    diff --git a/plugins/mapper-annotated-text/src/test/resources/rest-api-spec/test/mapper_annotatedtext/10_basic.yml b/plugins/mapper-annotated-text/src/test/resources/rest-api-spec/test/mapper_annotatedtext/10_basic.yml
    index 63516516252a9..aca6ba3059381 100644
    --- a/plugins/mapper-annotated-text/src/test/resources/rest-api-spec/test/mapper_annotatedtext/10_basic.yml
    +++ b/plugins/mapper-annotated-text/src/test/resources/rest-api-spec/test/mapper_annotatedtext/10_basic.yml
    @@ -3,10 +3,6 @@
     
     ---
     "annotated highlighter on annotated text":
    -  - skip:
    -      version: " - 6.4.99"
    -      reason: Annotated text type introduced in 6.5.0
    -
       - do:
           indices.create:
             index: annotated
    @@ -45,10 +41,6 @@
     
     ---
     "issue 39395 thread safety issue -requires multiple calls to reveal":
    -  - skip:
    -      version: " - 6.4.99"
    -      reason: Annotated text type introduced in 6.5.0
    -
       - do:
           indices.create:
             index: annotated
    diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/20_info.yml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/20_info.yml
    index 1b0f9830e26db..59657e2012c8a 100644
    --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/20_info.yml
    +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/20_info.yml
    @@ -61,10 +61,6 @@
     
     ---
     "skip_unavailable is returned as part of _remote/info response":
    -  - skip:
    -      version: " - 6.0.99"
    -      reason: "skip_unavailable is only returned from 6.1.0 on"
    -
       - do:
           cluster.get_settings:
             include_defaults: true
    diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yml b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yml
    index c121d542c86b1..0c0cf816b000d 100644
    --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yml
    +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yml
    @@ -1,10 +1,5 @@
     ---
     "Test with date processor":
    -  - skip:
    -      version: " - 6.9.99"
    -      reason: pre-7.0.0 requires the 8 prefix for Java time formats, so would treat the format in this test as a Joda time format
    -      features: "warnings"
    -
       - do:
           ingest.put_pipeline:
             id: "_id"
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml
    index 1401fcc086208..c04f749d0bd10 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/60_deprecated.yml
    @@ -2,11 +2,6 @@
     ---
     "Deprecated parameters should fail in Bulk query":
     
    -   - skip:
    -       version: " - 6.99.99"
    -       reason:  some parameters are removed starting from 7.0, their equivalents without underscore are used instead
    -       features: "warnings"
    -
        - do:
            catch:  bad_request
            bulk:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml
    index 902621cfba578..87d3d237d42cb 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/80_cas.yml
    @@ -1,10 +1,5 @@
     ---
     "Compare And Swap Sequence Numbers":
    -
    - - skip:
    -      version: " - 6.99.99"
    -      reason:  typeless API are add in 7.0.0
    -
      - do:
           index:
               index:  test_1
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml
    index 550582e9816eb..27534131782a5 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml
    @@ -1,10 +1,5 @@
     ---
     "Compare And Swap Sequence Numbers":
    -
    - - skip:
    -      version: " - 6.99.99"
    -      reason:  typesless api was introduces in 7.0
    -
      - do:
           index:
               index:  test_1
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml
    index f3629fbb7cc18..9770137cc6f5a 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml
    @@ -1,10 +1,5 @@
     ---
     "Index with typeless API on an index that has types":
    -
    - - skip:
    -      version: " - 6.99.99"
    -      reason:  Typeless APIs were introduced in 7.0.0
    -
      - do:
           indices.create: # not using include_type_name: false on purpose
               include_type_name: true
    @@ -64,11 +59,6 @@
     
     ---
     "Index call that introduces new field mappings":
    -
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  Typeless APIs were introduced in 7.0.0
    -
       - do:
           indices.create: # not using include_type_name: false on purpose
             include_type_name: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clear_cache/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clear_cache/10_basic.yml
    index 099226e41e6d3..94b23fb63adb5 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clear_cache/10_basic.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.clear_cache/10_basic.yml
    @@ -11,10 +11,6 @@
     
     ---
     "clear_cache with fielddata set to true":
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: fielddata was deprecated before 6.3.0
    -
       - do:
           indices.clear_cache:
             fielddata: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/20_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/20_mix_typeless_typeful.yml
    index a05134866628b..e4fedae6d1aa5 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/20_mix_typeless_typeful.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/20_mix_typeless_typeful.yml
    @@ -1,10 +1,5 @@
     ---
     "Create a typeless index while there is a typed template":
    -
    - - skip:
    -      version: " - 6.6.99"
    -      reason: Merging typeless/typed mappings/templates was added in 6.7
    -
      - do:
           indices.put_template:
               include_type_name: true
    @@ -38,11 +33,6 @@
     
     ---
     "Create a typed index while there is a typeless template":
    -
    - - skip:
    -      version: " - 6.6.99"
    -      reason: Merging typeless/typed mappings/templates was added in 6.7
    -
      - do:
           indices.put_template:
               include_type_name: false
    @@ -77,11 +67,6 @@
     
     ---
     "Implicitly create a typed index while there is a typeless template":
    -
    - - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name only supported as of 6.7
    -
      - do:
           indices.put_template:
               include_type_name: false
    @@ -102,11 +87,6 @@
     
     ---
     "Implicitly create a typeless index while there is a typed template":
    -
    - - skip:
    -      version: " - 6.99.99"
    -      reason: needs typeless index operations to work on typed indices
    -
      - do:
           indices.put_template:
               include_type_name: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_alias/30_wildcards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_alias/30_wildcards.yml
    index 08b3009be0e88..389166a03136e 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_alias/30_wildcards.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_alias/30_wildcards.yml
    @@ -26,9 +26,6 @@ setup:
     
     ---
     "Get aliases wildcard and simple exclusion":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Exclusions in the alias expression are not handled
       - do:
           indices.get_alias:
             name: test_blias_2,test_alias*,-test_alias_1
    @@ -41,9 +38,6 @@ setup:
     
     ---
     "Get aliases and wildcard exclusion":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Exclusions in the alias expression are not handled
       - do:
           indices.get_alias:
             name: test_alias_1,test_blias_1,-test_alias*
    @@ -66,9 +60,6 @@ setup:
     
     ---
     "Non-existent exclusion alias before wildcard returns 404":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Exclusions in the alias expression are not handled
       - do:
           catch: missing
           indices.get_alias:
    @@ -97,9 +88,6 @@ setup:
     
     ---
     "Missing exclusions does not fire 404":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Exclusions in the alias expression are not handled
       - do:
           indices.get_alias:
             name: test_alias*,-non-existent,test_blias*,-test
    @@ -112,9 +100,6 @@ setup:
     
     ---
     "Exclusion of non wildcarded aliases":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Exclusions in the alias expression are not handled
       - do:
           indices.get_alias:
             name: test_alias_1,test_blias_2,-test_alias*,-test_blias_2
    @@ -123,9 +108,6 @@ setup:
     
     ---
     "Wildcard exclusions does not trigger 404":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Exclusions in the alias expression are not handled
       - do:
           catch: missing
           indices.get_alias:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml
    index c3addd95469d4..e46f67326a8d2 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml
    @@ -1,8 +1,5 @@
     ---
     setup:
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0
       - do:
             indices.create:
               index: test_1
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml
    index 5a7624265ecc9..be3acc433ac8f 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml
    @@ -14,9 +14,6 @@
     
     ---
     "Index missing, ignore_unavailable=true":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: ignore_unavailable was ignored in previous versions
       - do:
           indices.get_mapping:
             index: test_index
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml
    index d3f15b3292285..7f6f3999c868d 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml
    @@ -102,9 +102,6 @@ setup:
     
     ---
     "Get test-* with wildcard_expansion=none":
    - - skip:
    -    version: " - 6.99.99"
    -    reason: allow_no_indices (defaults to true) was ignored in previous versions
      - do:
         indices.get_mapping:
             index: test-x*
    @@ -113,9 +110,6 @@ setup:
      - match: { '':  {} }
     ---
     "Get test-* with wildcard_expansion=none allow_no_indices=false":
    - - skip:
    -    version: " - 6.99.99"
    -    reason: allow_no_indices was ignored in previous versions
      - do:
         catch: missing
         indices.get_mapping:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_settings/30_defaults.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_settings/30_defaults.yml
    index 2e3f4af03ebef..83e77140facbc 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_settings/30_defaults.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_settings/30_defaults.yml
    @@ -10,9 +10,6 @@ setup:
             index: test-index
     ---
     Test retrieval of default settings:
    -  - skip:
    -      version: " - 6.3.99"
    -      reason: include_defaults will not work in mixed-mode clusters containing nodes pre-6.4
       - do:
           indices.get_settings:
             flat_settings: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml
    index c1aac94bf1d84..9becbd54a3773 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml
    @@ -1,7 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0.0
       - do:
           indices.put_template:
             name: test
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml
    index 2c4658afad2db..959e08d75dabf 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml
    @@ -1,8 +1,5 @@
     ---
     "Test Create and update mapping":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0
       - do:
           indices.create:
             index: test_index
    @@ -53,10 +50,6 @@
     
     ---
     "Create index with invalid mappings":
    -
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0
       - do:
           indices.create:
             index: test_index
    @@ -71,10 +64,6 @@
     
     ---
     "Put mappings with explicit _doc type":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0
    -
       - do:
           indices.create:
             index: test_index
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml
    index 13cb3321841cf..e91ea9210d897 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml
    @@ -53,12 +53,6 @@
     
     ---
     "PUT mapping with _doc on an index that has types":
    -
    - - skip:
    -      version: " - 6.6.99"
    -      reason: include_type_name is only supported as of 6.7
    -
    -
      - do:
           indices.create:
               include_type_name: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml
    index 182ec017e0d30..ad80a61f1763b 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml
    @@ -1,7 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0
       - do:
           indices.create:
             index: test_index1
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/30_max_size_condition.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/30_max_size_condition.yml
    index 6e4df0f292915..08bae450ea756 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/30_max_size_condition.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/30_max_size_condition.yml
    @@ -1,10 +1,6 @@
     ---
     "Rollover with max_size condition":
     
    -  - skip:
    -      version: " - 6.0.99"
    -      reason: max_size condition is introduced in 6.1.0
    -
       # create index with alias and replica
       - do:
           indices.create:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml
    index 47b004326a457..040ffd534c0ab 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.rollover/40_mapping.yml
    @@ -1,9 +1,5 @@
     ---
     "Typeless mapping":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  include_type_name defaults to true before 7.0.0
    -
       - do:
           indices.create:
             index: logs-1
    @@ -44,10 +40,6 @@
     
     ---
     "Mappings with explicit _doc type":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: include_type_name defaults to true before 7.0
    -
       - do:
           indices.create:
             index: logs-1
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml
    index 2baa82ea78842..01d41856fe1ae 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/10_basic.yml
    @@ -29,11 +29,6 @@ setup:
     
     ---
     "Split index via API":
    -  - skip:
    -      version: " - 6.9.99"
    -      reason: pre-7.0.0 will send warnings
    -      features: "warnings"
    -
       # make it read-only
       - do:
           indices.put_settings:
    @@ -190,11 +185,6 @@ setup:
     
     ---
     "Create illegal split indices":
    -  - skip:
    -      version: " - 6.9.99"
    -      reason: pre-7.0.0 will send warnings
    -      features: "warnings"
    -
       # try to do an illegal split with number_of_routing_shards set
       - do:
           catch: /illegal_argument_exception/
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml
    index 3740167a0253a..f600b4817fd91 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml
    @@ -1,9 +1,5 @@
     ---
     "Split index ignores target template mapping":
    -  - skip:
    -      version: " - 6.9.99"
    -      reason: pre-7.0.0 will send warnings
    -      features: "warnings"
     
       # create index
       - do:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml
    index df9eae0adf340..ba876b0ec4e11 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml
    @@ -1,9 +1,5 @@
     ---
     "Copy settings during split index":
    -  - skip:
    -      version: " - 6.9.99"
    -      reason: expects warnings that pre-7.0.0 will not send
    -      features: [arbitrary_key, warnings]
     
       - do:
           nodes.info:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/20_keep_alive.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/20_keep_alive.yml
    index 6217f66c2648e..e3b4dcc46230b 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/20_keep_alive.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/20_keep_alive.yml
    @@ -10,9 +10,6 @@
     
     ---
     "Max keep alive":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: search.max_keep_alive was added in 7.0.0
     
       - do:
           index:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml
    index 11e3d5906d7e9..cde56fa41e3d9 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/200_top_hits_metric.yml
    @@ -49,10 +49,6 @@ setup:
     
     ---
     "top_hits aggregation with nested documents":
    -  - skip:
    -          version: " - 6.1.99"
    -          reason: "<= 6.1 nodes don't always include index or id in nested top hits"
    -
       - do:
           search:
             rest_total_hits_as_int: true
    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 8532b40fbc1e1..4a01da6e6c213 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
    @@ -60,10 +60,6 @@ setup:
     
     ---
     "Simple Composite aggregation":
    -  - skip:
    -      version: " - 6.0.99"
    -      reason:  this uses a new API that has been added in 6.1
    -
       - do:
           search:
             rest_total_hits_as_int: true
    @@ -89,11 +85,6 @@ setup:
     
     ---
     "Nested Composite aggregation":
    -  - skip:
    -      version: " - 6.0.99"
    -      reason:  this uses a new API that has been added in 6.1
    -
    -
       - do:
           search:
             rest_total_hits_as_int: true
    @@ -139,11 +130,6 @@ setup:
     
     ---
     "Aggregate After":
    -  - skip:
    -      version: " - 6.0.99"
    -      reason:  this uses a new API that has been added in 6.1
    -
    -
       - do:
           search:
             rest_total_hits_as_int: true
    @@ -181,11 +167,6 @@ setup:
     
     ---
     "Aggregate After Missing":
    -  - skip:
    -      version: " - 6.1.99"
    -      reason:  bug fixed in 6.2.0
    -
    -
       - do:
           search:
             rest_total_hits_as_int: true
    @@ -212,10 +193,6 @@ setup:
     
     ---
     "Invalid Composite aggregation":
    -  - skip:
    -      version: " - 6.0.99"
    -      reason:  this uses a new API that has been added in 6.1
    -
       - do:
           catch:      /\[composite\] aggregation cannot be used with a parent aggregation/
           search:
    @@ -241,10 +218,6 @@ setup:
     
     ---
     "Composite aggregation with format":
    -  - skip:
    -      version: " - 6.2.99"
    -      reason:  this uses a new option (format) added in 6.3.0
    -
       - do:
           search:
             rest_total_hits_as_int: true
    @@ -302,10 +275,6 @@ setup:
     
     ---
     "Composite aggregation with after_key in the response":
    -  - skip:
    -      version: " - 6.2.99"
    -      reason:  starting in 6.3.0 after_key is returned in the response
    -
       - do:
             search:
               rest_total_hits_as_int: true
    @@ -331,10 +300,6 @@ setup:
     
     ---
     "Composite aggregation and array size":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  starting in 7.0 the composite aggregation throws an execption if the provided size is greater than search.max_buckets.
    -
       - do:
             catch: /.*Trying to create too many buckets.*/
             search:
    @@ -357,10 +322,6 @@ setup:
     
     ---
     "Composite aggregation with nested parent":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  the ability to set a nested parent aggregation was added in 7.0.
    -
       - do:
             search:
               rest_total_hits_as_int: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/240_max_buckets.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/240_max_buckets.yml
    index 3dd8d345043c3..981fcbd574b15 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/240_max_buckets.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/240_max_buckets.yml
    @@ -75,10 +75,6 @@ setup:
     
     ---
     "Max bucket":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  search.max_buckets limit has been added in 7.0
    -
       - do:
           cluster.put_settings:
             body:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/250_moving_fn.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/250_moving_fn.yml
    index 0a7affd276aea..2bfdf4703307d 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/250_moving_fn.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/250_moving_fn.yml
    @@ -1,8 +1,3 @@
    -setup:
    -  - skip:
    -        version: " - 6.3.99"
    -        reason:  "moving_fn added in 6.4.0"
    -
     ---
     "Bad window":
     
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/260_weighted_avg.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/260_weighted_avg.yml
    index c5988fc9e5dc4..6b17132c751de 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/260_weighted_avg.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/260_weighted_avg.yml
    @@ -1,7 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.3.99"
    -      reason: weighted_avg is only available as of 6.4.0
       - do:
           indices.create:
               index: test_1
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/270_median_absolute_deviation_metric.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/270_median_absolute_deviation_metric.yml
    index 0cba08fccae9b..03797503436fb 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/270_median_absolute_deviation_metric.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/270_median_absolute_deviation_metric.yml
    @@ -1,7 +1,4 @@
     setup:
    -  - skip:
    -        version: " - 6.5.99"
    -        reason:  "added in 6.6.0"
       - do:
             indices.create:
                 index: test
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.highlight/30_max_analyzed_offset.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.highlight/30_max_analyzed_offset.yml
    index 334708b54b066..a9506b053b4ac 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.highlight/30_max_analyzed_offset.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.highlight/30_max_analyzed_offset.yml
    @@ -28,9 +28,7 @@ setup:
     
     ---
     "Unified highlighter on a field WITHOUT OFFSETS exceeding index.highlight.max_analyzed_offset should FAIL":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: index.highlight.max_analyzed_offset setting has been added in 7.0.0
    +
       - do:
           catch: bad_request
           search:
    @@ -42,9 +40,7 @@ setup:
     
     ---
     "Plain highlighter on a field WITHOUT OFFSETS exceeding index.highlight.max_analyzed_offset should FAIL":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: index.highlight.max_analyzed_offset setting has been added in 7.0.0
    +
       - do:
           catch: bad_request
           search:
    @@ -56,9 +52,7 @@ setup:
     
     ---
     "Unified highlighter on a field WITH OFFSETS exceeding index.highlight.max_analyzed_offset should SUCCEED":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: index.highligt.max_analyzed_offset setting has been added in 7.0.0
    +
       - do:
           search:
               rest_total_hits_as_int: true
    @@ -69,9 +63,7 @@ setup:
     
     ---
     "Plain highlighter on a field WITH OFFSETS exceeding index.highlight.max_analyzed_offset should FAIL":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: index.highlight.max_analyzed_offset setting has been added in 7.0.0
    +
       - do:
           catch: bad_request
           search:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml
    index 89ea24618c68f..82ccb816f2314 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml
    @@ -1,8 +1,5 @@
     ---
     "Terms Query with No.of terms exceeding index.max_terms_count should FAIL":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: index.max_terms_count setting has been added in 7.0.0
       - do:
           indices.create:
               index: test_index
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/190_index_prefix_search.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/190_index_prefix_search.yml
    index 40c80b88cfb1b..f667786f557f9 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/190_index_prefix_search.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/190_index_prefix_search.yml
    @@ -1,8 +1,4 @@
     setup:
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: index_prefixes is only available as of 6.3.0
    -
       - do:
           indices.create:
             index:  test
    @@ -27,9 +23,6 @@ setup:
     
     ---
     "search with index prefixes":
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: index_prefixes is only available as of 6.3.0
       - do:
           search:
             rest_total_hits_as_int: true
    @@ -85,9 +78,6 @@ setup:
     
     ---
     "search index prefixes with span_multi":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: span_multi throws an exception with prefix fields on < versions
     
       - do:
           search:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_ignore_malformed.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_ignore_malformed.yml
    index 71ddb32302396..162da4fc3c9e5 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_ignore_malformed.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_ignore_malformed.yml
    @@ -1,8 +1,5 @@
     ---
     setup:
    -  - skip:
    -      version: " - 6.3.99"
    -      reason: _ignored was added in 6.4.0
     
       - do:
           indices.create:
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_index_phrase_search.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_index_phrase_search.yml
    index b48857be4e7a1..d37c0c8cb3f72 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_index_phrase_search.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/200_index_phrase_search.yml
    @@ -1,8 +1,6 @@
     ---
     "search with indexed phrases":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: index_phrase is only available as of 7.0.0
    +
       - do:
           indices.create:
             index:  test
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/210_rescore_explain.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/210_rescore_explain.yml
    index 92bb049980dff..4d63a81a99595 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/210_rescore_explain.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/210_rescore_explain.yml
    @@ -1,8 +1,5 @@
     ---
     "Score should match explanation in rescore":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Explanation for rescoring was corrected after these versions
       - do:
           bulk:
             refresh: true
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/50_completion_with_multi_fields.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/50_completion_with_multi_fields.yml
    index a29019183e199..8e793e4beb6f0 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/50_completion_with_multi_fields.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/50_completion_with_multi_fields.yml
    @@ -2,10 +2,6 @@
     ---
     "Search by suggestion and by keyword sub-field should work":
     
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  "Search by suggestion with multi-fields was introduced 7.0.0"
    -
       - do:
           indices.create:
             index: completion_with_sub_keyword
    @@ -64,10 +60,6 @@
     ---
     "Search by suggestion on sub field should work":
     
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  "Search by suggestion with multi-fields was introduced 7.0.0"
    -
       - do:
           indices.create:
             index: completion_with_sub_completion
    @@ -114,10 +106,6 @@
     ---
     "Search by suggestion on sub field with context should work":
     
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  "Search by suggestion with multi-fields was introduced 7.0.0"
    -
       - do:
           indices.create:
             index: completion_with_context
    @@ -183,10 +171,6 @@
     ---
     "Search by suggestion on sub field with weight should work":
     
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  "Search by suggestion with multi-fields was introduced 7.0.0"
    -
       - do:
           indices.create:
             index: completion_with_weight
    @@ -239,10 +223,6 @@
     ---
     "Search by suggestion on geofield-hash on sub field should work":
     
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  "Search by suggestion with multi-fields was introduced 7.0.0"
    -
       - do:
           indices.create:
             index: geofield_with_completion
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/update/35_if_seq_no.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/update/35_if_seq_no.yml
    index f982adf693ad0..318ecd3a7c0ce 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/update/35_if_seq_no.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/update/35_if_seq_no.yml
    @@ -1,10 +1,5 @@
     ---
     "Update with if_seq_no":
    -
    - - skip:
    -      version: " - 6.6.99"
    -      reason: if_seq_no was added in 6.7.0
    -
      - do:
           catch:        missing
           update:
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/graph/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/graph/10_basic.yml
    index c7aa714032f92..6c691f4f60e14 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/graph/10_basic.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/graph/10_basic.yml
    @@ -15,9 +15,6 @@ setup:
     
     ---
     "Test basic graph explore":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: "graph endpoints changed in 7.0.0 to not include _xpack in the path but this is not supported in 6.x"
       - do:
           index:
               index:  test_1
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/indices.freeze/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/indices.freeze/10_basic.yml
    index 16a0aace0e444..4ba49e53308d2 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/indices.freeze/10_basic.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/indices.freeze/10_basic.yml
    @@ -1,10 +1,5 @@
     ---
     "Basic":
    -
    -- skip:
    -    version: " - 6.99.99"
    -    reason: types are required in requests before 7.0.0
    -
     - do:
         index:
           index: test
    @@ -84,11 +79,6 @@
     
     ---
     "Test index options":
    -
    -- skip:
    -    version: " - 6.99.99"
    -    reason: types are required in requests before 7.0.0
    -
     - do:
         index:
           index: test
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_rollup_index_caps.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_rollup_index_caps.yml
    index e4b98b9492087..c666663ad760e 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_rollup_index_caps.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/get_rollup_index_caps.yml
    @@ -290,10 +290,6 @@ setup:
     ---
     "Verify job caps by rollup index comma delimited list":
     
    -- skip:
    -    version: " - 6.99.99"
    -    reason: "comma delimited index support was fixed in 7.0"
    -
     - do:
         headers:
           Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml
    index 7e8b6b3f61af0..db15be229214c 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/rollup/stop_job.yml
    @@ -85,10 +85,6 @@ setup:
     
     ---
     "Test wait_for_completion default timeout":
    -  - skip:
    -      version: " - 6.5.99"
    -      reason: wait_for_completion option was added in 6.6
    -
       - do:
           headers:
             Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
    @@ -106,10 +102,6 @@ setup:
     
     ---
     "Test wait_for_completion with custom timeout":
    -  - skip:
    -      version: " - 6.5.99"
    -      reason: wait_for_completion option was added in 6.6
    -
       - do:
           headers:
             Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/sql/translate.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/sql/translate.yml
    index 57f275af5e26b..dd4ce58dc80f7 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/sql/translate.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/sql/translate.yml
    @@ -1,9 +1,5 @@
     ---
     "Translate SQL":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: Triggers warnings before 7.0
    -
       - do:
           bulk:
             refresh: true
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml
    index 077ddd2d30825..02191f0b680a9 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml
    @@ -117,10 +117,6 @@ setup:
     
     ---
     "Test putting a watch with a redacted password with old seq no returns an error":
    -  - skip:
    -      version: " - 6.6.99"
    -      reason:  seq no powered concurrency was added in 6.7.0
    -
       # version 1
       - do:
           watcher.put_watch:
    diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/stats/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/stats/10_basic.yml
    index f5a8b149fe952..b737b040f212d 100644
    --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/stats/10_basic.yml
    +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/watcher/stats/10_basic.yml
    @@ -15,10 +15,6 @@
     
     ---
     "Test watcher stats current watches":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  metrics were fixed in 7.0.0
    -
       - do:
           watcher.stats:
             metric: "current_watches"
    @@ -28,10 +24,6 @@
     
     ---
     "Test watcher stats queued watches":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason:  metrics were fixed in 7.0.0
    -
       - do:
           watcher.stats:
             metric: "queued_watches"
    @@ -42,7 +34,6 @@
     ---
     "Test watcher stats queued watches using pending_watches":
       - skip:
    -      version: " - 6.99.99"
           reason:  metrics were fixed in 7.0.0
           features: warnings
     
    diff --git a/x-pack/qa/core-rest-tests-with-security/src/test/resources/rest-api-spec/test/rankeval/10_rankeval.yml b/x-pack/qa/core-rest-tests-with-security/src/test/resources/rest-api-spec/test/rankeval/10_rankeval.yml
    index 4720377909375..ed00c34874d21 100644
    --- a/x-pack/qa/core-rest-tests-with-security/src/test/resources/rest-api-spec/test/rankeval/10_rankeval.yml
    +++ b/x-pack/qa/core-rest-tests-with-security/src/test/resources/rest-api-spec/test/rankeval/10_rankeval.yml
    @@ -2,11 +2,6 @@
     # pull in rest test from modules in core
     ---
     "Basic Rankeval test":
    -
    -  - skip:
    -      version: " - 6.2.99"
    -      reason: response format was updated in 6.3
    -
       - do:
          indices.create:
              index: foo
    diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_ml_jobs_crud.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_ml_jobs_crud.yml
    index 3401d6e212717..bca44b5f16a38 100644
    --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_ml_jobs_crud.yml
    +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/30_ml_jobs_crud.yml
    @@ -133,50 +133,8 @@
                 }
               }
     
    ----
    -"Test job with pre 6.4 rules":
    -
    -  - skip:
    -      version:  "6.4.0 - "
    -      reason:   "Rules were replaced by custom_rules on 6.4.0"
    -
    -  - do:
    -      ml.put_job:
    -        job_id: job-with-old-rules
    -        body:  >
    -          {
    -            "analysis_config": {
    -              "detectors": [
    -                {
    -                  "function": "count",
    -                  "by_field_name": "country",
    -                  "detector_rules": [
    -                    {
    -                      "rule_action": "filter_results",
    -                      "rule_conditions": [
    -                        {
    -                          "condition_type":"numerical_actual",
    -                          "field_name":"country",
    -                          "field_value": "uk",
    -                          "condition": {"operator":"lt","value":"33.3"}
    -                        },
    -                        {"condition_type":"categorical", "field_name":"country", "value_filter": "foo"}
    -                      ]
    -                    }
    -                  ]
    -                }
    -              ]
    -            },
    -            "data_description" : {}
    -          }
    -
     ---
     "Test job with pre 6.4 rules - dummy job 6.4 onwards":
    -
    -  - skip:
    -      version:  " - 6.3.99"
    -      reason:   "Rules replaced by custom_rules on 6.4.0"
    -
       - do:
           ml.put_job:
             job_id: job-with-old-rules
    diff --git a/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/30_search_input.yml b/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/30_search_input.yml
    index d712ddba3a498..7866b54de7c77 100644
    --- a/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/30_search_input.yml
    +++ b/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/mustache/30_search_input.yml
    @@ -168,9 +168,6 @@ setup:
     
     ---
     "Test search input mustache integration (using request body and rest_total_hits_as_int)":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: "rest_total_hits_as_int support was added in 7.0"
       - do:
           watcher.execute_watch:
             body: >
    @@ -222,10 +219,6 @@ setup:
     
     ---
     "Test search input mustache integration (using request template and rest_total_hits_as_int)":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: "rest_total_hits_as_int support was added in 7.0"
    -
       - do:
           put_script:
             id: "search-template"
    diff --git a/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/painless/10_basic.yml b/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/painless/10_basic.yml
    index 7f6db2a6d6614..6ec0adcc93310 100644
    --- a/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/painless/10_basic.yml
    +++ b/x-pack/qa/smoke-test-watcher/src/test/resources/rest-api-spec/test/painless/10_basic.yml
    @@ -123,10 +123,6 @@
     
     ---
     "Test execute watch api with rest_total_hits_as_int":
    -  - skip:
    -      version: " - 6.99.99"
    -      reason: "rest_total_hits_as_int support was added in 7.0"
    -
       - do:
           cluster.health:
             wait_for_status: green
    
    From bd2b33d6eaca616a5acd846204e2d12f905854d4 Mon Sep 17 00:00:00 2001
    From: Costin Leau 
    Date: Thu, 18 Apr 2019 22:44:16 +0300
    Subject: [PATCH 101/260] SQL: Predicate diff takes into account all values
     (#41346)
    
    Fix bug in predicate subtraction that caused the evaluation to be
    skipped on the first mismatch instead of evaluating the whole list. In
    some cases this caused not only an incorrect result but one that kept on
    growing causing the engine to bail
    
    Fix #40835
    ---
     .../sql/capabilities/UnresolvedException.java |  6 +----
     .../sql/expression/predicate/Predicates.java  | 17 +++++++-----
     .../xpack/sql/optimizer/Optimizer.java        |  2 +-
     .../xpack/sql/rule/RuleExecutor.java          |  4 +--
     .../xpack/sql/optimizer/OptimizerTests.java   | 26 ++++++++++++++++---
     5 files changed, 38 insertions(+), 17 deletions(-)
    
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/capabilities/UnresolvedException.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/capabilities/UnresolvedException.java
    index d6c7543f6afa2..0db87c6b944e4 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/capabilities/UnresolvedException.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/capabilities/UnresolvedException.java
    @@ -7,16 +7,12 @@
     
     import org.elasticsearch.xpack.sql.ServerSqlException;
     
    -import java.util.Locale;
    -
    -import static java.lang.String.format;
    -
     /**
      * Thrown when we accidentally attempt to resolve something on on an unresolved entity. Throwing this
      * is always a bug.
      */
     public class UnresolvedException extends ServerSqlException {
         public UnresolvedException(String action, Object target) {
    -        super(format(Locale.ROOT, "Invalid call to %s on an unresolved object %s", action, target));
    +        super("Invalid call to {} on an unresolved object {}", action, target);
         }
     }
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Predicates.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Predicates.java
    index c280d974e8850..83a4b96f8295b 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Predicates.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Predicates.java
    @@ -97,14 +97,19 @@ public static List inCommon(List l, List r)
             return common.isEmpty() ? emptyList() : common;
         }
     
    -    public static List subtract(List from, List r) {
    -        List diff = new ArrayList<>(Math.min(from.size(), r.size()));
    -        for (Expression lExp : from) {
    -            for (Expression rExp : r) {
    -                if (!lExp.semanticEquals(rExp)) {
    -                    diff.add(lExp);
    +    public static List subtract(List from, List list) {
    +        List diff = new ArrayList<>(Math.min(from.size(), list.size()));
    +        for (Expression f : from) {
    +            boolean found = false;
    +            for (Expression l : list) {
    +                if (f.semanticEquals(l)) {
    +                    found = true;
    +                    break;
                     }
                 }
    +            if (found == false) {
    +                diff.add(f);
    +            }
             }
             return diff.isEmpty() ? emptyList() : diff;
         }
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
    index d6e4c4fe07d7e..eafdf21b11930 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
    @@ -1236,7 +1236,7 @@ static class BooleanSimplification extends OptimizerExpressionRule {
     
             @Override
             protected Expression rule(Expression e) {
    -            if (e instanceof BinaryPredicate) {
    +            if (e instanceof And || e instanceof Or) {
                     return simplifyAndOr((BinaryPredicate) e);
                 }
                 if (e instanceof Not) {
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/rule/RuleExecutor.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/rule/RuleExecutor.java
    index 2ed68def135ec..689b0b338a99e 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/rule/RuleExecutor.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/rule/RuleExecutor.java
    @@ -38,7 +38,7 @@ public Limiter(int maximumRuns) {
     
             boolean reached(int runs) {
                 if (runs >= this.runs) {
    -                throw new RuleExecutionException("Rule execution limit %d reached", runs);
    +                throw new RuleExecutionException("Rule execution limit [{}] reached", runs);
                 }
                 return false;
             }
    @@ -139,7 +139,7 @@ protected ExecutionInfo executeWithInfo(TreeType plan) {
     
             for (Batch batch : batches) {
                 int batchRuns = 0;
    -            List tfs = new ArrayList();
    +            List tfs = new ArrayList<>();
                 transformations.put(batch, tfs);
     
                 boolean hasChanged = false;
    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 a23d88b595630..c95206c29e9ed 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
    @@ -181,9 +181,12 @@ private static Literal L(Object value) {
         }
     
         private static FieldAttribute getFieldAttribute() {
    -        return new FieldAttribute(EMPTY, "a", new EsField("af", DataType.INTEGER, emptyMap(), true));
    +        return getFieldAttribute("a");
         }
     
    +    private static FieldAttribute getFieldAttribute(String name) {
    +        return new FieldAttribute(EMPTY, name, new EsField(name + "f", DataType.INTEGER, emptyMap(), true));
    +    }
     
         public void testPruneSubqueryAliases() {
             ShowTables s = new ShowTables(EMPTY, null, null);
    @@ -1145,6 +1148,23 @@ public void testCombineBinaryComparisonsWithDifferentUpperLimitInclusion() {
             assertEquals(or, exp);
         }
     
    +    // (a = 1 AND b = 3 AND c = 4) OR (a = 2 AND b = 3 AND c = 4) -> (b = 3 AND c = 4) AND (a = 1 OR a = 2)
    +    public void testBooleanSimplificationCommonExpressionSubstraction() {
    +        FieldAttribute fa = getFieldAttribute("a");
    +        FieldAttribute fb = getFieldAttribute("b");
    +        FieldAttribute fc = getFieldAttribute("c");
    +
    +        Expression a1 = new Equals(EMPTY, fa, ONE);
    +        Expression a2 = new Equals(EMPTY, fa, TWO);
    +        And common = new And(EMPTY, new Equals(EMPTY, fb, THREE), new Equals(EMPTY, fc, FOUR));
    +        And left = new And(EMPTY, a1, common);
    +        And right = new And(EMPTY, a2, common);
    +        Or or = new Or(EMPTY, left, right);
    +
    +        Expression exp = new BooleanSimplification().rule(or);
    +        assertEquals(new And(EMPTY, common, new Or(EMPTY, a1, a2)), exp);
    +    }
    +
         // (0 < a <= 1) OR (0 < a < 2) -> 0 < a < 2
         public void testRangesOverlappingNoLowerBoundary() {
             FieldAttribute fa = getFieldAttribute();
    @@ -1289,7 +1309,7 @@ public void testSortAggregateOnOrderByWithTwoFields() {
             Order firstOrderBy = new Order(EMPTY, firstField, OrderDirection.ASC, Order.NullsPosition.LAST);
             Order secondOrderBy = new Order(EMPTY, secondField, OrderDirection.ASC, Order.NullsPosition.LAST);
             
    -        OrderBy orderByPlan = new OrderBy(EMPTY, 
    +        OrderBy orderByPlan = new OrderBy(EMPTY,
                     new Aggregate(EMPTY, FROM(), Arrays.asList(secondField, firstField), Arrays.asList(secondAlias, firstAlias)),
                     Arrays.asList(firstOrderBy, secondOrderBy));
             LogicalPlan result = new Optimizer.SortAggregateOnOrderBy().apply(orderByPlan);
    @@ -1321,7 +1341,7 @@ public void testSortAggregateOnOrderByOnlyAliases() {
             Order firstOrderBy = new Order(EMPTY, firstAlias, OrderDirection.ASC, Order.NullsPosition.LAST);
             Order secondOrderBy = new Order(EMPTY, secondAlias, OrderDirection.ASC, Order.NullsPosition.LAST);
             
    -        OrderBy orderByPlan = new OrderBy(EMPTY, 
    +        OrderBy orderByPlan = new OrderBy(EMPTY,
                     new Aggregate(EMPTY, FROM(), Arrays.asList(secondAlias, firstAlias), Arrays.asList(secondAlias, firstAlias)),
                     Arrays.asList(firstOrderBy, secondOrderBy));
             LogicalPlan result = new Optimizer.SortAggregateOnOrderBy().apply(orderByPlan);
    
    From e596d0b00a6aba10e830bd9844e4f52f615be3b9 Mon Sep 17 00:00:00 2001
    From: Mark Vieira 
    Date: Thu, 18 Apr 2019 13:11:31 -0700
    Subject: [PATCH 102/260] Upgrade to Gradle 5.4 (#41357)
    
    ---
     buildSrc/src/main/resources/minimumGradleVersion | 2 +-
     gradle/wrapper/gradle-wrapper.properties         | 4 ++--
     2 files changed, 3 insertions(+), 3 deletions(-)
    
    diff --git a/buildSrc/src/main/resources/minimumGradleVersion b/buildSrc/src/main/resources/minimumGradleVersion
    index 11aa145248e68..48c32b26a12f0 100644
    --- a/buildSrc/src/main/resources/minimumGradleVersion
    +++ b/buildSrc/src/main/resources/minimumGradleVersion
    @@ -1 +1 @@
    -5.3
    \ No newline at end of file
    +5.4
    \ No newline at end of file
    diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
    index 8d172843af1d6..74b0ec22fcf7f 100644
    --- a/gradle/wrapper/gradle-wrapper.properties
    +++ b/gradle/wrapper/gradle-wrapper.properties
    @@ -1,6 +1,6 @@
     distributionBase=GRADLE_USER_HOME
     distributionPath=wrapper/dists
    -distributionUrl=https\://services.gradle.org/distributions/gradle-5.3-all.zip
    +distributionUrl=https\://services.gradle.org/distributions/gradle-5.4-all.zip
     zipStoreBase=GRADLE_USER_HOME
     zipStorePath=wrapper/dists
    -distributionSha256Sum=f4d820c2a9685710eba5b92f10e0e4fb20e0d6c0dd1f46971e658160f25e7147
    +distributionSha256Sum=f177768e7a032727e4338c8fd047f8f263e5bd283f67a7766c1ba4182c8455a6
    
    From 64270a3ce2f92016d0a021dfa22d6cf767164a87 Mon Sep 17 00:00:00 2001
    From: Jay Modi 
    Date: Thu, 18 Apr 2019 14:17:16 -0600
    Subject: [PATCH 103/260] Clean up docs regarding recommended JVM (#41356)
    
    This change clarifies the documentation around the recommended JVM. The
    recommended JVM is the bundled JVM. If a user does not use our
    recommended JVM we suggest that they use a supported LTS version of the
    JVM.
    
    Closes #41132
    ---
     docs/reference/setup.asciidoc | 11 +++++------
     1 file changed, 5 insertions(+), 6 deletions(-)
    
    diff --git a/docs/reference/setup.asciidoc b/docs/reference/setup.asciidoc
    index d2d46670809a2..8fd5a44443ddc 100644
    --- a/docs/reference/setup.asciidoc
    +++ b/docs/reference/setup.asciidoc
    @@ -25,16 +25,15 @@ platforms, but it is possible that it will work on other platforms too.
     
     Elasticsearch is built using Java, and includes a bundled version of
     http://openjdk.java.net[OpenJDK] from the JDK maintainers (GPLv2+CE)
    -within each distribution. The bundled JVM exists within the `jdk` directory of
    -the Elasticsearch home directory.
    +within each distribution. The bundled JVM is the recommended JVM and
    +is located within the `jdk` directory of the Elasticsearch home directory.
     
     To use your own version of Java, set the `JAVA_HOME` environment variable.
    -When using your own version, the bundled JVM directory may be removed.
    -If not using the bundled JVM, we recommend installing Java version
    - *{jdk} or a later version in the Java {jdk_major} release series*. We recommend
    -using a link:/support/matrix[supported]
    +If you must use a version of Java that is different from the bundled JVM,
    +we recommend using a link:/support/matrix[supported]
     http://www.oracle.com/technetwork/java/eol-135779.html[LTS version of Java].
     Elasticsearch will refuse to start if a known-bad version of Java is used.
    +The bundled JVM directory may be removed when using your own JVM.
     
     --
     
    
    From 17a6ac1a723148f47b113e35fd8e66d9e9cce016 Mon Sep 17 00:00:00 2001
    From: Jim Ferenczi 
    Date: Thu, 18 Apr 2019 22:29:36 +0200
    Subject: [PATCH 104/260] more_like_this query to throw an error if the like
     fields is not provided (#40632)
    
    With the removal of the `_all` field the `mlt` query cannot infer a field name
    to use to analyze the provided (un)like text if the `fields` parameter is not
    explicitly set in the query and the `index.query.default_field` is not changed
    in the index settings (by default it is set to `*`). For this reason the like text
    is ignored and queries are only built from the provided document ids.
    This change fixes this bug by throwing an error if the fields option is not set
    and the `index.query.default_field` is equals to `*`. The error is thrown only
    if like or unlike texts are provided in the query.
    ---
     .../index/query/MoreLikeThisQueryBuilder.java |  7 +++
     .../query/MoreLikeThisQueryBuilderTests.java  | 48 +++++++++++++++++--
     .../search/morelikethis/MoreLikeThisIT.java   |  8 ++--
     3 files changed, 56 insertions(+), 7 deletions(-)
    
    diff --git a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
    index b90a1e60ffa0b..11530ce5f30b3 100644
    --- a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
    +++ b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java
    @@ -1050,6 +1050,13 @@ protected Query doToQuery(QueryShardContext context) throws IOException {
             List moreLikeFields = new ArrayList<>();
             if (useDefaultField) {
                 moreLikeFields = context.defaultFields();
    +            if (moreLikeFields.size() == 1
    +                    && moreLikeFields.get(0).equals("*")
    +                    && (likeTexts.length > 0 || unlikeTexts.length > 0)) {
    +                throw new IllegalArgumentException("[more_like_this] query cannot infer the field to analyze the free text, " +
    +                    "you should update the [index.query.default_field] index setting to a field that exists in the mapping or " +
    +                    "set the [fields] option in the query.");
    +            }
             } else {
                 for (String field : fields) {
                     MappedFieldType fieldType = context.fieldMapper(field);
    diff --git a/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java
    index 62613139b50fd..56e7a32347265 100644
    --- a/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java
    @@ -31,10 +31,12 @@
     import org.elasticsearch.action.termvectors.MultiTermVectorsResponse;
     import org.elasticsearch.action.termvectors.TermVectorsRequest;
     import org.elasticsearch.action.termvectors.TermVectorsResponse;
    +import org.elasticsearch.cluster.metadata.IndexMetaData;
     import org.elasticsearch.common.Strings;
     import org.elasticsearch.common.bytes.BytesReference;
     import org.elasticsearch.common.io.stream.BytesStreamOutput;
     import org.elasticsearch.common.lucene.search.MoreLikeThisQuery;
    +import org.elasticsearch.common.settings.Settings;
     import org.elasticsearch.common.xcontent.ToXContent;
     import org.elasticsearch.common.xcontent.XContentBuilder;
     import org.elasticsearch.common.xcontent.XContentFactory;
    @@ -160,13 +162,13 @@ protected MoreLikeThisQueryBuilder doCreateTestQueryBuilder() {
             } else {
                 likeItems = randomLikeItems;
             }
    -        if (randomBoolean()) { // for the default field
    -            queryBuilder = new MoreLikeThisQueryBuilder(likeTexts, likeItems);
    +        if (randomBoolean() && likeItems != null && likeItems.length > 0) { // for the default field
    +            queryBuilder = new MoreLikeThisQueryBuilder(null, likeItems);
             } else {
                 queryBuilder = new MoreLikeThisQueryBuilder(randomFields, likeTexts, likeItems);
             }
     
    -        if (randomBoolean()) {
    +        if (randomBoolean() && queryBuilder.fields() != null) {
                 queryBuilder.unlike(generateRandomStringArray(5, 5, false, false));
             }
             if (randomBoolean()) {
    @@ -305,6 +307,39 @@ public void testUnsupportedFields() throws IOException {
             assertThat(e.getMessage(), containsString("more_like_this only supports text/keyword fields"));
         }
     
    +    public void testDefaultField() throws IOException {
    +        QueryShardContext context = createShardContext();
    +
    +        {
    +            MoreLikeThisQueryBuilder builder =
    +                new MoreLikeThisQueryBuilder(new String[]{"hello world"}, null);
    +            IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
    +                () -> builder.toQuery(context));
    +            assertThat(e.getMessage(), containsString("[more_like_this] query cannot infer"));
    +        }
    +
    +        {
    +            context.getIndexSettings().updateIndexMetaData(
    +                newIndexMeta("index",
    +                    context.getIndexSettings().getSettings(),
    +                    Settings.builder().putList("index.query.default_field", STRING_FIELD_NAME).build()
    +                )
    +            );
    +            try {
    +                MoreLikeThisQueryBuilder builder = new MoreLikeThisQueryBuilder(new String[]{"hello world"}, null);
    +                builder.toQuery(context);
    +            } finally {
    +                // Reset the default value
    +                context.getIndexSettings().updateIndexMetaData(
    +                    newIndexMeta("index",
    +                        context.getIndexSettings().getSettings(),
    +                        Settings.builder().putList("index.query.default_field", "*").build()
    +                    )
    +                );
    +            }
    +        }
    +    }
    +
         public void testMoreLikeThisBuilder() throws Exception {
             Query parsedQuery =
                 parseQuery(moreLikeThisQuery(new String[]{"name.first", "name.last"}, new String[]{"something"}, null)
    @@ -390,4 +425,11 @@ protected QueryBuilder parseQuery(XContentParser parser) throws IOException {
             }
             return query;
         }
    +
    +    private static IndexMetaData newIndexMeta(String name, Settings oldIndexSettings, Settings indexSettings) {
    +        Settings build = Settings.builder().put(oldIndexSettings)
    +            .put(indexSettings)
    +            .build();
    +        return IndexMetaData.builder(name).settings(build).build();
    +    }
     }
    diff --git a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java
    index 2e29c7c5a3815..4492353f6f15b 100644
    --- a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java
    @@ -343,10 +343,10 @@ public void testNumericField() throws Exception {
                     new MoreLikeThisQueryBuilder(new String[] {"string_value", "int_value"}, null,
                             new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)), SearchPhaseExecutionException.class);
     
    -        // mlt query with no field -> No results (because _all is not enabled)
    -        searchResponse = client().prepareSearch().setQuery(moreLikeThisQuery(new String[] {"index"}).minTermFreq(1).minDocFreq(1))
    -                .get();
    -        assertHitCount(searchResponse, 0L);
    +        // mlt query with no field -> exception because _all is not enabled)
    +        assertThrows(client().prepareSearch()
    +            .setQuery(moreLikeThisQuery(new String[] {"index"}).minTermFreq(1).minDocFreq(1)),
    +            SearchPhaseExecutionException.class);
     
             // mlt query with string fields
             searchResponse = client().prepareSearch().setQuery(moreLikeThisQuery(new String[]{"string_value"}, new String[] {"index"}, null)
    
    From 577f58dbf31e12d8b2284803a5bd3024b5b8fffd Mon Sep 17 00:00:00 2001
    From: Jim Ferenczi 
    Date: Thu, 18 Apr 2019 22:30:51 +0200
    Subject: [PATCH 105/260] Unified highlighter should ignore terms that targets
     the _id field (#41275)
    
    The `_id` field uses a binary encoding to index terms that is not compatible with
    the utf8 automaton that the unified highlighter creates to reanalyze the input.
    For these reason this commit ignores terms that target the `_id` field when
    `require_field_match` is set to false.
    
    Closes #37525
    ---
     .../highlight/UnifiedHighlighter.java         |  5 +++-
     .../highlight/HighlighterSearchIT.java        | 23 +++++++++++++++++++
     2 files changed, 27 insertions(+), 1 deletion(-)
    
    diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java
    index 2d570d2b7c793..2a75e9c58f4fc 100644
    --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java
    +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/UnifiedHighlighter.java
    @@ -35,6 +35,7 @@
     import org.elasticsearch.common.text.Text;
     import org.elasticsearch.index.IndexSettings;
     import org.elasticsearch.index.mapper.DocumentMapper;
    +import org.elasticsearch.index.mapper.IdFieldMapper;
     import org.elasticsearch.index.mapper.MappedFieldType;
     import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
     import org.elasticsearch.search.fetch.FetchSubPhase;
    @@ -109,7 +110,9 @@ public HighlightField highlight(HighlighterContext highlighterContext) {
                     final String fieldName = highlighterContext.fieldName;
                     highlighter.setFieldMatcher((name) -> fieldName.equals(name));
                 } else {
    -                highlighter.setFieldMatcher((name) -> true);
    +                // ignore terms that targets the _id field since they use a different encoding
    +                // that is not compatible with utf8
    +                highlighter.setFieldMatcher(name -> IdFieldMapper.NAME.equals(name) == false);
                 }
     
                 Snippet[] fieldSnippets = highlighter.highlightField(highlighterContext.fieldName,
    diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java
    index d1a669695313c..1467fd1f0971e 100644
    --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java
    @@ -2947,6 +2947,29 @@ public void testWithNormalizer() throws Exception {
             }
         }
     
    +    public void testDisableHighlightIdField() throws Exception {
    +        assertAcked(prepareCreate("test")
    +            .addMapping("doc", "keyword", "type=keyword"));
    +        ensureGreen();
    +
    +        client().prepareIndex("test", "doc", "d33f85bf1e51e84d9ab38948db9f3a068e1fe5294f1d8603914ac8c7bcc39ca1")
    +            .setSource("keyword", "Hello World")
    +            .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
    +            .get();
    +
    +        for (String highlighterType : new String[] {"plain", "unified"}) {
    +            SearchResponse searchResponse = client().prepareSearch()
    +                .setQuery(matchQuery("_id", "d33f85bf1e51e84d9ab38948db9f3a068e1fe5294f1d8603914ac8c7bcc39ca1"))
    +                .highlighter(
    +                    new HighlightBuilder().field(new Field("*")
    +                        .highlighterType(highlighterType).requireFieldMatch(false))
    +                )
    +                .get();
    +            assertHitCount(searchResponse, 1);
    +            assertNull(searchResponse.getHits().getAt(0).getHighlightFields().get("_id"));
    +        }
    +    }
    +
         public static class MockAnalysisPlugin extends Plugin implements AnalysisPlugin {
     
             public final class MockSnowBall extends TokenFilter {
    
    From 6de2e2628ca3ad2895b10eacaeb1ce051639a9a7 Mon Sep 17 00:00:00 2001
    From: Mark Vieira 
    Date: Thu, 18 Apr 2019 13:38:07 -0700
    Subject: [PATCH 106/260] Revert "Upgrade to Gradle 5.4 (#41357)"
    
    This reverts commit e596d0b0
    ---
     buildSrc/src/main/resources/minimumGradleVersion | 2 +-
     gradle/wrapper/gradle-wrapper.properties         | 4 ++--
     2 files changed, 3 insertions(+), 3 deletions(-)
    
    diff --git a/buildSrc/src/main/resources/minimumGradleVersion b/buildSrc/src/main/resources/minimumGradleVersion
    index 48c32b26a12f0..11aa145248e68 100644
    --- a/buildSrc/src/main/resources/minimumGradleVersion
    +++ b/buildSrc/src/main/resources/minimumGradleVersion
    @@ -1 +1 @@
    -5.4
    \ No newline at end of file
    +5.3
    \ No newline at end of file
    diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
    index 74b0ec22fcf7f..8d172843af1d6 100644
    --- a/gradle/wrapper/gradle-wrapper.properties
    +++ b/gradle/wrapper/gradle-wrapper.properties
    @@ -1,6 +1,6 @@
     distributionBase=GRADLE_USER_HOME
     distributionPath=wrapper/dists
    -distributionUrl=https\://services.gradle.org/distributions/gradle-5.4-all.zip
    +distributionUrl=https\://services.gradle.org/distributions/gradle-5.3-all.zip
     zipStoreBase=GRADLE_USER_HOME
     zipStorePath=wrapper/dists
    -distributionSha256Sum=f177768e7a032727e4338c8fd047f8f263e5bd283f67a7766c1ba4182c8455a6
    +distributionSha256Sum=f4d820c2a9685710eba5b92f10e0e4fb20e0d6c0dd1f46971e658160f25e7147
    
    From 419a159ffbf8e79f78ea5c9dff7ed6ca86a11c81 Mon Sep 17 00:00:00 2001
    From: Jim Ferenczi 
    Date: Thu, 18 Apr 2019 22:48:22 +0200
    Subject: [PATCH 107/260] Fix unmapped field handling in the composite
     aggregation (#41280)
    
    The `composite` aggregation maps unknown fields as numerics, this means that
    any `after` value that is set on a query with an unmapped field on some indices
    will fail if the provided value is not numeric. This commit changes the default
    value source to use keyword instead in order to be able to parse any type of after
    values.
    ---
     .../test/search.aggregation/230_composite.yml | 105 +++++++++++++++++-
     .../composite/TermsValuesSourceBuilder.java   |   4 +-
     2 files changed, 107 insertions(+), 2 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 4a01da6e6c213..67ab217006ab4 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
    @@ -18,6 +18,22 @@ setup:
                           nested_long:
                             type: long
     
    +  - do:
    +        indices.create:
    +          index: other
    +          body:
    +            mappings:
    +              properties:
    +                date:
    +                  type: date
    +                long:
    +                  type: long
    +                nested:
    +                  type: nested
    +                  properties:
    +                    nested_long:
    +                      type: long
    +
       - do:
           index:
             index: test
    @@ -54,9 +70,15 @@ setup:
             id:    6
             body:  { "date": "2017-10-21T07:00:00" }
     
    +  - do:
    +      index:
    +        index: other
    +        id:    0
    +        body:  { "date": "2017-10-20T03:08:45" }
    +
       - do:
           indices.refresh:
    -        index: [test]
    +        index: [test, other]
     
     ---
     "Simple Composite aggregation":
    @@ -380,3 +402,84 @@ setup:
       - match: { aggregations.1.2.buckets.0.doc_count:  2 }
       - match: { aggregations.1.2.buckets.1.key.nested: 1000 }
       - match: { aggregations.1.2.buckets.1.doc_count:  1 }
    +
    +---
    +"Composite aggregation with unmapped field":
    +  - skip:
    +      version: " - 7.99.99"
    +      reason:  starting in 8.0 the composite aggregation handles unmapped fields as keywords
    +
    +  - do:
    +      search:
    +        rest_total_hits_as_int: true
    +        index: [test, other]
    +        body:
    +          aggregations:
    +            test:
    +              composite:
    +                sources: [
    +                {
    +                  "long": {
    +                    "terms": {
    +                      "field": "long"
    +                    }
    +                  }
    +                },
    +                {
    +                  "kw": {
    +                    "terms": {
    +                      "field": "keyword"
    +                    }
    +                  }
    +                }
    +                ]
    +
    +  - match: {hits.total: 7}
    +  - length: { aggregations.test.buckets: 5 }
    +  - match: { aggregations.test.buckets.0.key.long: 0}
    +  - match: { aggregations.test.buckets.0.key.kw: "bar" }
    +  - match: { aggregations.test.buckets.0.doc_count: 2 }
    +  - match: { aggregations.test.buckets.1.key.long: 10 }
    +  - match: { aggregations.test.buckets.1.key.kw: "foo"}
    +  - match: { aggregations.test.buckets.1.doc_count: 1 }
    +  - match: { aggregations.test.buckets.2.key.long: 20 }
    +  - match: { aggregations.test.buckets.2.key.kw: "foo" }
    +  - match: { aggregations.test.buckets.2.doc_count: 1 }
    +  - match: { aggregations.test.buckets.3.key.long: 100}
    +  - match: { aggregations.test.buckets.3.key.kw: "bar" }
    +  - match: { aggregations.test.buckets.3.doc_count: 1 }
    +  - match: { aggregations.test.buckets.4.key.long: 1000 }
    +  - match: { aggregations.test.buckets.4.key.kw: "bar" }
    +  - match: { aggregations.test.buckets.4.doc_count: 1 }
    +
    +  - do:
    +      search:
    +        rest_total_hits_as_int: true
    +        index: [test, other]
    +        body:
    +          aggregations:
    +            test:
    +              composite:
    +                after: { "long": 100, "kw": "bar" }
    +                sources: [
    +                {
    +                  "long": {
    +                    "terms": {
    +                      "field": "long"
    +                    }
    +                  }
    +                },
    +                {
    +                  "kw": {
    +                    "terms": {
    +                      "field": "keyword"
    +                    }
    +                  }
    +                }
    +                ]
    +
    +  - match: {hits.total: 7}
    +  - length: { aggregations.test.buckets: 1 }
    +  - match: { aggregations.test.buckets.0.key.long: 1000 }
    +  - match: { aggregations.test.buckets.0.key.kw: "bar" }
    +  - match: { aggregations.test.buckets.0.doc_count: 1 }
    diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java
    index 75bdd02855e3d..3ef871f59b93c 100644
    --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java
    +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java
    @@ -83,7 +83,9 @@ public String type() {
         protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig config) throws IOException {
             ValuesSource vs = config.toValuesSource(context.getQueryShardContext());
             if (vs == null) {
    -            vs = ValuesSource.Numeric.EMPTY;
    +            // The field is unmapped so we use a value source that can parse any type of values.
    +            // This is needed because the after values are parsed even when there are no values to process.
    +            vs = ValuesSource.Bytes.WithOrdinals.EMPTY;
             }
             final MappedFieldType fieldType = config.fieldContext() != null ? config.fieldContext().fieldType() : null;
             final DocValueFormat format;
    
    From f4d5a34a4bd55c2817c82d6e89da0e5701eeadb8 Mon Sep 17 00:00:00 2001
    From: Lisa Cawley 
    Date: Thu, 18 Apr 2019 17:30:08 -0700
    Subject: [PATCH 108/260] [DOCS] Fixes deprecation notice in pagerduty action
     (#41362)
    
    ---
     x-pack/docs/en/watcher/actions/pagerduty.asciidoc | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/x-pack/docs/en/watcher/actions/pagerduty.asciidoc b/x-pack/docs/en/watcher/actions/pagerduty.asciidoc
    index 87a7b06b2cfa6..1cd9132a57b14 100644
    --- a/x-pack/docs/en/watcher/actions/pagerduty.asciidoc
    +++ b/x-pack/docs/en/watcher/actions/pagerduty.asciidoc
    @@ -155,7 +155,7 @@ must specify an account name and integration key, (see {ref}/secure-settings.htm
     bin/elasticsearch-keystore add xpack.notification.pagerduty.account.my_pagerduty_account.secure_service_api_key
     --------------------------------------------------
     
    -deprecated[Storing the service api key in the YAML file or via cluster update settings is still supported, but the keystore setting should be used]
    +deprecated[7.0.0, "Storing the service api key in the YAML file or via cluster update settings is still supported, but the keystore setting should be used."]
     
     You can also specify defaults for the <>:
    
    From 2d29807b9cdd2bb3e78dbd2f8b1d33fa9d35081e Mon Sep 17 00:00:00 2001
    From: Ryan Ernst 
    Date: Thu, 18 Apr 2019 18:19:43 -0700
    Subject: [PATCH 109/260] Remove several uses of 6.x version constants (#41162)
    
    This commit removes many conditionals in serialization, as well as tests
    which no longer need to be compatible with 6.x now that master is 8.0.
    Additionally, many tests are changed to not depend on a fixed version,
    but instead use new helper methods for getting the range of index
    compatible versions with the current version, which will make the tests
    not need changes in the future when version constants are removed.
    ---
     .../percolator/PercolateQueryBuilder.java     | 94 ++++++-------------
     .../percolator/PercolatorFieldMapper.java     | 24 ++---
     .../percolator/QueryBuilderStoreTests.java    |  2 +-
     .../action/index/IndexRequest.java            |  7 +-
     .../index/engine/TranslogLeafReader.java      | 11 +--
     .../recovery/RecoverySourceHandler.java       |  5 +-
     .../indices/recovery/RecoveryTarget.java      |  3 -
     .../org/elasticsearch/plugins/PluginInfo.java |  4 -
     .../ExceptionSerializationTests.java          |  2 +-
     .../java/org/elasticsearch/VersionTests.java  |  5 +-
     .../ClusterSearchShardsRequestTests.java      |  2 +-
     .../ClusterSearchShardsResponseTests.java     |  2 +-
     .../common/geo/GeoJsonShapeParserTests.java   |  2 +-
     .../mapper/ExternalFieldMapperTests.java      |  3 +-
     .../index/mapper/TypeFieldTypeTests.java      |  2 +-
     .../indices/IndicesServiceTests.java          |  2 +-
     .../indices/analysis/AnalysisModuleTests.java |  7 +-
     .../plugins/PluginsServiceTests.java          |  2 +-
     .../aggregations/bucket/GeoDistanceIT.java    |  2 +-
     .../aggregations/bucket/GeoHashGridIT.java    |  3 +-
     .../functionscore/DecayFunctionScoreIT.java   |  2 +-
     .../search/geo/GeoBoundingBoxIT.java          |  9 +-
     .../search/geo/GeoDistanceIT.java             |  3 +-
     .../elasticsearch/search/geo/GeoFilterIT.java |  3 +-
     .../search/geo/GeoPolygonIT.java              |  3 +-
     .../search/sort/GeoDistanceIT.java            | 12 +--
     .../search/sort/GeoDistanceSortBuilderIT.java |  6 +-
     .../test/AbstractBuilderTestCase.java         |  3 +-
     .../org/elasticsearch/test/VersionUtils.java  |  7 ++
     .../xpack/core/ml/job/results/ModelPlot.java  | 30 +-----
     .../monitoring/action/MonitoringBulkDoc.java  | 32 +------
     .../monitoring/exporter/MonitoringDoc.java    | 20 +---
     .../security/SecurityFeatureSetUsage.java     |  8 --
     .../ClusterStatsMonitoringDocTests.java       |  2 +-
     .../IndexRecoveryMonitoringDocTests.java      |  2 +-
     .../ml/JobStatsMonitoringDocTests.java        |  2 +-
     .../node/NodeStatsMonitoringDocTests.java     |  2 +-
     37 files changed, 91 insertions(+), 239 deletions(-)
    
    diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java
    index 44200823b6d66..c883bb5893c9f 100644
    --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java
    +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java
    @@ -57,7 +57,6 @@
     import org.elasticsearch.common.logging.DeprecationLogger;
     import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
     import org.elasticsearch.common.xcontent.NamedXContentRegistry;
    -import org.elasticsearch.common.xcontent.XContent;
     import org.elasticsearch.common.xcontent.XContentBuilder;
     import org.elasticsearch.common.xcontent.XContentFactory;
     import org.elasticsearch.common.xcontent.XContentHelper;
    @@ -90,8 +89,6 @@
     import java.util.Objects;
     import java.util.function.Supplier;
     
    -import static org.elasticsearch.percolator.PercolatorFieldMapper.parseQuery;
    -
     public class PercolateQueryBuilder extends AbstractQueryBuilder {
         public static final String NAME = "percolate";
     
    @@ -247,14 +244,8 @@ public PercolateQueryBuilder(String field, String documentType, String indexedDo
         PercolateQueryBuilder(StreamInput in) throws IOException {
             super(in);
             field = in.readString();
    -        if (in.getVersion().onOrAfter(Version.V_6_1_0)) {
    -            name = in.readOptionalString();
    -        }
    -        if (in.getVersion().before(Version.V_6_0_0_beta1)) {
    -            documentType = in.readString();
    -        } else {
    -            documentType = in.readOptionalString();
    -        }
    +        name = in.readOptionalString();
    +        documentType = in.readOptionalString();
             indexedDocumentIndex = in.readOptionalString();
             indexedDocumentType = in.readOptionalString();
             indexedDocumentId = in.readOptionalString();
    @@ -265,12 +256,7 @@ public PercolateQueryBuilder(String field, String documentType, String indexedDo
             } else {
                 indexedDocumentVersion = null;
             }
    -        if (in.getVersion().onOrAfter(Version.V_6_1_0)) {
    -            documents = in.readList(StreamInput::readBytesReference);
    -        } else {
    -            BytesReference document = in.readOptionalBytesReference();
    -            documents = document != null ? Collections.singletonList(document) : Collections.emptyList();
    -        }
    +        documents = in.readList(StreamInput::readBytesReference);
             if (documents.isEmpty() == false) {
                 documentXContentType = in.readEnum(XContentType.class);
             } else {
    @@ -294,14 +280,8 @@ protected void doWriteTo(StreamOutput out) throws IOException {
                 throw new IllegalStateException("supplier must be null, can't serialize suppliers, missing a rewriteAndFetch?");
             }
             out.writeString(field);
    -        if (out.getVersion().onOrAfter(Version.V_6_1_0)) {
    -            out.writeOptionalString(name);
    -        }
    -        if (out.getVersion().before(Version.V_6_0_0_beta1)) {
    -            out.writeString(documentType);
    -        } else {
    -            out.writeOptionalString(documentType);
    -        }
    +        out.writeOptionalString(name);
    +        out.writeOptionalString(documentType);
             out.writeOptionalString(indexedDocumentIndex);
             out.writeOptionalString(indexedDocumentType);
             out.writeOptionalString(indexedDocumentId);
    @@ -685,50 +665,30 @@ static PercolateQuery.QueryStore createStore(MappedFieldType queryBuilderFieldTy
                 if (binaryDocValues == null) {
                     return docId -> null;
                 }
    -            if (indexVersion.onOrAfter(Version.V_6_0_0_beta2)) {
    -                return docId -> {
    -                    if (binaryDocValues.advanceExact(docId)) {
    -                        BytesRef qbSource = binaryDocValues.binaryValue();
    -                        try (InputStream in = new ByteArrayInputStream(qbSource.bytes, qbSource.offset, qbSource.length)) {
    -                            try (StreamInput input = new NamedWriteableAwareStreamInput(
    -                                    new InputStreamStreamInput(in, qbSource.length), registry)) {
    -                                input.setVersion(indexVersion);
    -                                // Query builder's content is stored via BinaryFieldMapper, which has a custom encoding
    -                                // to encode multiple binary values into a single binary doc values field.
    -                                // This is the reason we need to first need to read the number of values and
    -                                // then the length of the field value in bytes.
    -                                int numValues = input.readVInt();
    -                                assert numValues == 1;
    -                                int valueLength = input.readVInt();
    -                                assert valueLength > 0;
    -                                QueryBuilder queryBuilder = input.readNamedWriteable(QueryBuilder.class);
    -                                assert in.read() == -1;
    -                                return PercolatorFieldMapper.toQuery(context, mapUnmappedFieldsAsString, queryBuilder);
    -                            }
    +            return docId -> {
    +                if (binaryDocValues.advanceExact(docId)) {
    +                    BytesRef qbSource = binaryDocValues.binaryValue();
    +                    try (InputStream in = new ByteArrayInputStream(qbSource.bytes, qbSource.offset, qbSource.length)) {
    +                        try (StreamInput input = new NamedWriteableAwareStreamInput(
    +                                new InputStreamStreamInput(in, qbSource.length), registry)) {
    +                            input.setVersion(indexVersion);
    +                            // Query builder's content is stored via BinaryFieldMapper, which has a custom encoding
    +                            // to encode multiple binary values into a single binary doc values field.
    +                            // This is the reason we need to first need to read the number of values and
    +                            // then the length of the field value in bytes.
    +                            int numValues = input.readVInt();
    +                            assert numValues == 1;
    +                            int valueLength = input.readVInt();
    +                            assert valueLength > 0;
    +                            QueryBuilder queryBuilder = input.readNamedWriteable(QueryBuilder.class);
    +                            assert in.read() == -1;
    +                            return PercolatorFieldMapper.toQuery(context, mapUnmappedFieldsAsString, queryBuilder);
                             }
    -                    } else {
    -                        return null;
                         }
    -                };
    -            } else {
    -                return docId -> {
    -                    if (binaryDocValues.advanceExact(docId)) {
    -                        BytesRef qbSource = binaryDocValues.binaryValue();
    -                        if (qbSource.length > 0) {
    -                            XContent xContent = PercolatorFieldMapper.QUERY_BUILDER_CONTENT_TYPE.xContent();
    -                            try (XContentParser sourceParser = xContent
    -                                    .createParser(context.getXContentRegistry(), LoggingDeprecationHandler.INSTANCE,
    -                                        qbSource.bytes, qbSource.offset, qbSource.length)) {
    -                                return parseQuery(context, mapUnmappedFieldsAsString, sourceParser);
    -                            }
    -                        } else {
    -                            return null;
    -                        }
    -                    } else {
    -                        return null;
    -                    }
    -                };
    -            }
    +                } else {
    +                    return null;
    +                }
    +            };
             };
         }
     
    diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java
    index 6ac073ef90a02..9ad660b4e548c 100644
    --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java
    +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java
    @@ -53,8 +53,6 @@
     import org.elasticsearch.common.lucene.search.Queries;
     import org.elasticsearch.common.settings.Setting;
     import org.elasticsearch.common.settings.Settings;
    -import org.elasticsearch.common.xcontent.XContentBuilder;
    -import org.elasticsearch.common.xcontent.XContentFactory;
     import org.elasticsearch.common.xcontent.XContentLocation;
     import org.elasticsearch.common.xcontent.XContentParser;
     import org.elasticsearch.common.xcontent.XContentType;
    @@ -84,7 +82,6 @@
     import java.nio.ByteBuffer;
     import java.util.ArrayList;
     import java.util.Arrays;
    -import java.util.Collections;
     import java.util.HashMap;
     import java.util.Iterator;
     import java.util.List;
    @@ -423,21 +420,12 @@ public boolean convertNowRangeToMatchAll() {
     
         static void createQueryBuilderField(Version indexVersion, BinaryFieldMapper qbField,
                                             QueryBuilder queryBuilder, ParseContext context) throws IOException {
    -        if (indexVersion.onOrAfter(Version.V_6_0_0_beta2)) {
    -            try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
    -                try (OutputStreamStreamOutput out  = new OutputStreamStreamOutput(stream)) {
    -                    out.setVersion(indexVersion);
    -                    out.writeNamedWriteable(queryBuilder);
    -                    byte[] queryBuilderAsBytes = stream.toByteArray();
    -                    qbField.parse(context.createExternalValueContext(queryBuilderAsBytes));
    -                }
    -            }
    -        } else {
    -            try (XContentBuilder builder = XContentFactory.contentBuilder(QUERY_BUILDER_CONTENT_TYPE)) {
    -                queryBuilder.toXContent(builder, new MapParams(Collections.emptyMap()));
    -                builder.flush();
    -                byte[] queryBuilderAsBytes = BytesReference.toBytes(BytesReference.bytes(builder));
    -                context.doc().add(new Field(qbField.name(), queryBuilderAsBytes, qbField.fieldType()));
    +        try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
    +            try (OutputStreamStreamOutput out  = new OutputStreamStreamOutput(stream)) {
    +                out.setVersion(indexVersion);
    +                out.writeNamedWriteable(queryBuilder);
    +                byte[] queryBuilderAsBytes = stream.toByteArray();
    +                qbField.parse(context.createExternalValueContext(queryBuilderAsBytes));
                 }
             }
         }
    diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java
    index 1c7ae3681ac63..30cad734023d5 100644
    --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java
    +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/QueryBuilderStoreTests.java
    @@ -74,7 +74,7 @@ public void testStoringQueryBuilders() throws IOException {
                 BinaryFieldMapper fieldMapper = PercolatorFieldMapper.Builder.createQueryBuilderFieldBuilder(
                     new Mapper.BuilderContext(settings, new ContentPath(0)));
     
    -            Version version = Version.V_6_0_0_beta2;
    +            Version version = Version.CURRENT;
                 try (IndexWriter indexWriter = new IndexWriter(directory, config)) {
                     for (int i = 0; i < queryBuilders.length; i++) {
                         queryBuilders[i] = new TermQueryBuilder(randomAlphaOfLength(4), randomAlphaOfLength(8));
    diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java
    index 13bac8fa2a49c..1ad029f616d02 100644
    --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java
    +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java
    @@ -603,12 +603,7 @@ public void process(Version indexCreatedVersion, @Nullable MappingMetaData mappi
                 assert ifSeqNo == UNASSIGNED_SEQ_NO;
                 assert ifPrimaryTerm == UNASSIGNED_PRIMARY_TERM;
                 autoGeneratedTimestamp = Math.max(0, System.currentTimeMillis()); // extra paranoia
    -            String uid;
    -            if (indexCreatedVersion.onOrAfter(Version.V_6_0_0_beta1)) {
    -                uid = UUIDs.base64UUID();
    -            } else {
    -                uid = UUIDs.legacyBase64UUID();
    -            }
    +            String uid = UUIDs.base64UUID();
                 id(uid);
             }
         }
    diff --git a/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java b/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java
    index c7e11e85f7da5..c1f92966196a3 100644
    --- a/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java
    +++ b/server/src/main/java/org/elasticsearch/index/engine/TranslogLeafReader.java
    @@ -161,14 +161,9 @@ public void document(int docID, StoredFieldVisitor visitor) throws IOException {
                 visitor.stringField(FAKE_ROUTING_FIELD, operation.routing().getBytes(StandardCharsets.UTF_8));
             }
             if (visitor.needsField(FAKE_ID_FIELD) == StoredFieldVisitor.Status.YES) {
    -            final byte[] id;
    -            if (indexVersionCreated.onOrAfter(Version.V_6_0_0)) {
    -                BytesRef bytesRef = Uid.encodeId(operation.id());
    -                id = new byte[bytesRef.length];
    -                System.arraycopy(bytesRef.bytes, bytesRef.offset, id, 0, bytesRef.length);
    -            } else { // TODO this can go away in 7.0 after backport
    -                id = operation.id().getBytes(StandardCharsets.UTF_8);
    -            }
    +            BytesRef bytesRef = Uid.encodeId(operation.id());
    +            final byte[] id = new byte[bytesRef.length];
    +            System.arraycopy(bytesRef.bytes, bytesRef.offset, id, 0, bytesRef.length);
                 visitor.stringField(FAKE_ID_FIELD, id);
             }
         }
    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 cbd0d1dc2cd64..22f64a9e5e3ca 100644
    --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java
    +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java
    @@ -253,9 +253,8 @@ public void recoverToTarget(ActionListener listener) {
     
         private boolean isTargetSameHistory() {
             final String targetHistoryUUID = request.metadataSnapshot().getHistoryUUID();
    -        assert targetHistoryUUID != null || shard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1) :
    -            "incoming target history N/A but index was created after or on 6.0.0-rc1";
    -        return targetHistoryUUID != null && targetHistoryUUID.equals(shard.getHistoryUUID());
    +        assert targetHistoryUUID != null : "incoming target history missing";
    +        return targetHistoryUUID.equals(shard.getHistoryUUID());
         }
     
         static void runUnderPrimaryPermit(CancellableThreads.Interruptible runnable, String reason,
    diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java
    index fbc0dbde51098..e15750b2feb06 100644
    --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java
    +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java
    @@ -395,9 +395,6 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada
             store.incRef();
             try {
                 store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData);
    -            if (indexShard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1)) {
    -                store.ensureIndexHasHistoryUUID();
    -            }
                 assert globalCheckpoint >= Long.parseLong(sourceMetaData.getCommitUserData().get(SequenceNumbers.MAX_SEQ_NO))
                     || indexShard.indexSettings().getIndexVersionCreated().before(Version.V_7_1_0) :
                     "invalid global checkpoint[" + globalCheckpoint + "] source_meta_data [" + sourceMetaData.getCommitUserData() + "]";
    diff --git a/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java b/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java
    index 32add31ffd50e..dd939c4b2d524 100644
    --- a/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java
    +++ b/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java
    @@ -196,10 +196,6 @@ public static PluginInfo readFromProperties(final Path path) throws IOException
                 }
             }
     
    -        if (esVersion.before(Version.V_6_3_0) && esVersion.onOrAfter(Version.V_6_0_0_beta2)) {
    -            propsMap.remove("requires.keystore");
    -        }
    -
             if (propsMap.isEmpty() == false) {
                 throw new IllegalArgumentException("Unknown properties in plugin descriptor: " + propsMap.keySet());
             }
    diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
    index 8f62e5c8156c9..1fac56886de45 100644
    --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
    +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java
    @@ -878,7 +878,7 @@ public void testElasticsearchRemoteException() throws IOException {
         public void testShardLockObtainFailedException() throws IOException {
             ShardId shardId = new ShardId("foo", "_na_", 1);
             ShardLockObtainFailedException orig = new ShardLockObtainFailedException(shardId, "boom");
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             ShardLockObtainFailedException ex = serialize(orig, version);
             assertEquals(orig.getMessage(), ex.getMessage());
             assertEquals(orig.getShardId(), ex.getShardId());
    diff --git a/server/src/test/java/org/elasticsearch/VersionTests.java b/server/src/test/java/org/elasticsearch/VersionTests.java
    index 99077cbea70db..7a4b859bb2bc2 100644
    --- a/server/src/test/java/org/elasticsearch/VersionTests.java
    +++ b/server/src/test/java/org/elasticsearch/VersionTests.java
    @@ -101,8 +101,7 @@ public void testMax() {
         }
     
         public void testMinimumIndexCompatibilityVersion() {
    -        assertEquals(Version.fromId(5000099), Version.V_6_0_0_beta1.minimumIndexCompatibilityVersion());
    -        assertEquals(Version.fromId(2000099), Version.fromId(5000099).minimumIndexCompatibilityVersion());
    +        assertEquals(Version.fromId(5000099), Version.fromId(6000099).minimumIndexCompatibilityVersion());
             assertEquals(Version.fromId(2000099),
                     Version.fromId(5010000).minimumIndexCompatibilityVersion());
             assertEquals(Version.fromId(2000099),
    @@ -161,7 +160,7 @@ public void testVersionNoPresentInSettings() {
     
         public void testIndexCreatedVersion() {
             // an actual index has a IndexMetaData.SETTING_INDEX_UUID
    -        final Version version = Version.V_6_0_0_beta1;
    +        final Version version = VersionUtils.randomVersion(random());
             assertEquals(version, Version.indexCreated(
                 Settings.builder()
                     .put(IndexMetaData.SETTING_INDEX_UUID, "foo")
    diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsRequestTests.java
    index 5f5fe54321bbb..600920267fcb3 100644
    --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsRequestTests.java
    +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsRequestTests.java
    @@ -54,7 +54,7 @@ public void testSerialization() throws Exception {
                 request.routing(routings);
             }
     
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             try (BytesStreamOutput out = new BytesStreamOutput()) {
                 out.setVersion(version);
                 request.writeTo(out);
    diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java
    index fbfe0e497017f..534cea2ae4604 100644
    --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java
    +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java
    @@ -77,7 +77,7 @@ public void testSerialization() throws Exception {
             List entries = new ArrayList<>();
             entries.addAll(searchModule.getNamedWriteables());
             NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(entries);
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             try(BytesStreamOutput out = new BytesStreamOutput()) {
                 out.setVersion(version);
                 clusterSearchShardsResponse.writeTo(out);
    diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java
    index 74024ddcada5e..5806875ce109b 100644
    --- a/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java
    +++ b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java
    @@ -291,7 +291,7 @@ public void testParse3DPolygon() throws IOException {
             shellCoordinates.add(new Coordinate(100, 0, 10));
             Coordinate[] coordinates = shellCoordinates.toArray(new Coordinate[shellCoordinates.size()]);
     
    -        Version randomVersion = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version randomVersion = VersionUtils.randomIndexCompatibleVersion(random());
             Settings indexSettings = Settings.builder()
                 .put(IndexMetaData.SETTING_VERSION_CREATED, randomVersion)
                 .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
    diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java
    index 89baa7d2c8a69..d80776007aba8 100644
    --- a/server/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java
    @@ -63,8 +63,7 @@ protected boolean forbidPrivateIndexSettings() {
         }
     
         public void testExternalValues() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             IndexService indexService = createIndex("test", settings);
             MapperRegistry mapperRegistry = new MapperRegistry(
    diff --git a/server/src/test/java/org/elasticsearch/index/mapper/TypeFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/TypeFieldTypeTests.java
    index dc6f14cb0148a..4e6f504e99263 100644
    --- a/server/src/test/java/org/elasticsearch/index/mapper/TypeFieldTypeTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/mapper/TypeFieldTypeTests.java
    @@ -39,7 +39,7 @@ protected MappedFieldType createDefaultFieldType() {
     
         public void testTermsQuery() throws Exception {
             QueryShardContext context = Mockito.mock(QueryShardContext.class);
    -        Version indexVersionCreated = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version indexVersionCreated = VersionUtils.randomIndexCompatibleVersion(random());
             Settings indexSettings = Settings.builder()
                     .put(IndexMetaData.SETTING_VERSION_CREATED, indexVersionCreated)
                     .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
    diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java
    index 769d7a6c6866a..1ccf858ed1590 100644
    --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java
    +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java
    @@ -524,7 +524,7 @@ public void testStatsByShardDoesNotDieFromExpectedExceptions() {
     
         public void testIsMetaDataField() {
             IndicesService indicesService = getIndicesService();
    -        final Version randVersion = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        final Version randVersion = VersionUtils.randomIndexCompatibleVersion(random());
             assertFalse(indicesService.isMetaDataField(randVersion, randomAlphaOfLengthBetween(10, 15)));
             for (String builtIn : IndicesModule.getBuiltInMetaDataFields()) {
                 assertTrue(indicesService.isMetaDataField(randVersion, builtIn));
    diff --git a/server/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java b/server/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java
    index f991351d99ad2..a56834a4caf90 100644
    --- a/server/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java
    +++ b/server/src/test/java/org/elasticsearch/indices/analysis/AnalysisModuleTests.java
    @@ -133,10 +133,11 @@ public void testSimpleConfigurationYaml() throws IOException {
     
         public void testVersionedAnalyzers() throws Exception {
             String yaml = "/org/elasticsearch/index/analysis/test1.yml";
    +        Version version = VersionUtils.randomVersion(random());
             Settings settings2 = Settings.builder()
                     .loadFromStream(yaml, getClass().getResourceAsStream(yaml), false)
                     .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
    -                .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_6_0_0)
    +                .put(IndexMetaData.SETTING_VERSION_CREATED, version)
                     .build();
             AnalysisRegistry newRegistry = getNewRegistry(settings2);
             IndexAnalyzers indexAnalyzers = getIndexAnalyzers(newRegistry, settings2);
    @@ -149,9 +150,9 @@ public void testVersionedAnalyzers() throws Exception {
     
             // analysis service has the expected version
             assertThat(indexAnalyzers.get("standard").analyzer(), is(instanceOf(StandardAnalyzer.class)));
    -        assertEquals(Version.V_6_0_0.luceneVersion,
    +        assertEquals(version.luceneVersion,
                     indexAnalyzers.get("standard").analyzer().getVersion());
    -        assertEquals(Version.V_6_0_0.luceneVersion,
    +        assertEquals(version.luceneVersion,
                     indexAnalyzers.get("stop").analyzer().getVersion());
     
             assertThat(indexAnalyzers.get("custom7").analyzer(), is(instanceOf(StandardAnalyzer.class)));
    diff --git a/server/src/test/java/org/elasticsearch/plugins/PluginsServiceTests.java b/server/src/test/java/org/elasticsearch/plugins/PluginsServiceTests.java
    index e447107aac8b5..562bb3b4d9ecb 100644
    --- a/server/src/test/java/org/elasticsearch/plugins/PluginsServiceTests.java
    +++ b/server/src/test/java/org/elasticsearch/plugins/PluginsServiceTests.java
    @@ -601,7 +601,7 @@ public void testNonExtensibleDep() throws Exception {
         }
     
         public void testIncompatibleElasticsearchVersion() throws Exception {
    -        PluginInfo info = new PluginInfo("my_plugin", "desc", "1.0", Version.V_6_0_0,
    +        PluginInfo info = new PluginInfo("my_plugin", "desc", "1.0", Version.fromId(6000099),
                 "1.8", "FakePlugin", Collections.emptyList(), false);
             IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> PluginsService.verifyCompatibility(info));
             assertThat(e.getMessage(), containsString("was built for Elasticsearch version 6.0.0"));
    diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoDistanceIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoDistanceIT.java
    index 02313694210e6..55ffd1719ac88 100644
    --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoDistanceIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoDistanceIT.java
    @@ -64,7 +64,7 @@ protected boolean forbidPrivateIndexSettings() {
             return false;
         }
     
    -    private Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +    private Version version = VersionUtils.randomIndexCompatibleVersion(random());
     
         private IndexRequestBuilder indexCity(String idx, String name, String... latLons) throws Exception {
             XContentBuilder source = jsonBuilder().startObject().field("city", name);
    diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java
    index a77eb02c1a034..4ea9db378e1ef 100644
    --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java
    @@ -61,8 +61,7 @@ protected boolean forbidPrivateIndexSettings() {
             return false;
         }
     
    -    private Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -            Version.CURRENT);
    +    private Version version = VersionUtils.randomIndexCompatibleVersion(random());
     
         static ObjectIntMap expectedDocCountsForGeoHash = null;
         static ObjectIntMap multiValuedExpectedDocCountsForGeoHash = null;
    diff --git a/server/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java b/server/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java
    index 794e3baa0d565..0b8deef0831c6 100644
    --- a/server/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java
    @@ -610,7 +610,7 @@ public void testDateWithoutOrigin() throws Exception {
         }
     
         public void testManyDocsLin() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = jsonBuilder().startObject().startObject("type").startObject("properties")
                     .startObject("test").field("type", "text").endObject().startObject("date").field("type", "date")
    diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java
    index 4306ae680aadb..6ffdf7394af6b 100644
    --- a/server/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java
    @@ -47,8 +47,7 @@ protected boolean forbidPrivateIndexSettings() {
         }
     
         public void testSimpleBoundingBoxTest() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("type1")
                     .startObject("properties").startObject("location").field("type", "geo_point");
    @@ -119,8 +118,7 @@ public void testSimpleBoundingBoxTest() throws Exception {
         }
     
         public void testLimit2BoundingBox() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("type1")
                     .startObject("properties").startObject("location").field("type", "geo_point");
    @@ -174,8 +172,7 @@ public void testLimit2BoundingBox() throws Exception {
         }
     
         public void testCompleteLonRange() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("type1")
                     .startObject("properties").startObject("location").field("type", "geo_point");
    diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java
    index d78e24a05cda9..e47ca503e020e 100644
    --- a/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java
    @@ -104,8 +104,7 @@ protected boolean forbidPrivateIndexSettings() {
     
         @Before
         public void setupTestIndex() throws IOException {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("type1")
                     .startObject("properties").startObject("location").field("type", "geo_point");
    diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java
    index b26a7ff510a3d..b21d6724693fc 100644
    --- a/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java
    @@ -367,8 +367,7 @@ public void testShapeRelations() throws Exception {
     
         public void testBulk() throws Exception {
             byte[] bulkAction = unZipData("/org/elasticsearch/search/geo/gzippedmap.gz");
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()
                     .startObject()
    diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java
    index 80f7fa32f01ec..f10c7da15c122 100644
    --- a/server/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoPolygonIT.java
    @@ -49,8 +49,7 @@ protected boolean forbidPrivateIndexSettings() {
     
         @Override
         protected void setupSuiteScopeCluster() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
     
             assertAcked(prepareCreate("test").setSettings(settings).addMapping("type1", "location",
    diff --git a/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java b/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java
    index f9e9dab70b239..53235997283a4 100644
    --- a/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java
    @@ -55,8 +55,7 @@ protected boolean forbidPrivateIndexSettings() {
         }
     
         public void testDistanceSortingMVFields() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
                     .startObject("locations").field("type", "geo_point");
    @@ -182,8 +181,7 @@ public void testDistanceSortingMVFields() throws Exception {
         // Regression bug:
         // https://github.com/elastic/elasticsearch/issues/2851
         public void testDistanceSortingWithMissingGeoPoint() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
                     .startObject("locations").field("type", "geo_point");
    @@ -225,8 +223,7 @@ public void testDistanceSortingWithMissingGeoPoint() throws Exception {
         }
     
         public void testDistanceSortingNestedFields() throws Exception {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().startObject().startObject("company").startObject("properties")
                     .startObject("name").field("type", "text").endObject().startObject("branches").field("type", "nested")
    @@ -374,8 +371,7 @@ public void testDistanceSortingNestedFields() throws Exception {
          * Issue 3073
          */
         public void testGeoDistanceFilter() throws IOException {
    -        Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0,
    -                Version.CURRENT);
    +        Version version = VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             double lat = 40.720611;
             double lon = -73.998776;
    diff --git a/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceSortBuilderIT.java b/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceSortBuilderIT.java
    index 97d9361fc46fe..114e9de570066 100644
    --- a/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceSortBuilderIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceSortBuilderIT.java
    @@ -69,7 +69,7 @@ public void testManyToManyGeoPoints() throws ExecutionException, InterruptedExce
              * 1   2   3   4   5   6   7
              */
             Version version = randomBoolean() ? Version.CURRENT
    -                : VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +                : VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             assertAcked(prepareCreate("index").setSettings(settings).addMapping("type", LOCATION_FIELD, "type=geo_point"));
             XContentBuilder d1Builder = jsonBuilder();
    @@ -143,7 +143,7 @@ public void testSingeToManyAvgMedian() throws ExecutionException, InterruptedExc
              * d2 = (0, 1), (0, 5), (0, 6); so avg. distance is 4, median distance is 5
              */
             Version version = randomBoolean() ? Version.CURRENT
    -                : VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +                : VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             assertAcked(prepareCreate("index").setSettings(settings).addMapping("type", LOCATION_FIELD, "type=geo_point"));
             XContentBuilder d1Builder = jsonBuilder();
    @@ -208,7 +208,7 @@ public void testManyToManyGeoPointsWithDifferentFormats() throws ExecutionExcept
              * 1   2   3   4   5   6
              */
             Version version = randomBoolean() ? Version.CURRENT
    -                : VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +                : VersionUtils.randomIndexCompatibleVersion(random());
             Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
             assertAcked(prepareCreate("index").setSettings(settings).addMapping("type", LOCATION_FIELD, "type=geo_point"));
             XContentBuilder d1Builder = jsonBuilder();
    diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java
    index 0f1a3fb3dcb7c..d607d73336826 100644
    --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java
    +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java
    @@ -179,8 +179,7 @@ protected static String createUniqueRandomName() {
     
         protected Settings createTestIndexSettings() {
             // we have to prefer CURRENT since with the range of versions we support it's rather unlikely to get the current actually.
    -        Version indexVersionCreated = randomBoolean() ? Version.CURRENT
    -                : VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.CURRENT);
    +        Version indexVersionCreated = randomBoolean() ? Version.CURRENT : VersionUtils.randomIndexCompatibleVersion(random());
             return Settings.builder()
                 .put(IndexMetaData.SETTING_VERSION_CREATED, indexVersionCreated)
                 .build();
    diff --git a/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java b/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java
    index f775f9f5b0121..990ae8e1f09a2 100644
    --- a/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java
    +++ b/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java
    @@ -248,4 +248,11 @@ public static Version maxCompatibleVersion(Version version) {
             assert compatible.size() > 0;
             return compatible.get(compatible.size() - 1);
         }
    +
    +    /**
    +     * Returns a random version index compatible with the current version.
    +     */
    +    public static Version randomIndexCompatibleVersion(Random random) {
    +        return randomVersionBetween(random, Version.CURRENT.minimumIndexCompatibilityVersion(), Version.CURRENT);
    +    }
     }
    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ModelPlot.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ModelPlot.java
    index c17ed54c78826..60c8c30b94230 100644
    --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ModelPlot.java
    +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/results/ModelPlot.java
    @@ -5,7 +5,6 @@
      */
     package org.elasticsearch.xpack.core.ml.job.results;
     
    -import org.elasticsearch.Version;
     import org.elasticsearch.common.ParseField;
     import org.elasticsearch.common.io.stream.StreamInput;
     import org.elasticsearch.common.io.stream.StreamOutput;
    @@ -112,18 +111,9 @@ public ModelPlot(StreamInput in) throws IOException {
             modelLower = in.readDouble();
             modelUpper = in.readDouble();
             modelMedian = in.readDouble();
    -        if (in.getVersion().before(Version.V_6_0_0_rc1)) {
    -            actual = in.readDouble();
    -        } else {
    -            actual = in.readOptionalDouble();
    -        }
    +        actual = in.readOptionalDouble();
             bucketSpan = in.readLong();
    -        if (in.getVersion().onOrAfter(Version.V_6_1_0)) {
    -            detectorIndex = in.readInt();
    -        } else {
    -            // default to -1 as marker for no detector index
    -            detectorIndex = -1;
    -        }
    +        detectorIndex = in.readInt();
         }
     
         @Override
    @@ -140,21 +130,9 @@ public void writeTo(StreamOutput out) throws IOException {
             out.writeDouble(modelLower);
             out.writeDouble(modelUpper);
             out.writeDouble(modelMedian);
    -        if (out.getVersion().before(Version.V_6_0_0_rc1)) {
    -            if (actual == null) {
    -                // older versions cannot accommodate null, so we have no choice but to propagate the bug of
    -                // https://github.com/elastic/x-pack-elasticsearch/issues/2528
    -                out.writeDouble(0.0);
    -            } else {
    -                out.writeDouble(actual);
    -            }
    -        } else {
    -            out.writeOptionalDouble(actual);
    -        }
    +        out.writeOptionalDouble(actual);
             out.writeLong(bucketSpan);
    -        if (out.getVersion().onOrAfter(Version.V_6_1_0)) {
    -            out.writeInt(detectorIndex);
    -        }
    +        out.writeInt(detectorIndex);
         }
     
         @Override
    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/action/MonitoringBulkDoc.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/action/MonitoringBulkDoc.java
    index 9ddc2c6d9ec69..0de18220c1010 100644
    --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/action/MonitoringBulkDoc.java
    +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/action/MonitoringBulkDoc.java
    @@ -5,7 +5,6 @@
      */
     package org.elasticsearch.xpack.core.monitoring.action;
     
    -import org.elasticsearch.Version;
     import org.elasticsearch.common.Nullable;
     import org.elasticsearch.common.Strings;
     import org.elasticsearch.common.bytes.BytesArray;
    @@ -15,8 +14,6 @@
     import org.elasticsearch.common.io.stream.Writeable;
     import org.elasticsearch.common.xcontent.XContentType;
     import org.elasticsearch.xpack.core.monitoring.MonitoredSystem;
    -import org.elasticsearch.xpack.core.monitoring.exporter.MonitoringDoc;
    -import org.elasticsearch.xpack.core.monitoring.exporter.MonitoringTemplateUtils;
     
     import java.io.IOException;
     import java.util.Objects;
    @@ -54,52 +51,29 @@ public MonitoringBulkDoc(final MonitoredSystem system,
          */
         public static MonitoringBulkDoc readFrom(StreamInput in) throws IOException {
             final MonitoredSystem system = MonitoredSystem.fromSystem(in.readOptionalString());
    -
    -        if (in.getVersion().before(Version.V_6_0_0_rc1)) {
    -            in.readOptionalString(); // Monitoring version, removed in 6.0 rc1
    -            in.readOptionalString(); // Cluster UUID, removed in 6.0 rc1
    -        }
    -
             final long timestamp = in.readVLong();
     
    -        if (in.getVersion().before(Version.V_6_0_0_rc1)) {
    -            in.readOptionalWriteable(MonitoringDoc.Node::new);// Source node, removed in 6.0 rc1
    -            MonitoringIndex.readFrom(in);// Monitoring index, removed in 6.0 rc1
    -        }
    -
             final String type = in.readOptionalString();
             final String id = in.readOptionalString();
             final BytesReference source = in.readBytesReference();
             final XContentType xContentType = (source != BytesArray.EMPTY) ? in.readEnum(XContentType.class) : XContentType.JSON;
    +        long interval = in.readVLong();
     
    -        long interval = 0L;
    -        if (in.getVersion().onOrAfter(Version.V_6_0_0_rc1)) {
    -            interval = in.readVLong();
    -        }
             return new MonitoringBulkDoc(system, type, id, timestamp, interval, source, xContentType);
         }
     
         @Override
         public void writeTo(StreamOutput out) throws IOException {
             out.writeOptionalString(system.getSystem());
    -        if (out.getVersion().before(Version.V_6_0_0_rc1)) {
    -            out.writeOptionalString(MonitoringTemplateUtils.TEMPLATE_VERSION);
    -            out.writeOptionalString(null);
    -        }
             out.writeVLong(timestamp);
    -        if (out.getVersion().before(Version.V_6_0_0_rc1)) {
    -            out.writeOptionalWriteable(null);
    -            MonitoringIndex.IGNORED_DATA.writeTo(out);
    -        }
             out.writeOptionalString(type);
             out.writeOptionalString(id);
             out.writeBytesReference(source);
             if (source != BytesArray.EMPTY) {
                 out.writeEnum(xContentType);
             }
    -        if (out.getVersion().onOrAfter(Version.V_6_0_0_rc1)) {
    -            out.writeVLong(intervalMillis);
    -        }
    +        out.writeVLong(intervalMillis);
    +
         }
     
         public MonitoredSystem getSystem() {
    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringDoc.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringDoc.java
    index 21ceb6097ae3a..caf13c3df3e1b 100644
    --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringDoc.java
    +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringDoc.java
    @@ -5,7 +5,6 @@
      */
     package org.elasticsearch.xpack.core.monitoring.exporter;
     
    -import org.elasticsearch.Version;
     import org.elasticsearch.common.Nullable;
     import org.elasticsearch.common.io.stream.StreamInput;
     import org.elasticsearch.common.io.stream.StreamOutput;
    @@ -163,17 +162,7 @@ public Node(StreamInput in) throws IOException {
                 transportAddress = in.readOptionalString();
                 ip = in.readOptionalString();
                 name = in.readOptionalString();
    -            if (in.getVersion().onOrAfter(Version.V_6_0_0_rc1)) {
    -                timestamp = in.readVLong();
    -            } else {
    -                // Read the node attributes (removed in 6.0 rc1)
    -                int size = in.readVInt();
    -                for (int i = 0; i < size; i++) {
    -                    in.readString();
    -                    in.readString();
    -                }
    -                timestamp = 0L;
    -            }
    +            timestamp = in.readVLong();
             }
     
             @Override
    @@ -183,12 +172,7 @@ public void writeTo(StreamOutput out) throws IOException {
                 out.writeOptionalString(transportAddress);
                 out.writeOptionalString(ip);
                 out.writeOptionalString(name);
    -            if (out.getVersion().onOrAfter(Version.V_6_0_0_rc1)) {
    -                out.writeVLong(timestamp);
    -            } else {
    -                // Write an empty map of node attributes (removed in 6.0 rc1)
    -                out.writeVInt(0);
    -            }
    +            out.writeVLong(timestamp);
             }
     
             public String getUUID() {
    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/SecurityFeatureSetUsage.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/SecurityFeatureSetUsage.java
    index 618414426f085..50696345398e9 100644
    --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/SecurityFeatureSetUsage.java
    +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/SecurityFeatureSetUsage.java
    @@ -49,10 +49,6 @@ public SecurityFeatureSetUsage(StreamInput in) throws IOException {
             }
             auditUsage = in.readMap();
             ipFilterUsage = in.readMap();
    -        if (in.getVersion().before(Version.V_6_0_0_beta1)) {
    -            // system key has been removed but older send its usage, so read the map and ignore
    -            in.readMap();
    -        }
             anonymousUsage = in.readMap();
             roleMappingStoreUsage = in.readMap();
         }
    @@ -86,10 +82,6 @@ public void writeTo(StreamOutput out) throws IOException {
             }
             out.writeMap(auditUsage);
             out.writeMap(ipFilterUsage);
    -        if (out.getVersion().before(Version.V_6_0_0_beta1)) {
    -            // system key has been removed but older versions still expected it so send a empty map
    -            out.writeMap(Collections.emptyMap());
    -        }
             out.writeMap(anonymousUsage);
             out.writeMap(roleMappingStoreUsage);
         }
    diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java
    index a8fc173551d5a..e03f07740095a 100644
    --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java
    +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java
    @@ -202,7 +202,7 @@ public void testToXContent() throws IOException {
                                                                     transportAddress,
                                                                     singletonMap("attr", "value"),
                                                                     singleton(DiscoveryNode.Role.MASTER),
    -                                                                Version.V_6_0_0_beta1);
    +                                                                Version.CURRENT);
     
             final ClusterState clusterState = ClusterState.builder(clusterName)
                                                             .metaData(MetaData.builder()
    diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexRecoveryMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexRecoveryMonitoringDocTests.java
    index b36a8c8db0b7e..49fdd9ad244ac 100644
    --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexRecoveryMonitoringDocTests.java
    +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/indices/IndexRecoveryMonitoringDocTests.java
    @@ -79,7 +79,7 @@ public void testToXContent() throws IOException {
                                                                         new TransportAddress(TransportAddress.META_ADDRESS, 9300),
                                                                         singletonMap("attr", "value_0"),
                                                                         singleton(DiscoveryNode.Role.MASTER),
    -                                                                    Version.V_6_0_0_beta1);
    +                                                                    Version.CURRENT);
     
             final DiscoveryNode discoveryNodeOne = new DiscoveryNode("_node_1",
                                                                         "_node_id_1",
    diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsMonitoringDocTests.java
    index 15b0f324d1a47..2bc3ac48b99ec 100644
    --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsMonitoringDocTests.java
    +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsMonitoringDocTests.java
    @@ -86,7 +86,7 @@ public void testToXContent() throws IOException {
                                                                  new TransportAddress(TransportAddress.META_ADDRESS, 9300),
                                                                  singletonMap("attr", "value"),
                                                                  singleton(DiscoveryNode.Role.MASTER),
    -                                                             Version.V_6_0_0_beta1);
    +                                                             Version.CURRENT);
     
             final ModelSizeStats modelStats = new ModelSizeStats.Builder("_model")
                                                                 .setModelBytes(100L)
    diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java
    index 753930d2b6a51..4296bedfe0da3 100644
    --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java
    +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/node/NodeStatsMonitoringDocTests.java
    @@ -364,7 +364,7 @@ private static NodeStats mockNodeStats() {
                                                                     new TransportAddress(TransportAddress.META_ADDRESS, 1234),
                                                                     emptyMap(),
                                                                     emptySet(),
    -                                                                Version.V_6_0_0_beta1);
    +                                                                Version.CURRENT);
     
             return new NodeStats(discoveryNode, no, indices, os, process, jvm, threadPool, fs, null, null, null, null, null, null, null);
         }
    
    From 056194fe5729538c381a486a31756f40a8a354d8 Mon Sep 17 00:00:00 2001
    From: Alpar Torok 
    Date: Fri, 19 Apr 2019 08:55:10 +0300
    Subject: [PATCH 110/260] Improove the configuration time if the build (#41251)
    
    This will help with reproduction lines and running tests form IDEs and
    other operations that are quick and executed often enough for the
    configuration time to matter.
    
    Running Gradle with a FIPS JVM is not supproted, so if the runtime JVM
    is the same one, no need to spend time checking for fips support.
    
    Verification of the JAVA_HOME env vars is now async and
    parallel so it doesn't block configuration.
    ---
     .../elasticsearch/gradle/BuildPlugin.groovy   | 61 +++++++++++++------
     1 file changed, 41 insertions(+), 20 deletions(-)
    
    diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    index 3d8ba1f3b36a5..f32ed078e09b6 100644
    --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    @@ -61,6 +61,9 @@ import org.gradle.util.GradleVersion
     import java.nio.charset.StandardCharsets
     import java.time.ZoneOffset
     import java.time.ZonedDateTime
    +import java.util.concurrent.ExecutorService
    +import java.util.concurrent.Executors
    +import java.util.concurrent.Future
     import java.util.regex.Matcher
     
     /**
    @@ -153,8 +156,12 @@ class BuildPlugin implements Plugin {
                     runtimeJavaVersionEnum = JavaVersion.toVersion(findJavaSpecificationVersion(project, runtimeJavaHome))
                 }
     
    -            String inFipsJvmScript = 'print(java.security.Security.getProviders()[0].name.toLowerCase().contains("fips"));'
    -            boolean inFipsJvm = Boolean.parseBoolean(runJavaAsScript(project, runtimeJavaHome, inFipsJvmScript))
    +            boolean inFipsJvm = false
    +            if (new File(runtimeJavaHome).canonicalPath != gradleJavaHome.canonicalPath) {
    +                // We don't expect Gradle to be running in a FIPS JVM
    +                String inFipsJvmScript = 'print(java.security.Security.getProviders()[0].name.toLowerCase().contains("fips"));'
    +                inFipsJvm = Boolean.parseBoolean(runJavaAsScript(project, runtimeJavaHome, inFipsJvmScript))
    +            }
     
                 // Build debugging info
                 println '======================================='
    @@ -190,24 +197,38 @@ class BuildPlugin implements Plugin {
                     throw new GradleException(message)
                 }
     
    -            for (final Map.Entry javaVersionEntry : javaVersions.entrySet()) {
    -                final String javaHome = javaVersionEntry.getValue()
    -                if (javaHome == null) {
    -                    continue
    -                }
    -                JavaVersion javaVersionEnum = JavaVersion.toVersion(findJavaSpecificationVersion(project, javaHome))
    -                final JavaVersion expectedJavaVersionEnum
    -                final int version = javaVersionEntry.getKey()
    -                if (version < 9) {
    -                    expectedJavaVersionEnum = JavaVersion.toVersion("1." + version)
    -                } else {
    -                    expectedJavaVersionEnum = JavaVersion.toVersion(Integer.toString(version))
    -                }
    -                if (javaVersionEnum != expectedJavaVersionEnum) {
    -                    final String message =
    -                            "the environment variable JAVA" + version + "_HOME must be set to a JDK installation directory for Java" +
    -                                    " ${expectedJavaVersionEnum} but is [${javaHome}] corresponding to [${javaVersionEnum}]"
    -                    throw new GradleException(message)
    +            ExecutorService exec = Executors.newFixedThreadPool(javaVersions.size())
    +            Set> results = new HashSet<>()
    +
    +            javaVersions.entrySet().stream()
    +                    .filter { it.getValue() != null }
    +                    .forEach { javaVersionEntry ->
    +                        results.add(exec.submit {
    +                            final String javaHome = javaVersionEntry.getValue()
    +                            final int version = javaVersionEntry.getKey()
    +                            if (project.file(javaHome).exists() == false) {
    +                                throw new GradleException("Invalid JAVA${version}_HOME=${javaHome} location does not exist")
    +                            }
    +
    +                            JavaVersion javaVersionEnum = JavaVersion.toVersion(findJavaSpecificationVersion(project, javaHome))
    +                            final JavaVersion expectedJavaVersionEnum = version < 9 ?
    +                                    JavaVersion.toVersion("1." + version) :
    +                                    JavaVersion.toVersion(Integer.toString(version))
    +
    +                            if (javaVersionEnum != expectedJavaVersionEnum) {
    +                                final String message =
    +                                        "the environment variable JAVA" + version + "_HOME must be set to a JDK installation directory for Java" +
    +                                                " ${expectedJavaVersionEnum} but is [${javaHome}] corresponding to [${javaVersionEnum}]"
    +                                throw new GradleException(message)
    +                            }
    +                        })
    +            }
    +
    +            project.gradle.taskGraph.whenReady {
    +                try {
    +                    results.forEach { it.get() }
    +                } finally {
    +                    exec.shutdown();
                     }
                 }
     
    
    From e76914cd456f9220d47262db25fbc20a8af8e8fc Mon Sep 17 00:00:00 2001
    From: Alpar Torok 
    Date: Fri, 19 Apr 2019 09:19:28 +0300
    Subject: [PATCH 111/260] Clean up build tool dependencies (#41336)
    
    We are no longer using these dependencies.
    
    Relates to #41061 since the class that seems to be leaking is both part
    of Gradle and the logging jar.
    ---
     buildSrc/build.gradle | 13 +------------
     1 file changed, 1 insertion(+), 12 deletions(-)
    
    diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle
    index 70f50f2c541ee..32845566a16f7 100644
    --- a/buildSrc/build.gradle
    +++ b/buildSrc/build.gradle
    @@ -113,8 +113,6 @@ repositories {
     
     dependencies {
       compile localGroovy()
    -  compile "com.carrotsearch.randomizedtesting:junit4-ant:${props.getProperty('randomizedrunner')}"
    -  compile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}"
       
       compile 'com.netflix.nebula:gradle-extra-configurations-plugin:3.0.3'
       compile 'com.netflix.nebula:nebula-publishing-plugin:4.4.4'
    @@ -127,16 +125,7 @@ dependencies {
       compile 'de.thetaphi:forbiddenapis:2.6'
       compile 'com.avast.gradle:gradle-docker-compose-plugin:0.8.12'
       testCompile "junit:junit:${props.getProperty('junit')}"
    -}
    -
    -
    -// Gradle 2.14+ removed ProgressLogger(-Factory) classes from the public APIs
    -// Use logging dependency instead
    -// Gradle 4.3.1 stopped releasing the logging jars to jcenter, just use the last available one
    -GradleVersion logVersion = GradleVersion.current() > GradleVersion.version('4.3') ? GradleVersion.version('4.3') : GradleVersion.current()
    -
    -dependencies {
    -  compileOnly "org.gradle:gradle-logging:${logVersion.getVersion()}"
    +  testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}"
     }
     
     /*****************************************************************************
    
    From b1f9884a8d9fcd1a780b4a4709e1deaa5dd491fa Mon Sep 17 00:00:00 2001
    From: Alpar Torok 
    Date: Fri, 19 Apr 2019 09:34:23 +0300
    Subject: [PATCH 112/260] Convert repository-hdfs to testclusters (#41252)
    
    * Convert repository-hdfs to testclusters
    
    Relates #40862
    ---
     plugins/build.gradle                 |   6 +-
     plugins/repository-hdfs/build.gradle | 122 +++++++++++----------------
     2 files changed, 51 insertions(+), 77 deletions(-)
    
    diff --git a/plugins/build.gradle b/plugins/build.gradle
    index 585f26c3780f8..89a4fe7384c9c 100644
    --- a/plugins/build.gradle
    +++ b/plugins/build.gradle
    @@ -20,11 +20,7 @@
     // only configure immediate children of plugins dir
     configure(subprojects.findAll { it.parent.path == project.path }) {
       group = 'org.elasticsearch.plugin'
    -  // TODO exclude some plugins as they require features not yet supproted by testclusters
    -  if (false ==  name in ['repository-hdfs']) {
    -      apply plugin: 'elasticsearch.testclusters'       
    -  }
    -
    +  apply plugin: 'elasticsearch.testclusters'       
       apply plugin: 'elasticsearch.esplugin'
     
       esplugin {
    diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle
    index 65820ff874752..3d6dcd29d6d19 100644
    --- a/plugins/repository-hdfs/build.gradle
    +++ b/plugins/repository-hdfs/build.gradle
    @@ -18,7 +18,6 @@
      */
     
     import org.apache.tools.ant.taskdefs.condition.Os
    -import org.elasticsearch.gradle.test.ClusterConfiguration
     import org.elasticsearch.gradle.test.RestIntegTestTask
     
     import java.nio.file.Files
    @@ -64,15 +63,17 @@ dependencies {
       compile "org.apache.logging.log4j:log4j-slf4j-impl:${versions.log4j}"
     
       hdfsFixture project(':test:fixtures:hdfs-fixture')
    +  // Set the keytab files in the classpath so that we can access them from test code without the security manager
    +  // freaking out.
    +  testRuntime fileTree(dir: project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab").parent, include: ['*.keytab'])
     }
     
     dependencyLicenses {
       mapping from: /hadoop-.*/, to: 'hadoop'
     }
     
    -
     String realm = "BUILD.ELASTIC.CO"
    -
    +String krb5conf = project(':test:fixtures:krb5kdc-fixture').ext.krb5Conf("hdfs")
     
     // Create HDFS File System Testing Fixtures for HA/Secure combinations
     for (String fixtureName : ['hdfsFixture', 'haHdfsFixture', 'secureHdfsFixture', 'secureHaHdfsFixture']) {
    @@ -91,9 +92,8 @@ for (String fixtureName : ['hdfsFixture', 'haHdfsFixture', 'secureHdfsFixture',
     
         // If it's a secure fixture, then depend on Kerberos Fixture and principals + add the krb5conf to the JVM options
         if (fixtureName.equals('secureHdfsFixture') || fixtureName.equals('secureHaHdfsFixture')) {
    -      miniHDFSArgs.add("-Djava.security.krb5.conf=${project(':test:fixtures:krb5kdc-fixture').ext.krb5Conf("hdfs")}");
    +      miniHDFSArgs.add("-Djava.security.krb5.conf=${krb5conf}")
         }
    -
         // If it's an HA fixture, set a nameservice to use in the JVM options
         if (fixtureName.equals('haHdfsFixture') || fixtureName.equals('secureHaHdfsFixture')) {
           miniHDFSArgs.add("-Dha-nameservice=ha-hdfs")
    @@ -107,8 +107,7 @@ for (String fixtureName : ['hdfsFixture', 'haHdfsFixture', 'secureHdfsFixture',
         if (fixtureName.equals('secureHdfsFixture') || fixtureName.equals('secureHaHdfsFixture')) {
           miniHDFSArgs.add("hdfs/hdfs.build.elastic.co@${realm}")
           miniHDFSArgs.add(
    -              project(':test:fixtures:krb5kdc-fixture')
    -                      .ext.krb5Keytabs("hdfs", "hdfs_hdfs.build.elastic.co.keytab")
    +              project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs", "hdfs_hdfs.build.elastic.co.keytab")
           )
         }
     
    @@ -116,61 +115,36 @@ for (String fixtureName : ['hdfsFixture', 'haHdfsFixture', 'secureHdfsFixture',
       }
     }
     
    -// The following closure must execute before the afterEvaluate block in the constructor of the following integrationTest tasks:
    -project.afterEvaluate {
    -  for (String integTestTaskName : ['integTestHa', 'integTestSecure', 'integTestSecureHa']) {
    -    ClusterConfiguration cluster = project.extensions.getByName("${integTestTaskName}Cluster") as ClusterConfiguration
    -    cluster.dependsOn(project.bundlePlugin)
    -
    -    Task restIntegTestTask = project.tasks.getByName(integTestTaskName)
    -    restIntegTestTask.clusterConfig.plugin(project.path)
    -
    -    // Default jvm arguments for all test clusters
    -    String jvmArgs = "-Xms" + System.getProperty('tests.heap.size', '512m') +
    -            " " + "-Xmx" + System.getProperty('tests.heap.size', '512m') +
    -            " " + System.getProperty('tests.jvm.argline', '')
    -
    -    // If it's a secure cluster, add the keytab as an extra config, and set the krb5 conf in the JVM options.
    -    if (integTestTaskName.equals('integTestSecure') || integTestTaskName.equals('integTestSecureHa')) {
    -      String krb5conf = project(':test:fixtures:krb5kdc-fixture').ext.krb5Conf("hdfs")
    -      restIntegTestTask.clusterConfig.extraConfigFile(
    -              "repository-hdfs/krb5.keytab",
    -              "${project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs", "elasticsearch.keytab")}"
    -      )
    -      jvmArgs = jvmArgs + " " + "-Djava.security.krb5.conf=${krb5conf}"
    -
    -      // If it's the HA + Secure tests then also set the Kerberos settings for the integration test JVM since we'll
    -      // need to auth to HDFS to trigger namenode failovers.
    -      if (integTestTaskName.equals('integTestSecureHa')) {
    -        Task restIntegTestTaskRunner = project.tasks.getByName("${integTestTaskName}Runner")
    -        restIntegTestTaskRunner.systemProperty "test.krb5.principal.es", "elasticsearch@${realm}"
    -        restIntegTestTaskRunner.systemProperty "test.krb5.principal.hdfs", "hdfs/hdfs.build.elastic.co@${realm}"
    -        restIntegTestTaskRunner.jvmArgs "-Djava.security.krb5.conf=${krb5conf}"
    -        restIntegTestTaskRunner.systemProperty (
    +for (String integTestTaskName : ['integTestHa', 'integTestSecure', 'integTestSecureHa']) {
    +  task "${integTestTaskName}"(type: RestIntegTestTask) {
    +    description = "Runs rest tests against an elasticsearch cluster with HDFS."
    +    dependsOn(project.bundlePlugin)
    +    runner {
    +      if (integTestTaskName.contains("Secure")) {
    +        dependsOn secureHdfsFixture
    +        systemProperty "test.krb5.principal.es", "elasticsearch@${realm}"
    +        systemProperty "test.krb5.principal.hdfs", "hdfs/hdfs.build.elastic.co@${realm}"
    +        jvmArgs "-Djava.security.krb5.conf=${krb5conf}"
    +        systemProperty (
                     "test.krb5.keytab.hdfs",
                     project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab")
             )
           }
         }
    -
    -    restIntegTestTask.clusterConfig.jvmArgs = jvmArgs
       }
    -}
     
    -// Create a Integration Test suite just for HA based tests
    -RestIntegTestTask integTestHa = project.tasks.create('integTestHa', RestIntegTestTask.class) {
    -  description = "Runs rest tests against an elasticsearch cluster with HDFS configured with HA Namenode."
    -}
    -
    -// Create a Integration Test suite just for security based tests
    -RestIntegTestTask integTestSecure = project.tasks.create('integTestSecure', RestIntegTestTask.class) {
    -  description = "Runs rest tests against an elasticsearch cluster with HDFS secured by MIT Kerberos."
    +  testClusters."${integTestTaskName}" {
    +    plugin(file(bundlePlugin.archiveFile))
    +    if (integTestTaskName.contains("Secure")) {
    +      systemProperty "java.security.krb5.conf", krb5conf
    +      extraConfigFile(
    +              "repository-hdfs/krb5.keytab",
    +              file("${project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs", "elasticsearch.keytab")}")
    +      )
    +    }
    +  }
     }
     
    -// Create a Integration Test suite just for HA related security based tests
    -RestIntegTestTask integTestSecureHa = project.tasks.create('integTestSecureHa', RestIntegTestTask.class) {
    -  description = "Runs rest tests against an elasticsearch cluster with HDFS configured with HA Namenode and secured by MIT Kerberos."
    -}
     
     // Determine HDFS Fixture compatibility for the current build environment.
     boolean fixtureSupported = false
    @@ -199,21 +173,27 @@ if (legalPath == false) {
     
     // Always ignore HA integration tests in the normal integration test runner, they are included below as
     // part of their own HA-specific integration test tasks.
    -integTestRunner.exclude('**/Ha*TestSuiteIT.class')
    +integTest.runner {
    +  exclude('**/Ha*TestSuiteIT.class')
    +}
     
     if (fixtureSupported) {
       // Check depends on the HA test. Already depends on the standard test.
       project.check.dependsOn(integTestHa)
     
       // Both standard and HA tests depend on their respective HDFS fixtures
    -  integTestCluster.dependsOn hdfsFixture
    -  integTestHaCluster.dependsOn haHdfsFixture
    +  integTest.dependsOn hdfsFixture
    +  integTestHa.dependsOn haHdfsFixture
     
       // The normal test runner only runs the standard hdfs rest tests
    -  integTestRunner.systemProperty 'tests.rest.suite', 'hdfs_repository'
    +  integTest.runner {
    +    systemProperty 'tests.rest.suite', 'hdfs_repository'
    +  }
     
       // Only include the HA integration tests for the HA test task
    -  integTestHaRunner.setIncludes(['**/Ha*TestSuiteIT.class'])
    +  integTestHa.runner {
    +    setIncludes(['**/Ha*TestSuiteIT.class'])
    +  }
     } else {
       if (legalPath) {
         logger.warn("hdfsFixture unsupported, please set HADOOP_HOME and put HADOOP_HOME\\bin in PATH")
    @@ -222,29 +202,27 @@ if (fixtureSupported) {
       }
     
       // The normal integration test runner will just test that the plugin loads
    -  integTestRunner.systemProperty 'tests.rest.suite', 'hdfs_repository/10_basic'
    +  integTest.runner {
    +    systemProperty 'tests.rest.suite', 'hdfs_repository/10_basic'
    +  }
       // HA fixture is unsupported. Don't run them.
       integTestHa.setEnabled(false)
     }
     
     check.dependsOn(integTestSecure, integTestSecureHa)
     
    -// Fixture dependencies
    -integTestSecureCluster.dependsOn secureHdfsFixture
    -integTestSecureHaCluster.dependsOn secureHaHdfsFixture
    -
    -// Set the keytab files in the classpath so that we can access them from test code without the security manager
    -// freaking out.
    -project.dependencies {
    -  testRuntime fileTree(dir: project(':test:fixtures:krb5kdc-fixture').ext.krb5Keytabs("hdfs","hdfs_hdfs.build.elastic.co.keytab").parent, include: ['*.keytab'])
    -}
    -
     // Run just the secure hdfs rest test suite.
    -integTestSecureRunner.systemProperty 'tests.rest.suite', 'secure_hdfs_repository'
    +integTestSecure.runner {
    +  systemProperty 'tests.rest.suite', 'secure_hdfs_repository'
    +}
     // Ignore HA integration Tests. They are included below as part of integTestSecureHa test runner.
    -integTestSecureRunner.exclude('**/Ha*TestSuiteIT.class')
    +integTestSecure.runner {
    +  exclude('**/Ha*TestSuiteIT.class')
    +}
     // Only include the HA integration tests for the HA test task
    -integTestSecureHaRunner.setIncludes(['**/Ha*TestSuiteIT.class'])
    +integTestSecureHa.runner {
    +  setIncludes(['**/Ha*TestSuiteIT.class'])
    +}
     
     thirdPartyAudit {
         ignoreMissingClasses()
    
    From f4757b900379744c1c558d3867b290357d9f5b36 Mon Sep 17 00:00:00 2001
    From: David Turner 
    Date: Fri, 19 Apr 2019 07:37:22 +0100
    Subject: [PATCH 113/260] Allow ops to be blocked after primary promotion
     (#41360)
    
    Today we assert that there are no operations in flight in this test. However we
    will sometimes be in a situation where the operations are blocked, and we
    distinguish these cases since #41271 causing the assertion to fail. This commit
    addresses this by allowing operations to be blocked sometimes after a primary
    promotion.
    
    Fixes #41333.
    ---
     .../org/elasticsearch/index/shard/IndexShardTests.java | 10 ++++++++--
     1 file changed, 8 insertions(+), 2 deletions(-)
    
    diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
    index 183cd5519269b..c8f1777be246f 100644
    --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
    @@ -182,6 +182,7 @@
     import static org.hamcrest.Matchers.hasToString;
     import static org.hamcrest.Matchers.instanceOf;
     import static org.hamcrest.Matchers.isIn;
    +import static org.hamcrest.Matchers.isOneOf;
     import static org.hamcrest.Matchers.lessThan;
     import static org.hamcrest.Matchers.lessThanOrEqualTo;
     import static org.hamcrest.Matchers.not;
    @@ -630,7 +631,7 @@ public void onFailure(Exception e) {
             closeShards(indexShard);
         }
     
    -    public void testOperationPermitsOnPrimaryShards() throws InterruptedException, ExecutionException, IOException {
    +    public void testOperationPermitsOnPrimaryShards() throws Exception {
             final ShardId shardId = new ShardId("test", "_na_", 0);
             final IndexShard indexShard;
     
    @@ -638,6 +639,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E
                 // relocation target
                 indexShard = newShard(newShardRouting(shardId, "local_node", "other node",
                     true, ShardRoutingState.INITIALIZING, AllocationId.newRelocation(AllocationId.newInitializing())));
    +            assertEquals(0, indexShard.getActiveOperationsCount());
             } else if (randomBoolean()) {
                 // simulate promotion
                 indexShard = newStartedShard(false);
    @@ -654,11 +656,15 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E
                     new IndexShardRoutingTable.Builder(indexShard.shardId()).addShard(primaryRouting).build(),
                     Collections.emptySet());
                 latch.await();
    +            assertThat(indexShard.getActiveOperationsCount(), isOneOf(0, IndexShard.OPERATIONS_BLOCKED));
    +            if (randomBoolean()) {
    +                assertBusy(() -> assertEquals(0, indexShard.getActiveOperationsCount()));
    +            }
             } else {
                 indexShard = newStartedShard(true);
    +            assertEquals(0, indexShard.getActiveOperationsCount());
             }
             final long primaryTerm = indexShard.getPendingPrimaryTerm();
    -        assertEquals(0, indexShard.getActiveOperationsCount());
             Releasable operation1 = acquirePrimaryOperationPermitBlockingly(indexShard);
             assertEquals(1, indexShard.getActiveOperationsCount());
             Releasable operation2 = acquirePrimaryOperationPermitBlockingly(indexShard);
    
    From 92483d4199509324c963eefcef13290aa658800f Mon Sep 17 00:00:00 2001
    From: Alpar Torok 
    Date: Fri, 19 Apr 2019 09:46:21 +0300
    Subject: [PATCH 114/260] Add FIPS specific testclusters configuration (#41199)
    
    ClusterFormationTasks auto configured these properties for clusters.
    This PR adds FIPS specific configuration across all test clusters from
    the main build script to prevent coupling betwwen testclusters and the
    build plugin.
    
    Closes #40904
    ---
     build.gradle                                      | 15 +++++++++++++++
     .../org/elasticsearch/gradle/BuildPlugin.groovy   |  6 ------
     modules/reindex/build.gradle                      |  6 ------
     3 files changed, 15 insertions(+), 12 deletions(-)
    
    diff --git a/build.gradle b/build.gradle
    index c6ace445e9310..8e05cf17ce0c3 100644
    --- a/build.gradle
    +++ b/build.gradle
    @@ -598,6 +598,21 @@ allprojects {
       } 
     }
     
    +subprojects {
    +    // Common config when running with a FIPS-140 runtime JVM
    +    if (project.ext.has("inFipsJvm") && project.ext.inFipsJvm) {
    +        tasks.withType(Test) {
    +          systemProperty 'javax.net.ssl.trustStorePassword', 'password'
    +          systemProperty 'javax.net.ssl.keyStorePassword', 'password'
    +        }
    +        project.pluginManager.withPlugin("elasticsearch.testclusters") {
    +          project.testClusters.all {
    +            systemProperty 'javax.net.ssl.trustStorePassword', 'password'
    +            systemProperty 'javax.net.ssl.keyStorePassword', 'password'
    +          }
    +        }
    +    }
    +}
     
     
     
    diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    index f32ed078e09b6..c712174f1bedf 100644
    --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    @@ -993,12 +993,6 @@ class BuildPlugin implements Plugin {
                     // TODO: remove this once ctx isn't added to update script params in 7.0
                     systemProperty 'es.scripting.update.ctx_in_params', 'false'
     
    -                // Set the system keystore/truststore password if we're running tests in a FIPS-140 JVM
    -                if (project.inFipsJvm) {
    -                    systemProperty 'javax.net.ssl.trustStorePassword', 'password'
    -                    systemProperty 'javax.net.ssl.keyStorePassword', 'password'
    -                }
    -
                     testLogging {
                         showExceptions = true
                         showCauses = true
    diff --git a/modules/reindex/build.gradle b/modules/reindex/build.gradle
    index 48888d1bfce99..da184deedaa11 100644
    --- a/modules/reindex/build.gradle
    +++ b/modules/reindex/build.gradle
    @@ -95,12 +95,6 @@ dependencies {
       es090 'org.elasticsearch:elasticsearch:0.90.13@zip'
     }
     
    -// Issue tracked in https://github.com/elastic/elasticsearch/issues/40904
    -if (project.inFipsJvm) {
    -  testingConventions.enabled = false
    -  integTest.enabled = false
    -}
    -
     if (Os.isFamily(Os.FAMILY_WINDOWS)) {
       logger.warn("Disabling reindex-from-old tests because we can't get the pid file on windows")
       integTest.runner {
    
    From 5375465ddcfd5711311b8fe588090dc985183920 Mon Sep 17 00:00:00 2001
    From: Jim Ferenczi 
    Date: Fri, 19 Apr 2019 09:17:35 +0200
    Subject: [PATCH 115/260] Handle unmapped fields in _field_caps API (#34071)
    
    Today the `_field_caps` API returns the list of indices where a field
    is present only if this field has different types within the requested indices.
    However if the request is an index pattern (or an alias, or both...) there
    is no way to infer the indices if the response contains only fields that have
    the same type in all indices. This commit changes the response to always return
    the list of indices in the response. It also adds a way to retrieve unmapped field
    in a specific section per field called `unmapped`. This section is created for each field
    that is present in some indices but not all if the parameter `include_unmapped` is set to
    true in the request (defaults to false).
    ---
     .../org/elasticsearch/client/SearchIT.java    |   2 +
     docs/reference/search/field-caps.asciidoc     |  55 +++++-
     .../rest-api-spec/api/field_caps.json         |   5 +
     .../test/field_caps/10_basic.yml              |  29 ++-
     .../action/fieldcaps/FieldCapabilities.java   |  33 ++--
     .../FieldCapabilitiesIndexRequest.java        |   3 +-
     .../FieldCapabilitiesIndexResponse.java       |   8 +-
     .../fieldcaps/FieldCapabilitiesRequest.java   |  33 +++-
     .../FieldCapabilitiesRequestBuilder.java      |   5 +
     .../fieldcaps/FieldCapabilitiesResponse.java  |  73 ++++++--
     .../TransportFieldCapabilitiesAction.java     |  66 +++++--
     .../action/RestFieldCapabilitiesAction.java   |   1 +
     .../FieldCapabilitiesRequestTests.java        |   2 +
     .../FieldCapabilitiesResponseTests.java       | 123 +------------
     .../fieldcaps/FieldCapabilitiesTests.java     |   4 +-
     .../MergedFieldCapabilitiesResponseTests.java | 173 ++++++++++++++++++
     .../mapper/FieldFilterMapperPluginTests.java  |   2 +-
     .../search/fieldcaps/FieldCapabilitiesIT.java |  79 +++++++-
     .../DocumentAndFieldLevelSecurityTests.java   |   2 +-
     19 files changed, 501 insertions(+), 197 deletions(-)
     create mode 100644 server/src/test/java/org/elasticsearch/action/fieldcaps/MergedFieldCapabilitiesResponseTests.java
    
    diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java
    index 54826e963cb83..00d905aa14006 100644
    --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java
    +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java
    @@ -1255,6 +1255,8 @@ public void testFieldCaps() throws IOException {
             FieldCapabilitiesResponse response = execute(request,
                 highLevelClient()::fieldCaps, highLevelClient()::fieldCapsAsync);
     
    +        assertEquals(new String[] {"index1", "index2"}, response.getIndices());
    +
             // Check the capabilities for the 'rating' field.
             assertTrue(response.get().containsKey("rating"));
             Map ratingResponse = response.getField("rating");
    diff --git a/docs/reference/search/field-caps.asciidoc b/docs/reference/search/field-caps.asciidoc
    index 034bc92930b10..d150e8e5fa6db 100644
    --- a/docs/reference/search/field-caps.asciidoc
    +++ b/docs/reference/search/field-caps.asciidoc
    @@ -71,6 +71,7 @@ GET _field_caps?fields=rating,title
     --------------------------------------------------
     {
         "fields": {
    +        "indices": ["index1", "index2", "index3", "index4", "index5"],
             "rating": { <1>
                 "long": {
                     "searchable": true,
    @@ -103,9 +104,61 @@ and as a `keyword` in `index3` and `index4`.
     <3> The field `rating` is not searchable in `index4`.
     <4> The field `title` is defined as `text` in all indices.
     
    +[float]
    +=== Unmapped fields
     
    +By default unmapped fields are ignored. You can include them in the response by
    +adding a parameter called `include_unmapped` in the request:
     
    +[source,js]
    +--------------------------------------------------
    +GET _field_caps?fields=rating,title&include_unmapped
    +--------------------------------------------------
    +// CONSOLE
     
    +In which case the response will contain an entry for each field that is present in
    +some indices but not all:
     
    +[source,js]
    +--------------------------------------------------
    +{
    +    "fields": {
    +        "indices": ["index1", "index2", "index3"],
    +        "rating": {
    +            "long": {
    +                "searchable": true,
    +                "aggregatable": false,
    +                "indices": ["index1", "index2"],
    +                "non_aggregatable_indices": ["index1"]
    +            },
    +            "keyword": {
    +                "searchable": false,
    +                "aggregatable": true,
    +                "indices": ["index3", "index4"],
    +                "non_searchable_indices": ["index4"]
    +            },
    +            "unmapped": { <1>
    +                "indices": ["index5"],
    +                "searchable": false,
    +                "aggregatable": false
    +            }
    +        },
    +        "title": {
    +            "text": {
    +                "indices": ["index1", "index2", "index3", "index4"],
    +                "searchable": true,
    +                "aggregatable": false
    +            },
    +            "unmapped": { <2>
    +                "indices": ["index5"]
    +                "searchable": false,
    +                "aggregatable": false
    +            }
    +        }
    +    }
    +}
    +--------------------------------------------------
    +// NOTCONSOLE
     
    -
    +<1> The `rating` field is unmapped` in `index5`.
    +<2> The `title` field is unmapped` in `index5`.
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json
    index 3ba09ca314b03..ebb520fd433b9 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json
    @@ -32,6 +32,11 @@
               "options" : ["open","closed","none","all"],
               "default" : "open",
               "description" : "Whether to expand wildcard expression to concrete indices that are open, closed or both."
    +        },
    +        "include_unmapped": {
    +          "type": "boolean",
    +          "default": false,
    +          "description": "Indicates whether unmapped fields should be included in the response."
             }
           }
         },
    diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml
    index f65d295fa89d8..89cdaf4a842c6 100644
    --- a/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml
    +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml
    @@ -14,8 +14,8 @@ setup:
                         type:     double
                       geo:
                         type:     geo_point
    -                  date:
    -                    type:     date
    +                  misc:
    +                    type:     text
                       object:
                         type: object
                         properties:
    @@ -294,3 +294,28 @@ setup:
       - match: {fields.geo.keyword.indices:                 ["test3"]}
       - is_false: fields.geo.keyword.non_searchable_indices
       - is_false: fields.geo.keyword.on_aggregatable_indices
    +
    +---
    +"Field caps with include_unmapped":
    +  - skip:
    +      version: " - 7.99.99"
    +      reason: include_unmapped has been added in 7.1.0
    +
    +  - do:
    +      field_caps:
    +        include_unmapped: true
    +        index: 'test1,test2,test3'
    +        fields: [text, misc]
    +
    +  - match: {fields.text.text.searchable:                true}
    +  - match: {fields.text.text.aggregatable:              false}
    +  - is_false: fields.text.text.indices
    +  - is_false: fields.text.text.non_searchable_indices
    +  - is_false: fields.text.text.non_aggregatable_indices
    +  - match: {fields.misc.text.searchable:                true}
    +  - match: {fields.misc.text.aggregatable:              false}
    +  - match: {fields.misc.text.indices:                   ["test1"]}
    +  - match: {fields.misc.unmapped.searchable:            false}
    +  - match: {fields.misc.unmapped.aggregatable:          false}
    +  - match: {fields.misc.unmapped.indices:               ["test2", "test3"]}
    +
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilities.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilities.java
    index 5cfdba9294634..20f525716a218 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilities.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilities.java
    @@ -20,6 +20,7 @@
     package org.elasticsearch.action.fieldcaps;
     
     import org.elasticsearch.common.ParseField;
    +import org.elasticsearch.common.Strings;
     import org.elasticsearch.common.io.stream.StreamInput;
     import org.elasticsearch.common.io.stream.StreamOutput;
     import org.elasticsearch.common.io.stream.Writeable;
    @@ -34,6 +35,8 @@
     import java.util.Collections;
     import java.util.Comparator;
     import java.util.List;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
     
     /**
      * Describes the capabilities of a field optionally merged across multiple indices.
    @@ -214,30 +217,30 @@ public String[] nonAggregatableIndices() {
         public boolean equals(Object o) {
             if (this == o) return true;
             if (o == null || getClass() != o.getClass()) return false;
    -
             FieldCapabilities that = (FieldCapabilities) o;
    -
    -        if (isSearchable != that.isSearchable) return false;
    -        if (isAggregatable != that.isAggregatable) return false;
    -        if (!name.equals(that.name)) return false;
    -        if (!type.equals(that.type)) return false;
    -        if (!Arrays.equals(indices, that.indices)) return false;
    -        if (!Arrays.equals(nonSearchableIndices, that.nonSearchableIndices)) return false;
    -        return Arrays.equals(nonAggregatableIndices, that.nonAggregatableIndices);
    +        return isSearchable == that.isSearchable &&
    +            isAggregatable == that.isAggregatable &&
    +            Objects.equals(name, that.name) &&
    +            Objects.equals(type, that.type) &&
    +            Arrays.equals(indices, that.indices) &&
    +            Arrays.equals(nonSearchableIndices, that.nonSearchableIndices) &&
    +            Arrays.equals(nonAggregatableIndices, that.nonAggregatableIndices);
         }
     
         @Override
         public int hashCode() {
    -        int result = name.hashCode();
    -        result = 31 * result + type.hashCode();
    -        result = 31 * result + (isSearchable ? 1 : 0);
    -        result = 31 * result + (isAggregatable ? 1 : 0);
    +        int result = Objects.hash(name, type, isSearchable, isAggregatable);
             result = 31 * result + Arrays.hashCode(indices);
             result = 31 * result + Arrays.hashCode(nonSearchableIndices);
             result = 31 * result + Arrays.hashCode(nonAggregatableIndices);
             return result;
         }
     
    +    @Override
    +    public String toString() {
    +        return Strings.toString(this);
    +    }
    +
         static class Builder {
             private String name;
             private String type;
    @@ -260,6 +263,10 @@ void add(String index, boolean search, boolean agg) {
                 this.isAggregatable &= agg;
             }
     
    +        List getIndices() {
    +            return indiceList.stream().map(c -> c.name).collect(Collectors.toList());
    +        }
    +
             FieldCapabilities build(boolean withIndices) {
                 final String[] indices;
                 /* Eclipse can't deal with o -> o.name, maybe because of
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java
    index 1d28230c1218e..81266f12e37b3 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexRequest.java
    @@ -28,8 +28,7 @@
     
     import java.io.IOException;
     
    -public class FieldCapabilitiesIndexRequest
    -    extends SingleShardRequest {
    +public class FieldCapabilitiesIndexRequest extends SingleShardRequest {
     
         private String[] fields;
         private OriginalIndices originalIndices;
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexResponse.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexResponse.java
    index 1e4686245165b..8295e1aad25e4 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexResponse.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesIndexResponse.java
    @@ -26,6 +26,7 @@
     
     import java.io.IOException;
     import java.util.Map;
    +import java.util.Objects;
     
     /**
      * Response for {@link FieldCapabilitiesIndexRequest} requests.
    @@ -89,14 +90,13 @@ public void writeTo(StreamOutput out) throws IOException {
         public boolean equals(Object o) {
             if (this == o) return true;
             if (o == null || getClass() != o.getClass()) return false;
    -
             FieldCapabilitiesIndexResponse that = (FieldCapabilitiesIndexResponse) o;
    -
    -        return responseMap.equals(that.responseMap);
    +        return Objects.equals(indexName, that.indexName) &&
    +            Objects.equals(responseMap, that.responseMap);
         }
     
         @Override
         public int hashCode() {
    -        return responseMap.hashCode();
    +        return Objects.hash(indexName, responseMap);
         }
     }
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java
    index e9e77df5f9030..250d656afdd6d 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java
    @@ -19,6 +19,7 @@
     
     package org.elasticsearch.action.fieldcaps;
     
    +import org.elasticsearch.Version;
     import org.elasticsearch.action.ActionRequest;
     import org.elasticsearch.action.ActionRequestValidationException;
     import org.elasticsearch.action.IndicesRequest;
    @@ -44,6 +45,7 @@ public final class FieldCapabilitiesRequest extends ActionRequest implements Ind
         private String[] indices = Strings.EMPTY_ARRAY;
         private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen();
         private String[] fields = Strings.EMPTY_ARRAY;
    +    private boolean includeUnmapped = false;
         // pkg private API mainly for cross cluster search to signal that we do multiple reductions ie. the results should not be merged
         private boolean mergeResults = true;
     
    @@ -51,8 +53,7 @@ public final class FieldCapabilitiesRequest extends ActionRequest implements Ind
             new ObjectParser<>(NAME, FieldCapabilitiesRequest::new);
     
         static {
    -        PARSER.declareStringArray(fromList(String.class, FieldCapabilitiesRequest::fields),
    -            FIELDS_FIELD);
    +        PARSER.declareStringArray(fromList(String.class, FieldCapabilitiesRequest::fields), FIELDS_FIELD);
         }
     
         public FieldCapabilitiesRequest() {}
    @@ -83,6 +84,11 @@ public void readFrom(StreamInput in) throws IOException {
             indices = in.readStringArray();
             indicesOptions = IndicesOptions.readIndicesOptions(in);
             mergeResults = in.readBoolean();
    +        if (in.getVersion().onOrAfter(Version.V_8_0_0)) {
    +            includeUnmapped = in.readBoolean();
    +        } else {
    +            includeUnmapped = false;
    +        }
         }
     
         @Override
    @@ -92,6 +98,9 @@ public void writeTo(StreamOutput out) throws IOException {
             out.writeStringArray(indices);
             indicesOptions.writeIndicesOptions(out);
             out.writeBoolean(mergeResults);
    +        if (out.getVersion().onOrAfter(Version.V_8_0_0)) {
    +            out.writeBoolean(includeUnmapped);
    +        }
         }
     
         /**
    @@ -123,6 +132,11 @@ public FieldCapabilitiesRequest indicesOptions(IndicesOptions indicesOptions) {
             return this;
         }
     
    +    public FieldCapabilitiesRequest includeUnmapped(boolean includeUnmapped) {
    +        this.includeUnmapped = includeUnmapped;
    +        return this;
    +    }
    +
         @Override
         public String[] indices() {
             return indices;
    @@ -133,12 +147,15 @@ public IndicesOptions indicesOptions() {
             return indicesOptions;
         }
     
    +    public boolean includeUnmapped() {
    +        return includeUnmapped;
    +    }
    +
         @Override
         public ActionRequestValidationException validate() {
             ActionRequestValidationException validationException = null;
             if (fields == null || fields.length == 0) {
    -            validationException =
    -                ValidateActions.addValidationError("no fields specified", validationException);
    +            validationException = ValidateActions.addValidationError("no fields specified", validationException);
             }
             return validationException;
         }
    @@ -152,14 +169,12 @@ public boolean equals(Object o) {
             return  Arrays.equals(indices, that.indices) &&
                 Objects.equals(indicesOptions, that.indicesOptions) &&
                 Arrays.equals(fields, that.fields) &&
    -            Objects.equals(mergeResults, that.mergeResults);
    +            Objects.equals(mergeResults, that.mergeResults) &&
    +            includeUnmapped == that.includeUnmapped;
         }
     
         @Override
         public int hashCode() {
    -        return Objects.hash(Arrays.hashCode(indices),
    -            indicesOptions,
    -            Arrays.hashCode(fields),
    -            mergeResults);
    +        return Objects.hash(Arrays.hashCode(indices), indicesOptions, Arrays.hashCode(fields), mergeResults, includeUnmapped);
         }
     }
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java
    index 8da925e8fc98a..477bf3c733d5d 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java
    @@ -36,4 +36,9 @@ public FieldCapabilitiesRequestBuilder setFields(String... fields) {
             request().fields(fields);
             return this;
         }
    +
    +    public FieldCapabilitiesRequestBuilder setIncludeUnmapped(boolean includeUnmapped) {
    +        request().includeUnmapped(includeUnmapped);
    +        return this;
    +    }
     }
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java
    index f908ec7b1b289..4f7b75d879445 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java
    @@ -19,8 +19,10 @@
     
     package org.elasticsearch.action.fieldcaps;
     
    +import org.elasticsearch.Version;
     import org.elasticsearch.action.ActionResponse;
     import org.elasticsearch.common.ParseField;
    +import org.elasticsearch.common.Strings;
     import org.elasticsearch.common.collect.Tuple;
     import org.elasticsearch.common.io.stream.StreamInput;
     import org.elasticsearch.common.io.stream.StreamOutput;
    @@ -31,6 +33,7 @@
     import org.elasticsearch.common.xcontent.XContentParserUtils;
     
     import java.io.IOException;
    +import java.util.Arrays;
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.List;
    @@ -42,32 +45,43 @@
      * Response for {@link FieldCapabilitiesRequest} requests.
      */
     public class FieldCapabilitiesResponse extends ActionResponse implements ToXContentObject {
    +    private static final ParseField INDICES_FIELD = new ParseField("indices");
         private static final ParseField FIELDS_FIELD = new ParseField("fields");
     
    +    private String[] indices;
         private Map> responseMap;
         private List indexResponses;
     
    -    FieldCapabilitiesResponse(Map> responseMap) {
    -        this(responseMap, Collections.emptyList());
    +    FieldCapabilitiesResponse(String[] indices, Map> responseMap) {
    +        this(indices, responseMap, Collections.emptyList());
         }
     
         FieldCapabilitiesResponse(List indexResponses) {
    -        this(Collections.emptyMap(), indexResponses);
    +        this(Strings.EMPTY_ARRAY, Collections.emptyMap(), indexResponses);
         }
     
    -    private FieldCapabilitiesResponse(Map> responseMap,
    +    private FieldCapabilitiesResponse(String[] indices, Map> responseMap,
                                           List indexResponses) {
             this.responseMap = Objects.requireNonNull(responseMap);
             this.indexResponses = Objects.requireNonNull(indexResponses);
    +        this.indices = indices;
         }
     
         /**
          * Used for serialization
          */
         FieldCapabilitiesResponse() {
    -        this(Collections.emptyMap(), Collections.emptyList());
    +        this(Strings.EMPTY_ARRAY, Collections.emptyMap(), Collections.emptyList());
         }
     
    +    /**
    +     * Get the concrete list of indices that were requested.
    +     */
    +    public String[] getIndices() {
    +        return indices;
    +    }
    +
    +
         /**
          * Get the field capabilities map.
          */
    @@ -94,8 +108,12 @@ public Map getField(String field) {
         @Override
         public void readFrom(StreamInput in) throws IOException {
             super.readFrom(in);
    -        this.responseMap =
    -            in.readMap(StreamInput::readString, FieldCapabilitiesResponse::readField);
    +        if (in.getVersion().onOrAfter(Version.V_8_0_0)) {
    +            indices = in.readStringArray();
    +        } else {
    +            indices = Strings.EMPTY_ARRAY;
    +        }
    +        this.responseMap = in.readMap(StreamInput::readString, FieldCapabilitiesResponse::readField);
             indexResponses = in.readList(FieldCapabilitiesIndexResponse::new);
         }
     
    @@ -106,20 +124,27 @@ private static Map readField(StreamInput in) throws I
         @Override
         public void writeTo(StreamOutput out) throws IOException {
             super.writeTo(out);
    +        if (out.getVersion().onOrAfter(Version.V_8_0_0)) {
    +            out.writeStringArray(indices);
    +        }
             out.writeMap(responseMap, StreamOutput::writeString, FieldCapabilitiesResponse::writeField);
             out.writeList(indexResponses);
         }
     
    -    private static void writeField(StreamOutput out,
    -                           Map map) throws IOException {
    +    private static void writeField(StreamOutput out, Map map) throws IOException {
             out.writeMap(map, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut));
         }
     
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
    -        return builder.startObject()
    -            .field(FIELDS_FIELD.getPreferredName(), responseMap)
    -            .endObject();
    +        if (indexResponses.size() > 0) {
    +            throw new IllegalStateException("cannot serialize non-merged response");
    +        }
    +        builder.startObject();
    +        builder.field(INDICES_FIELD.getPreferredName(), indices);
    +        builder.field(FIELDS_FIELD.getPreferredName(), responseMap);
    +        builder.endObject();
    +        return builder;
         }
     
         public static FieldCapabilitiesResponse fromXContent(XContentParser parser) throws IOException {
    @@ -129,11 +154,14 @@ public static FieldCapabilitiesResponse fromXContent(XContentParser parser) thro
         @SuppressWarnings("unchecked")
         private static final ConstructingObjectParser PARSER =
             new ConstructingObjectParser<>("field_capabilities_response", true,
    -            a -> new FieldCapabilitiesResponse(
    -                ((List>>) a[0]).stream()
    -                    .collect(Collectors.toMap(Tuple::v1, Tuple::v2))));
    +            a -> {
    +                List indices = a[0] == null ? Collections.emptyList() : (List) a[0];
    +                return new FieldCapabilitiesResponse(indices.stream().toArray(String[]::new),
    +                    ((List>>) a[1]).stream().collect(Collectors.toMap(Tuple::v1, Tuple::v2)));
    +            });
     
         static {
    +        PARSER.declareStringArray(ConstructingObjectParser.optionalConstructorArg(), INDICES_FIELD);
             PARSER.declareNamedObjects(ConstructingObjectParser.constructorArg(), (p, c, n) -> {
                 Map typeToCapabilities = parseTypeToCapabilities(p, n);
                 return new Tuple<>(n, typeToCapabilities);
    @@ -158,14 +186,21 @@ private static Map parseTypeToCapabilities(XContentPa
         public boolean equals(Object o) {
             if (this == o) return true;
             if (o == null || getClass() != o.getClass()) return false;
    -
             FieldCapabilitiesResponse that = (FieldCapabilitiesResponse) o;
    -
    -        return responseMap.equals(that.responseMap);
    +        return Arrays.equals(indices, that.indices) &&
    +            Objects.equals(responseMap, that.responseMap) &&
    +            Objects.equals(indexResponses, that.indexResponses);
         }
     
         @Override
         public int hashCode() {
    -        return responseMap.hashCode();
    +        int result = Objects.hash(responseMap, indexResponses);
    +        result = 31 * result + Arrays.hashCode(indices);
    +        return result;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return Strings.toString(this);
         }
     }
    diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java
    index c177734cdd7c4..ffb98dd5224ba 100644
    --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java
    +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java
    @@ -37,10 +37,13 @@
     import org.elasticsearch.transport.TransportService;
     
     import java.util.ArrayList;
    +import java.util.Arrays;
     import java.util.Collections;
     import java.util.HashMap;
    +import java.util.HashSet;
     import java.util.List;
     import java.util.Map;
    +import java.util.Set;
     
     public class TransportFieldCapabilitiesAction extends HandledTransportAction {
         private final ThreadPool threadPool;
    @@ -75,20 +78,21 @@ protected void doExecute(Task task, FieldCapabilitiesRequest request, final Acti
             } else {
                 concreteIndices = indexNameExpressionResolver.concreteIndexNames(clusterState, localIndices);
             }
    +        final String[] allIndices = mergeIndiceNames(concreteIndices, remoteClusterIndices);
             final int totalNumRequest = concreteIndices.length + remoteClusterIndices.size();
             final CountDown completionCounter = new CountDown(totalNumRequest);
             final List indexResponses = Collections.synchronizedList(new ArrayList<>());
             final Runnable onResponse = () -> {
                 if (completionCounter.countDown()) {
                     if (request.isMergeResults()) {
    -                    listener.onResponse(merge(indexResponses));
    +                    listener.onResponse(merge(allIndices, indexResponses, request.includeUnmapped()));
                     } else {
                         listener.onResponse(new FieldCapabilitiesResponse(indexResponses));
                     }
                 }
             };
             if (totalNumRequest == 0) {
    -            listener.onResponse(new FieldCapabilitiesResponse(Collections.emptyMap()));
    +            listener.onResponse(new FieldCapabilitiesResponse(allIndices, Collections.emptyMap()));
             } else {
                 ActionListener innerListener = new ActionListener() {
                     @Override
    @@ -129,35 +133,61 @@ public void onFailure(Exception e) {
             }
         }
     
    -    private FieldCapabilitiesResponse merge(List indexResponses) {
    -        Map> responseMapBuilder = new HashMap<> ();
    +    private String[] mergeIndiceNames(String[] localIndices, Map remoteIndices) {
    +        Set allIndices = new HashSet<>();
    +        Arrays.stream(localIndices).forEach(allIndices::add);
    +        for (Map.Entry entry : remoteIndices.entrySet()) {
    +            for (String index : entry.getValue().indices()) {
    +                allIndices.add(RemoteClusterAware.buildRemoteIndexName(entry.getKey(), index));
    +            }
    +        }
    +        return allIndices.stream().toArray(String[]::new);
    +    }
    +
    +    private FieldCapabilitiesResponse merge(String[] indices, List indexResponses,
    +                                            boolean includeUnmapped) {
    +        final Map> responseMapBuilder = new HashMap<> ();
             for (FieldCapabilitiesIndexResponse response : indexResponses) {
                 innerMerge(responseMapBuilder, response.getIndexName(), response.get());
             }
    -
    -        Map> responseMap = new HashMap<>();
    -        for (Map.Entry> entry :
    -            responseMapBuilder.entrySet()) {
    -            Map typeMap = new HashMap<>();
    -            boolean multiTypes = entry.getValue().size() > 1;
    -            for (Map.Entry fieldEntry :
    -                entry.getValue().entrySet()) {
    +        final Map> responseMap = new HashMap<>();
    +        for (Map.Entry> entry : responseMapBuilder.entrySet()) {
    +            final Map typeMapBuilder = entry.getValue();
    +            if (includeUnmapped) {
    +                addUnmappedFields(indices, entry.getKey(), typeMapBuilder);
    +            }
    +            boolean multiTypes = typeMapBuilder.size() > 1;
    +            final Map typeMap = new HashMap<>();
    +            for (Map.Entry fieldEntry : typeMapBuilder.entrySet()) {
                     typeMap.put(fieldEntry.getKey(), fieldEntry.getValue().build(multiTypes));
                 }
    -            responseMap.put(entry.getKey(), typeMap);
    +            responseMap.put(entry.getKey(), Collections.unmodifiableMap(typeMap));
             }
     
    -        return new FieldCapabilitiesResponse(responseMap);
    +        return new FieldCapabilitiesResponse(indices, Collections.unmodifiableMap(responseMap));
    +    }
    +
    +    private void addUnmappedFields(String[] indices, String field, Map typeMap) {
    +        Set unmappedIndices = new HashSet<>();
    +        Arrays.stream(indices).forEach(unmappedIndices::add);
    +        typeMap.values().stream().forEach((b) -> b.getIndices().stream().forEach(unmappedIndices::remove));
    +        if (unmappedIndices.isEmpty() == false) {
    +            FieldCapabilities.Builder unmapped = new FieldCapabilities.Builder(field, "unmapped");
    +            typeMap.put("unmapped", unmapped);
    +            for (String index : unmappedIndices) {
    +                unmapped.add(index, false, false);
    +            }
    +        }
         }
     
    -    private void innerMerge(Map> responseMapBuilder, String indexName,
    -                            Map map) {
    +    private void innerMerge(Map> responseMapBuilder,
    +                                String indexName, Map map) {
             for (Map.Entry entry : map.entrySet()) {
                 final String field = entry.getKey();
                 final FieldCapabilities fieldCap = entry.getValue();
                 Map typeMap = responseMapBuilder.computeIfAbsent(field, f -> new HashMap<>());
    -            FieldCapabilities.Builder builder = typeMap.computeIfAbsent(fieldCap.getType(), key -> new FieldCapabilities.Builder(field,
    -                key));
    +            FieldCapabilities.Builder builder = typeMap.computeIfAbsent(fieldCap.getType(),
    +                key -> new FieldCapabilities.Builder(field, key));
                 builder.add(indexName, fieldCap.isSearchable(), fieldCap.isAggregatable());
             }
         }
    diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java
    index 4b9d37ace91c6..24e26713ed6a5 100644
    --- a/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java
    +++ b/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java
    @@ -57,6 +57,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request,
     
             fieldRequest.indicesOptions(
                 IndicesOptions.fromRequest(request, fieldRequest.indicesOptions()));
    +        fieldRequest.includeUnmapped(request.paramAsBoolean("include_unmapped", false));
             return channel -> client.fieldCaps(fieldRequest, new RestToXContentListener<>(channel));
         }
     }
    diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java
    index 4e4bd7fdf5534..dbab3104d4494 100644
    --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java
    +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java
    @@ -50,6 +50,7 @@ protected FieldCapabilitiesRequest createTestInstance() {
             if (randomBoolean()) {
                 request.indicesOptions(randomBoolean() ? IndicesOptions.strictExpand() : IndicesOptions.lenientExpandOpen());
             }
    +        request.includeUnmapped(randomBoolean());
             return request;
         }
     
    @@ -75,6 +76,7 @@ protected FieldCapabilitiesRequest mutateInstance(FieldCapabilitiesRequest insta
                 request.indicesOptions(indicesOptions);
             });
             mutators.add(request -> request.setMergeResults(!request.isMergeResults()));
    +        mutators.add(request -> request.includeUnmapped(!request.includeUnmapped()));
     
             FieldCapabilitiesRequest mutatedInstance = copyInstance(instance);
             Consumer mutator = randomFrom(mutators);
    diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java
    index 90b730660ddd9..e75dede451b16 100644
    --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java
    +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java
    @@ -19,32 +19,18 @@
     
     package org.elasticsearch.action.fieldcaps;
     
    -import org.elasticsearch.common.bytes.BytesReference;
    -import org.elasticsearch.common.xcontent.ToXContent;
    -import org.elasticsearch.common.xcontent.XContentBuilder;
    -import org.elasticsearch.common.xcontent.XContentFactory;
    -import org.elasticsearch.common.xcontent.XContentParser;
    -import org.elasticsearch.common.xcontent.XContentType;
    -import org.elasticsearch.test.AbstractStreamableXContentTestCase;
    +import org.elasticsearch.test.AbstractStreamableTestCase;
     
    -import java.io.IOException;
     import java.util.ArrayList;
     import java.util.Collections;
     import java.util.HashMap;
     import java.util.List;
     import java.util.Map;
    -import java.util.function.Predicate;
     
     import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiLettersOfLength;
     
     
    -public class FieldCapabilitiesResponseTests extends AbstractStreamableXContentTestCase {
    -
    -    @Override
    -    protected FieldCapabilitiesResponse doParseInstance(XContentParser parser) throws IOException {
    -        return FieldCapabilitiesResponse.fromXContent(parser);
    -    }
    -
    +public class FieldCapabilitiesResponseTests extends AbstractStreamableTestCase {
         @Override
         protected FieldCapabilitiesResponse createBlankInstance() {
             return new FieldCapabilitiesResponse();
    @@ -52,36 +38,15 @@ protected FieldCapabilitiesResponse createBlankInstance() {
     
         @Override
         protected FieldCapabilitiesResponse createTestInstance() {
    -        if (randomBoolean()) {
    -            // merged responses
    -            Map> responses = new HashMap<>();
    -
    -            String[] fields = generateRandomStringArray(5, 10, false, true);
    -            assertNotNull(fields);
    -
    -            for (String field : fields) {
    -                Map typesToCapabilities = new HashMap<>();
    -                String[] types = generateRandomStringArray(5, 10, false, false);
    -                assertNotNull(types);
    +        List responses = new ArrayList<>();
    +        int numResponse = randomIntBetween(0, 10);
     
    -                for (String type : types) {
    -                    typesToCapabilities.put(type, FieldCapabilitiesTests.randomFieldCaps(field));
    -                }
    -                responses.put(field, typesToCapabilities);
    -            }
    -            return new FieldCapabilitiesResponse(responses);
    -        } else {
    -            // non-merged responses
    -            List responses = new ArrayList<>();
    -            int numResponse = randomIntBetween(0, 10);
    -            for (int i = 0; i < numResponse; i++) {
    -                responses.add(createRandomIndexResponse());
    -            }
    -            return new FieldCapabilitiesResponse(responses);
    +        for (int i = 0; i < numResponse; i++) {
    +            responses.add(createRandomIndexResponse());
             }
    +        return new FieldCapabilitiesResponse(responses);
         }
     
    -
         private FieldCapabilitiesIndexResponse createRandomIndexResponse() {
             Map responses = new HashMap<>();
     
    @@ -118,78 +83,6 @@ protected FieldCapabilitiesResponse mutateInstance(FieldCapabilitiesResponse res
                         FieldCapabilitiesTests.randomFieldCaps(toReplace)));
                     break;
             }
    -        return new FieldCapabilitiesResponse(mutatedResponses);
    -    }
    -
    -    @Override
    -    protected Predicate getRandomFieldsExcludeFilter() {
    -        // Disallow random fields from being inserted under the 'fields' key, as this
    -        // map only contains field names, and also under 'fields.FIELD_NAME', as these
    -        // maps only contain type names.
    -        return field -> field.matches("fields(\\.\\w+)?");
    -    }
    -
    -    public void testToXContent() throws IOException {
    -        FieldCapabilitiesResponse response = createSimpleResponse();
    -
    -        XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON);
    -        response.toXContent(builder, ToXContent.EMPTY_PARAMS);
    -
    -        String generatedResponse = BytesReference.bytes(builder).utf8ToString();
    -        assertEquals((
    -            "{" +
    -            "    \"fields\": {" +
    -            "        \"rating\": { " +
    -            "            \"keyword\": {" +
    -            "                \"type\": \"keyword\"," +
    -            "                \"searchable\": false," +
    -            "                \"aggregatable\": true," +
    -            "                \"indices\": [\"index3\", \"index4\"]," +
    -            "                \"non_searchable_indices\": [\"index4\"] " +
    -            "            }," +
    -            "            \"long\": {" +
    -            "                \"type\": \"long\"," +
    -            "                \"searchable\": true," +
    -            "                \"aggregatable\": false," +
    -            "                \"indices\": [\"index1\", \"index2\"]," +
    -            "                \"non_aggregatable_indices\": [\"index1\"] " +
    -            "            }" +
    -            "        }," +
    -            "        \"title\": { " +
    -            "            \"text\": {" +
    -            "                \"type\": \"text\"," +
    -            "                \"searchable\": true," +
    -            "                \"aggregatable\": false" +
    -            "            }" +
    -            "        }" +
    -            "    }" +
    -            "}").replaceAll("\\s+", ""), generatedResponse);
    -    }
    -
    -    public void testEmptyResponse() throws IOException {
    -        FieldCapabilitiesResponse testInstance = new FieldCapabilitiesResponse();
    -        assertSerialization(testInstance);
    -    }
    -
    -    private static FieldCapabilitiesResponse createSimpleResponse() {
    -        Map titleCapabilities = new HashMap<>();
    -        titleCapabilities.put("text", new FieldCapabilities("title", "text", true, false));
    -
    -        Map ratingCapabilities = new HashMap<>();
    -        ratingCapabilities.put("long", new FieldCapabilities("rating", "long",
    -            true, false,
    -            new String[]{"index1", "index2"},
    -            null,
    -            new String[]{"index1"}));
    -        ratingCapabilities.put("keyword", new FieldCapabilities("rating", "keyword",
    -            false, true,
    -            new String[]{"index3", "index4"},
    -            new String[]{"index4"},
    -            null));
    -
    -        Map> responses = new HashMap<>();
    -        responses.put("title", titleCapabilities);
    -        responses.put("rating", ratingCapabilities);
    -        return new FieldCapabilitiesResponse(responses);
    +        return new FieldCapabilitiesResponse(null, mutatedResponses);
         }
     }
    diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesTests.java
    index 5dc27f8e6ec1f..deeae3351ec96 100644
    --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesTests.java
    +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesTests.java
    @@ -86,8 +86,8 @@ public void testBuilder() {
                 assertThat(cap2.isAggregatable(), equalTo(false));
                 assertThat(cap2.indices().length, equalTo(3));
                 assertThat(cap2.indices(), equalTo(new String[]{"index1", "index2", "index3"}));
    -            assertThat(cap1.nonSearchableIndices(), equalTo(new String[]{"index1", "index3"}));
    -            assertThat(cap1.nonAggregatableIndices(), equalTo(new String[]{"index2", "index3"}));
    +            assertThat(cap2.nonSearchableIndices(), equalTo(new String[]{"index1", "index3"}));
    +            assertThat(cap2.nonAggregatableIndices(), equalTo(new String[]{"index2", "index3"}));
             }
         }
     
    diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/MergedFieldCapabilitiesResponseTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/MergedFieldCapabilitiesResponseTests.java
    new file mode 100644
    index 0000000000000..45097af07ea11
    --- /dev/null
    +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/MergedFieldCapabilitiesResponseTests.java
    @@ -0,0 +1,173 @@
    +/*
    + * 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.action.fieldcaps;
    +
    +import org.elasticsearch.common.bytes.BytesReference;
    +import org.elasticsearch.common.xcontent.ToXContent;
    +import org.elasticsearch.common.xcontent.XContentBuilder;
    +import org.elasticsearch.common.xcontent.XContentFactory;
    +import org.elasticsearch.common.xcontent.XContentParser;
    +import org.elasticsearch.common.xcontent.XContentType;
    +import org.elasticsearch.test.AbstractStreamableXContentTestCase;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.function.Predicate;
    +
    +public class MergedFieldCapabilitiesResponseTests extends AbstractStreamableXContentTestCase {
    +
    +    @Override
    +    protected FieldCapabilitiesResponse doParseInstance(XContentParser parser) throws IOException {
    +        return FieldCapabilitiesResponse.fromXContent(parser);
    +    }
    +
    +    @Override
    +    protected FieldCapabilitiesResponse createBlankInstance() {
    +        return new FieldCapabilitiesResponse();
    +    }
    +
    +    @Override
    +    protected FieldCapabilitiesResponse createTestInstance() {
    +        // merged responses
    +        Map> responses = new HashMap<>();
    +
    +        String[] fields = generateRandomStringArray(5, 10, false, true);
    +        assertNotNull(fields);
    +
    +        for (String field : fields) {
    +            Map typesToCapabilities = new HashMap<>();
    +            String[] types = generateRandomStringArray(5, 10, false, false);
    +            assertNotNull(types);
    +
    +            for (String type : types) {
    +                typesToCapabilities.put(type, FieldCapabilitiesTests.randomFieldCaps(field));
    +            }
    +            responses.put(field, typesToCapabilities);
    +        }
    +        int numIndices = randomIntBetween(1, 10);
    +        String[] indices = new String[numIndices];
    +        for (int i = 0; i < numIndices; i++) {
    +            indices[i] = randomAlphaOfLengthBetween(5, 10);
    +        }
    +        return new FieldCapabilitiesResponse(indices, responses);
    +    }
    +
    +    @Override
    +    protected FieldCapabilitiesResponse mutateInstance(FieldCapabilitiesResponse response) {
    +        Map> mutatedResponses = new HashMap<>(response.get());
    +
    +        int mutation = response.get().isEmpty() ? 0 : randomIntBetween(0, 2);
    +
    +        switch (mutation) {
    +            case 0:
    +                String toAdd = randomAlphaOfLength(10);
    +                mutatedResponses.put(toAdd, Collections.singletonMap(
    +                    randomAlphaOfLength(10),
    +                    FieldCapabilitiesTests.randomFieldCaps(toAdd)));
    +                break;
    +            case 1:
    +                String toRemove = randomFrom(mutatedResponses.keySet());
    +                mutatedResponses.remove(toRemove);
    +                break;
    +            case 2:
    +                String toReplace = randomFrom(mutatedResponses.keySet());
    +                mutatedResponses.put(toReplace, Collections.singletonMap(
    +                    randomAlphaOfLength(10),
    +                    FieldCapabilitiesTests.randomFieldCaps(toReplace)));
    +                break;
    +        }
    +        return new FieldCapabilitiesResponse(null, mutatedResponses);
    +    }
    +
    +    @Override
    +    protected Predicate getRandomFieldsExcludeFilter() {
    +        // Disallow random fields from being inserted under the 'fields' key, as this
    +        // map only contains field names, and also under 'fields.FIELD_NAME', as these
    +        // maps only contain type names.
    +        return field -> field.matches("fields(\\.\\w+)?");
    +    }
    +
    +    public void testToXContent() throws IOException {
    +        FieldCapabilitiesResponse response = createSimpleResponse();
    +
    +        XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON);
    +        response.toXContent(builder, ToXContent.EMPTY_PARAMS);
    +
    +        String generatedResponse = BytesReference.bytes(builder).utf8ToString();
    +        assertEquals((
    +            "{" +
    +            "    \"indices\": null," +
    +            "    \"fields\": {" +
    +            "        \"rating\": { " +
    +            "            \"keyword\": {" +
    +            "                \"type\": \"keyword\"," +
    +            "                \"searchable\": false," +
    +            "                \"aggregatable\": true," +
    +            "                \"indices\": [\"index3\", \"index4\"]," +
    +            "                \"non_searchable_indices\": [\"index4\"] " +
    +            "            }," +
    +            "            \"long\": {" +
    +            "                \"type\": \"long\"," +
    +            "                \"searchable\": true," +
    +            "                \"aggregatable\": false," +
    +            "                \"indices\": [\"index1\", \"index2\"]," +
    +            "                \"non_aggregatable_indices\": [\"index1\"] " +
    +            "            }" +
    +            "        }," +
    +            "        \"title\": { " +
    +            "            \"text\": {" +
    +            "                \"type\": \"text\"," +
    +            "                \"searchable\": true," +
    +            "                \"aggregatable\": false" +
    +            "            }" +
    +            "        }" +
    +            "    }" +
    +            "}").replaceAll("\\s+", ""), generatedResponse);
    +    }
    +
    +    public void testEmptyResponse() throws IOException {
    +        FieldCapabilitiesResponse testInstance = new FieldCapabilitiesResponse();
    +        assertSerialization(testInstance);
    +    }
    +
    +    private static FieldCapabilitiesResponse createSimpleResponse() {
    +        Map titleCapabilities = new HashMap<>();
    +        titleCapabilities.put("text", new FieldCapabilities("title", "text", true, false));
    +
    +        Map ratingCapabilities = new HashMap<>();
    +        ratingCapabilities.put("long", new FieldCapabilities("rating", "long",
    +            true, false,
    +            new String[]{"index1", "index2"},
    +            null,
    +            new String[]{"index1"}));
    +        ratingCapabilities.put("keyword", new FieldCapabilities("rating", "keyword",
    +            false, true,
    +            new String[]{"index3", "index4"},
    +            new String[]{"index4"},
    +            null));
    +
    +        Map> responses = new HashMap<>();
    +        responses.put("title", titleCapabilities);
    +        responses.put("rating", ratingCapabilities);
    +        return new FieldCapabilitiesResponse(null, responses);
    +    }
    +}
    diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldFilterMapperPluginTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldFilterMapperPluginTests.java
    index 07a80a31debb6..2608e16497214 100644
    --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldFilterMapperPluginTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldFilterMapperPluginTests.java
    @@ -113,7 +113,7 @@ public void testFieldCapabilities() {
         }
     
         private static void assertFieldCaps(FieldCapabilitiesResponse fieldCapabilitiesResponse, Collection expectedFields) {
    -        Map> responseMap = fieldCapabilitiesResponse.get();
    +        Map> responseMap = new HashMap<>(fieldCapabilitiesResponse.get());
             Set builtInMetaDataFields = IndicesModule.getBuiltInMetaDataFields();
             for (String field : builtInMetaDataFields) {
                 Map remove = responseMap.remove(field);
    diff --git a/server/src/test/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java b/server/src/test/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java
    index 14640b191a2b8..7adc447a20736 100644
    --- a/server/src/test/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java
    +++ b/server/src/test/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java
    @@ -28,6 +28,7 @@
     import org.elasticsearch.test.ESIntegTestCase;
     import org.junit.Before;
     
    +import java.util.Arrays;
     import java.util.Collection;
     import java.util.Collections;
     import java.util.Map;
    @@ -60,6 +61,13 @@ public void setUp() throws Exception {
                                 .field("type", "alias")
                                 .field("path", "playlist")
                             .endObject()
    +                        .startObject("old_field")
    +                            .field("type", "long")
    +                        .endObject()
    +                        .startObject("new_field")
    +                            .field("type", "alias")
    +                            .field("path", "old_field")
    +                        .endObject()
                         .endObject()
                     .endObject()
                 .endObject();
    @@ -75,10 +83,14 @@ public void setUp() throws Exception {
                             .startObject("route_length_miles")
                                 .field("type", "double")
                             .endObject()
    +                        .startObject("new_field")
    +                            .field("type", "long")
    +                        .endObject()
                         .endObject()
                     .endObject()
                 .endObject();
             assertAcked(prepareCreate("new_index").addMapping("_doc", newIndexMapping));
    +        assertAcked(client().admin().indices().prepareAliases().addAlias("new_index", "current"));
         }
     
         public static class FieldFilterPlugin extends Plugin implements MapperPlugin {
    @@ -94,9 +106,9 @@ protected Collection> nodePlugins() {
         }
     
         public void testFieldAlias() {
    -        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields("distance", "route_length_miles")
    -            .get();
    +        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields("distance", "route_length_miles").get();
     
    +        assertIndices(response, "old_index", "new_index");
             // Ensure the response has entries for both requested fields.
             assertTrue(response.get().containsKey("distance"));
             assertTrue(response.get().containsKey("route_length_miles"));
    @@ -126,26 +138,73 @@ public void testFieldAlias() {
         }
     
         public void testFieldAliasWithWildcard() {
    -        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields("route*")
    -            .get();
    +        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields("route*").get();
     
    +        assertIndices(response, "old_index", "new_index");
             assertEquals(1, response.get().size());
             assertTrue(response.get().containsKey("route_length_miles"));
         }
     
         public void testFieldAliasFiltering() {
    -        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields(
    -            "secret-soundtrack", "route_length_miles")
    -            .get();
    +        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields("secret-soundtrack", "route_length_miles").get();
    +        assertIndices(response, "old_index", "new_index");
             assertEquals(1, response.get().size());
             assertTrue(response.get().containsKey("route_length_miles"));
         }
     
         public void testFieldAliasFilteringWithWildcard() {
    -        FieldCapabilitiesResponse response = client().prepareFieldCaps()
    -            .setFields("distance", "secret*")
    -            .get();
    +        FieldCapabilitiesResponse response = client().prepareFieldCaps().setFields("distance", "secret*").get();
    +        assertIndices(response, "old_index", "new_index");
             assertEquals(1, response.get().size());
             assertTrue(response.get().containsKey("distance"));
         }
    +
    +    public void testWithUnmapped() {
    +        FieldCapabilitiesResponse response = client().prepareFieldCaps()
    +            .setFields("new_field", "old_field")
    +            .setIncludeUnmapped(true)
    +            .get();
    +        assertIndices(response, "old_index", "new_index");
    +
    +        assertEquals(2, response.get().size());
    +        assertTrue(response.get().containsKey("old_field"));
    +
    +        Map oldField = response.getField("old_field");
    +        assertEquals(2, oldField.size());
    +
    +        assertTrue(oldField.containsKey("long"));
    +        assertEquals(
    +            new FieldCapabilities("old_field", "long", true, true, new String[] {"old_index"}, null, null),
    +            oldField.get("long"));
    +
    +        assertTrue(oldField.containsKey("unmapped"));
    +        assertEquals(
    +            new FieldCapabilities("old_field", "unmapped", false, false, new String[] {"new_index"}, null, null),
    +            oldField.get("unmapped"));
    +
    +        Map newField = response.getField("new_field");
    +        assertEquals(1, newField.size());
    +
    +        assertTrue(newField.containsKey("long"));
    +        assertEquals(
    +            new FieldCapabilities("new_field", "long", true, true),
    +            newField.get("long"));
    +    }
    +
    +    public void testWithIndexAlias() {
    +        FieldCapabilitiesResponse response = client().prepareFieldCaps("current").setFields("*").get();
    +        assertIndices(response, "new_index");
    +
    +        FieldCapabilitiesResponse response1 = client().prepareFieldCaps("current", "old_index").setFields("*").get();
    +        assertIndices(response1, "old_index", "new_index");
    +        FieldCapabilitiesResponse response2 = client().prepareFieldCaps("current", "old_index", "new_index").setFields("*").get();
    +        assertEquals(response1, response2);
    +    }
    +
    +    private void assertIndices(FieldCapabilitiesResponse response, String... indices) {
    +        assertNotNull(response.getIndices());
    +        Arrays.sort(indices);
    +        Arrays.sort(response.getIndices());
    +        assertArrayEquals(indices, response.getIndices());
    +    }
     }
    diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DocumentAndFieldLevelSecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DocumentAndFieldLevelSecurityTests.java
    index b28b379494c42..ce235895e0fa2 100644
    --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DocumentAndFieldLevelSecurityTests.java
    +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DocumentAndFieldLevelSecurityTests.java
    @@ -410,7 +410,7 @@ private static void assertExpectedFields(ImmutableOpenMap> responseMap = fieldCapabilitiesResponse.get();
    +        Map> responseMap = new HashMap<>(fieldCapabilitiesResponse.get());
             Set builtInMetaDataFields = IndicesModule.getBuiltInMetaDataFields();
             for (String field : builtInMetaDataFields) {
                 Map remove = responseMap.remove(field);
    
    From b89491e30d56087a8ae4102c75f074bea8a714c4 Mon Sep 17 00:00:00 2001
    From: Ryan Ernst 
    Date: Fri, 19 Apr 2019 00:42:55 -0700
    Subject: [PATCH 116/260] Fix openjdk use for package test (#41277)
    
    The gc log filename on linux is different with openjdk vs the oracle
    openjdk. This commit fixes the filename checked, which has changed
    because we now use oracle's openjdk 11.
    
    Additionally, windows packaging tests are fixed to download openjdk 11, and use it.
    ---
     Vagrantfile                                          | 12 +++++++-----
     .../gradle/vagrant/VagrantTestPlugin.groovy          |  2 +-
     .../resources/packaging/tests/70_sysv_initd.bats     |  2 +-
     3 files changed, 9 insertions(+), 7 deletions(-)
    
    diff --git a/Vagrantfile b/Vagrantfile
    index 6ba1d095e40c4..96745ddfc5f05 100644
    --- a/Vagrantfile
    +++ b/Vagrantfile
    @@ -64,9 +64,6 @@ Vagrant.configure(2) do |config|
           SHELL
         end
       end
    -  # Wheezy's backports don't contain Openjdk 8 and the backflips
    -  # required to get the sun jdk on there just aren't worth it. We have
    -  # jessie and stretch for testing debian and it works fine.
       'debian-8'.tap do |box|
         config.vm.define box, define_opts do |config|
           config.vm.box = 'elastic/debian-8-x86_64'
    @@ -406,11 +403,16 @@ def windows_common(config, name)
         $ps_prompt | Out-File $PsHome/Microsoft.PowerShell_profile.ps1
       SHELL
     
    +  config.vm.provision 'windows-jdk-11', type: 'shell', inline: <<-SHELL
    +    New-Item -ItemType Directory -Force -Path "C:/java"
    +    Invoke-WebRequest "https://download.java.net/java/GA/jdk11/9/GPL/openjdk-11.0.2_windows-x64_bin.zip" -OutFile "C:/java/jdk-11.zip"
    +    Expand-Archive -Path "C:/java/jdk-11.zip" -DestinationPath "C:/java/"
    +  SHELL
    +
       config.vm.provision 'set env variables', type: 'shell', inline: <<-SHELL
         $ErrorActionPreference = "Stop"
         [Environment]::SetEnvironmentVariable("PACKAGING_ARCHIVES", "C:/project/build/packaging/archives", "Machine")
    -    $javaHome = [Environment]::GetEnvironmentVariable("JAVA_HOME", "Machine")
    -    [Environment]::SetEnvironmentVariable("SYSTEM_JAVA_HOME", $javaHome, "Machine")
    +    [Environment]::SetEnvironmentVariable("SYSTEM_JAVA_HOME", "C:\java\jdk-11.0.2", "Machine")
         [Environment]::SetEnvironmentVariable("PACKAGING_TESTS", "C:/project/build/packaging/tests", "Machine")
         [Environment]::SetEnvironmentVariable("JAVA_HOME", $null, "Machine")
       SHELL
    diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy
    index 121b40821bbc3..870524da43399 100644
    --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy
    +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy
    @@ -300,7 +300,7 @@ class VagrantTestPlugin implements Plugin {
                          } else {
                            \$testArgs = \$args
                          }
    -                     "\$Env:SYSTEM_JAVA_HOME"/bin/java -cp "\$Env:PACKAGING_TESTS/*" org.elasticsearch.packaging.VMTestRunner @testArgs
    +                     & "\$Env:SYSTEM_JAVA_HOME"/bin/java -cp "\$Env:PACKAGING_TESTS/*" org.elasticsearch.packaging.VMTestRunner @testArgs
                          exit \$LASTEXITCODE
                          """
             }
    diff --git a/qa/vagrant/src/test/resources/packaging/tests/70_sysv_initd.bats b/qa/vagrant/src/test/resources/packaging/tests/70_sysv_initd.bats
    index b20e756e701f4..668e869243dbd 100644
    --- a/qa/vagrant/src/test/resources/packaging/tests/70_sysv_initd.bats
    +++ b/qa/vagrant/src/test/resources/packaging/tests/70_sysv_initd.bats
    @@ -160,7 +160,7 @@ setup() {
     
     @test "[INIT.D] GC logs exist" {
         start_elasticsearch_service
    -    assert_file_exist /var/log/elasticsearch/gc.log.0.current
    +    assert_file_exist /var/log/elasticsearch/gc.log
         stop_elasticsearch_service
     }
     
    
    From f23d900184bdcf55598e57b03efe9f3d588e7ddc Mon Sep 17 00:00:00 2001
    From: Arlind 
    Date: Fri, 19 Apr 2019 10:24:29 +0200
    Subject: [PATCH 117/260] Update glossary.asciidoc (#41364)
    
    ---
     docs/reference/glossary.asciidoc | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/docs/reference/glossary.asciidoc b/docs/reference/glossary.asciidoc
    index 7221a38cce7d0..e59e57f3c5a53 100644
    --- a/docs/reference/glossary.asciidoc
    +++ b/docs/reference/glossary.asciidoc
    @@ -132,7 +132,7 @@ more primary shards to scale the number of <>
     that your index can handle.
     +
     You cannot change the number of primary shards in an index, once the index is
    -index is created. However, an index can be split into a new index using the
    +created. However, an index can be split into a new index using the
     <>.
     +
     See also <>
    
    From 02c1196d706ce8c8d6ecfe88fe50f654463e788a Mon Sep 17 00:00:00 2001
    From: Alpar Torok 
    Date: Fri, 19 Apr 2019 11:46:58 +0300
    Subject: [PATCH 118/260] Mute failing test
    
    tracking #39331
    ---
     .../java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java    | 1 +
     1 file changed, 1 insertion(+)
    
    diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java
    index bfa142a64e789..2215526942947 100644
    --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java
    +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java
    @@ -266,6 +266,7 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception {
     
         }
     
    +    @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/39331")
         public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws Exception {
             final String leaderIndex = "leader";
             final int numberOfShards = randomIntBetween(1, 3);
    
    From 3d98b78955a680b10cc35980b598048ec7a6a5f1 Mon Sep 17 00:00:00 2001
    From: Albert Zaharovits 
    Date: Fri, 19 Apr 2019 13:56:52 +0300
    Subject: [PATCH 119/260] Fix role mapping DN field wildcards for users with
     NULL DNs (#41343)
    
    The `DistinguishedNamePredicate`, used for matching users to role mapping
    expressions, should handle users with null DNs. But it fails to do so (and this is
    a NPE bug), if the role mapping expression contains a lucene regexp or a wildcard.
    
    The fix simplifies `DistinguishedNamePredicate` to not handle null DNs at all, and
    instead use the `ExpressionModel#NULL_PREDICATE` for the DN field, just like
    any other missing user field.
    ---
     .../authc/support/UserRoleMapper.java         | 25 ++++++-----
     .../DistinguishedNamePredicateTests.java      | 10 ++---
     .../mapper/ExpressionRoleMappingTests.java    | 45 +++++++++++++++++--
     3 files changed, 58 insertions(+), 22 deletions(-)
    
    diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java
    index e55530bb5def0..dbc323810610f 100644
    --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java
    +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java
    @@ -79,8 +79,12 @@ public UserData(String username, @Nullable String dn, Collection groups,
             public ExpressionModel asModel() {
                 final ExpressionModel model = new ExpressionModel();
                 model.defineField("username", username);
    -            model.defineField("dn", dn, new DistinguishedNamePredicate(dn));
    +            if (dn != null) {
    +                // null dn fields get the default NULL_PREDICATE
    +                model.defineField("dn", dn, new DistinguishedNamePredicate(dn));
    +            }
                 model.defineField("groups", groups, groups.stream()
    +                    .filter(group -> group != null)
                         .>map(DistinguishedNamePredicate::new)
                         .reduce(Predicate::or)
                         .orElse(fieldValue -> false)
    @@ -165,22 +169,19 @@ class DistinguishedNamePredicate implements Predicate predicate = new UserRoleMapper.DistinguishedNamePredicate(null);
    -        assertPredicate(predicate, null, true);
    -        assertPredicate(predicate, "", false);
    -        assertPredicate(predicate, randomAlphaOfLengthBetween(1, 8), false);
    -
    -        predicate = new UserRoleMapper.DistinguishedNamePredicate("");
    +        Predicate predicate = new UserRoleMapper.DistinguishedNamePredicate("");
             assertPredicate(predicate, null, false);
             assertPredicate(predicate, "", true);
             assertPredicate(predicate, randomAlphaOfLengthBetween(1, 8), false);
    +        assertPredicate(predicate, randomAlphaOfLengthBetween(1, 8) + "*", false);
     
             predicate = new UserRoleMapper.DistinguishedNamePredicate("foo=");
             assertPredicate(predicate, null, false);
             assertPredicate(predicate, "foo", false);
             assertPredicate(predicate, "foo=", true);
             assertPredicate(predicate, randomAlphaOfLengthBetween(5, 12), false);
    +        assertPredicate(predicate, randomAlphaOfLengthBetween(5, 12) + "*", false);
     
             predicate = new UserRoleMapper.DistinguishedNamePredicate("=bar");
             assertPredicate(predicate, null, false);
             assertPredicate(predicate, "bar", false);
             assertPredicate(predicate, "=bar", true);
             assertPredicate(predicate, randomAlphaOfLengthBetween(5, 12), false);
    +        assertPredicate(predicate, randomAlphaOfLengthBetween(5, 12) + "*", false);
         }
     
         private void assertPredicate(Predicate predicate, Object value, boolean expected) {
    diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/ExpressionRoleMappingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/ExpressionRoleMappingTests.java
    index 57db60051194e..42652676d39f1 100644
    --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/ExpressionRoleMappingTests.java
    +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/ExpressionRoleMappingTests.java
    @@ -31,6 +31,7 @@
     import org.elasticsearch.xpack.core.security.authc.support.mapper.ExpressionRoleMapping;
     import org.elasticsearch.xpack.core.security.authc.support.mapper.TemplateRoleName;
     import org.elasticsearch.xpack.core.security.authc.support.mapper.expressiondsl.AllExpression;
    +import org.elasticsearch.xpack.core.security.authc.support.mapper.expressiondsl.AnyExpression;
     import org.elasticsearch.xpack.core.security.authc.support.mapper.expressiondsl.FieldExpression;
     import org.elasticsearch.xpack.security.authc.support.UserRoleMapper;
     import org.hamcrest.Matchers;
    @@ -59,7 +60,7 @@ public void setupMapping() throws Exception {
                 Settings.EMPTY, Mockito.mock(Environment.class), new ThreadContext(Settings.EMPTY));
         }
     
    -    public void testParseValidJsonWithFixedRoleNames() throws Exception {
    +    public void testValidExpressionWithFixedRoleNames() throws Exception {
             String json = "{"
                 + "\"roles\": [  \"kibana_user\", \"sales\" ], "
                 + "\"enabled\": true, "
    @@ -69,7 +70,7 @@ public void testParseValidJsonWithFixedRoleNames() throws Exception {
                 + "    { \"except\": { \"field\": { \"metadata.active\" : false } } }"
                 + "  ]}"
                 + "}";
    -        final ExpressionRoleMapping mapping = parse(json, "ldap_sales");
    +        ExpressionRoleMapping mapping = parse(json, "ldap_sales");
             assertThat(mapping.getRoles(), Matchers.containsInAnyOrder("kibana_user", "sales"));
             assertThat(mapping.getExpression(), instanceOf(AllExpression.class));
     
    @@ -96,12 +97,48 @@ public void testParseValidJsonWithFixedRoleNames() throws Exception {
                 Collections.emptyList(), Collections.singletonMap("active", true), realm
             );
     
    +        final UserRoleMapper.UserData user4 = new UserRoleMapper.UserData(
    +                "peter.null", null, Collections.emptyList(), Collections.singletonMap("active", true), realm
    +        );
    +
             assertThat(mapping.getExpression().match(user1a.asModel()), equalTo(true));
             assertThat(mapping.getExpression().match(user1b.asModel()), equalTo(true));
             assertThat(mapping.getExpression().match(user1c.asModel()), equalTo(true));
             assertThat(mapping.getExpression().match(user1d.asModel()), equalTo(true));
    -        assertThat(mapping.getExpression().match(user2.asModel()), equalTo(false));
    -        assertThat(mapping.getExpression().match(user3.asModel()), equalTo(false));
    +        assertThat(mapping.getExpression().match(user2.asModel()), equalTo(false)); // metadata.active == false
    +        assertThat(mapping.getExpression().match(user3.asModel()), equalTo(false)); // dn != ou=sales,dc=example,dc=com
    +        assertThat(mapping.getExpression().match(user4.asModel()), equalTo(false)); // dn == null
    +
    +        // expression without dn
    +        json = "{"
    +                + "\"roles\": [  \"superuser\", \"system_admin\", \"admin\" ], "
    +                + "\"enabled\": true, "
    +                + "\"rules\": { "
    +                + "  \"any\": [ "
    +                + "    { \"field\": { \"username\" : \"tony.stark\" } }, "
    +                + "    { \"field\": { \"groups\": \"cn=admins,dc=stark-enterprises,dc=com\" } }"
    +                + "  ]}"
    +                + "}";
    +        mapping = parse(json, "stark_admin");
    +            assertThat(mapping.getRoles(), Matchers.containsInAnyOrder("superuser", "system_admin", "admin"));
    +            assertThat(mapping.getExpression(), instanceOf(AnyExpression.class));
    +
    +        final UserRoleMapper.UserData userTony = new UserRoleMapper.UserData(
    +                "tony.stark", null, Collections.singletonList("Audi R8 owners"), Collections.singletonMap("boss", true), realm
    +        );
    +        final UserRoleMapper.UserData userPepper = new UserRoleMapper.UserData(
    +                "pepper.potts", null, Arrays.asList("marvel", "cn=admins,dc=stark-enterprises,dc=com"), null, realm
    +        );
    +        final UserRoleMapper.UserData userMax = new UserRoleMapper.UserData(
    +                "max.rockatansky", null, Collections.singletonList("bronze"), Collections.singletonMap("mad", true), realm
    +        );
    +        final UserRoleMapper.UserData userFinn = new UserRoleMapper.UserData(
    +                "finn.hackleberry", null, Arrays.asList("hacker", null), null, realm
    +        );
    +        assertThat(mapping.getExpression().match(userTony.asModel()), equalTo(true));
    +        assertThat(mapping.getExpression().match(userPepper.asModel()), equalTo(true));
    +        assertThat(mapping.getExpression().match(userMax.asModel()), equalTo(false));
    +        assertThat(mapping.getExpression().match(userFinn.asModel()), equalTo(false));
         }
     
         public void testParseValidJsonWithTemplatedRoleNames() throws Exception {
    
    From 328ba09f84b2ca084e1a322e48f79aee3ce440be Mon Sep 17 00:00:00 2001
    From: David Turner 
    Date: Fri, 19 Apr 2019 16:02:35 +0100
    Subject: [PATCH 120/260] Omit non-masters in ClusterFormationFailureHelper
     (#41344)
    
    Today the `ClusterFormationFailureHelper` says `... discovery will continue
    using ... from last-known cluster state` and lists all the nodes in the
    last-known cluster state. In fact we ignore the master-ineligible nodes in the
    last-known cluster state during discovery. This commit fixes this by listing
    only the master-eligible nodes from the cluster state in this message.
    ---
     .../ClusterFormationFailureHelper.java        |  4 +-
     .../ClusterFormationFailureHelperTests.java   | 42 +++++++++++++++----
     2 files changed, 37 insertions(+), 9 deletions(-)
    
    diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java
    index aaae94d0297e5..a707a9ae980b7 100644
    --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java
    +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java
    @@ -133,8 +133,8 @@ static class ClusterFormationState {
             }
     
             String getDescription() {
    -            final List clusterStateNodes
    -                = StreamSupport.stream(clusterState.nodes().spliterator(), false).map(DiscoveryNode::toString).collect(Collectors.toList());
    +            final List clusterStateNodes = StreamSupport.stream(clusterState.nodes().getMasterNodes().values().spliterator(), false)
    +                .map(n -> n.value.toString()).collect(Collectors.toList());
     
                 final String discoveryWillContinueDescription = String.format(Locale.ROOT,
                     "discovery will continue using %s from hosts providers and %s from last-known cluster state; " +
    diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java
    index 8b08c9c3fc01e..16740b0761ff5 100644
    --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java
    +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java
    @@ -31,6 +31,7 @@
     import org.elasticsearch.test.ESTestCase;
     
     import java.util.Arrays;
    +import java.util.HashSet;
     import java.util.concurrent.atomic.AtomicLong;
     import java.util.stream.Collectors;
     
    @@ -43,6 +44,7 @@
     import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
     import static org.hamcrest.Matchers.equalTo;
     import static org.hamcrest.Matchers.is;
    +import static org.hamcrest.Matchers.isOneOf;
     
     public class ClusterFormationFailureHelperTests extends ESTestCase {
         public void testScheduling() {
    @@ -72,7 +74,7 @@ public void testScheduling() {
                     warningCount.incrementAndGet();
                     return new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L);
                 },
    -            deterministicTaskQueue.getThreadPool(), () -> logLastFailedJoinAttemptWarningCount.incrementAndGet());
    +            deterministicTaskQueue.getThreadPool(), logLastFailedJoinAttemptWarningCount::incrementAndGet);
     
             deterministicTaskQueue.runAllTasks();
             assertThat("should not schedule anything yet", warningCount.get(), is(0L));
    @@ -139,19 +141,18 @@ public void testDescriptionOnMasterIneligibleNodes() {
                 .version(12L).nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build();
     
             assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 15L).getDescription(),
    -            is("master not discovered yet: have discovered []; discovery will continue using [] from hosts providers and [" + localNode +
    -                "] from last-known cluster state; node term 15, last-accepted version 12 in term 0"));
    +            is("master not discovered yet: have discovered []; discovery will continue using [] from hosts providers " +
    +                "and [] from last-known cluster state; node term 15, last-accepted version 12 in term 0"));
     
             final TransportAddress otherAddress = buildNewFakeTransportAddress();
             assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 16L).getDescription(),
                 is("master not discovered yet: have discovered []; discovery will continue using [" + otherAddress +
    -                "] from hosts providers and [" + localNode +
    -                "] from last-known cluster state; node term 16, last-accepted version 12 in term 0"));
    +                "] from hosts providers and [] from last-known cluster state; node term 16, last-accepted version 12 in term 0"));
     
             final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT);
             assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 17L).getDescription(),
    -            is("master not discovered yet: have discovered [" + otherNode + "]; discovery will continue using [] from hosts providers and ["
    -                + localNode + "] from last-known cluster state; node term 17, last-accepted version 12 in term 0"));
    +            is("master not discovered yet: have discovered [" + otherNode + "]; discovery will continue using [] from hosts providers " +
    +                "and [] from last-known cluster state; node term 17, last-accepted version 12 in term 0"));
         }
     
         public void testDescriptionBeforeBootstrapping() {
    @@ -349,5 +350,32 @@ public void testDescriptionAfterBootstrapping() {
                     "have discovered [] which is not a quorum; " +
                     "discovery will continue using [] from hosts providers and [" + localNode +
                     "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"));
    +
    +        final DiscoveryNode otherMasterNode = new DiscoveryNode("other-master", buildNewFakeTransportAddress(), Version.CURRENT);
    +        final DiscoveryNode otherNonMasterNode = new DiscoveryNode("other-non-master", buildNewFakeTransportAddress(), emptyMap(),
    +            new HashSet<>(randomSubsetOf(Arrays.stream(DiscoveryNode.Role.values())
    +                .filter(r -> r != DiscoveryNode.Role.MASTER).collect(Collectors.toList()))),
    +            Version.CURRENT);
    +
    +        String[] configNodeIds = new String[]{"n1", "n2"};
    +        final ClusterState stateWithOtherNodes = ClusterState.builder(ClusterName.DEFAULT)
    +            .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).add(otherMasterNode).add(otherNonMasterNode))
    +            .metaData(MetaData.builder().coordinationMetaData(CoordinationMetaData.builder()
    +                .lastAcceptedConfiguration(config(configNodeIds))
    +                .lastCommittedConfiguration(config(configNodeIds)).build())).build();
    +
    +        assertThat(new ClusterFormationState(Settings.EMPTY, stateWithOtherNodes, emptyList(), emptyList(), 0L).getDescription(), isOneOf(
    +
    +            // nodes from last-known cluster state could be in either order
    +
    +            "master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " +
    +                "have discovered [] which is not a quorum; " +
    +                "discovery will continue using [] from hosts providers and [" + localNode + ", " + otherMasterNode +
    +                "] from last-known cluster state; node term 0, last-accepted version 0 in term 0",
    +
    +            "master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " +
    +                "have discovered [] which is not a quorum; " +
    +                "discovery will continue using [] from hosts providers and [" + otherMasterNode + ", " + localNode +
    +                "] from last-known cluster state; node term 0, last-accepted version 0 in term 0"));
         }
     }
    
    From c39980a65dd593363f1d8d1b038b26cb0ce02aaf Mon Sep 17 00:00:00 2001
    From: Marios Trivyzas 
    Date: Fri, 19 Apr 2019 19:03:28 +0300
    Subject: [PATCH 121/260] SQL: Fix bug with optimization of null related
     conditionals (#41355)
    
    The SimplifyConditional rule is removing NULL literals from those
    functions to simplify their evaluation. This happens in the Optimizer
    and a new instance of the conditional function is generated. Previously,
    the dataType was not set properly (defaulted to DataType.NULL) for
    those new instances and since the resolveType() wasn't called again
    it resulted in returning always null.
    
    E.g.:
    
    SELECT COALESCE(null, 'foo', null, 'bar')
    
    COALESCE(null, 'foo', null, 'bar')
    -----------------
    null
    
    This issue was not visible before because the tests always used an alias
    for the conditional function which caused the resolveType() to be
    called which sets the dataType properly.
    
    E.g.:
    
    SELECT COALESCE(null, 'foo', null, 'bar') as c
    
    c
    -----------------
    foo
    ---
     x-pack/plugin/sql/qa/src/main/resources/null.csv-spec    | 7 +++++++
     .../predicate/conditional/ConditionalFunction.java       | 9 +++++++--
     .../xpack/sql/optimizer/OptimizerTests.java              | 6 ++++++
     3 files changed, 20 insertions(+), 2 deletions(-)
    
    diff --git a/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec
    index 19541cf5d9f32..610217b233314 100644
    --- a/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec
    +++ b/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec
    @@ -61,6 +61,13 @@ c:i
     ;
     
     coalesceMixed
    +SELECT COALESCE(null, 123, null, 321);
    +
    +COALESCE(null, 123, null, 321):i
    +123
    +;
    +
    +coalesceMixedWithAlias
     SELECT COALESCE(null, 123, null, 321) AS c;
     
     c:i
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java
    index 3de85185e8a4f..b3841f09e825e 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java
    @@ -25,7 +25,7 @@
      */
     public abstract class ConditionalFunction extends ScalarFunction {
     
    -    protected DataType dataType = DataType.NULL;
    +    protected DataType dataType = null;
     
         ConditionalFunction(Source source, List fields) {
             super(source, fields);
    @@ -33,6 +33,12 @@ public abstract class ConditionalFunction extends ScalarFunction {
     
         @Override
         public DataType dataType() {
    +        if (dataType == null) {
    +            dataType = DataType.NULL;
    +            for (Expression exp : children()) {
    +                dataType = DataTypeConversion.commonType(dataType, exp.dataType());
    +            }
    +        }
             return dataType;
         }
     
    @@ -61,7 +67,6 @@ protected TypeResolution resolveType() {
                             child.dataType().typeName));
                     }
                 }
    -            dataType = DataTypeConversion.commonType(dataType, child.dataType());
             }
             return TypeResolution.TYPE_RESOLVED;
         }
    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 c95206c29e9ed..eb8ac2b4d1595 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
    @@ -501,6 +501,7 @@ public void testSimplifyCoalesceRandomNullsWithValue() {
                             randomListOfNulls())));
             assertEquals(1, e.children().size());
             assertEquals(TRUE, e.children().get(0));
    +        assertEquals(DataType.BOOLEAN, e.dataType());
         }
     
         private List randomListOfNulls() {
    @@ -514,6 +515,7 @@ public void testSimplifyCoalesceFirstLiteral() {
             assertEquals(Coalesce.class, e.getClass());
             assertEquals(1, e.children().size());
             assertEquals(TRUE, e.children().get(0));
    +        assertEquals(DataType.BOOLEAN, e.dataType());
         }
     
         public void testSimplifyIfNullNulls() {
    @@ -527,11 +529,13 @@ public void testSimplifyIfNullWithNullAndValue() {
             assertEquals(IfNull.class, e.getClass());
             assertEquals(1, e.children().size());
             assertEquals(ONE, e.children().get(0));
    +        assertEquals(DataType.INTEGER, e.dataType());
     
             e = new SimplifyConditional().rule(new IfNull(EMPTY, ONE, NULL));
             assertEquals(IfNull.class, e.getClass());
             assertEquals(1, e.children().size());
             assertEquals(ONE, e.children().get(0));
    +        assertEquals(DataType.INTEGER, e.dataType());
         }
     
         public void testFoldNullNotAppliedOnNullIf() {
    @@ -559,6 +563,7 @@ public void testSimplifyGreatestRandomNullsWithValue() {
             assertEquals(2, e.children().size());
             assertEquals(ONE, e.children().get(0));
             assertEquals(TWO, e.children().get(1));
    +        assertEquals(DataType.INTEGER, e.dataType());
         }
     
         public void testSimplifyLeastNulls() {
    @@ -580,6 +585,7 @@ public void testSimplifyLeastRandomNullsWithValue() {
             assertEquals(2, e.children().size());
             assertEquals(ONE, e.children().get(0));
             assertEquals(TWO, e.children().get(1));
    +        assertEquals(DataType.INTEGER, e.dataType());
         }
         
         public void testConcatFoldingIsNotNull() {
    
    From b0f133f664c981d8e189359533103faa5e044bc5 Mon Sep 17 00:00:00 2001
    From: Ryan Ernst 
    Date: Fri, 19 Apr 2019 09:49:36 -0700
    Subject: [PATCH 122/260] Add a rule for task dependencies (#41322)
    
    This commit adds a task rule to print the task dependencies of any task.
    It only prints the direct dependencies, but makes debugging missing
    dependencies a lot easier.
    ---
     build.gradle | 25 +++++++++++++++++++++++--
     1 file changed, 23 insertions(+), 2 deletions(-)
    
    diff --git a/build.gradle b/build.gradle
    index 8e05cf17ce0c3..583569ef7cd45 100644
    --- a/build.gradle
    +++ b/build.gradle
    @@ -584,9 +584,30 @@ allprojects {
             configurations.findAll { it.isCanBeResolved() }.each { it.resolve() }
           }
       }
    -}
     
    -allprojects {
    +  // helper task to print direct dependencies of a single task
    +  project.tasks.addRule("Pattern: Dependencies") { String taskName ->
    +    if (taskName.endsWith("Dependencies") == false) {
    +      return
    +    }
    +    if (project.tasks.findByName(taskName) != null) {
    +      return
    +    }
    +    String realTaskName = taskName.substring(0, taskName.length() - "Dependencies".length())
    +    Task realTask = project.tasks.findByName(realTaskName)
    +    if (realTask == null) {
    +      return
    +    }
    +    project.tasks.create(taskName) {
    +      doLast {
    +        println("${realTask.path} dependencies:")
    +        for (Task dep : realTask.getTaskDependencies().getDependencies(realTask)) {
    +          println("  - ${dep.path}")
    +        }
    +      }
    +    }
    +  }
    +
       task checkPart1
       task checkPart2 
       tasks.matching { it.name == "check" }.all { check ->
    
    From aab0695a51963284301ad7e371c9bf618a56b1c2 Mon Sep 17 00:00:00 2001
    From: Jason Tedor 
    Date: Fri, 19 Apr 2019 14:08:22 -0400
    Subject: [PATCH 123/260] Fix build issue if no specific Java version are set
     (#41379)
    
    If no Java versions are set then when we size the executor thread pool
    we end up with zero threads, which is illegal. This commit avoids that
    problem by only starting the executor when needed.
    ---
     .../elasticsearch/gradle/BuildPlugin.groovy   | 75 ++++++++++---------
     1 file changed, 39 insertions(+), 36 deletions(-)
    
    diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    index c712174f1bedf..7db0be891842b 100644
    --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    @@ -130,13 +130,6 @@ class BuildPlugin implements Plugin {
                 String runtimeJavaHome = findRuntimeJavaHome(compilerJavaHome)
                 File gradleJavaHome = Jvm.current().javaHome
     
    -            final Map javaVersions = [:]
    -            for (int version = 8; version <= Integer.parseInt(minimumCompilerVersion.majorVersion); version++) {
    -                if(System.getenv(getJavaHomeEnvVarName(version.toString())) != null) {
    -                    javaVersions.put(version, findJavaHome(version.toString()));
    -                }
    -            }
    -
                 String javaVendor = System.getProperty('java.vendor')
                 String gradleJavaVersion = System.getProperty('java.version')
                 String gradleJavaVersionDetails = "${javaVendor} ${gradleJavaVersion}" +
    @@ -197,38 +190,48 @@ class BuildPlugin implements Plugin {
                     throw new GradleException(message)
                 }
     
    -            ExecutorService exec = Executors.newFixedThreadPool(javaVersions.size())
    -            Set> results = new HashSet<>()
    -
    -            javaVersions.entrySet().stream()
    -                    .filter { it.getValue() != null }
    -                    .forEach { javaVersionEntry ->
    -                        results.add(exec.submit {
    -                            final String javaHome = javaVersionEntry.getValue()
    -                            final int version = javaVersionEntry.getKey()
    -                            if (project.file(javaHome).exists() == false) {
    -                                throw new GradleException("Invalid JAVA${version}_HOME=${javaHome} location does not exist")
    -                            }
    +            final Map javaVersions = [:]
    +            for (int version = 8; version <= Integer.parseInt(minimumCompilerVersion.majorVersion); version++) {
    +                if(System.getenv(getJavaHomeEnvVarName(version.toString())) != null) {
    +                    javaVersions.put(version, findJavaHome(version.toString()));
    +                }
    +            }
     
    -                            JavaVersion javaVersionEnum = JavaVersion.toVersion(findJavaSpecificationVersion(project, javaHome))
    -                            final JavaVersion expectedJavaVersionEnum = version < 9 ?
    -                                    JavaVersion.toVersion("1." + version) :
    -                                    JavaVersion.toVersion(Integer.toString(version))
    +            if (javaVersions.isEmpty() == false) {
     
    -                            if (javaVersionEnum != expectedJavaVersionEnum) {
    -                                final String message =
    -                                        "the environment variable JAVA" + version + "_HOME must be set to a JDK installation directory for Java" +
    -                                                " ${expectedJavaVersionEnum} but is [${javaHome}] corresponding to [${javaVersionEnum}]"
    -                                throw new GradleException(message)
    -                            }
    -                        })
    -            }
    +                ExecutorService exec = Executors.newFixedThreadPool(javaVersions.size())
    +                Set> results = new HashSet<>()
    +
    +                javaVersions.entrySet().stream()
    +                        .filter { it.getValue() != null }
    +                        .forEach { javaVersionEntry ->
    +                    results.add(exec.submit {
    +                        final String javaHome = javaVersionEntry.getValue()
    +                        final int version = javaVersionEntry.getKey()
    +                        if (project.file(javaHome).exists() == false) {
    +                            throw new GradleException("Invalid JAVA${version}_HOME=${javaHome} location does not exist")
    +                        }
     
    -            project.gradle.taskGraph.whenReady {
    -                try {
    -                    results.forEach { it.get() }
    -                } finally {
    -                    exec.shutdown();
    +                        JavaVersion javaVersionEnum = JavaVersion.toVersion(findJavaSpecificationVersion(project, javaHome))
    +                        final JavaVersion expectedJavaVersionEnum = version < 9 ?
    +                                JavaVersion.toVersion("1." + version) :
    +                                JavaVersion.toVersion(Integer.toString(version))
    +
    +                        if (javaVersionEnum != expectedJavaVersionEnum) {
    +                            final String message =
    +                                    "the environment variable JAVA" + version + "_HOME must be set to a JDK installation directory for Java" +
    +                                            " ${expectedJavaVersionEnum} but is [${javaHome}] corresponding to [${javaVersionEnum}]"
    +                            throw new GradleException(message)
    +                        }
    +                    })
    +                }
    +
    +                project.gradle.taskGraph.whenReady {
    +                    try {
    +                        results.forEach { it.get() }
    +                    } finally {
    +                        exec.shutdown();
    +                    }
                     }
                 }
     
    
    From 0ecbae85ea34f7a3c8c5076e1345bf490713efa4 Mon Sep 17 00:00:00 2001
    From: Ryan Ernst 
    Date: Fri, 19 Apr 2019 11:33:57 -0700
    Subject: [PATCH 124/260] Move keystore-cli to its own tools project (#40787)
    
    This commit moves the keystore cli into its own project, so that the
    test dependencies can be isolated from the rest of server.
    ---
     distribution/build.gradle                     |  3 ++
     distribution/src/bin/elasticsearch-keystore   |  1 +
     .../src/bin/elasticsearch-keystore.bat        |  1 +
     distribution/tools/keystore-cli/build.gradle  | 28 +++++++++++++
     .../settings/AddFileKeyStoreCommand.java      |  0
     .../settings/AddStringKeyStoreCommand.java    |  0
     .../settings/CreateKeyStoreCommand.java       |  0
     .../common/settings/KeyStoreCli.java          |  0
     .../common/settings/ListKeyStoreCommand.java  |  0
     .../RemoveSettingKeyStoreCommand.java         |  0
     .../bootstrap/BootstrapTests.java             |  0
     .../settings/AddFileKeyStoreCommandTests.java |  0
     .../AddStringKeyStoreCommandTests.java        |  0
     .../settings/CreateKeyStoreCommandTests.java  |  0
     .../settings/KeyStoreCommandTestCase.java     |  0
     .../common/settings/KeyStoreWrapperTests.java |  0
     .../settings/ListKeyStoreCommandTests.java    |  0
     .../RemoveSettingKeyStoreCommandTests.java    |  0
     server/build.gradle                           |  2 -
     .../MetaDataIndexStateServiceTests.java       |  4 +-
     .../index/mapper/ObjectMapperMergeTests.java  | 42 +++++++++----------
     .../composite/InternalCompositeTests.java     | 42 +++++++++----------
     settings.gradle                               |  1 +
     23 files changed, 78 insertions(+), 46 deletions(-)
     create mode 100644 distribution/tools/keystore-cli/build.gradle
     rename {server => distribution/tools/keystore-cli}/src/main/java/org/elasticsearch/common/settings/AddFileKeyStoreCommand.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/main/java/org/elasticsearch/common/settings/AddStringKeyStoreCommand.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/main/java/org/elasticsearch/common/settings/CreateKeyStoreCommand.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/main/java/org/elasticsearch/common/settings/KeyStoreCli.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/main/java/org/elasticsearch/common/settings/ListKeyStoreCommand.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/main/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommand.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/KeyStoreCommandTestCase.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java (100%)
     rename {server => distribution/tools/keystore-cli}/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java (100%)
    
    diff --git a/distribution/build.gradle b/distribution/build.gradle
    index e27d37bb513a0..2949993903080 100644
    --- a/distribution/build.gradle
    +++ b/distribution/build.gradle
    @@ -296,6 +296,9 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) {
               from { project(':distribution:tools:plugin-cli').jar }
               from { project(':distribution:tools:plugin-cli').configurations.runtime }
             }
    +        into('tools/keystore-cli') {
    +          from { project(':distribution:tools:keystore-cli').jar }
    +        }
             if (oss == false) {
               into('tools/security-cli') {
                 from { project(':x-pack:plugin:security:cli').jar }
    diff --git a/distribution/src/bin/elasticsearch-keystore b/distribution/src/bin/elasticsearch-keystore
    index 49e1aa7437a08..334eda30d1a21 100755
    --- a/distribution/src/bin/elasticsearch-keystore
    +++ b/distribution/src/bin/elasticsearch-keystore
    @@ -1,5 +1,6 @@
     #!/bin/bash
     
     ES_MAIN_CLASS=org.elasticsearch.common.settings.KeyStoreCli \
    +  ES_ADDITIONAL_CLASSPATH_DIRECTORIES=lib/tools/keystore-cli \
       "`dirname "$0"`"/elasticsearch-cli \
       "$@"
    diff --git a/distribution/src/bin/elasticsearch-keystore.bat b/distribution/src/bin/elasticsearch-keystore.bat
    index 83372248fb61a..9c0fb37ce6efa 100644
    --- a/distribution/src/bin/elasticsearch-keystore.bat
    +++ b/distribution/src/bin/elasticsearch-keystore.bat
    @@ -4,6 +4,7 @@ setlocal enabledelayedexpansion
     setlocal enableextensions
     
     set ES_MAIN_CLASS=org.elasticsearch.common.settings.KeyStoreCli
    +set ES_ADDITIONAL_CLASSPATH_DIRECTORIES=lib/tools/keystore-cli
     call "%~dp0elasticsearch-cli.bat" ^
       %%* ^
       || goto exit
    diff --git a/distribution/tools/keystore-cli/build.gradle b/distribution/tools/keystore-cli/build.gradle
    new file mode 100644
    index 0000000000000..5d57ef2e05597
    --- /dev/null
    +++ b/distribution/tools/keystore-cli/build.gradle
    @@ -0,0 +1,28 @@
    +/*
    + * 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.
    + */
    +
    +apply plugin: 'elasticsearch.build'
    +
    +dependencies {
    +  compileOnly "org.elasticsearch:elasticsearch:${version}"
    +  compileOnly "org.elasticsearch:elasticsearch-cli:${version}"
    +  testCompile "org.elasticsearch.test:framework:${version}"
    +  testCompile 'com.google.jimfs:jimfs:1.1'
    +  testCompile 'com.google.guava:guava:18.0'
    +}
    diff --git a/server/src/main/java/org/elasticsearch/common/settings/AddFileKeyStoreCommand.java b/distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/AddFileKeyStoreCommand.java
    similarity index 100%
    rename from server/src/main/java/org/elasticsearch/common/settings/AddFileKeyStoreCommand.java
    rename to distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/AddFileKeyStoreCommand.java
    diff --git a/server/src/main/java/org/elasticsearch/common/settings/AddStringKeyStoreCommand.java b/distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/AddStringKeyStoreCommand.java
    similarity index 100%
    rename from server/src/main/java/org/elasticsearch/common/settings/AddStringKeyStoreCommand.java
    rename to distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/AddStringKeyStoreCommand.java
    diff --git a/server/src/main/java/org/elasticsearch/common/settings/CreateKeyStoreCommand.java b/distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/CreateKeyStoreCommand.java
    similarity index 100%
    rename from server/src/main/java/org/elasticsearch/common/settings/CreateKeyStoreCommand.java
    rename to distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/CreateKeyStoreCommand.java
    diff --git a/server/src/main/java/org/elasticsearch/common/settings/KeyStoreCli.java b/distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/KeyStoreCli.java
    similarity index 100%
    rename from server/src/main/java/org/elasticsearch/common/settings/KeyStoreCli.java
    rename to distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/KeyStoreCli.java
    diff --git a/server/src/main/java/org/elasticsearch/common/settings/ListKeyStoreCommand.java b/distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/ListKeyStoreCommand.java
    similarity index 100%
    rename from server/src/main/java/org/elasticsearch/common/settings/ListKeyStoreCommand.java
    rename to distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/ListKeyStoreCommand.java
    diff --git a/server/src/main/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommand.java b/distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommand.java
    similarity index 100%
    rename from server/src/main/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommand.java
    rename to distribution/tools/keystore-cli/src/main/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommand.java
    diff --git a/server/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/bootstrap/BootstrapTests.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/AddFileKeyStoreCommandTests.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/AddStringKeyStoreCommandTests.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/CreateKeyStoreCommandTests.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/KeyStoreCommandTestCase.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreCommandTestCase.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/KeyStoreCommandTestCase.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreCommandTestCase.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/ListKeyStoreCommandTests.java
    diff --git a/server/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java
    similarity index 100%
    rename from server/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java
    rename to distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/RemoveSettingKeyStoreCommandTests.java
    diff --git a/server/build.gradle b/server/build.gradle
    index f1c17f8dbe00c..ce5e4cc807fdf 100644
    --- a/server/build.gradle
    +++ b/server/build.gradle
    @@ -132,8 +132,6 @@ dependencies {
           exclude group: 'org.elasticsearch', module: 'elasticsearch'
         }
       }
    -  testCompile 'com.google.jimfs:jimfs:1.1'
    -  testCompile 'com.google.guava:guava:18.0'
     }
     
     if (isEclipse) {
    diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java
    index 8189e0b2b047d..e4dd37ede4255 100644
    --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java
    +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java
    @@ -19,7 +19,6 @@
     
     package org.elasticsearch.cluster.metadata;
     
    -import com.google.common.collect.ImmutableList;
     import org.elasticsearch.Version;
     import org.elasticsearch.action.support.master.AcknowledgedResponse;
     import org.elasticsearch.cluster.ClusterName;
    @@ -375,7 +374,8 @@ private static ClusterState addRestoredIndex(final String index, final int numSh
     
             final Snapshot snapshot = new Snapshot(randomAlphaOfLength(10), new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5)));
             final RestoreInProgress.Entry entry =
    -            new RestoreInProgress.Entry("_uuid", snapshot, RestoreInProgress.State.INIT, ImmutableList.of(index), shardsBuilder.build());
    +            new RestoreInProgress.Entry("_uuid", snapshot, RestoreInProgress.State.INIT,
    +                Collections.singletonList(index), shardsBuilder.build());
             return ClusterState.builder(newState)
                 .putCustom(RestoreInProgress.TYPE, new RestoreInProgress.Builder().add(entry).build())
                 .build();
    diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java
    index adba78546c1b0..76547098d280c 100644
    --- a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java
    @@ -18,7 +18,6 @@
      */
     package org.elasticsearch.index.mapper;
     
    -import com.google.common.collect.ImmutableMap;
     import org.elasticsearch.Version;
     import org.elasticsearch.common.settings.Settings;
     import org.elasticsearch.index.mapper.FieldMapper.CopyTo;
    @@ -27,6 +26,8 @@
     import org.elasticsearch.test.ESTestCase;
     import org.junit.AfterClass;
     
    +import java.util.Collections;
    +import java.util.HashMap;
     import java.util.Map;
     
     import static java.util.Collections.emptyMap;
    @@ -38,11 +39,7 @@ public class ObjectMapperMergeTests extends ESTestCase {
         private static FieldMapper barFieldMapper = createTextFieldMapper("bar");
         private static FieldMapper bazFieldMapper = createTextFieldMapper("baz");
     
    -    private static RootObjectMapper rootObjectMapper = createRootObjectMapper(
    -        "type1", true, ImmutableMap.of(
    -            "disabled", createObjectMapper("disabled", false, emptyMap()),
    -            "foo", createObjectMapper("foo", true, ImmutableMap.of(
    -                "bar", barFieldMapper))));
    +    private static RootObjectMapper rootObjectMapper = createMapping(false, true, true, false);
     
         @AfterClass
         public static void cleanupReferences() {
    @@ -51,14 +48,24 @@ public static void cleanupReferences() {
             rootObjectMapper = null;
         }
     
    +    private static RootObjectMapper createMapping(boolean disabledFieldEnabled, boolean fooFieldEnabled,
    +                                                  boolean includeBarField, boolean includeBazField) {
    +        Map mappers = new HashMap<>();
    +        mappers.put("disabled", createObjectMapper("disabled", disabledFieldEnabled, emptyMap()));
    +        Map fooMappers = new HashMap<>();
    +        if (includeBarField) {
    +            fooMappers.put("bar", barFieldMapper);
    +        }
    +        if (includeBazField) {
    +            fooMappers.put("baz", bazFieldMapper);
    +        }
    +        mappers.put("foo", createObjectMapper("foo", fooFieldEnabled,  Collections.unmodifiableMap(fooMappers)));
    +        return createRootObjectMapper("type1", true, Collections.unmodifiableMap(mappers));
    +    }
    +
         public void testMerge() {
             // GIVEN an enriched mapping with "baz" new field
    -        ObjectMapper mergeWith = createRootObjectMapper(
    -            "type1", true, ImmutableMap.of(
    -                "disabled", createObjectMapper("disabled", false, emptyMap()),
    -                "foo", createObjectMapper("foo", true, ImmutableMap.of(
    -                    "bar", barFieldMapper,
    -                    "baz", bazFieldMapper))));
    +        ObjectMapper mergeWith = createMapping(false, true, true, true);
     
             // WHEN merging mappings
             final ObjectMapper merged = rootObjectMapper.merge(mergeWith);
    @@ -71,10 +78,7 @@ public void testMerge() {
     
         public void testMergeWhenDisablingField() {
             // GIVEN a mapping with "foo" field disabled
    -        ObjectMapper mergeWith = createRootObjectMapper(
    -            "type1", true, ImmutableMap.of(
    -                "disabled", createObjectMapper("disabled", false, emptyMap()),
    -                "foo", createObjectMapper("foo", false, emptyMap())));
    +        ObjectMapper mergeWith = createMapping(false, false, false, false);
     
             // WHEN merging mappings
             // THEN a MapperException is thrown with an excepted message
    @@ -84,11 +88,7 @@ public void testMergeWhenDisablingField() {
     
         public void testMergeWhenEnablingField() {
             // GIVEN a mapping with "disabled" field enabled
    -        ObjectMapper mergeWith = createRootObjectMapper(
    -            "type1", true, ImmutableMap.of(
    -                "disabled", createObjectMapper("disabled", true, emptyMap()),
    -                "foo", createObjectMapper("foo", true, ImmutableMap.of(
    -                    "bar", barFieldMapper))));
    +        ObjectMapper mergeWith = createMapping(true, true, true, false);
     
             // WHEN merging mappings
             // THEN a MapperException is thrown with an excepted message
    diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/InternalCompositeTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/InternalCompositeTests.java
    index 725600b3551d6..98263d2ebb680 100644
    --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/InternalCompositeTests.java
    +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/InternalCompositeTests.java
    @@ -19,7 +19,6 @@
     
     package org.elasticsearch.search.aggregations.bucket.composite;
     
    -import com.google.common.collect.Lists;
     import org.apache.lucene.util.BytesRef;
     import org.elasticsearch.common.io.stream.Writeable;
     import org.elasticsearch.common.time.DateFormatter;
    @@ -36,6 +35,7 @@
     import java.io.IOException;
     import java.time.ZoneOffset;
     import java.util.ArrayList;
    +import java.util.Arrays;
     import java.util.Collections;
     import java.util.Comparator;
     import java.util.HashMap;
    @@ -251,11 +251,11 @@ public void testReduceSame() throws IOException {
     
         public void testCompareCompositeKeyBiggerFieldName() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field3", "field2"),
    +            Arrays.asList("field3", "field2"),
                 new Comparable[]{1, 2}
             );
             assertThat(key1.compareTo(key2), lessThan(0));
    @@ -263,11 +263,11 @@ public void testCompareCompositeKeyBiggerFieldName() {
     
         public void testCompareCompositeKeySmallerFieldName() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field3", "field2"),
    +            Arrays.asList("field3", "field2"),
                 new Comparable[]{1, 2}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
             assertThat(key1.compareTo(key2), greaterThan(0));
    @@ -275,11 +275,11 @@ public void testCompareCompositeKeySmallerFieldName() {
     
         public void testCompareCompositeKeyBiggerValue() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field3", "field2"),
    +            Arrays.asList("field3", "field2"),
                 new Comparable[]{2, 3}
             );
             assertThat(key1.compareTo(key2), lessThan(0));
    @@ -287,11 +287,11 @@ public void testCompareCompositeKeyBiggerValue() {
     
         public void testCompareCompositeKeySmallerValue() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field3", "field2"),
    +            Arrays.asList("field3", "field2"),
                 new Comparable[]{1, 2}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{2, 3}
             );
             assertThat(key1.compareTo(key2), greaterThan(0));
    @@ -299,11 +299,11 @@ public void testCompareCompositeKeySmallerValue() {
     
         public void testCompareCompositeKeyNullValueIsSmaller1() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{null, 20}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
             assertThat(key1.compareTo(key2), lessThan(0));
    @@ -311,11 +311,11 @@ public void testCompareCompositeKeyNullValueIsSmaller1() {
     
         public void testCompareCompositeKeyNullValueIsSmaller2() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{null, 20}
             );
             assertThat(key1.compareTo(key2), greaterThan(0));
    @@ -323,29 +323,29 @@ public void testCompareCompositeKeyNullValueIsSmaller2() {
     
         public void testCompareCompositeKeyMoreFieldsIsGreater() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
    -        InternalComposite.ArrayMap key2 = createMap(Lists.newArrayList("field1", "field2", "field3"),new Comparable[]{1, 2, null});
    +        InternalComposite.ArrayMap key2 = createMap(Arrays.asList("field1", "field2", "field3"),new Comparable[]{1, 2, null});
             assertThat(key1.compareTo(key2), lessThan(0));
         }
     
         public void testCompareCompositeKeyLessFieldsIsLesser() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2", "field3"),
    +            Arrays.asList("field1", "field2", "field3"),
                 new Comparable[]{1, 2, null}
             );
    -        InternalComposite.ArrayMap key2 = createMap(Lists.newArrayList("field1", "field2"),new Comparable[]{1, 2});
    +        InternalComposite.ArrayMap key2 = createMap(Arrays.asList("field1", "field2"),new Comparable[]{1, 2});
             assertThat(key1.compareTo(key2), greaterThan(0));
         }
     
         public void testCompareCompositeKeyEqual() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2", "field3"),
    +            Arrays.asList("field1", "field2", "field3"),
                 new Comparable[]{null, 1, 2}
             );
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field1", "field2", "field3"),
    +            Arrays.asList("field1", "field2", "field3"),
                 new Comparable[]{null, 1, 2}
             );
             assertThat(key1.compareTo(key1), equalTo(0));
    @@ -358,12 +358,12 @@ public void testCompareCompositeKeyEqual() {
     
         public void testCompareCompositeKeyValuesHaveDifferentTypes() {
             InternalComposite.ArrayMap key1 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{1, 2}
             );
     
             InternalComposite.ArrayMap key2 = createMap(
    -            Lists.newArrayList("field1", "field2"),
    +            Arrays.asList("field1", "field2"),
                 new Comparable[]{"1", 2}
             );
     
    diff --git a/settings.gradle b/settings.gradle
    index a05e0bf000337..76fab9a657648 100644
    --- a/settings.gradle
    +++ b/settings.gradle
    @@ -44,6 +44,7 @@ List projects = [
       'distribution:tools:java-version-checker',
       'distribution:tools:launchers',
       'distribution:tools:plugin-cli',
    +  'distribution:tools:keystore-cli',
       'server',
       'server:cli',
       'test:framework',
    
    From 0dca576a35fa6fb199ae8dbb7d01603c21991f7e Mon Sep 17 00:00:00 2001
    From: Ryan Ernst 
    Date: Fri, 19 Apr 2019 11:34:16 -0700
    Subject: [PATCH 125/260] Only include relevant platform files from modules
     (#41089)
    
    This commit adds a filter to the files include from modules to only
    include platform specific files relevant to the distribution being
    built. For example, the deb files on linux would now only include linux
    ML binaries, and not windows or macos files.
    ---
     distribution/archives/build.gradle | 24 ++++++++++++------------
     distribution/build.gradle          | 18 +++++++++++++++---
     distribution/packages/build.gradle |  2 +-
     3 files changed, 28 insertions(+), 16 deletions(-)
    
    diff --git a/distribution/archives/build.gradle b/distribution/archives/build.gradle
    index b48ebeb2c6ac0..c7c58ad4ebdc5 100644
    --- a/distribution/archives/build.gradle
    +++ b/distribution/archives/build.gradle
    @@ -113,25 +113,25 @@ task buildIntegTestZip(type: Zip) {
     task buildWindowsZip(type: Zip) {
       configure(commonZipConfig)
       archiveClassifier = 'windows-x86_64'
    -  with archiveFiles(modulesFiles(false), 'zip', 'windows', false, true)
    +  with archiveFiles(modulesFiles(false, 'windows'), 'zip', 'windows', false, true)
     }
     
     task buildOssWindowsZip(type: Zip) {
       configure(commonZipConfig)
       archiveClassifier = 'windows-x86_64'
    -  with archiveFiles(modulesFiles(true), 'zip', 'windows', true, true)
    +  with archiveFiles(modulesFiles(true, 'windows'), 'zip', 'windows', true, true)
     }
     
     task buildNoJdkWindowsZip(type: Zip) {
       configure(commonZipConfig)
       archiveClassifier = 'no-jdk-windows-x86_64'
    -  with archiveFiles(modulesFiles(false), 'zip', 'windows', false, false)
    +  with archiveFiles(modulesFiles(false, 'windows'), 'zip', 'windows', false, false)
     }
     
     task buildOssNoJdkWindowsZip(type: Zip) {
       configure(commonZipConfig)
       archiveClassifier = 'no-jdk-windows-x86_64'
    -  with archiveFiles(modulesFiles(true), 'zip', 'windows', true, false)
    +  with archiveFiles(modulesFiles(true, 'windows'), 'zip', 'windows', true, false)
     }
     
     Closure commonTarConfig = {
    @@ -144,49 +144,49 @@ Closure commonTarConfig = {
     task buildDarwinTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'darwin-x86_64'
    -  with archiveFiles(modulesFiles(false), 'tar', 'darwin', false, true)
    +  with archiveFiles(modulesFiles(false, 'darwin'), 'tar', 'darwin', false, true)
     }
     
     task buildOssDarwinTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'darwin-x86_64'
    -  with archiveFiles(modulesFiles(true), 'tar', 'darwin', true, true)
    +  with archiveFiles(modulesFiles(true, 'darwin'), 'tar', 'darwin', true, true)
     }
     
     task buildNoJdkDarwinTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'no-jdk-darwin-x86_64'
    -  with archiveFiles(modulesFiles(false), 'tar', 'darwin', false, false)
    +  with archiveFiles(modulesFiles(false, 'darwin'), 'tar', 'darwin', false, false)
     }
     
     task buildOssNoJdkDarwinTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'no-jdk-darwin-x86_64'
    -  with archiveFiles(modulesFiles(true), 'tar', 'darwin', true, false)
    +  with archiveFiles(modulesFiles(true, 'darwin'), 'tar', 'darwin', true, false)
     }
     
     task buildLinuxTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'linux-x86_64'
    -  with archiveFiles(modulesFiles(false), 'tar', 'linux', false, true)
    +  with archiveFiles(modulesFiles(false, 'linux'), 'tar', 'linux', false, true)
     }
     
     task buildOssLinuxTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'linux-x86_64'
    -  with archiveFiles(modulesFiles(true), 'tar', 'linux', true, true)
    +  with archiveFiles(modulesFiles(true, 'linux'), 'tar', 'linux', true, true)
     }
     
     task buildNoJdkLinuxTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'no-jdk-linux-x86_64'
    -  with archiveFiles(modulesFiles(false), 'tar', 'linux', false, false)
    +  with archiveFiles(modulesFiles(false, 'linux'), 'tar', 'linux', false, false)
     }
     
     task buildOssNoJdkLinuxTar(type: Tar) {
       configure(commonTarConfig)
       archiveClassifier = 'no-jdk-linux-x86_64'
    -  with archiveFiles(modulesFiles(true), 'tar', 'linux', true, false)
    +  with archiveFiles(modulesFiles(true, 'linux'), 'tar', 'linux', true, false)
     }
     
     Closure tarExists = { it -> new File('/bin/tar').exists() || new File('/usr/bin/tar').exists() || new File('/usr/local/bin/tar').exists() }
    diff --git a/distribution/build.gradle b/distribution/build.gradle
    index 2949993903080..bbf11923011c9 100644
    --- a/distribution/build.gradle
    +++ b/distribution/build.gradle
    @@ -308,7 +308,7 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) {
           }
         }
     
    -    modulesFiles = { oss ->
    +    modulesFiles = { oss, platform ->
           copySpec {
             eachFile {
               if (it.relativePath.segments[-2] == 'bin') {
    @@ -318,10 +318,22 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) {
                 it.mode = 0644
               }
             }
    +        Task buildModules
             if (oss) {
    -          from project(':distribution').buildOssModules
    +          buildModules = project(':distribution').buildOssModules
             } else {
    -          from project(':distribution').buildDefaultModules
    +          buildModules = project(':distribution').buildDefaultModules
    +        }
    +        List excludePlatforms = ['linux', 'windows', 'darwin']
    +        if (platform != null) {
    +           excludePlatforms.remove(excludePlatforms.indexOf(platform))
    +        } else {
    +           excludePlatforms = []
    +        }
    +        from(buildModules) {
    +          for (String excludePlatform : excludePlatforms) {
    +            exclude "**/platform/${excludePlatform}-x86_64/**"
    +          }
             }
           }
         }
    diff --git a/distribution/packages/build.gradle b/distribution/packages/build.gradle
    index 1c7fc466ba845..11c56bc66e09a 100644
    --- a/distribution/packages/build.gradle
    +++ b/distribution/packages/build.gradle
    @@ -139,7 +139,7 @@ Closure commonPackageConfig(String type, boolean oss, boolean jdk) {
             with libFiles(oss)
           }
           into('modules') {
    -        with modulesFiles(oss)
    +        with modulesFiles(oss, 'linux')
           }
           if (jdk) {
             into('jdk') {
    
    From a5ad5f7bb9164cfeb460fb87c0ead37ce7d475fe Mon Sep 17 00:00:00 2001
    From: Jason Tedor 
    Date: Fri, 19 Apr 2019 15:26:56 -0400
    Subject: [PATCH 126/260] Use collections conveniences in static initializers
     (#41374)
    
    This commit replaces the construction of some collections in static
    initializers with new collection convenience methods that are available
    now that we have bumped the minimum Java language level to be higher
    than Java 8.
    ---
     .../plugins/InstallPluginCommand.java         |  42 +++---
     .../common/SnowballAnalyzerProvider.java      |  22 +---
     .../percolator/QueryAnalyzer.java             |  59 ++++-----
     .../http/TestDeprecationHeaderRestAction.java |  19 +--
     .../cluster/coordination/Reconfigurator.java  |   2 +-
     .../common/inject/internal/MoreTypes.java     |  26 ++--
     .../elasticsearch/common/util/set/Sets.java   |  69 +++++++++-
     .../admin/indices/RestGetMappingAction.java   |   4 +-
     .../elasticsearch/threadpool/ThreadPool.java  |  53 ++++----
     .../common/util/set/SetsTests.java            |  17 ++-
     .../xpack/core/scheduler/Cron.java            |  52 ++++----
     .../persistence/DatafeedConfigProvider.java   |  14 +-
     .../xpack/ml/process/NativeController.java    |  11 +-
     .../xpack/ml/utils/DomainSplitFunction.java   | 120 +++++++++---------
     .../authc/kerberos/KerberosTestCase.java      |  62 +++++----
     15 files changed, 287 insertions(+), 285 deletions(-)
    
    diff --git a/distribution/tools/plugin-cli/src/main/java/org/elasticsearch/plugins/InstallPluginCommand.java b/distribution/tools/plugin-cli/src/main/java/org/elasticsearch/plugins/InstallPluginCommand.java
    index fc2da33c3a688..06b1aae2bce32 100644
    --- a/distribution/tools/plugin-cli/src/main/java/org/elasticsearch/plugins/InstallPluginCommand.java
    +++ b/distribution/tools/plugin-cli/src/main/java/org/elasticsearch/plugins/InstallPluginCommand.java
    @@ -44,6 +44,8 @@
     import org.elasticsearch.common.SuppressForbidden;
     import org.elasticsearch.common.collect.Tuple;
     import org.elasticsearch.common.hash.MessageDigests;
    +import org.elasticsearch.common.io.Streams;
    +import org.elasticsearch.common.util.set.Sets;
     import org.elasticsearch.core.internal.io.IOUtils;
     import org.elasticsearch.env.Environment;
     
    @@ -52,6 +54,7 @@
     import java.io.InputStream;
     import java.io.InputStreamReader;
     import java.io.OutputStream;
    +import java.io.UncheckedIOException;
     import java.net.HttpURLConnection;
     import java.net.URI;
     import java.net.URISyntaxException;
    @@ -82,7 +85,6 @@
     import java.util.Map;
     import java.util.Objects;
     import java.util.Set;
    -import java.util.TreeSet;
     import java.util.stream.Collectors;
     import java.util.zip.ZipEntry;
     import java.util.zip.ZipInputStream;
    @@ -130,36 +132,28 @@ class InstallPluginCommand extends EnvironmentAwareCommand {
         static final int PLUGIN_MALFORMED = 2;
     
         /** The builtin modules, which are plugins, but cannot be installed or removed. */
    -    static final Set MODULES;
    +    private static final Set MODULES;
         static {
    -        try (InputStream stream = InstallPluginCommand.class.getResourceAsStream("/modules.txt");
    -            BufferedReader reader = new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8))) {
    -            Set modules = new HashSet<>();
    -            String line = reader.readLine();
    -            while (line != null) {
    -                modules.add(line.trim());
    -                line = reader.readLine();
    -            }
    -            MODULES = Collections.unmodifiableSet(modules);
    -        } catch (IOException e) {
    -            throw new RuntimeException(e);
    +        try (var stream = InstallPluginCommand.class.getResourceAsStream("/modules.txt")) {
    +            MODULES = Streams.readAllLines(stream)
    +                .stream()
    +                .map(String::trim)
    +                .collect(Collectors.toUnmodifiableSet());
    +        } catch (final IOException e) {
    +            throw new UncheckedIOException(e);
             }
         }
     
         /** The official plugins that can be installed simply by name. */
         static final Set OFFICIAL_PLUGINS;
         static {
    -        try (InputStream stream = InstallPluginCommand.class.getResourceAsStream("/plugins.txt");
    -            BufferedReader reader = new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8))) {
    -            Set plugins = new TreeSet<>(); // use tree set to get sorting for help command
    -            String line = reader.readLine();
    -            while (line != null) {
    -                plugins.add(line.trim());
    -                line = reader.readLine();
    -            }
    -            OFFICIAL_PLUGINS = Collections.unmodifiableSet(plugins);
    -        } catch (IOException e) {
    -            throw new RuntimeException(e);
    +        try (var stream = InstallPluginCommand.class.getResourceAsStream("/plugins.txt")) {
    +            OFFICIAL_PLUGINS = Streams.readAllLines(stream)
    +                .stream()
    +                .map(String::trim)
    +                .collect(Sets.toUnmodifiableSortedSet());
    +        } catch (final IOException e) {
    +            throw new UncheckedIOException(e);
             }
         }
     
    diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SnowballAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SnowballAnalyzerProvider.java
    index 6eec01570a881..c7706391557cd 100644
    --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SnowballAnalyzerProvider.java
    +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/SnowballAnalyzerProvider.java
    @@ -29,11 +29,8 @@
     import org.elasticsearch.index.analysis.AbstractIndexAnalyzerProvider;
     import org.elasticsearch.index.analysis.Analysis;
     
    -import java.util.HashMap;
     import java.util.Map;
     
    -import static java.util.Collections.unmodifiableMap;
    -
     /**
      * Creates a SnowballAnalyzer initialized with stopwords and Snowball filter. Only
      * supports Dutch, English (default), French, German and German2 where stopwords
    @@ -48,17 +45,12 @@
      *
      */
     public class SnowballAnalyzerProvider extends AbstractIndexAnalyzerProvider {
    -    private static final Map DEFAULT_LANGUAGE_STOPWORDS;
    -
    -    static {
    -        Map defaultLanguageStopwords = new HashMap<>();
    -        defaultLanguageStopwords.put("English", EnglishAnalyzer.ENGLISH_STOP_WORDS_SET);
    -        defaultLanguageStopwords.put("Dutch", DutchAnalyzer.getDefaultStopSet());
    -        defaultLanguageStopwords.put("German", GermanAnalyzer.getDefaultStopSet());
    -        defaultLanguageStopwords.put("German2", GermanAnalyzer.getDefaultStopSet());
    -        defaultLanguageStopwords.put("French", FrenchAnalyzer.getDefaultStopSet());
    -        DEFAULT_LANGUAGE_STOPWORDS = unmodifiableMap(defaultLanguageStopwords);
    -    }
    +    private static final Map DEFAULT_LANGUAGE_STOP_WORDS = Map.of(
    +        "English", EnglishAnalyzer.ENGLISH_STOP_WORDS_SET,
    +        "Dutch", DutchAnalyzer.getDefaultStopSet(),
    +        "German", GermanAnalyzer.getDefaultStopSet(),
    +        "German2", GermanAnalyzer.getDefaultStopSet(),
    +        "French", FrenchAnalyzer.getDefaultStopSet());
     
         private final SnowballAnalyzer analyzer;
     
    @@ -66,7 +58,7 @@ public class SnowballAnalyzerProvider extends AbstractIndexAnalyzerProvider, BiFunction> queryProcessors;
    -
    -    static {
    -        Map, BiFunction> map = new HashMap<>();
    -        map.put(MatchNoDocsQuery.class, matchNoDocsQuery());
    -        map.put(MatchAllDocsQuery.class, matchAllDocsQuery());
    -        map.put(ConstantScoreQuery.class, constantScoreQuery());
    -        map.put(BoostQuery.class, boostQuery());
    -        map.put(TermQuery.class, termQuery());
    -        map.put(TermInSetQuery.class, termInSetQuery());
    -        map.put(CommonTermsQuery.class, commonTermsQuery());
    -        map.put(BlendedTermQuery.class, blendedTermQuery());
    -        map.put(PhraseQuery.class, phraseQuery());
    -        map.put(MultiPhraseQuery.class, multiPhraseQuery());
    -        map.put(SpanTermQuery.class, spanTermQuery());
    -        map.put(SpanNearQuery.class, spanNearQuery());
    -        map.put(SpanOrQuery.class, spanOrQuery());
    -        map.put(SpanFirstQuery.class, spanFirstQuery());
    -        map.put(SpanNotQuery.class, spanNotQuery());
    -        map.put(BooleanQuery.class, booleanQuery());
    -        map.put(DisjunctionMaxQuery.class, disjunctionMaxQuery());
    -        map.put(SynonymQuery.class, synonymQuery());
    -        map.put(FunctionScoreQuery.class, functionScoreQuery());
    -        map.put(PointRangeQuery.class, pointRangeQuery());
    -        map.put(IndexOrDocValuesQuery.class, indexOrDocValuesQuery());
    -        map.put(ESToParentBlockJoinQuery.class, toParentBlockJoinQuery());
    -        queryProcessors = Collections.unmodifiableMap(map);
    -    }
    +    private static final Map, BiFunction> QUERY_PROCESSORS = Map.ofEntries(
    +        entry(MatchNoDocsQuery.class, matchNoDocsQuery()),
    +        entry(MatchAllDocsQuery.class, matchAllDocsQuery()),
    +        entry(ConstantScoreQuery.class, constantScoreQuery()),
    +        entry(BoostQuery.class, boostQuery()),
    +        entry(TermQuery.class, termQuery()),
    +        entry(TermInSetQuery.class, termInSetQuery()),
    +        entry(CommonTermsQuery.class, commonTermsQuery()),
    +        entry(BlendedTermQuery.class, blendedTermQuery()),
    +        entry(PhraseQuery.class, phraseQuery()),
    +        entry(MultiPhraseQuery.class, multiPhraseQuery()),
    +        entry(SpanTermQuery.class, spanTermQuery()),
    +        entry(SpanNearQuery.class, spanNearQuery()),
    +        entry(SpanOrQuery.class, spanOrQuery()),
    +        entry(SpanFirstQuery.class, spanFirstQuery()),
    +        entry(SpanNotQuery.class, spanNotQuery()),
    +        entry(BooleanQuery.class, booleanQuery()),
    +        entry(DisjunctionMaxQuery.class, disjunctionMaxQuery()),
    +        entry(SynonymQuery.class, synonymQuery()),
    +        entry(FunctionScoreQuery.class, functionScoreQuery()),
    +        entry(PointRangeQuery.class, pointRangeQuery()),
    +        entry(IndexOrDocValuesQuery.class, indexOrDocValuesQuery()),
    +        entry(ESToParentBlockJoinQuery.class, toParentBlockJoinQuery()));
     
         private QueryAnalyzer() {
         }
    @@ -130,11 +125,11 @@ private QueryAnalyzer() {
         static Result analyze(Query query, Version indexVersion) {
             Class queryClass = query.getClass();
             if (queryClass.isAnonymousClass()) {
    -            // Sometimes queries have anonymous classes in that case we need the direct super class.
    -            // (for example blended term query)
    +            // sometimes queries have anonymous classes in that case we need the direct super class (e.g., blended term query)
                 queryClass = queryClass.getSuperclass();
             }
    -        BiFunction queryProcessor = queryProcessors.get(queryClass);
    +        assert Query.class.isAssignableFrom(queryClass) : query.getClass();
    +        BiFunction queryProcessor = QUERY_PROCESSORS.get(queryClass);
             if (queryProcessor != null) {
                 return queryProcessor.apply(query, indexVersion);
             } else {
    diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/TestDeprecationHeaderRestAction.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/TestDeprecationHeaderRestAction.java
    index 27b2f18b0919f..b609e28a2ac0b 100644
    --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/TestDeprecationHeaderRestAction.java
    +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/TestDeprecationHeaderRestAction.java
    @@ -18,8 +18,8 @@
      */
     package org.elasticsearch.http;
     
    -import org.apache.logging.log4j.Logger;
     import org.apache.logging.log4j.LogManager;
    +import org.apache.logging.log4j.Logger;
     import org.elasticsearch.client.node.NodeClient;
     import org.elasticsearch.common.logging.DeprecationLogger;
     import org.elasticsearch.common.settings.Setting;
    @@ -33,8 +33,6 @@
     import org.elasticsearch.rest.RestStatus;
     
     import java.io.IOException;
    -import java.util.Collections;
    -import java.util.HashMap;
     import java.util.List;
     import java.util.Map;
     
    @@ -58,17 +56,10 @@ public class TestDeprecationHeaderRestAction extends BaseRestHandler {
             Setting.boolSetting("test.setting.not_deprecated", false,
                                 Setting.Property.NodeScope, Setting.Property.Dynamic);
     
    -    private static final Map> SETTINGS_MAP;
    -
    -    static {
    -        Map> settingsMap = new HashMap<>(3);
    -
    -        settingsMap.put(TEST_DEPRECATED_SETTING_TRUE1.getKey(), TEST_DEPRECATED_SETTING_TRUE1);
    -        settingsMap.put(TEST_DEPRECATED_SETTING_TRUE2.getKey(), TEST_DEPRECATED_SETTING_TRUE2);
    -        settingsMap.put(TEST_NOT_DEPRECATED_SETTING.getKey(), TEST_NOT_DEPRECATED_SETTING);
    -
    -        SETTINGS_MAP = Collections.unmodifiableMap(settingsMap);
    -    }
    +    private static final Map> SETTINGS_MAP = Map.of(
    +        TEST_DEPRECATED_SETTING_TRUE1.getKey(), TEST_DEPRECATED_SETTING_TRUE1,
    +        TEST_DEPRECATED_SETTING_TRUE2.getKey(), TEST_DEPRECATED_SETTING_TRUE2,
    +        TEST_NOT_DEPRECATED_SETTING.getKey(), TEST_NOT_DEPRECATED_SETTING);
     
         public static final String DEPRECATED_ENDPOINT = "[/_test_cluster/deprecated_settings] exists for deprecated tests";
         public static final String DEPRECATED_USAGE = "[deprecated_settings] usage is deprecated. use [settings] instead";
    diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java
    index 44e6532df2daa..81a12ea4d47b4 100644
    --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java
    +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java
    @@ -123,7 +123,7 @@ public VotingConfiguration reconfigure(Set liveNodes, Set
             final Set liveInConfigIds = new TreeSet<>(currentConfig.getNodeIds());
             liveInConfigIds.retainAll(liveNodeIds);
     
    -        final Set inConfigNotLiveIds = Sets.sortedDifference(currentConfig.getNodeIds(), liveInConfigIds);
    +        final Set inConfigNotLiveIds = Sets.unmodifiableSortedDifference(currentConfig.getNodeIds(), liveInConfigIds);
             final Set nonRetiredInConfigNotLiveIds = new TreeSet<>(inConfigNotLiveIds);
             nonRetiredInConfigNotLiveIds.removeAll(retiredNodeIds);
     
    diff --git a/server/src/main/java/org/elasticsearch/common/inject/internal/MoreTypes.java b/server/src/main/java/org/elasticsearch/common/inject/internal/MoreTypes.java
    index 7e4d54867fb30..8ae2248f9ea8b 100644
    --- a/server/src/main/java/org/elasticsearch/common/inject/internal/MoreTypes.java
    +++ b/server/src/main/java/org/elasticsearch/common/inject/internal/MoreTypes.java
    @@ -32,13 +32,11 @@
     import java.lang.reflect.TypeVariable;
     import java.lang.reflect.WildcardType;
     import java.util.Arrays;
    -import java.util.HashMap;
     import java.util.Map;
     import java.util.NoSuchElementException;
     import java.util.Objects;
     
     import static java.util.Collections.singleton;
    -import static java.util.Collections.unmodifiableMap;
     
     /**
      * Static methods for working with types that we aren't publishing in the
    @@ -53,20 +51,16 @@ public class MoreTypes {
         private MoreTypes() {
         }
     
    -    private static final Map, TypeLiteral> PRIMITIVE_TO_WRAPPER;
    -    static {
    -        Map, TypeLiteral> primitiveToWrapper = new HashMap<>();
    -        primitiveToWrapper.put(TypeLiteral.get(boolean.class), TypeLiteral.get(Boolean.class));
    -        primitiveToWrapper.put(TypeLiteral.get(byte.class), TypeLiteral.get(Byte.class));
    -        primitiveToWrapper.put(TypeLiteral.get(short.class), TypeLiteral.get(Short.class));
    -        primitiveToWrapper.put(TypeLiteral.get(int.class), TypeLiteral.get(Integer.class));
    -        primitiveToWrapper.put(TypeLiteral.get(long.class), TypeLiteral.get(Long.class));
    -        primitiveToWrapper.put(TypeLiteral.get(float.class), TypeLiteral.get(Float.class));
    -        primitiveToWrapper.put(TypeLiteral.get(double.class), TypeLiteral.get(Double.class));
    -        primitiveToWrapper.put(TypeLiteral.get(char.class), TypeLiteral.get(Character.class));
    -        primitiveToWrapper.put(TypeLiteral.get(void.class), TypeLiteral.get(Void.class));
    -        PRIMITIVE_TO_WRAPPER = unmodifiableMap(primitiveToWrapper);
    -    }
    +    private static final Map, TypeLiteral> PRIMITIVE_TO_WRAPPER = Map.of(
    +        TypeLiteral.get(boolean.class), TypeLiteral.get(Boolean.class),
    +        TypeLiteral.get(byte.class), TypeLiteral.get(Byte.class),
    +        TypeLiteral.get(short.class), TypeLiteral.get(Short.class),
    +        TypeLiteral.get(int.class), TypeLiteral.get(Integer.class),
    +        TypeLiteral.get(long.class), TypeLiteral.get(Long.class),
    +        TypeLiteral.get(float.class), TypeLiteral.get(Float.class),
    +        TypeLiteral.get(double.class), TypeLiteral.get(Double.class),
    +        TypeLiteral.get(char.class), TypeLiteral.get(Character.class),
    +        TypeLiteral.get(void.class), TypeLiteral.get(Void.class));
     
         /**
          * Returns an equivalent type that's safe for use in a key. The returned type will be free of
    diff --git a/server/src/main/java/org/elasticsearch/common/util/set/Sets.java b/server/src/main/java/org/elasticsearch/common/util/set/Sets.java
    index 950d6433510bf..9ab4ee59b4189 100644
    --- a/server/src/main/java/org/elasticsearch/common/util/set/Sets.java
    +++ b/server/src/main/java/org/elasticsearch/common/util/set/Sets.java
    @@ -96,13 +96,50 @@ public static  Set difference(Set left, Set right) {
          * @param    the type of the elements of the sets
          * @return the sorted relative complement of the left set with respect to the right set
          */
    -    public static  SortedSet sortedDifference(Set left, Set right) {
    +    public static  SortedSet sortedDifference(final Set left, final Set right) {
             Objects.requireNonNull(left);
             Objects.requireNonNull(right);
    -        return left.stream().filter(k -> !right.contains(k)).collect(new SortedSetCollector<>());
    +        return left.stream().filter(k -> right.contains(k) == false).collect(toSortedSet());
         }
     
    -    private static class SortedSetCollector implements Collector, SortedSet> {
    +    /**
    +     * The relative complement, or difference, of the specified left and right set, returned as a sorted set. Namely, the resulting set
    +     * contains all the elements that are in the left set but not in the right set, and the set is sorted using the natural ordering of
    +     * element type. Neither input is mutated by this operation, an entirely new set is returned. The resulting set is unmodifiable.
    +     *
    +     * @param left  the left set
    +     * @param right the right set
    +     * @param    the type of the elements of the sets
    +     * @return the unmodifiable sorted relative complement of the left set with respect to the right set
    +     */
    +    public static  Set unmodifiableSortedDifference(final Set left, final Set right) {
    +        Objects.requireNonNull(left);
    +        Objects.requireNonNull(right);
    +        return left.stream().filter(k -> right.contains(k) == false).collect(toUnmodifiableSortedSet());
    +    }
    +
    +    /**
    +     * Returns a {@link Collector} that accumulates the input elements into a sorted set.
    +     *
    +     * @param  the type of the input elements
    +     * @return a sorted set
    +     */
    +    public static  Collector, SortedSet> toSortedSet() {
    +        return new SortedSetCollector<>();
    +    }
    +
    +    /**
    +     * Returns a {@link Collector} that accumulates the input elements into a sorted set and finishes the resulting set into an
    +     * unmodifiable set. The resulting read-only view through the unmodifiable set is a sorted set.
    +     *
    +     * @param  the type of the input elements
    +     * @return an unmodifiable set where the underlying set is sorted
    +     */
    +    public static  Collector, Set> toUnmodifiableSortedSet() {
    +        return new UnmodifiableSortedSetCollector<>();
    +    }
    +
    +    abstract static class AbstractSortedSetCollector> implements Collector, R> {
     
             @Override
             public Supplier> supplier() {
    @@ -111,7 +148,7 @@ public Supplier> supplier() {
     
             @Override
             public BiConsumer, T> accumulator() {
    -            return (s, e) -> s.add(e);
    +            return SortedSet::add;
             }
     
             @Override
    @@ -122,13 +159,21 @@ public BinaryOperator> combiner() {
                 };
             }
     
    +        public abstract Function, R> finisher();
    +
    +        public abstract Set characteristics();
    +
    +    }
    +
    +    private static class SortedSetCollector extends AbstractSortedSetCollector> {
    +
             @Override
             public Function, SortedSet> finisher() {
                 return Function.identity();
             }
     
             static final Set CHARACTERISTICS =
    -                Collections.unmodifiableSet(EnumSet.of(Collector.Characteristics.IDENTITY_FINISH));
    +            Collections.unmodifiableSet(EnumSet.of(Characteristics.IDENTITY_FINISH));
     
             @Override
             public Set characteristics() {
    @@ -137,6 +182,20 @@ public Set characteristics() {
     
         }
     
    +    private static class UnmodifiableSortedSetCollector extends AbstractSortedSetCollector> {
    +
    +        @Override
    +        public Function, Set> finisher() {
    +            return Collections::unmodifiableSet;
    +        }
    +
    +        @Override
    +        public Set characteristics() {
    +            return Collections.emptySet();
    +        }
    +
    +    }
    +
         public static  Set union(Set left, Set right) {
             Objects.requireNonNull(left);
             Objects.requireNonNull(right);
    diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java
    index 74f451ab30cd2..a1854eaf27e6d 100644
    --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java
    +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java
    @@ -20,7 +20,6 @@
     package org.elasticsearch.rest.action.admin.indices;
     
     import com.carrotsearch.hppc.cursors.ObjectCursor;
    -
     import org.apache.logging.log4j.LogManager;
     import org.apache.logging.log4j.Logger;
     import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest;
    @@ -51,7 +50,6 @@
     import java.util.List;
     import java.util.Locale;
     import java.util.Set;
    -import java.util.SortedSet;
     import java.util.stream.Collectors;
     
     import static org.elasticsearch.rest.RestRequest.Method.GET;
    @@ -118,7 +116,7 @@ public RestResponse buildResponse(final GetMappingsResponse response, final XCon
                         }
                     }
     
    -                final SortedSet difference = Sets.sortedDifference(Arrays.stream(types).collect(Collectors.toSet()), typeNames);
    +                final Set difference = Sets.sortedDifference(Arrays.stream(types).collect(Collectors.toSet()), typeNames);
     
                     // now remove requested aliases that contain wildcards that are simple matches
                     final List matches = new ArrayList<>();
    diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java
    index 56053476fa22f..3468d3d30212b 100644
    --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java
    +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java
    @@ -42,6 +42,7 @@
     import java.io.Closeable;
     import java.io.IOException;
     import java.util.ArrayList;
    +import java.util.Arrays;
     import java.util.Collection;
     import java.util.Collections;
     import java.util.HashMap;
    @@ -55,9 +56,11 @@
     import java.util.concurrent.ScheduledThreadPoolExecutor;
     import java.util.concurrent.ThreadPoolExecutor;
     import java.util.concurrent.TimeUnit;
    +import java.util.function.Function;
     import java.util.stream.Collectors;
     
     import static java.util.Collections.unmodifiableMap;
    +import static java.util.Map.entry;
     
     public class ThreadPool implements Scheduler, Closeable {
     
    @@ -98,15 +101,8 @@ public String getType() {
                 this.type = type;
             }
     
    -        private static final Map TYPE_MAP;
    -
    -        static {
    -            Map typeMap = new HashMap<>();
    -            for (ThreadPoolType threadPoolType : ThreadPoolType.values()) {
    -                typeMap.put(threadPoolType.getType(), threadPoolType);
    -            }
    -            TYPE_MAP = Collections.unmodifiableMap(typeMap);
    -        }
    +        private static final Map TYPE_MAP =
    +            Arrays.stream(ThreadPoolType.values()).collect(Collectors.toUnmodifiableMap(ThreadPoolType::getType, Function.identity()));
     
             public static ThreadPoolType fromType(String type) {
                 ThreadPoolType threadPoolType = TYPE_MAP.get(type);
    @@ -117,28 +113,23 @@ public static ThreadPoolType fromType(String type) {
             }
         }
     
    -    public static final Map THREAD_POOL_TYPES;
    -
    -    static {
    -        HashMap map = new HashMap<>();
    -        map.put(Names.SAME, ThreadPoolType.DIRECT);
    -        map.put(Names.GENERIC, ThreadPoolType.SCALING);
    -        map.put(Names.LISTENER, ThreadPoolType.FIXED);
    -        map.put(Names.GET, ThreadPoolType.FIXED);
    -        map.put(Names.ANALYZE, ThreadPoolType.FIXED);
    -        map.put(Names.WRITE, ThreadPoolType.FIXED);
    -        map.put(Names.SEARCH, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE);
    -        map.put(Names.MANAGEMENT, ThreadPoolType.SCALING);
    -        map.put(Names.FLUSH, ThreadPoolType.SCALING);
    -        map.put(Names.REFRESH, ThreadPoolType.SCALING);
    -        map.put(Names.WARMER, ThreadPoolType.SCALING);
    -        map.put(Names.SNAPSHOT, ThreadPoolType.SCALING);
    -        map.put(Names.FORCE_MERGE, ThreadPoolType.FIXED);
    -        map.put(Names.FETCH_SHARD_STARTED, ThreadPoolType.SCALING);
    -        map.put(Names.FETCH_SHARD_STORE, ThreadPoolType.SCALING);
    -        map.put(Names.SEARCH_THROTTLED, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE);
    -        THREAD_POOL_TYPES = Collections.unmodifiableMap(map);
    -    }
    +    public static final Map THREAD_POOL_TYPES = Map.ofEntries(
    +        entry(Names.SAME, ThreadPoolType.DIRECT),
    +        entry(Names.GENERIC, ThreadPoolType.SCALING),
    +        entry(Names.LISTENER, ThreadPoolType.FIXED),
    +        entry(Names.GET, ThreadPoolType.FIXED),
    +        entry(Names.ANALYZE, ThreadPoolType.FIXED),
    +        entry(Names.WRITE, ThreadPoolType.FIXED),
    +        entry(Names.SEARCH, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE),
    +        entry(Names.MANAGEMENT, ThreadPoolType.SCALING),
    +        entry(Names.FLUSH, ThreadPoolType.SCALING),
    +        entry(Names.REFRESH, ThreadPoolType.SCALING),
    +        entry(Names.WARMER, ThreadPoolType.SCALING),
    +        entry(Names.SNAPSHOT, ThreadPoolType.SCALING),
    +        entry(Names.FORCE_MERGE, ThreadPoolType.FIXED),
    +        entry(Names.FETCH_SHARD_STARTED, ThreadPoolType.SCALING),
    +        entry(Names.FETCH_SHARD_STORE, ThreadPoolType.SCALING),
    +        entry(Names.SEARCH_THROTTLED, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE));
     
         private final Map executors;
     
    diff --git a/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java b/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java
    index f4337daf4346c..df6f264bdd58d 100644
    --- a/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java
    +++ b/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java
    @@ -25,6 +25,8 @@
     import java.util.HashSet;
     import java.util.Iterator;
     import java.util.Set;
    +import java.util.function.BiFunction;
    +import java.util.function.Consumer;
     import java.util.stream.Collectors;
     import java.util.stream.IntStream;
     
    @@ -42,9 +44,21 @@ public void testDifference() {
         }
     
         public void testSortedDifference() {
    +        runSortedDifferenceTest(Sets::sortedDifference, set -> {});
    +    }
    +
    +    public void testUnmodifiableSortedDifference() {
    +        runSortedDifferenceTest(
    +                // assert the resulting difference us unmodifiable
    +                Sets::unmodifiableSortedDifference, set -> expectThrows(UnsupportedOperationException.class, () -> set.add(randomInt())));
    +    }
    +
    +    private void runSortedDifferenceTest(
    +        final BiFunction, Set, Set> sortedDifference,
    +        final Consumer> asserter) {
             final int endExclusive = randomIntBetween(0, 256);
             final Tuple, Set> sets = randomSets(endExclusive);
    -        final Set difference = Sets.sortedDifference(sets.v1(), sets.v2());
    +        final Set difference = sortedDifference.apply(sets.v1(), sets.v2());
             assertDifference(endExclusive, sets, difference);
             final Iterator it = difference.iterator();
             if (it.hasNext()) {
    @@ -55,6 +69,7 @@ public void testSortedDifference() {
                     current = next;
                 }
             }
    +        asserter.accept(difference);
         }
     
         public void testIntersection() {
    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/Cron.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/Cron.java
    index 6391251bbcb26..cd1f999c1004e 100644
    --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/Cron.java
    +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/Cron.java
    @@ -13,7 +13,6 @@
     import java.io.IOException;
     import java.time.ZoneOffset;
     import java.util.Calendar;
    -import java.util.HashMap;
     import java.util.Iterator;
     import java.util.Locale;
     import java.util.Map;
    @@ -23,6 +22,7 @@
     import java.util.TimeZone;
     import java.util.TreeSet;
     
    +import static java.util.Map.entry;
     import static org.elasticsearch.xpack.core.watcher.support.Exceptions.illegalArgument;
     
     
    @@ -213,30 +213,28 @@ public class Cron implements ToXContentFragment {
         private static final Integer ALL_SPEC = ALL_SPEC_INT;
         private static final Integer NO_SPEC = NO_SPEC_INT;
     
    -    private static final Map monthMap = new HashMap<>(20);
    -    private static final Map dayMap = new HashMap<>(60);
    -    static {
    -        monthMap.put("JAN", 0);
    -        monthMap.put("FEB", 1);
    -        monthMap.put("MAR", 2);
    -        monthMap.put("APR", 3);
    -        monthMap.put("MAY", 4);
    -        monthMap.put("JUN", 5);
    -        monthMap.put("JUL", 6);
    -        monthMap.put("AUG", 7);
    -        monthMap.put("SEP", 8);
    -        monthMap.put("OCT", 9);
    -        monthMap.put("NOV", 10);
    -        monthMap.put("DEC", 11);
    -
    -        dayMap.put("SUN", 1);
    -        dayMap.put("MON", 2);
    -        dayMap.put("TUE", 3);
    -        dayMap.put("WED", 4);
    -        dayMap.put("THU", 5);
    -        dayMap.put("FRI", 6);
    -        dayMap.put("SAT", 7);
    -    }
    +    private static final Map MONTH_MAP = Map.ofEntries(
    +        entry("JAN", 0),
    +        entry("FEB", 1),
    +        entry("MAR", 2),
    +        entry("APR", 3),
    +        entry("MAY", 4),
    +        entry("JUN", 5),
    +        entry("JUL", 6),
    +        entry("AUG", 7),
    +        entry("SEP", 8),
    +        entry("OCT", 9),
    +        entry("NOV", 10),
    +        entry("DEC", 11));
    +
    +    private static final Map DAY_MAP = Map.of(
    +        "SUN", 1,
    +        "MON", 2,
    +        "TUE", 3,
    +        "WED", 4,
    +        "THU", 5,
    +        "FRI", 6,
    +        "SAT", 7);
     
         private final String expression;
     
    @@ -1413,7 +1411,7 @@ private int getNumericValue(String s, int i) {
         }
     
         private int getMonthNumber(String s) {
    -        Integer integer = monthMap.get(s);
    +        Integer integer = MONTH_MAP.get(s);
     
             if (integer == null) {
                 return -1;
    @@ -1423,7 +1421,7 @@ private int getMonthNumber(String s) {
         }
     
         private int getDayOfWeekNumber(String s) {
    -        Integer integer = dayMap.get(s);
    +        Integer integer = DAY_MAP.get(s);
     
             if (integer == null) {
                 return -1;
    diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java
    index 1c0d6dea5988d..d24b91d07cef0 100644
    --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java
    +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/persistence/DatafeedConfigProvider.java
    @@ -45,20 +45,18 @@
     import org.elasticsearch.search.SearchHit;
     import org.elasticsearch.search.builder.SearchSourceBuilder;
     import org.elasticsearch.xpack.core.ClientHelper;
    +import org.elasticsearch.xpack.core.action.util.ExpandedIdsMatcher;
     import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig;
     import org.elasticsearch.xpack.core.ml.datafeed.DatafeedUpdate;
     import org.elasticsearch.xpack.core.ml.job.config.Job;
     import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex;
     import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper;
     import org.elasticsearch.xpack.core.ml.utils.ToXContentParams;
    -import org.elasticsearch.xpack.core.action.util.ExpandedIdsMatcher;
     
     import java.io.IOException;
     import java.io.InputStream;
     import java.util.ArrayList;
     import java.util.Collection;
    -import java.util.Collections;
    -import java.util.HashMap;
     import java.util.HashSet;
     import java.util.List;
     import java.util.Map;
    @@ -86,13 +84,9 @@ public class DatafeedConfigProvider {
         private final Client client;
         private final NamedXContentRegistry xContentRegistry;
     
    -    public static final Map TO_XCONTENT_PARAMS;
    -    static {
    -        Map modifiable = new HashMap<>();
    -        modifiable.put(ToXContentParams.FOR_INTERNAL_STORAGE, "true");
    -        modifiable.put(ToXContentParams.INCLUDE_TYPE, "true");
    -        TO_XCONTENT_PARAMS = Collections.unmodifiableMap(modifiable);
    -    }
    +    public static final Map TO_XCONTENT_PARAMS = Map.of(
    +        ToXContentParams.FOR_INTERNAL_STORAGE, "true",
    +        ToXContentParams.INCLUDE_TYPE, "true");
     
         public DatafeedConfigProvider(Client client, NamedXContentRegistry xContentRegistry) {
             this.client = client;
    diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/NativeController.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/NativeController.java
    index 721e07721e32d..2dc86825a1209 100644
    --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/NativeController.java
    +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/NativeController.java
    @@ -17,8 +17,6 @@
     import java.io.OutputStream;
     import java.nio.charset.StandardCharsets;
     import java.time.Duration;
    -import java.util.Collections;
    -import java.util.HashMap;
     import java.util.List;
     import java.util.Map;
     import java.util.concurrent.TimeoutException;
    @@ -43,14 +41,7 @@ public class NativeController {
         private static final String START_COMMAND = "start";
         private static final String KILL_COMMAND = "kill";
     
    -    public static final Map UNKNOWN_NATIVE_CODE_INFO;
    -
    -    static {
    -        Map unknownInfo = new HashMap<>(2);
    -        unknownInfo.put("version", "N/A");
    -        unknownInfo.put("build_hash", "N/A");
    -        UNKNOWN_NATIVE_CODE_INFO = Collections.unmodifiableMap(unknownInfo);
    -    }
    +    public static final Map UNKNOWN_NATIVE_CODE_INFO = Map.of("version", "N/A", "build_hash", "N/A");
     
         private final CppLogMessageHandler cppLogHandler;
         private final OutputStream commandStream;
    diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/DomainSplitFunction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/DomainSplitFunction.java
    index 62ee074aecdfa..452c1f5e9ed81 100644
    --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/DomainSplitFunction.java
    +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/DomainSplitFunction.java
    @@ -9,16 +9,18 @@
     import org.elasticsearch.common.io.Streams;
     import org.elasticsearch.common.logging.DeprecationLogger;
     
    -import java.io.InputStream;
    +import java.io.IOException;
    +import java.io.UncheckedIOException;
     import java.security.AccessController;
     import java.security.PrivilegedAction;
     import java.util.Arrays;
    -import java.util.Collections;
    -import java.util.HashMap;
     import java.util.List;
     import java.util.Locale;
     import java.util.Map;
     import java.util.StringJoiner;
    +import java.util.stream.Collectors;
    +
    +import static java.util.Map.entry;
     
     public final class DomainSplitFunction {
     
    @@ -28,68 +30,60 @@ public final class DomainSplitFunction {
         private static final int MAX_DOMAIN_PART_LENGTH = 63;
     
         private static final Map exact;
    -    private static final Map under;
    -    private static final Map excluded;
    -    static {
    -        Map exactMap = new HashMap<>(2048);
    -
    -        String exactResourceName = "org/elasticsearch/xpack/ml/transforms/exact.properties";
    +    private static final Map under = Map.ofEntries(
    +        entry("bd", "i"),
    +        entry("np", "i"),
    +        entry("jm", "i"),
    +        entry("fj", "i"),
    +        entry("fk", "i"),
    +        entry("ye", "i"),
    +        entry("sch.uk", "i"),
    +        entry("bn", "i"),
    +        entry("kitakyushu.jp", "i"),
    +        entry("kobe.jp", "i"),
    +        entry("ke", "i"),
    +        entry("sapporo.jp", "i"),
    +        entry("kh", "i"),
    +        entry("mm", "i"),
    +        entry("il", "i"),
    +        entry("yokohama.jp", "i"),
    +        entry("ck", "i"),
    +        entry("nagoya.jp", "i"),
    +        entry("sendai.jp", "i"),
    +        entry("kw", "i"),
    +        entry("er", "i"),
    +        entry("mz", "i"),
    +        entry("platform.sh", "p"),
    +        entry("gu", "i"),
    +        entry("nom.br", "i"),
    +        entry("zm", "i"),
    +        entry("pg", "i"),
    +        entry("ni", "i"),
    +        entry("kawasaki.jp", "i"),
    +        entry("zw", "i"));
    +
    +    private static final Map excluded =
    +        Map.of(
    +            "city.yokohama.jp", "i",
    +            "teledata.mz", "i",
    +            "city.kobe.jp", "i",
    +            "city.sapporo.jp", "i",
    +            "city.kawasaki.jp", "i",
    +            "city.nagoya.jp", "i",
    +            "www.ck", "i",
    +            "city.sendai.jp", "i",
    +            "city.kitakyushu.jp", "i");
     
    -        try (InputStream resource = DomainSplitFunction.class.getClassLoader().getResourceAsStream(exactResourceName)) {
    -            List lines = Streams.readAllLines(resource);
    -            for (String line : lines) {
    -                String[] split = line.split("=");
    -                exactMap.put(split[0].trim(), split[1].trim());
    -            }
    -        } catch (Exception e) {
    -            throw new RuntimeException("Could not load DomainSplit resource", e);
    +    static {
    +        try (var stream =
    +                 DomainSplitFunction.class.getClassLoader().getResourceAsStream("org/elasticsearch/xpack/ml/transforms/exact.properties")) {
    +            exact = Streams.readAllLines(stream)
    +                .stream()
    +                .map(line -> line.split("="))
    +                .collect(Collectors.toUnmodifiableMap(split -> split[0], split -> split[1]));
    +        } catch (final IOException e) {
    +            throw new UncheckedIOException(e);
             }
    -        exact = Collections.unmodifiableMap(exactMap);
    -
    -        Map underMap = new HashMap<>(30);
    -        underMap.put("bd", "i");
    -        underMap.put("np", "i");
    -        underMap.put("jm", "i");
    -        underMap.put("fj", "i");
    -        underMap.put("fk", "i");
    -        underMap.put("ye", "i");
    -        underMap.put("sch.uk", "i");
    -        underMap.put("bn", "i");
    -        underMap.put("kitakyushu.jp", "i");
    -        underMap.put("kobe.jp", "i");
    -        underMap.put("ke", "i");
    -        underMap.put("sapporo.jp", "i");
    -        underMap.put("kh", "i");
    -        underMap.put("mm", "i");
    -        underMap.put("il", "i");
    -        underMap.put("yokohama.jp", "i");
    -        underMap.put("ck", "i");
    -        underMap.put("nagoya.jp", "i");
    -        underMap.put("sendai.jp", "i");
    -        underMap.put("kw", "i");
    -        underMap.put("er", "i");
    -        underMap.put("mz", "i");
    -        underMap.put("platform.sh", "p");
    -        underMap.put("gu", "i");
    -        underMap.put("nom.br", "i");
    -        underMap.put("zm", "i");
    -        underMap.put("pg", "i");
    -        underMap.put("ni", "i");
    -        underMap.put("kawasaki.jp", "i");
    -        underMap.put("zw", "i");
    -        under = Collections.unmodifiableMap(underMap);
    -
    -        Map excludedMap = new HashMap<>(9);
    -        excludedMap.put("city.yokohama.jp", "i");
    -        excludedMap.put("teledata.mz", "i");
    -        excludedMap.put("city.kobe.jp", "i");
    -        excludedMap.put("city.sapporo.jp", "i");
    -        excludedMap.put("city.kawasaki.jp", "i");
    -        excludedMap.put("city.nagoya.jp", "i");
    -        excludedMap.put("www.ck", "i");
    -        excludedMap.put("city.sendai.jp", "i");
    -        excludedMap.put("city.kitakyushu.jp", "i");
    -        excluded = Collections.unmodifiableMap(excludedMap);
         }
     
         private DomainSplitFunction() {}
    diff --git a/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTestCase.java b/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTestCase.java
    index 6754b1acb9347..8245ecf36a627 100644
    --- a/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTestCase.java
    +++ b/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTestCase.java
    @@ -21,13 +21,13 @@
     import org.junit.BeforeClass;
     
     import javax.security.auth.Subject;
    +
     import java.io.IOException;
     import java.nio.file.Path;
     import java.security.AccessController;
     import java.security.PrivilegedActionException;
     import java.security.PrivilegedExceptionAction;
     import java.util.ArrayList;
    -import java.util.HashSet;
     import java.util.List;
     import java.util.Locale;
     import java.util.Set;
    @@ -54,37 +54,33 @@ public abstract class KerberosTestCase extends ESTestCase {
         protected SimpleKdcLdapServer simpleKdcLdapServer;
     
         private static Locale restoreLocale;
    -    private static Set unsupportedLocaleLanguages;
    -
    -    static {
    -        unsupportedLocaleLanguages = new HashSet<>();
    -        /*
    -         * arabic and other languages have problem due to handling of GeneralizedTime in
    -         * SimpleKdcServer For more look at :
    -         * org.apache.kerby.asn1.type.Asn1GeneralizedTime#toBytes()
    -         */
    -        unsupportedLocaleLanguages.add("ar");
    -        unsupportedLocaleLanguages.add("ja");
    -        unsupportedLocaleLanguages.add("th");
    -        unsupportedLocaleLanguages.add("hi");
    -        unsupportedLocaleLanguages.add("uz");
    -        unsupportedLocaleLanguages.add("fa");
    -        unsupportedLocaleLanguages.add("ks");
    -        unsupportedLocaleLanguages.add("ckb");
    -        unsupportedLocaleLanguages.add("ne");
    -        unsupportedLocaleLanguages.add("dz");
    -        unsupportedLocaleLanguages.add("mzn");
    -        unsupportedLocaleLanguages.add("mr");
    -        unsupportedLocaleLanguages.add("as");
    -        unsupportedLocaleLanguages.add("bn");
    -        unsupportedLocaleLanguages.add("lrc");
    -        unsupportedLocaleLanguages.add("my");
    -        unsupportedLocaleLanguages.add("ps");
    -        unsupportedLocaleLanguages.add("ur");
    -        unsupportedLocaleLanguages.add("pa");
    -        unsupportedLocaleLanguages.add("ig");
    -        unsupportedLocaleLanguages.add("sd");
    -    }
    +
    +    /*
    +     * Arabic and other language have problems due to handling of generalized time in SimpleKdcServer. For more, look at
    +     * org.apache.kerby.asn1.type.Asn1GeneralizedTime#toBytes
    +     */
    +    private static Set UNSUPPORTED_LOCALE_LANGUAGES = Set.of(
    +        "ar",
    +        "ja",
    +        "th",
    +        "hi",
    +        "uz",
    +        "fa",
    +        "ks",
    +        "ckb",
    +        "ne",
    +        "dz",
    +        "mzn",
    +        "mr",
    +        "as",
    +        "bn",
    +        "lrc",
    +        "my",
    +        "ps",
    +        "ur",
    +        "pa",
    +        "ig",
    +        "sd");
     
         @BeforeClass
         public static void setupKerberos() throws Exception {
    @@ -106,7 +102,7 @@ public static void restoreLocale() {
         }
     
         private static boolean isLocaleUnsupported() {
    -        return unsupportedLocaleLanguages.contains(Locale.getDefault().getLanguage());
    +        return UNSUPPORTED_LOCALE_LANGUAGES.contains(Locale.getDefault().getLanguage());
         }
     
         @Before
    
    From c1f83cec9c6623048c0bff73c2f9753338a2e1ca Mon Sep 17 00:00:00 2001
    From: Ryan Ernst 
    Date: Fri, 19 Apr 2019 12:34:52 -0700
    Subject: [PATCH 127/260] Remove /var/run/elasticsearch from packages (#41102)
    
    The pid dir for both systemd and init.d is already managed by those
    respective systems (tmpfiles.d and the init script, respectively). Since
    the /var/run dir is often mounted as tmpfs, it does not make sense to
    have the elasticsearch pid dir added by the package installation. This
    commit removes that empty dir from deb and rpm.
    ---
     distribution/packages/build.gradle                              | 2 --
     .../main/java/org/elasticsearch/packaging/util/Packages.java    | 2 --
     qa/vagrant/src/test/resources/packaging/utils/packages.bash     | 2 --
     3 files changed, 6 deletions(-)
    
    diff --git a/distribution/packages/build.gradle b/distribution/packages/build.gradle
    index 11c56bc66e09a..af1479d360cd6 100644
    --- a/distribution/packages/build.gradle
    +++ b/distribution/packages/build.gradle
    @@ -81,7 +81,6 @@ void addProcessFilesTask(String type, boolean oss, boolean jdk) {
     
         doLast {
           // create empty dirs, we set the permissions when configuring the packages
    -      mkdir "${packagingFiles}/var/run/elasticsearch"
           mkdir "${packagingFiles}/var/log/elasticsearch"
           mkdir "${packagingFiles}/var/lib/elasticsearch"
           mkdir "${packagingFiles}/usr/share/elasticsearch/plugins"
    @@ -253,7 +252,6 @@ Closure commonPackageConfig(String type, boolean oss, boolean jdk) {
             dirMode mode
           }
         }
    -    copyEmptyDir('/var/run/elasticsearch', 'elasticsearch', 'elasticsearch', 0755)
         copyEmptyDir('/var/log/elasticsearch', 'elasticsearch', 'elasticsearch', 02750)
         copyEmptyDir('/var/lib/elasticsearch', 'elasticsearch', 'elasticsearch', 02750)
         copyEmptyDir('/usr/share/elasticsearch/plugins', 'root', 'root', 0755)
    diff --git a/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/Packages.java b/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/Packages.java
    index c5dcc34af882f..70ac89dc3b7f5 100644
    --- a/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/Packages.java
    +++ b/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/Packages.java
    @@ -173,8 +173,6 @@ private static void verifyOssInstallation(Installation es, Distribution distribu
                 es.modules
             ).forEach(dir -> assertThat(dir, file(Directory, "root", "root", p755)));
     
    -        assertThat(es.pidDir, file(Directory, "elasticsearch", "elasticsearch", p755));
    -
             Stream.of(
                 es.data,
                 es.logs
    diff --git a/qa/vagrant/src/test/resources/packaging/utils/packages.bash b/qa/vagrant/src/test/resources/packaging/utils/packages.bash
    index a38f36c3d14c6..5df432c35b37d 100644
    --- a/qa/vagrant/src/test/resources/packaging/utils/packages.bash
    +++ b/qa/vagrant/src/test/resources/packaging/utils/packages.bash
    @@ -47,7 +47,6 @@ export_elasticsearch_paths() {
         export ESCONFIG="/etc/elasticsearch"
         export ESDATA="/var/lib/elasticsearch"
         export ESLOG="/var/log/elasticsearch"
    -    export ESPIDDIR="/var/run/elasticsearch"
         export ESENVFILE=$(env_file)
         export PACKAGE_NAME=${PACKAGE_NAME:-"elasticsearch-oss"}
     }
    @@ -132,7 +131,6 @@ verify_package_installation() {
         assert_file "$ESLOG" d elasticsearch elasticsearch 2750
         assert_file "$ESPLUGINS" d root root 755
         assert_file "$ESMODULES" d root root 755
    -    assert_file "$ESPIDDIR" d elasticsearch elasticsearch 755
         assert_file "$ESHOME/NOTICE.txt" f root root 644
         assert_file "$ESHOME/README.textile" f root root 644
     
    
    From 8fe7568f732f6561d9e015c276fd0fca162ea9bb Mon Sep 17 00:00:00 2001
    From: clement-tourriere 
    Date: Fri, 19 Apr 2019 22:17:00 +0200
    Subject: [PATCH 128/260] Add ignore_above in ICUCollationKeywordFieldMapper
     (#40414)
    
    Add the possibility to use ignore_above parameter in ICUCollationKeywordFieldMapper.
    
    Close #40413
    ---
     docs/plugins/analysis-icu.asciidoc            |  8 ++++
     .../ICUCollationKeywordFieldMapper.java       | 29 ++++++++++--
     .../ICUCollationKeywordFieldMapperTests.java  | 47 +++++++++++++++++++
     3 files changed, 81 insertions(+), 3 deletions(-)
    
    diff --git a/docs/plugins/analysis-icu.asciidoc b/docs/plugins/analysis-icu.asciidoc
    index 51be1907c9869..9dc889674852a 100644
    --- a/docs/plugins/analysis-icu.asciidoc
    +++ b/docs/plugins/analysis-icu.asciidoc
    @@ -413,6 +413,14 @@ The following parameters are accepted by `icu_collation_keyword` fields:
         Accepts a string value which is substituted for any explicit `null`
         values.  Defaults to `null`, which means the field is treated as missing.
     
    +<>::
    +
    +    Strings longer than the `ignore_above` setting will be ignored.
    +    Checking is performed on the original string before the collation.
    +    The `ignore_above` setting can be updated on existing fields
    +    using the {ref}/indices-put-mapping.html[PUT mapping API].
    +    By default, there is no limit and all values will be indexed.
    +
     `store`::
     
         Whether the field value should be stored and retrievable separately from
    diff --git a/plugins/analysis-icu/src/main/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapper.java b/plugins/analysis-icu/src/main/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapper.java
    index a228283527d66..4b29d314356df 100644
    --- a/plugins/analysis-icu/src/main/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapper.java
    +++ b/plugins/analysis-icu/src/main/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapper.java
    @@ -70,6 +70,7 @@ public static class Defaults {
             }
     
             public static final String NULL_VALUE = null;
    +        public static final int IGNORE_ABOVE = Integer.MAX_VALUE;
         }
     
         public static final class CollationFieldType extends StringFieldType {
    @@ -226,6 +227,7 @@ public static class Builder extends FieldMapper.Builder field
                 }
             }
     
    -        if (value == null) {
    +        if (value == null || value.length() > ignoreAbove) {
                 return;
             }
     
    diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperTests.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperTests.java
    index 103098d5a4620..058bd7dbc8935 100644
    --- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperTests.java
    +++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperTests.java
    @@ -403,4 +403,51 @@ public void testUpdateCollator() throws IOException {
             assertEquals("Can't merge because of conflicts: [Cannot update language setting for [" + FIELD_TYPE
                 + "], Cannot update strength setting for [" + FIELD_TYPE + "]]", e.getMessage());
         }
    +
    +
    +    public void testIgnoreAbove() throws IOException {
    +        String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
    +            .startObject("properties").startObject("field").field("type", FIELD_TYPE)
    +            .field("ignore_above", 5).endObject().endObject()
    +            .endObject().endObject());
    +
    +        DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
    +
    +        assertEquals(mapping, mapper.mappingSource().toString());
    +
    +        ParsedDocument doc = mapper.parse(new SourceToParse("test", "type", "1", BytesReference
    +            .bytes(XContentFactory.jsonBuilder()
    +                .startObject()
    +                .field("field", "elk")
    +                .endObject()),
    +            XContentType.JSON));
    +
    +        IndexableField[] fields = doc.rootDoc().getFields("field");
    +        assertEquals(2, fields.length);
    +
    +        doc = mapper.parse(new SourceToParse("test", "type", "1", BytesReference
    +            .bytes(XContentFactory.jsonBuilder()
    +                .startObject()
    +                .field("field", "elasticsearch")
    +                .endObject()),
    +            XContentType.JSON));
    +
    +        fields = doc.rootDoc().getFields("field");
    +        assertEquals(0, fields.length);
    +    }
    +
    +    public void testUpdateIgnoreAbove() throws IOException {
    +        String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
    +            .startObject("properties").startObject("field").field("type", FIELD_TYPE).endObject().endObject()
    +            .endObject().endObject());
    +
    +        indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE);
    +
    +        mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type")
    +            .startObject("properties").startObject("field").field("type", FIELD_TYPE)
    +            .field("ignore_above", 5).endObject().endObject()
    +            .endObject().endObject());
    +        indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE);
    +    }
    +
     }
    
    From f22f512399977cec575658023df3f3d776e78770 Mon Sep 17 00:00:00 2001
    From: Jason Tedor 
    Date: Fri, 19 Apr 2019 17:14:49 -0400
    Subject: [PATCH 129/260] Limit the number of forks getting Java versions
     (#41386)
    
    To reduce configuration time, we fork some threads to compute the Java
    version for the various configured Javas. However, as the number of
    JAVA${N}_HOME variable increases, the current implementation creates as
    many threads as there are such variables, which could be more than the
    number of physical cores on the machine. It is not likely that we would
    see benefits to trying to execute all of these once beyond the number of
    physical cores (maybe simultaneous multi-threading helps though, who
    knows. Therefore, this commit limits the parallelization here to the
    number number of physical cores.
    ---
     .../org/elasticsearch/gradle/BuildPlugin.groovy    | 14 +++++++++-----
     1 file changed, 9 insertions(+), 5 deletions(-)
    
    diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    index 7db0be891842b..2d398650ddead 100644
    --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy
    @@ -197,9 +197,10 @@ class BuildPlugin implements Plugin {
                     }
                 }
     
    +            final int numberOfPhysicalCores = numberOfPhysicalCores(project.rootProject)
                 if (javaVersions.isEmpty() == false) {
     
    -                ExecutorService exec = Executors.newFixedThreadPool(javaVersions.size())
    +                ExecutorService exec = Executors.newFixedThreadPool(numberOfPhysicalCores)
                     Set> results = new HashSet<>()
     
                     javaVersions.entrySet().stream()
    @@ -247,7 +248,7 @@ class BuildPlugin implements Plugin {
                 project.rootProject.ext.inFipsJvm = inFipsJvm
                 project.rootProject.ext.gradleJavaVersion = JavaVersion.toVersion(gradleJavaVersion)
                 project.rootProject.ext.java9Home = "${-> findJavaHome("9")}"
    -            project.rootProject.ext.defaultParallel = findDefaultParallel(project.rootProject)
    +            project.rootProject.ext.defaultParallel = numberOfPhysicalCores
             }
     
             project.targetCompatibility = project.rootProject.ext.minimumRuntimeVersion
    @@ -1013,7 +1014,7 @@ class BuildPlugin implements Plugin {
             }
         }
     
    -    private static int findDefaultParallel(Project project) {
    +    private static int numberOfPhysicalCores(Project project) {
             if (project.file("/proc/cpuinfo").exists()) {
                 // Count physical cores on any Linux distro ( don't count hyper-threading )
                 Map socketToCore = [:]
    @@ -1026,7 +1027,7 @@ class BuildPlugin implements Plugin {
                         if (name == "physical id") {
                             currentID = value
                         }
    -                    // Number  of cores not including hyper-threading
    +                    // number of cores not including hyper-threading
                         if (name == "cpu cores") {
                             assert currentID.isEmpty() == false
                             socketToCore[currentID] = Integer.valueOf(value)
    @@ -1044,8 +1045,11 @@ class BuildPlugin implements Plugin {
                     standardOutput = stdout
                 }
                 return Integer.parseInt(stdout.toString('UTF-8').trim())
    +        } else {
    +            // guess that it is half the number of processors (which is wrong on systems that do not have simultaneous multi-threading)
    +            // TODO: implement this on Windows
    +            return Runtime.getRuntime().availableProcessors() / 2
             }
    -        return Runtime.getRuntime().availableProcessors() / 2
         }
     
         private static configurePrecommit(Project project) {
    
    From 2e7520246b402c0b37725ddb0df10347583c907a Mon Sep 17 00:00:00 2001
    From: Jason Tedor 
    Date: Fri, 19 Apr 2019 17:23:15 -0400
    Subject: [PATCH 130/260] Fix reference to ignore_above from analyis-icu docs
    
    This commit fixes a reference to the docs for ignore_above from the
    analysis-icu plugin docs.
    ---
     docs/plugins/analysis-icu.asciidoc | 2 +-
     1 file changed, 1 insertion(+), 1 deletion(-)
    
    diff --git a/docs/plugins/analysis-icu.asciidoc b/docs/plugins/analysis-icu.asciidoc
    index 9dc889674852a..b6299139992d1 100644
    --- a/docs/plugins/analysis-icu.asciidoc
    +++ b/docs/plugins/analysis-icu.asciidoc
    @@ -413,7 +413,7 @@ The following parameters are accepted by `icu_collation_keyword` fields:
         Accepts a string value which is substituted for any explicit `null`
         values.  Defaults to `null`, which means the field is treated as missing.
     
    -<>::
    +{ref}/ignore-above.html[`ignore_above`]::
     
         Strings longer than the `ignore_above` setting will be ignored.
         Checking is performed on the original string before the collation.
    
    From b6b98f72ce8a6f6903e3c9fc26d3f9e0c3c6e34b Mon Sep 17 00:00:00 2001
    From: Joe Zack 
    Date: Sat, 20 Apr 2019 04:32:35 -0400
    Subject: [PATCH 131/260] Fix discovery config in docker-compose docs (#41394)
    
    Today's `docker-compose` docs are missing the `discovery.seed_nodes` config on
    one of the nodes. With today's configuration the cluster can still form the
    first time it is started, because `cluster.initial_master_nodes` requires both
    nodes to bootstrap the cluster which ensures that each discover the other.
    However if `es02` is elected master it will remove `es01` from the voting
    configuration and then when restarted it will form a cluster on its own without
    needing to do any discovery. Meanwhile `es01` doesn't know how to find `es02`
    after a restart so will be unable to join this cluster.
    
    This commit fixes this by adding the missing configuration.
    ---
     docs/reference/setup/install/docker.asciidoc | 1 +
     1 file changed, 1 insertion(+)
    
    diff --git a/docs/reference/setup/install/docker.asciidoc b/docs/reference/setup/install/docker.asciidoc
    index 9d03edb9e7e9d..76112f220dc3a 100644
    --- a/docs/reference/setup/install/docker.asciidoc
    +++ b/docs/reference/setup/install/docker.asciidoc
    @@ -169,6 +169,7 @@ services:
         container_name: es01
         environment:
           - node.name=es01
    +      - discovery.seed_hosts=es02
           - cluster.initial_master_nodes=es01,es02
           - cluster.name=docker-cluster
           - bootstrap.memory_lock=true
    
    From 8d57727f9eeffc80b95109b82ef6f0c136168daf Mon Sep 17 00:00:00 2001
    From: Jason Tedor 
    Date: Sat, 20 Apr 2019 08:21:10 -0400
    Subject: [PATCH 132/260] Reduce security permissions in CCR plugin (#41391)
    
    It looks like these permissions were copy/pasted from another plugin yet
    almost none of these permissions are needed for the CCR plugin. This
    commit removes all these unneeded permissions from the CCR plugin.
    ---
     .../plugin-metadata/plugin-security.policy    | 46 -------------------
     1 file changed, 46 deletions(-)
    
    diff --git a/x-pack/plugin/ccr/src/main/plugin-metadata/plugin-security.policy b/x-pack/plugin/ccr/src/main/plugin-metadata/plugin-security.policy
    index 45d92fd2b8aa1..16701ab74d8c9 100644
    --- a/x-pack/plugin/ccr/src/main/plugin-metadata/plugin-security.policy
    +++ b/x-pack/plugin/ccr/src/main/plugin-metadata/plugin-security.policy
    @@ -1,50 +1,4 @@
     grant {
    -  // needed because of problems in unbound LDAP library
    -  permission java.util.PropertyPermission "*", "read,write";
    -
    -  // required to configure the custom mailcap for watcher
    -  permission java.lang.RuntimePermission "setFactory";
    -
    -  // needed when sending emails for javax.activation
    -  // otherwise a classnotfound exception is thrown due to trying
    -  // to load the class with the application class loader
    -  permission java.lang.RuntimePermission "setContextClassLoader";
    -  permission java.lang.RuntimePermission "getClassLoader";
    -  // TODO: remove use of this jar as soon as possible!!!!
    -  permission java.lang.RuntimePermission "accessClassInPackage.com.sun.activation.registries";
    -
    -  // bouncy castle
    -  permission java.security.SecurityPermission "putProviderProperty.BC";
    -
    -  // needed for x-pack security extension
    -  permission java.security.SecurityPermission "createPolicy.JavaPolicy";
    -  permission java.security.SecurityPermission "getPolicy";
    -  permission java.security.SecurityPermission "setPolicy";
    -
       // needed for multiple server implementations used in tests
       permission java.net.SocketPermission "*", "accept,connect";
    -
    -  // needed for Windows named pipes in machine learning
    -  permission java.io.FilePermission "\\\\.\\pipe\\*", "read,write";
     };
    -
    -grant codeBase "${codebase.netty-common}" {
    -   // for reading the system-wide configuration for the backlog of established sockets
    -   permission java.io.FilePermission "/proc/sys/net/core/somaxconn", "read";
    -};
    -
    -grant codeBase "${codebase.netty-transport}" {
    -   // Netty NioEventLoop wants to change this, because of https://bugs.openjdk.java.net/browse/JDK-6427854
    -   // the bug says it only happened rarely, and that its fixed, but apparently it still happens rarely!
    -   permission java.util.PropertyPermission "sun.nio.ch.bugLevel", "write";
    -};
    -
    -grant codeBase "${codebase.elasticsearch-rest-client}" {
    -  // rest client uses system properties which gets the default proxy
    -  permission java.net.NetPermission "getProxySelector";
    -};
    -
    -grant codeBase "${codebase.httpasyncclient}" {
    -  // rest client uses system properties which gets the default proxy
    -  permission java.net.NetPermission "getProxySelector";
    -};
    \ No newline at end of file
    
    From fcfab3ed261545e7e7a4a2dda0f9bfe4c4c72ada Mon Sep 17 00:00:00 2001
    From: Jason Tedor 
    Date: Sat, 20 Apr 2019 08:22:56 -0400
    Subject: [PATCH 133/260] Remove script engine from X-Pack plugin (#41387)
    
    The X-Pack plugin implements ScriptEngine yet it does not actually
    implement any of the methods on the interface, effectively making this a
    no-op. This commit removes this interface from the X-Pack plugin.
    ---
     .../main/java/org/elasticsearch/xpack/core/XPackPlugin.java  | 5 ++---
     .../xpack/core/LocalStateCompositeXPackPlugin.java           | 1 -
     2 files changed, 2 insertions(+), 4 deletions(-)
    
    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java
    index 1a26e8aa88c17..2038b35b4e6e0 100644
    --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java
    +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java
    @@ -5,8 +5,8 @@
      */
     package org.elasticsearch.xpack.core;
     
    -import org.apache.logging.log4j.Logger;
     import org.apache.logging.log4j.LogManager;
    +import org.apache.logging.log4j.Logger;
     import org.apache.lucene.util.SetOnce;
     import org.elasticsearch.SpecialPermission;
     import org.elasticsearch.Version;
    @@ -49,7 +49,6 @@
     import org.elasticsearch.plugins.EnginePlugin;
     import org.elasticsearch.plugins.ExtensiblePlugin;
     import org.elasticsearch.plugins.RepositoryPlugin;
    -import org.elasticsearch.plugins.ScriptPlugin;
     import org.elasticsearch.repositories.Repository;
     import org.elasticsearch.rest.RestController;
     import org.elasticsearch.rest.RestHandler;
    @@ -86,7 +85,7 @@
     import java.util.stream.Collectors;
     import java.util.stream.StreamSupport;
     
    -public class XPackPlugin extends XPackClientPlugin implements ScriptPlugin, ExtensiblePlugin, RepositoryPlugin, EnginePlugin {
    +public class XPackPlugin extends XPackClientPlugin implements ExtensiblePlugin, RepositoryPlugin, EnginePlugin {
     
         private static Logger logger = LogManager.getLogger(XPackPlugin.class);
         private static DeprecationLogger deprecationLogger = new DeprecationLogger(logger);
    diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java
    index fe56bfbb785bf..3f8b279e5016c 100644
    --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java
    +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java
    @@ -256,7 +256,6 @@ public Settings additionalSettings() {
         @Override
         public List> getContexts() {
             List> contexts = new ArrayList<>();
    -        contexts.addAll(super.getContexts());
             filterPlugins(ScriptPlugin.class).stream().forEach(p -> contexts.addAll(p.getContexts()));
             return contexts;
         }
    
    From 035448c52a476969db8cb0ec2271023d76fed05a Mon Sep 17 00:00:00 2001
    From: Ignacio Vera 
    Date: Mon, 22 Apr 2019 10:08:41 +0200
    Subject: [PATCH 134/260] mute IntervalQueryBuilderTests#testMustRewrite
     (#41403)
    
    ---
     .../index/query/IntervalQueryBuilderTests.java              | 6 ++++++
     1 file changed, 6 insertions(+)
    
    diff --git a/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java
    index d01cc174c3025..9f9a8f1d20716 100644
    --- a/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java
    +++ b/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java
    @@ -383,5 +383,11 @@ public  FactoryType compile(Script script, ScriptContext
    Date: Mon, 22 Apr 2019 15:25:40 +0300
    Subject: [PATCH 135/260] [ML] Refactor autodetect service into its own class
     (#41378)
    
    This also improves aims to improve the corresponding unit tests
    with regard to readability and maintainability.
    ---
     .../autodetect/AutodetectProcessManager.java  | 104 --------
     .../AutodetectWorkerExecutorService.java      | 122 +++++++++
     .../AutodetectProcessManagerTests.java        | 247 +++++-------------
     .../AutodetectWorkerExecutorServiceTests.java | 100 +++++++
     4 files changed, 289 insertions(+), 284 deletions(-)
     create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorService.java
     create mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorServiceTests.java
    
    diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java
    index 1e35530fe1735..9a5d556e1c3f4 100644
    --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java
    +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java
    @@ -16,14 +16,12 @@
     import org.elasticsearch.cluster.ClusterStateListener;
     import org.elasticsearch.cluster.service.ClusterService;
     import org.elasticsearch.common.CheckedConsumer;
    -import org.elasticsearch.common.SuppressForbidden;
     import org.elasticsearch.common.collect.Tuple;
     import org.elasticsearch.common.settings.Setting;
     import org.elasticsearch.common.settings.Settings;
     import org.elasticsearch.common.unit.ByteSizeUnit;
     import org.elasticsearch.common.unit.ByteSizeValue;
     import org.elasticsearch.common.util.concurrent.AbstractRunnable;
    -import org.elasticsearch.common.util.concurrent.EsExecutors;
     import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
     import org.elasticsearch.common.util.concurrent.ThreadContext;
     import org.elasticsearch.common.xcontent.NamedXContentRegistry;
    @@ -77,20 +75,13 @@
     import java.nio.file.Path;
     import java.time.Duration;
     import java.time.ZonedDateTime;
    -import java.util.ArrayList;
     import java.util.Date;
     import java.util.Iterator;
    -import java.util.List;
     import java.util.Locale;
     import java.util.Optional;
    -import java.util.concurrent.AbstractExecutorService;
    -import java.util.concurrent.BlockingQueue;
     import java.util.concurrent.ConcurrentHashMap;
     import java.util.concurrent.ConcurrentMap;
    -import java.util.concurrent.CountDownLatch;
     import java.util.concurrent.ExecutorService;
    -import java.util.concurrent.LinkedBlockingQueue;
    -import java.util.concurrent.TimeUnit;
     import java.util.function.BiConsumer;
     import java.util.function.Consumer;
     
    @@ -791,99 +782,4 @@ public void clusterChanged(ClusterChangedEvent event) {
             upgradeInProgress = MlMetadata.getMlMetadata(event.state()).isUpgradeMode();
         }
     
    -    /*
    -     * The autodetect native process can only handle a single operation at a time. In order to guarantee that, all
    -     * operations are initially added to a queue and a worker thread from ml autodetect threadpool will process each
    -     * operation at a time.
    -     */
    -    static class AutodetectWorkerExecutorService extends AbstractExecutorService {
    -
    -        private final ThreadContext contextHolder;
    -        private final CountDownLatch awaitTermination = new CountDownLatch(1);
    -        private final BlockingQueue queue = new LinkedBlockingQueue<>(100);
    -
    -        private volatile boolean running = true;
    -
    -        @SuppressForbidden(reason = "properly rethrowing errors, see EsExecutors.rethrowErrors")
    -        AutodetectWorkerExecutorService(ThreadContext contextHolder) {
    -            this.contextHolder = contextHolder;
    -        }
    -
    -        @Override
    -        public void shutdown() {
    -            running = false;
    -        }
    -
    -        @Override
    -        public List shutdownNow() {
    -            throw new UnsupportedOperationException("not supported");
    -        }
    -
    -        @Override
    -        public boolean isShutdown() {
    -            return running == false;
    -        }
    -
    -        @Override
    -        public boolean isTerminated() {
    -            return awaitTermination.getCount() == 0;
    -        }
    -
    -        @Override
    -        public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
    -            return awaitTermination.await(timeout, unit);
    -        }
    -
    -        @Override
    -        public synchronized void execute(Runnable command) {
    -            if (isShutdown()) {
    -                EsRejectedExecutionException rejected = new EsRejectedExecutionException("autodetect worker service has shutdown", true);
    -                if (command instanceof AbstractRunnable) {
    -                    ((AbstractRunnable) command).onRejection(rejected);
    -                } else {
    -                    throw rejected;
    -                }
    -            }
    -
    -            boolean added = queue.offer(contextHolder.preserveContext(command));
    -            if (added == false) {
    -                throw new ElasticsearchStatusException("Unable to submit operation", RestStatus.TOO_MANY_REQUESTS);
    -            }
    -        }
    -
    -        void start() {
    -            try {
    -                while (running) {
    -                    Runnable runnable = queue.poll(500, TimeUnit.MILLISECONDS);
    -                    if (runnable != null) {
    -                        try {
    -                            runnable.run();
    -                        } catch (Exception e) {
    -                            logger.error("error handling job operation", e);
    -                        }
    -                        EsExecutors.rethrowErrors(contextHolder.unwrap(runnable));
    -                    }
    -                }
    -
    -                synchronized (this) {
    -                    // if shutdown with tasks pending notify the handlers
    -                    if (queue.isEmpty() == false) {
    -                        List notExecuted = new ArrayList<>();
    -                        queue.drainTo(notExecuted);
    -
    -                        for (Runnable runnable : notExecuted) {
    -                            if (runnable instanceof AbstractRunnable) {
    -                                ((AbstractRunnable) runnable).onRejection(
    -                                    new EsRejectedExecutionException("unable to process as autodetect worker service has shutdown", true));
    -                            }
    -                        }
    -                    }
    -                }
    -            } catch (InterruptedException e) {
    -                Thread.currentThread().interrupt();
    -            } finally {
    -                awaitTermination.countDown();
    -            }
    -        }
    -    }
     }
    diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorService.java
    new file mode 100644
    index 0000000000000..324815513b979
    --- /dev/null
    +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorService.java
    @@ -0,0 +1,122 @@
    +/*
    + * 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.job.process.autodetect;
    +
    +import org.apache.logging.log4j.LogManager;
    +import org.apache.logging.log4j.Logger;
    +import org.elasticsearch.ElasticsearchStatusException;
    +import org.elasticsearch.common.SuppressForbidden;
    +import org.elasticsearch.common.util.concurrent.AbstractRunnable;
    +import org.elasticsearch.common.util.concurrent.EsExecutors;
    +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
    +import org.elasticsearch.common.util.concurrent.ThreadContext;
    +import org.elasticsearch.rest.RestStatus;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.concurrent.AbstractExecutorService;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +
    +/*
    + * The autodetect native process can only handle a single operation at a time. In order to guarantee that, all
    + * operations are initially added to a queue and a worker thread from ml autodetect threadpool will process each
    + * operation at a time.
    + */
    +class AutodetectWorkerExecutorService extends AbstractExecutorService {
    +
    +    private static final Logger logger = LogManager.getLogger(AutodetectWorkerExecutorService.class);
    +
    +    private final ThreadContext contextHolder;
    +    private final CountDownLatch awaitTermination = new CountDownLatch(1);
    +    private final BlockingQueue queue = new LinkedBlockingQueue<>(100);
    +
    +    private volatile boolean running = true;
    +
    +    @SuppressForbidden(reason = "properly rethrowing errors, see EsExecutors.rethrowErrors")
    +    AutodetectWorkerExecutorService(ThreadContext contextHolder) {
    +        this.contextHolder = contextHolder;
    +    }
    +
    +    @Override
    +    public void shutdown() {
    +        running = false;
    +    }
    +
    +    @Override
    +    public List shutdownNow() {
    +        throw new UnsupportedOperationException("not supported");
    +    }
    +
    +    @Override
    +    public boolean isShutdown() {
    +        return running == false;
    +    }
    +
    +    @Override
    +    public boolean isTerminated() {
    +        return awaitTermination.getCount() == 0;
    +    }
    +
    +    @Override
    +    public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
    +        return awaitTermination.await(timeout, unit);
    +    }
    +
    +    @Override
    +    public synchronized void execute(Runnable command) {
    +        if (isShutdown()) {
    +            EsRejectedExecutionException rejected = new EsRejectedExecutionException("autodetect worker service has shutdown", true);
    +            if (command instanceof AbstractRunnable) {
    +                ((AbstractRunnable) command).onRejection(rejected);
    +            } else {
    +                throw rejected;
    +            }
    +        }
    +
    +        boolean added = queue.offer(contextHolder.preserveContext(command));
    +        if (added == false) {
    +            throw new ElasticsearchStatusException("Unable to submit operation", RestStatus.TOO_MANY_REQUESTS);
    +        }
    +    }
    +
    +    void start() {
    +        try {
    +            while (running) {
    +                Runnable runnable = queue.poll(500, TimeUnit.MILLISECONDS);
    +                if (runnable != null) {
    +                    try {
    +                        runnable.run();
    +                    } catch (Exception e) {
    +                        logger.error("error handling job operation", e);
    +                    }
    +                    EsExecutors.rethrowErrors(contextHolder.unwrap(runnable));
    +                }
    +            }
    +
    +            synchronized (this) {
    +                // if shutdown with tasks pending notify the handlers
    +                if (queue.isEmpty() == false) {
    +                    List notExecuted = new ArrayList<>();
    +                    queue.drainTo(notExecuted);
    +
    +                    for (Runnable runnable : notExecuted) {
    +                        if (runnable instanceof AbstractRunnable) {
    +                            ((AbstractRunnable) runnable).onRejection(
    +                                new EsRejectedExecutionException("unable to process as autodetect worker service has shutdown", true));
    +                        }
    +                    }
    +                }
    +            }
    +        } catch (InterruptedException e) {
    +            Thread.currentThread().interrupt();
    +        } finally {
    +            awaitTermination.countDown();
    +        }
    +    }
    +}
    diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java
    index ee02e5237c605..9a147dfd1bc10 100644
    --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java
    +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java
    @@ -15,7 +15,6 @@
     import org.elasticsearch.common.CheckedConsumer;
     import org.elasticsearch.common.settings.ClusterSettings;
     import org.elasticsearch.common.settings.Settings;
    -import org.elasticsearch.common.util.concurrent.AbstractRunnable;
     import org.elasticsearch.common.util.concurrent.EsExecutors;
     import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
     import org.elasticsearch.common.util.concurrent.ThreadContext;
    @@ -26,7 +25,6 @@
     import org.elasticsearch.index.analysis.AnalysisRegistry;
     import org.elasticsearch.test.ESTestCase;
     import org.elasticsearch.test.junit.annotations.TestLogging;
    -import org.elasticsearch.threadpool.TestThreadPool;
     import org.elasticsearch.threadpool.ThreadPool;
     import org.elasticsearch.xpack.core.ml.job.config.AnalysisConfig;
     import org.elasticsearch.xpack.core.ml.job.config.DataDescription;
    @@ -50,17 +48,14 @@
     import org.elasticsearch.xpack.ml.job.persistence.JobDataCountsPersister;
     import org.elasticsearch.xpack.ml.job.persistence.JobResultsPersister;
     import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider;
    -import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager.AutodetectWorkerExecutorService;
     import org.elasticsearch.xpack.ml.job.process.autodetect.params.AutodetectParams;
     import org.elasticsearch.xpack.ml.job.process.autodetect.params.DataLoadParams;
     import org.elasticsearch.xpack.ml.job.process.autodetect.params.FlushJobParams;
     import org.elasticsearch.xpack.ml.job.process.autodetect.params.TimeRange;
     import org.elasticsearch.xpack.ml.job.process.normalizer.NormalizerFactory;
     import org.elasticsearch.xpack.ml.notifications.Auditor;
    -import org.junit.After;
     import org.junit.Before;
     import org.mockito.ArgumentCaptor;
    -import org.mockito.Mockito;
     
     import java.io.ByteArrayInputStream;
     import java.io.IOException;
    @@ -76,11 +71,9 @@
     import java.util.TreeMap;
     import java.util.concurrent.Callable;
     import java.util.concurrent.CountDownLatch;
    -import java.util.concurrent.ExecutionException;
     import java.util.concurrent.ExecutorService;
     import java.util.concurrent.Future;
     import java.util.concurrent.TimeUnit;
    -import java.util.concurrent.atomic.AtomicBoolean;
     import java.util.concurrent.atomic.AtomicInteger;
     import java.util.concurrent.atomic.AtomicReference;
     import java.util.function.BiConsumer;
    @@ -93,7 +86,6 @@
     import static org.elasticsearch.mock.orig.Mockito.verify;
     import static org.elasticsearch.mock.orig.Mockito.verifyNoMoreInteractions;
     import static org.elasticsearch.mock.orig.Mockito.when;
    -import static org.hamcrest.Matchers.containsString;
     import static org.hamcrest.Matchers.is;
     import static org.hamcrest.Matchers.notNullValue;
     import static org.hamcrest.Matchers.nullValue;
    @@ -103,6 +95,7 @@
     import static org.mockito.Matchers.anyString;
     import static org.mockito.Matchers.eq;
     import static org.mockito.Matchers.same;
    +import static org.mockito.Mockito.doCallRealMethod;
     import static org.mockito.Mockito.mock;
     import static org.mockito.Mockito.spy;
     
    @@ -115,11 +108,15 @@
     public class AutodetectProcessManagerTests extends ESTestCase {
     
         private Environment environment;
    +    private Client client;
    +    private ThreadPool threadPool;
         private AnalysisRegistry analysisRegistry;
         private JobManager jobManager;
         private JobResultsProvider jobResultsProvider;
         private JobResultsPersister jobResultsPersister;
         private JobDataCountsPersister jobDataCountsPersister;
    +    private AutodetectCommunicator autodetectCommunicator;
    +    private AutodetectProcessFactory autodetectFactory;
         private NormalizerFactory normalizerFactory;
         private Auditor auditor;
         private ClusterState clusterState;
    @@ -131,18 +128,24 @@ public class AutodetectProcessManagerTests extends ESTestCase {
         private Quantiles quantiles = new Quantiles("foo", new Date(), "state");
         private Set filters = new HashSet<>();
     
    -    private ThreadPool threadPool;
    -
         @Before
         public void setup() throws Exception {
             Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build();
             environment = TestEnvironment.newEnvironment(settings);
    +        client = mock(Client.class);
    +
    +        threadPool = mock(ThreadPool.class);
    +        when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
    +        when(threadPool.executor(anyString())).thenReturn(EsExecutors.newDirectExecutorService());
    +
             analysisRegistry = CategorizationAnalyzerTests.buildTestAnalysisRegistry(environment);
             jobManager = mock(JobManager.class);
             jobResultsProvider = mock(JobResultsProvider.class);
             jobResultsPersister = mock(JobResultsPersister.class);
             when(jobResultsPersister.bulkPersisterBuilder(any())).thenReturn(mock(JobResultsPersister.Builder.class));
             jobDataCountsPersister = mock(JobDataCountsPersister.class);
    +        autodetectCommunicator = mock(AutodetectCommunicator.class);
    +        autodetectFactory = mock(AutodetectProcessFactory.class);
             normalizerFactory = mock(NormalizerFactory.class);
             auditor = mock(Auditor.class);
             clusterService = mock(ClusterService.class);
    @@ -170,25 +173,16 @@ public void setup() throws Exception {
                 handler.accept(buildAutodetectParams());
                 return null;
             }).when(jobResultsProvider).getAutodetectParams(any(), any(), any());
    -
    -        threadPool = new TestThreadPool("AutodetectProcessManagerTests");
    -    }
    -
    -    @After
    -    public void stopThreadPool() {
    -        terminate(threadPool);
         }
     
         public void testOpenJob() {
    -        Client client = mock(Client.class);
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
             doAnswer(invocationOnMock -> {
                 @SuppressWarnings("unchecked")
                 ActionListener listener = (ActionListener) invocationOnMock.getArguments()[1];
                 listener.onResponse(createJobDetails("foo"));
                 return null;
             }).when(jobManager).getJob(eq("foo"), any());
    -        AutodetectProcessManager manager = createManager(communicator, client);
    +        AutodetectProcessManager manager = createSpyManager();
     
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
    @@ -200,8 +194,6 @@ public void testOpenJob() {
         }
     
         public void testOpenJob_withoutVersion() {
    -        Client client = mock(Client.class);
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
             Job.Builder jobBuilder = new Job.Builder(createJobDetails("no_version"));
             jobBuilder.setJobVersion(null);
             Job job = jobBuilder.build();
    @@ -214,7 +206,7 @@ public void testOpenJob_withoutVersion() {
                 return null;
             }).when(jobManager).getJob(eq(job.getId()), any());
     
    -        AutodetectProcessManager manager = createManager(communicator, client);
    +        AutodetectProcessManager manager = createSpyManager();
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn(job.getId());
             AtomicReference errorHolder = new AtomicReference<>();
    @@ -235,25 +227,22 @@ public void testOpenJob_exceedMaxNumJobs() {
                 }).when(jobManager).getJob(eq(jobId), any());
             }
     
    -        Client client = mock(Client.class);
    -        ThreadPool threadPool = mock(ThreadPool.class);
    -        when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
             ThreadPool.Cancellable cancellable = mock(ThreadPool.Cancellable.class);
             when(threadPool.scheduleWithFixedDelay(any(), any(), any())).thenReturn(cancellable);
    -        ExecutorService executorService = mock(ExecutorService.class);
    -        Future future = mock(Future.class);
    -        when(executorService.submit(any(Callable.class))).thenReturn(future);
    -        when(threadPool.executor(anyString())).thenReturn(EsExecutors.newDirectExecutorService());
    +
             AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
             when(autodetectProcess.isProcessAlive()).thenReturn(true);
             when(autodetectProcess.readAutodetectResults()).thenReturn(Collections.emptyIterator());
    -        AutodetectProcessFactory autodetectProcessFactory =
    -                (j, autodetectParams, e, onProcessCrash) -> autodetectProcess;
    +
    +        autodetectFactory = (j, autodetectParams, e, onProcessCrash) -> autodetectProcess;
             Settings.Builder settings = Settings.builder();
             settings.put(MachineLearning.MAX_OPEN_JOBS_PER_NODE.getKey(), 3);
    -        AutodetectProcessManager manager = spy(new AutodetectProcessManager(environment, settings.build(), client, threadPool,
    -                jobManager, jobResultsProvider, jobResultsPersister, jobDataCountsPersister, autodetectProcessFactory,
    -                normalizerFactory, new NamedXContentRegistry(Collections.emptyList()), auditor, clusterService));
    +        AutodetectProcessManager manager = createSpyManager(settings.build());
    +        doCallRealMethod().when(manager).create(any(), any(), any(), any());
    +
    +        ExecutorService executorService = mock(ExecutorService.class);
    +        Future future = mock(Future.class);
    +        when(executorService.submit(any(Callable.class))).thenReturn(future);
             doReturn(executorService).when(manager).createAutodetectExecutorService(any());
     
             doAnswer(invocationOnMock -> {
    @@ -293,8 +282,7 @@ public void testOpenJob_exceedMaxNumJobs() {
         }
     
         public void testProcessData()  {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
             assertEquals(0, manager.numberOfOpenJobs());
     
             JobTask jobTask = mock(JobTask.class);
    @@ -307,8 +295,7 @@ public void testProcessData()  {
         }
     
         public void testProcessDataThrowsElasticsearchStatusException_onIoException() {
    -        AutodetectCommunicator communicator = Mockito.mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
     
             DataLoadParams params = mock(DataLoadParams.class);
             InputStream inputStream = createInputStream("");
    @@ -318,7 +305,7 @@ public void testProcessDataThrowsElasticsearchStatusException_onIoException() {
                 BiConsumer handler = (BiConsumer) invocationOnMock.getArguments()[4];
                 handler.accept(null, new IOException("blah"));
                 return null;
    -        }).when(communicator).writeToJob(eq(inputStream), same(analysisRegistry), same(xContentType), eq(params), any());
    +        }).when(autodetectCommunicator).writeToJob(eq(inputStream), same(analysisRegistry), same(xContentType), eq(params), any());
     
     
             JobTask jobTask = mock(JobTask.class);
    @@ -330,8 +317,7 @@ public void testProcessDataThrowsElasticsearchStatusException_onIoException() {
         }
     
         public void testCloseJob() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
             assertEquals(0, manager.numberOfOpenJobs());
     
             JobTask jobTask = mock(JobTask.class);
    @@ -350,7 +336,6 @@ public void testCloseJob() {
         // interleaved in the AutodetectProcessManager.close() call
         @TestLogging("org.elasticsearch.xpack.ml.job.process.autodetect:DEBUG")
         public void testCanCloseClosingJob() throws Exception {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
             AtomicInteger numberOfCommunicatorCloses = new AtomicInteger(0);
             doAnswer(invocationOnMock -> {
                 numberOfCommunicatorCloses.incrementAndGet();
    @@ -358,8 +343,8 @@ public void testCanCloseClosingJob() throws Exception {
                 // the middle of the AutodetectProcessManager.close() method
                 Thread.yield();
                 return null;
    -        }).when(communicator).close(anyBoolean(), anyString());
    -        AutodetectProcessManager manager = createManager(communicator);
    +        }).when(autodetectCommunicator).close(anyBoolean(), anyString());
    +        AutodetectProcessManager manager = createSpyManager();
             assertEquals(0, manager.numberOfOpenJobs());
     
             JobTask jobTask = mock(JobTask.class);
    @@ -395,19 +380,18 @@ public void testCanKillClosingJob() throws Exception {
             CountDownLatch closeStartedLatch = new CountDownLatch(1);
             CountDownLatch killLatch = new CountDownLatch(1);
             CountDownLatch closeInterruptedLatch = new CountDownLatch(1);
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
             doAnswer(invocationOnMock -> {
                 closeStartedLatch.countDown();
                 if (killLatch.await(3, TimeUnit.SECONDS)) {
                     closeInterruptedLatch.countDown();
                 }
                 return null;
    -        }).when(communicator).close(anyBoolean(), anyString());
    +        }).when(autodetectCommunicator).close(anyBoolean(), anyString());
             doAnswer(invocationOnMock -> {
                 killLatch.countDown();
                 return null;
    -        }).when(communicator).killProcess(anyBoolean(), anyBoolean(), anyBoolean());
    -        AutodetectProcessManager manager = createManager(communicator);
    +        }).when(autodetectCommunicator).killProcess(anyBoolean(), anyBoolean(), anyBoolean());
    +        AutodetectProcessManager manager = createSpyManager();
             assertEquals(0, manager.numberOfOpenJobs());
     
             JobTask jobTask = mock(JobTask.class);
    @@ -433,8 +417,7 @@ public void testCanKillClosingJob() throws Exception {
         }
     
         public void testBucketResetMessageIsSent() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
             XContentType xContentType = randomFrom(XContentType.values());
     
             DataLoadParams params = new DataLoadParams(TimeRange.builder().startTime("1000").endTime("2000").build(), Optional.empty());
    @@ -443,12 +426,11 @@ public void testBucketResetMessageIsSent() {
             when(jobTask.getJobId()).thenReturn("foo");
             manager.openJob(jobTask, clusterState, (e, b) -> {});
             manager.processData(jobTask, analysisRegistry, inputStream, xContentType, params, (dataCounts1, e) -> {});
    -        verify(communicator).writeToJob(same(inputStream), same(analysisRegistry), same(xContentType), same(params), any());
    +        verify(autodetectCommunicator).writeToJob(same(inputStream), same(analysisRegistry), same(xContentType), same(params), any());
         }
     
         public void testFlush() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
     
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
    @@ -460,12 +442,11 @@ public void testFlush() {
             FlushJobParams params = FlushJobParams.builder().build();
             manager.flushJob(jobTask, params, ActionListener.wrap(flushAcknowledgement -> {}, e -> fail(e.getMessage())));
     
    -        verify(communicator).flushJob(same(params), any());
    +        verify(autodetectCommunicator).flushJob(same(params), any());
         }
     
         public void testFlushThrows() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManagerAndCallProcessData(communicator, "foo");
    +        AutodetectProcessManager manager = createSpyManagerAndCallProcessData("foo");
     
             FlushJobParams params = FlushJobParams.builder().build();
             doAnswer(invocationOnMock -> {
    @@ -473,7 +454,7 @@ public void testFlushThrows() {
                 BiConsumer handler = (BiConsumer) invocationOnMock.getArguments()[1];
                 handler.accept(null, new IOException("blah"));
                 return null;
    -        }).when(communicator).flushJob(same(params), any());
    +        }).when(autodetectCommunicator).flushJob(same(params), any());
     
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
    @@ -483,12 +464,11 @@ public void testFlushThrows() {
         }
     
         public void testCloseThrows() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
     
             // let the communicator throw, simulating a problem with the underlying
             // autodetect, e.g. a crash
    -        doThrow(Exception.class).when(communicator).close(anyBoolean(), anyString());
    +        doThrow(Exception.class).when(autodetectCommunicator).close(anyBoolean(), anyString());
     
             // create a jobtask
             JobTask jobTask = mock(JobTask.class);
    @@ -507,8 +487,7 @@ public void testCloseThrows() {
         }
     
         public void testWriteUpdateProcessMessage() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManagerAndCallProcessData(communicator, "foo");
    +        AutodetectProcessManager manager = createSpyManagerAndCallProcessData("foo");
             ModelPlotConfig modelConfig = mock(ModelPlotConfig.class);
             List rules = Collections.singletonList(mock(DetectionRule.class));
             List detectorUpdates = Collections.singletonList(new JobUpdate.DetectorUpdate(2, null, rules));
    @@ -519,7 +498,7 @@ public void testWriteUpdateProcessMessage() {
             manager.writeUpdateProcessMessage(jobTask, updateParams, e -> {});
     
             ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateProcessMessage.class);
    -        verify(communicator).writeUpdateProcessMessage(captor.capture(), any());
    +        verify(autodetectCommunicator).writeUpdateProcessMessage(captor.capture(), any());
     
             UpdateProcessMessage updateProcessMessage = captor.getValue();
             assertThat(updateProcessMessage.getModelPlotConfig(), equalTo(modelConfig));
    @@ -527,8 +506,7 @@ public void testWriteUpdateProcessMessage() {
         }
     
         public void testJobHasActiveAutodetectProcess() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
             assertFalse(manager.jobHasActiveAutodetectProcess(jobTask));
    @@ -545,8 +523,7 @@ public void testJobHasActiveAutodetectProcess() {
         }
     
         public void testKillKillsAutodetectProcess() throws IOException {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
             assertFalse(manager.jobHasActiveAutodetectProcess(jobTask));
    @@ -559,12 +536,11 @@ public void testKillKillsAutodetectProcess() throws IOException {
     
             manager.killAllProcessesOnThisNode();
     
    -        verify(communicator).killProcess(false, false, true);
    +        verify(autodetectCommunicator).killProcess(false, false, true);
         }
     
         public void testKillingAMissingJobFinishesTheTask() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
    -        AutodetectProcessManager manager = createManager(communicator);
    +        AutodetectProcessManager manager = createSpyManager();
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
     
    @@ -574,14 +550,13 @@ public void testKillingAMissingJobFinishesTheTask() {
         }
     
         public void testProcessData_GivenStateNotOpened() {
    -        AutodetectCommunicator communicator = mock(AutodetectCommunicator.class);
             doAnswer(invocationOnMock -> {
                 @SuppressWarnings("unchecked")
                 BiConsumer handler = (BiConsumer) invocationOnMock.getArguments()[4];
                 handler.accept(new DataCounts("foo"), null);
                 return null;
    -        }).when(communicator).writeToJob(any(), any(), any(), any(), any());
    -        AutodetectProcessManager manager = createManager(communicator);
    +        }).when(autodetectCommunicator).writeToJob(any(), any(), any(), any(), any());
    +        AutodetectProcessManager manager = createSpyManager();
     
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("foo");
    @@ -595,8 +570,6 @@ public void testProcessData_GivenStateNotOpened() {
         }
     
         public void testCreate_notEnoughThreads() throws IOException {
    -        Client client = mock(Client.class);
    -        ThreadPool threadPool = mock(ThreadPool.class);
             when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
             ExecutorService executorService = mock(ExecutorService.class);
             doThrow(new EsRejectedExecutionException("")).when(executorService).submit(any(Runnable.class));
    @@ -611,11 +584,9 @@ public void testCreate_notEnoughThreads() throws IOException {
             }).when(jobManager).getJob(eq("my_id"), any());
     
             AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
    -        AutodetectProcessFactory autodetectProcessFactory =
    -                (j, autodetectParams, e, onProcessCrash) -> autodetectProcess;
    -        AutodetectProcessManager manager = new AutodetectProcessManager(environment, Settings.EMPTY,
    -                client, threadPool, jobManager, jobResultsProvider, jobResultsPersister, jobDataCountsPersister, autodetectProcessFactory,
    -                normalizerFactory, new NamedXContentRegistry(Collections.emptyList()), auditor, clusterService);
    +        autodetectFactory = (j, autodetectParams, e, onProcessCrash) -> autodetectProcess;
    +        AutodetectProcessManager manager = createSpyManager();
    +        doCallRealMethod().when(manager).create(any(), any(), any(), any());
     
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn("my_id");
    @@ -675,86 +646,7 @@ public void testCreate_givenNonZeroCountsAndNoModelSnapshotNorQuantiles() {
             verifyNoMoreInteractions(auditor);
         }
     
    -    public void testAutodetectWorkerExecutorServiceDoesNotSwallowErrors() {
    -        final ThreadPool threadPool = new TestThreadPool("testAutodetectWorkerExecutorServiceDoesNotSwallowErrors");
    -        try {
    -            final AutodetectWorkerExecutorService executor = new AutodetectWorkerExecutorService(threadPool.getThreadContext());
    -            if (randomBoolean()) {
    -                executor.submit(() -> {
    -                    throw new Error("future error");
    -                });
    -            } else {
    -                executor.execute(() -> {
    -                    throw new Error("future error");
    -                });
    -            }
    -            final Error e = expectThrows(Error.class, () -> executor.start());
    -            assertThat(e.getMessage(), containsString("future error"));
    -        } finally {
    -            ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS);
    -        }
    -    }
    -
    -    public void testAutodetectWorkerExecutorService_SubmitAfterShutdown() {
    -        AutodetectProcessManager.AutodetectWorkerExecutorService executor =
    -                new AutodetectWorkerExecutorService(new ThreadContext(Settings.EMPTY));
    -
    -        threadPool.generic().execute(() -> executor.start());
    -        executor.shutdown();
    -        expectThrows(EsRejectedExecutionException.class, () -> executor.execute(() -> {}));
    -    }
    -
    -    public void testAutodetectWorkerExecutorService_TasksNotExecutedCallHandlerOnShutdown()
    -            throws InterruptedException, ExecutionException {
    -        AutodetectProcessManager.AutodetectWorkerExecutorService executor =
    -                new AutodetectWorkerExecutorService(new ThreadContext(Settings.EMPTY));
    -
    -        CountDownLatch latch = new CountDownLatch(1);
    -
    -        Future executorFinished = threadPool.generic().submit(() -> executor.start());
    -
    -        // run a task that will block while the others are queued up
    -        executor.execute(() -> {
    -            try {
    -                latch.await();
    -            } catch (InterruptedException e) {
    -            }
    -        });
    -
    -        AtomicBoolean runnableShouldNotBeCalled = new AtomicBoolean(false);
    -        executor.execute(() -> runnableShouldNotBeCalled.set(true));
    -
    -        AtomicInteger onFailureCallCount = new AtomicInteger();
    -        AtomicInteger doRunCallCount = new AtomicInteger();
    -        for (int i=0; i<2; i++) {
    -            executor.execute(new AbstractRunnable() {
    -                @Override
    -                public void onFailure(Exception e) {
    -                    onFailureCallCount.incrementAndGet();
    -                }
    -
    -                @Override
    -                protected void doRun() {
    -                    doRunCallCount.incrementAndGet();
    -                }
    -            });
    -        }
    -
    -        // now shutdown
    -        executor.shutdown();
    -        latch.countDown();
    -        executorFinished.get();
    -
    -        assertFalse(runnableShouldNotBeCalled.get());
    -        // the AbstractRunnables should have had their callbacks called
    -        assertEquals(2, onFailureCallCount.get());
    -        assertEquals(0, doRunCallCount.get());
    -    }
    -
         private AutodetectProcessManager createNonSpyManager(String jobId) {
    -        Client client = mock(Client.class);
    -        ThreadPool threadPool = mock(ThreadPool.class);
    -        when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
             ExecutorService executorService = mock(ExecutorService.class);
             when(threadPool.executor(anyString())).thenReturn(executorService);
             when(threadPool.scheduleWithFixedDelay(any(), any(), any())).thenReturn(mock(ThreadPool.Cancellable.class));
    @@ -766,11 +658,8 @@ private AutodetectProcessManager createNonSpyManager(String jobId) {
             }).when(jobManager).getJob(eq(jobId), any());
     
             AutodetectProcess autodetectProcess = mock(AutodetectProcess.class);
    -        AutodetectProcessFactory autodetectProcessFactory =
    -                (j, autodetectParams, e, onProcessCrash) -> autodetectProcess;
    -        return new AutodetectProcessManager(environment, Settings.EMPTY, client, threadPool, jobManager,
    -                jobResultsProvider, jobResultsPersister, jobDataCountsPersister, autodetectProcessFactory,
    -                normalizerFactory, new NamedXContentRegistry(Collections.emptyList()), auditor, clusterService);
    +        autodetectFactory = (j, autodetectParams, e, onProcessCrash) -> autodetectProcess;
    +        return createManager(Settings.EMPTY);
         }
     
         private AutodetectParams buildAutodetectParams() {
    @@ -783,27 +672,25 @@ private AutodetectParams buildAutodetectParams() {
                     .build();
         }
     
    -    private AutodetectProcessManager createManager(AutodetectCommunicator communicator) {
    -        Client client = mock(Client.class);
    -        return createManager(communicator, client);
    +    private AutodetectProcessManager createSpyManager() {
    +        return createSpyManager(Settings.EMPTY);
         }
     
    -    private AutodetectProcessManager createManager(AutodetectCommunicator communicator, Client client) {
    -        ThreadPool threadPool = mock(ThreadPool.class);
    -        when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY));
    -        when(threadPool.executor(anyString())).thenReturn(EsExecutors.newDirectExecutorService());
    -        AutodetectProcessFactory autodetectProcessFactory = mock(AutodetectProcessFactory.class);
    -        AutodetectProcessManager manager = new AutodetectProcessManager(environment, Settings.EMPTY,
    -                client, threadPool, jobManager, jobResultsProvider, jobResultsPersister, jobDataCountsPersister,
    -                autodetectProcessFactory, normalizerFactory,
    -                new NamedXContentRegistry(Collections.emptyList()), auditor, clusterService);
    +    private AutodetectProcessManager createSpyManager(Settings settings) {
    +        AutodetectProcessManager manager = createManager(settings);
             manager = spy(manager);
    -        doReturn(communicator).when(manager).create(any(), any(), eq(buildAutodetectParams()), any());
    +        doReturn(autodetectCommunicator).when(manager).create(any(), any(), eq(buildAutodetectParams()), any());
             return manager;
         }
     
    -    private AutodetectProcessManager createManagerAndCallProcessData(AutodetectCommunicator communicator, String jobId) {
    -        AutodetectProcessManager manager = createManager(communicator);
    +    private AutodetectProcessManager createManager(Settings settings) {
    +        return new AutodetectProcessManager(environment, settings,
    +            client, threadPool, jobManager, jobResultsProvider, jobResultsPersister, jobDataCountsPersister,
    +            autodetectFactory, normalizerFactory,
    +            new NamedXContentRegistry(Collections.emptyList()), auditor, clusterService);
    +    }
    +    private AutodetectProcessManager createSpyManagerAndCallProcessData(String jobId) {
    +        AutodetectProcessManager manager = createSpyManager();
             JobTask jobTask = mock(JobTask.class);
             when(jobTask.getJobId()).thenReturn(jobId);
             manager.openJob(jobTask, clusterState, (e, b) -> {});
    diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorServiceTests.java
    new file mode 100644
    index 0000000000000..4e9afd38c995a
    --- /dev/null
    +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectWorkerExecutorServiceTests.java
    @@ -0,0 +1,100 @@
    +/*
    + * 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.job.process.autodetect;
    +
    +import org.elasticsearch.common.settings.Settings;
    +import org.elasticsearch.common.util.concurrent.AbstractRunnable;
    +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
    +import org.elasticsearch.common.util.concurrent.ThreadContext;
    +import org.elasticsearch.test.ESTestCase;
    +import org.elasticsearch.threadpool.TestThreadPool;
    +import org.elasticsearch.threadpool.ThreadPool;
    +import org.junit.After;
    +
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import static org.hamcrest.Matchers.containsString;
    +
    +public class AutodetectWorkerExecutorServiceTests extends ESTestCase {
    +
    +    private ThreadPool threadPool = new TestThreadPool("AutodetectWorkerExecutorServiceTests");
    +
    +    @After
    +    public void stopThreadPool() {
    +        terminate(threadPool);
    +    }
    +
    +    public void testAutodetectWorkerExecutorService_SubmitAfterShutdown() {
    +        AutodetectWorkerExecutorService executor = new AutodetectWorkerExecutorService(new ThreadContext(Settings.EMPTY));
    +
    +        threadPool.generic().execute(() -> executor.start());
    +        executor.shutdown();
    +        expectThrows(EsRejectedExecutionException.class, () -> executor.execute(() -> {}));
    +    }
    +
    +    public void testAutodetectWorkerExecutorService_TasksNotExecutedCallHandlerOnShutdown() throws Exception {
    +        AutodetectWorkerExecutorService executor = new AutodetectWorkerExecutorService(new ThreadContext(Settings.EMPTY));
    +
    +        CountDownLatch latch = new CountDownLatch(1);
    +
    +        Future executorFinished = threadPool.generic().submit(() -> executor.start());
    +
    +        // run a task that will block while the others are queued up
    +        executor.execute(() -> {
    +            try {
    +                latch.await();
    +            } catch (InterruptedException e) {
    +            }
    +        });
    +
    +        AtomicBoolean runnableShouldNotBeCalled = new AtomicBoolean(false);
    +        executor.execute(() -> runnableShouldNotBeCalled.set(true));
    +
    +        AtomicInteger onFailureCallCount = new AtomicInteger();
    +        AtomicInteger doRunCallCount = new AtomicInteger();
    +        for (int i=0; i<2; i++) {
    +            executor.execute(new AbstractRunnable() {
    +                @Override
    +                public void onFailure(Exception e) {
    +                    onFailureCallCount.incrementAndGet();
    +                }
    +
    +                @Override
    +                protected void doRun() {
    +                    doRunCallCount.incrementAndGet();
    +                }
    +            });
    +        }
    +
    +        // now shutdown
    +        executor.shutdown();
    +        latch.countDown();
    +        executorFinished.get();
    +
    +        assertFalse(runnableShouldNotBeCalled.get());
    +        // the AbstractRunnables should have had their callbacks called
    +        assertEquals(2, onFailureCallCount.get());
    +        assertEquals(0, doRunCallCount.get());
    +    }
    +
    +    public void testAutodetectWorkerExecutorServiceDoesNotSwallowErrors() {
    +        AutodetectWorkerExecutorService executor = new AutodetectWorkerExecutorService(threadPool.getThreadContext());
    +        if (randomBoolean()) {
    +            executor.submit(() -> {
    +                throw new Error("future error");
    +            });
    +        } else {
    +            executor.execute(() -> {
    +                throw new Error("future error");
    +            });
    +        }
    +        Error e = expectThrows(Error.class, () -> executor.start());
    +        assertThat(e.getMessage(), containsString("future error"));
    +    }
    +}
    
    From 7e2a968c3650c4790ab03e1544f593520709db01 Mon Sep 17 00:00:00 2001
    From: James Rodewig 
    Date: Mon, 22 Apr 2019 05:26:36 -0700
    Subject: [PATCH 136/260] [DOCS] Remove abbrevtitles for Asciidoctor migration
     (#41366)
    
    ---
     docs/reference/release-notes/highlights.asciidoc | 5 +----
     1 file changed, 1 insertion(+), 4 deletions(-)
    
    diff --git a/docs/reference/release-notes/highlights.asciidoc b/docs/reference/release-notes/highlights.asciidoc
    index 6ff7c75a235db..6e5c55d02192f 100644
    --- a/docs/reference/release-notes/highlights.asciidoc
    +++ b/docs/reference/release-notes/highlights.asciidoc
    @@ -1,8 +1,5 @@
     [[release-highlights]]
    -= {es} Release highlights
    -++++
    -Release highlights
    -++++
    += Release highlights
     
     [partintro]
     --
    
    From 1013101d7729959764e7632729d392ce4a32e177 Mon Sep 17 00:00:00 2001
    From: Ioannis Kakavas 
    Date: Mon, 22 Apr 2019 15:38:41 +0300
    Subject: [PATCH 137/260] OpenID Connect realm settings and rest API docs
     (#40740)
    
    This commit adds the relevant docs for the OpenID Connect
    realm settings and the REST APIs that are exposed.
    ---
     .../settings/security-settings.asciidoc       | 244 +++++++++++++++++-
     x-pack/docs/en/rest-api/security.asciidoc     |  12 +
     .../security/oidc-authenticate-api.asciidoc   |  68 +++++
     .../security/oidc-logout-api.asciidoc         |  53 ++++
     .../oidc-prepare-authentication-api.asciidoc  | 128 +++++++++
     5 files changed, 503 insertions(+), 2 deletions(-)
     create mode 100644 x-pack/docs/en/rest-api/security/oidc-authenticate-api.asciidoc
     create mode 100644 x-pack/docs/en/rest-api/security/oidc-logout-api.asciidoc
     create mode 100644 x-pack/docs/en/rest-api/security/oidc-prepare-authentication-api.asciidoc
    
    diff --git a/docs/reference/settings/security-settings.asciidoc b/docs/reference/settings/security-settings.asciidoc
    index b767b7869dbd9..04cef7e199f29 100644
    --- a/docs/reference/settings/security-settings.asciidoc
    +++ b/docs/reference/settings/security-settings.asciidoc
    @@ -1074,8 +1074,7 @@ they cannot have individual passwords.
     
     If you are loading the IdP metadata over SSL/TLS (that is, `idp.metadata.path` 
     is a URL using the `https` protocol), the following settings can be used to 
    -configure SSL. If these are not specified, then the 
    -<> are used.
    +configure SSL.
     
     NOTE: These settings are not used for any purpose other than loading metadata 
     over https.
    @@ -1204,6 +1203,247 @@ If this setting is used, then the Kerberos realm does not perform role mapping a
     instead loads the user from the listed realms.
     See {stack-ov}/realm-chains.html#authorization_realms[Delegating authorization to another realm]
     
    +[[ref-oidc-settings]]
    +[float]
    +===== OpenID Connect realm settings
    +
    +In addition to the <>, you
    +can specify the following settings:
    +
    +`op.issuer`::
    +A verifiable Identifier for your OpenID Connect Provider. An Issuer
    +Identifier is usually a case sensitive URL using the https scheme that contains
    +scheme, host, and optionally, port number and path components and no query or
    +fragment components. The value for this setting should be provided by your OpenID
    +Connect Provider.
    +
    +`op.authorization_endpoint`::
    +The URL for the Authorization Endpoint at the
    +OpenID Connect Provider. The value for this setting should be provided by your OpenID
    +Connect Provider.
    +
    +`op.token_endpoint`::
    +The URL for the Token Endpoint at the OpenID Connect Provider.
    +The value for this setting should be provided by your OpenID Connect Provider.
    +
    +`op.userinfo_endpoint`::
    +The URL for the User Info Endpoint at the OpenID Connect Provider.
    +The value for this setting should be provided by your OpenID Connect Provider.
    +
    +`op.endsession_endpoint`::
    +The URL for the End Session Endpoint at the OpenID Connect
    +Provider. The value for this setting should be provided by your OpenID Connect Provider.
    +
    +`op.jwkset_path`::
    +The path or URL to a JSON Web Key Set with the key material that the OpenID Connect
    +Provider uses for signing tokens and claims responses.
    +If a path is provided, then it is resolved relative to the {es} config
    +directory.
    +If a URL is provided, then it must be either a `file` URL or a `https` URL.
    +{es} automatically caches the retrieved JWK set to avoid unnecessary HTTP
    +requests but will attempt to refresh the JWK upon signature verification
    +failure, as this might indicate that the OpenID Connect Provider has
    +rotated the signing keys.
    +
    +File based resources are polled at a frequency determined by the global {es}
    +`resource.reload.interval.high` setting, which defaults to 5 seconds.
    +
    +`rp.client_id`::
    +The OAuth 2.0 Client Identifier that was assigned to {es} during registration
    +at the OpenID Connect Provider
    +
    +`rp.client_secret`(<>)::
    +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`.
    +
    +`rp.response_type`::
    +OAuth 2.0 Response Type value that determines the authorization
    +processing flow to be used. Can be `code` for authorization code grant flow,
    +or one of `id_token`, `id_token token` for the implicit flow.
    +
    +`rp.signature_algorithm`::
    +The signature algorithm that will be used by {es} in order to verify the
    +signature of the id tokens it will receive from the OpenID Connect Provider.
    +Defaults to `RSA256`
    +
    +`rp.requested_scopes`::
    +The scope values that will be requested by the OpenID Connect Provider as
    +part of the Authentication Request. Optional, defaults to `openid`
    +
    +`rp.post_logout_redirect_uri`::
    +The Redirect URI (usually within {kib}) that the OpenID Connect Provider
    +should redirect the browser to after a successful Single Logout.
    +
    +`claims.principal`::
    +The name of the OpenID Connect claim that contains the user's principal (username).
    +
    +`claims.groups`::
    +The name of the OpenID Connect claim that contains the user's groups.
    +
    +`claims.name`::
    +The name of the OpenID Connect claim that contains the user's full name.
    +
    +`claims.mail`::
    +The name of the OpenID Connect claim that contains the user's email address.
    +
    +`claims.dn`::
    +The name of the OpenID Connect claim that contains the user's X.509
    +_Distinguished Name_.
    +
    +`claim_patterns.principal`::
    +A Java regular expression that is matched against the OpenID Connect claim specified
    +by `claims.principal` before it is applied to the user's _principal_ property.
    +The attribute value must match the pattern and the value of the first
    +_capturing group_ is used as the principal. For example, `^([^@]+)@example\\.com$`
    +matches email addresses from the "example.com" domain and uses the local-part as
    +the principal.
    +
    +`claim_patterns.groups`::
    +As per `claim_patterns.principal`, but for the _group_ property.
    +
    +`claim_patterns.name`::
    +As per `claim_patterns.principal`, but for the _name_ property.
    +
    +`claim_patterns.mail`::
    +As per `claim_patterns.principal`, but for the _mail_ property.
    +
    +`claim_patterns.dn`::
    +As per `claim_patterns.principal`, but for the _dn_ property.
    +
    +
    +`allowed_clock_skew`::
    +The maximum allowed clock skew to be taken into consideration when validating
    +id tokens with regards to their creation and expiration times.
    +
    +`populate_user_metadata`::
    +Specifies whether to populate the {es} user's metadata with the values that are
    +provided by the OpenID Connect claims. Defaults to `true`.
    +
    +`http.connect_timeout`::
    +Controls the behavior of the http client used for back-channel communication to
    +the OpenID Connect Provider endpoints. Specifies the timeout until a connection
    + is established. A value of zero means the timeout is not used. Defaults to `5s`
    +
    +`http.connection_read_timeout`::
    +Controls the behavior of the http client used for back-channel communication to
    +the OpenID Connect Provider endpoints. Specifies the timeout used when
    +requesting a connection from the connection manager. Defaults to `5s`
    +
    +`http.socket_timeout`::
    +Controls the behavior of the http client used for back-channel communication to
    +the OpenID Connect Provider endpoints. Specifies the socket timeout (SO_TIMEOUT)
    +in milliseconds, which is the timeout for waiting for data or, put differently,
    +a maximum period inactivity between two consecutive data packets). Defaults to
    +`5s`
    +
    +`http.max_connections`::
    +Controls the behavior of the http client used for back-channel communication to
    +the OpenID Connect Provider endpoints. Specifies the maximum number of
    +connections allowed across all endpoints.
    +
    +`http.max_endpoint_connections`::
    +Controls the behavior of the http client used for back-channel communication to
    +the OpenID Connect Provider endpoints. Specifies the maximum number of
    +connections allowed per endpoint.
    +
    +[float]
    +[[ref-oidc-ssl-settings]]
    +===== OpenID Connect realm SSL settings
    +
    +The following settings can be used to configure SSL for all outgoing http connections
    +to the OpenID Connect Provider endpoints.
    +
    +NOTE: These settings are _only_ used for the back-channel communication between
    +{es} and the OpenID Connect Provider
    +
    +`ssl.key`::
    +Specifies the path to the PEM encoded private key to use for http client
    +authentication (if required). `ssl.key` and `ssl.keystore.path` cannot be used
    +at the same time.
    +
    +`ssl.key_passphrase`::
    +Specifies the passphrase to decrypt the PEM encoded private key (`ssl.key`) if it is
    +encrypted. Cannot be used with `ssl.secure_key_passphrase`.
    +
    +`ssl.secure_key_passphrase` (<>)::
    +Specifies the passphrase to decrypt the PEM encoded private key (`ssl.key`) if it is
    +encrypted. Cannot be used with `ssl.key_passphrase`.
    +
    +`ssl.certificate`::
    +Specifies the path to the PEM encoded certificate (or certificate chain) that is associated
    +with the key (`ssl.key`). This setting can be used only if `ssl.key` is set.
    +
    +`ssl.certificate_authorities`::
    +Specifies the paths to the PEM encoded certificate authority certificates that should be
    +trusted. `ssl.certificate_authorities` and `ssl.truststore.path` cannot be
    +used at the same time.
    +
    +`ssl.keystore.path`::
    +Specifies the path to the keystore that contains a private key and certificate.
    +Must be either a Java Keystore (jks) or a PKCS#12 file.
    +`ssl.key` and `ssl.keystore.path` cannot be used at the same time.
    +
    +`ssl.keystore.type`::
    +The type of the keystore (`ssl.keystore.path`). Must be either `jks` or `PKCS12`.
    +If the keystore path ends in ".p12", ".pfx" or "pkcs12", this setting defaults
    +to `PKCS12`. Otherwise, it defaults to `jks`.
    +
    +`ssl.keystore.password`::
    +The password to the keystore (`ssl.keystore.path`). This setting cannot be used
    +with `ssl.keystore.secure_password`.
    +
    +`ssl.keystore.secure_password` (<>)::
    +The password to the keystore (`ssl.keystore.path`).
    +This setting cannot be used with `ssl.keystore.password`.
    +
    +`ssl.keystore.key_password`::
    +The password for the key in the keystore (`ssl.keystore.path`).
    +Defaults to the keystore password. This setting cannot be used with
    +`ssl.keystore.secure_key_password`.
    +
    +`ssl.keystore.secure_key_password` (<>)::
    +The password for the key in the keystore (`ssl.keystore.path`).
    +Defaults to the keystore password. This setting cannot be used with
    +`ssl.keystore.key_password`.
    +
    +`ssl.truststore.path`::
    +The path to the keystore that contains the certificates to trust.
    +Must be either a Java Keystore (jks) or a PKCS#12 file.
    +`ssl.certificate_authorities` and `ssl.truststore.path` cannot be used at the
    +same time.
    +
    +`ssl.truststore.type`::
    +The type of the truststore (`ssl.truststore.path`). Must be either `jks` or
    +`PKCS12`. If the keystore path ends in ".p12", ".pfx" or "pkcs12", this setting
    +defaults to `PKCS12`. Otherwise, it defaults to `jks`.
    +
    +`ssl.truststore.password`::
    +The password to the truststore (`ssl.truststore.path`). This setting cannot be
    +used with `ssl.truststore.secure_password`.
    +
    +`ssl.truststore.secure_password` (<>)::
    +The password to the truststore (`ssl.truststore.path`). This setting cannot be
    +used with `ssl.truststore.password`.
    +
    +`ssl.verification_mode`::
    +One of `full` (verify the hostname and the certificate path), `certificate` (verify the
    +certificate path, but not the hostname) or `none` (perform no verification).
    +Defaults to `full`.
    ++
    +See <> for a more detailed explanation of these values.
    +
    +`ssl.supported_protocols`::
    +Specifies the supported protocols for TLS/SSL. Defaults to `TLSv1.3,TLSv1.2,TLSv1.1` if
    +the JVM supports TLSv1.3, otherwise `TLSv1.2,TLSv1.1`.
    +
    +`ssl.cipher_suites`::
    +Specifies the cipher suites that should be supported.
    +
     [float]
     [[load-balancing]]
     ===== Load balancing and failover
    diff --git a/x-pack/docs/en/rest-api/security.asciidoc b/x-pack/docs/en/rest-api/security.asciidoc
    index 7e14a6a0ee9b7..c04bae90801ee 100644
    --- a/x-pack/docs/en/rest-api/security.asciidoc
    +++ b/x-pack/docs/en/rest-api/security.asciidoc
    @@ -76,6 +76,15 @@ native realm:
     * <>
     * <>
     
    +=== OpenID Connect
    +
    +You can use the following APIs to authenticate users against an OpenID Connect
    +authentication realm
    +
    +* <>
    +* <>
    +* <>
    +
     
     include::security/authenticate.asciidoc[]
     include::security/change-password.asciidoc[]
    @@ -102,3 +111,6 @@ include::security/has-privileges.asciidoc[]
     include::security/invalidate-api-keys.asciidoc[]
     include::security/invalidate-tokens.asciidoc[]
     include::security/ssl.asciidoc[]
    +include::security/oidc-prepare-authentication-api.asciidoc[]
    +include::security/oidc-authenticate-api.asciidoc[]
    +include::security/oidc-logout-api.asciidoc[]
    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
    new file mode 100644
    index 0000000000000..0efb2b23145f7
    --- /dev/null
    +++ b/x-pack/docs/en/rest-api/security/oidc-authenticate-api.asciidoc
    @@ -0,0 +1,68 @@
    +[role="xpack"]
    +[[security-api-oidc-authenticate]]
    +
    +=== OpenID Connect Authenticate API
    +
    +Submits the response to an oAuth 2.0 authentication request for consumption from {es}. Upon successful validation, {es}
    +will respond with an {es} internal Access Token and Refresh Token that can be subsequently used for authentication. This
    +API endpoint basically exchanges successful OpenID Connect Authentication responses for {es} access and refresh tokens
    +to be used for authentication.
    +
    +{es} exposes all the necessary OpenID Connect related functionality via the OpenID Connect APIs. These APIs
    +are used internally by {kib} in order to provide OpenID Connect based authentication, but can also be used by other,
    +custom web applications or other clients. See also
    +<> and
    +<>
    +
    +==== Request
    +
    +`POST /_security/oidc/authenticate`
    +
    +==== Request Body
    +
    +`redirect_uri`::
    +The URL to which the OpenID Connect Provider redirected the User Agent in response to an authentication request, after a
    + successful authentication. This URL is expected to be provided as-is (URL encoded), taken from the body of the response
    + or as the value of a `Location` header in the response from the OpenID Connect Provider.
    +
    +`state`::
    +String value used to maintain state between the authentication request and the response. This value needs to be the same
    +as the one that was provided to the call to `/_security/oidc/prepare` earlier, or the one that was generated by {es}
    +and included in the response to that call.
    +
    +`nonce`::
    +String value used to associate a Client session with an ID Token, and to mitigate replay attacks. This value needs to be
    +the same as the one that was provided to the call to `/_security/oidc/prepare` earlier, or the one that was generated by {es}
    +and included in the response to that call.
    +
    +==== Examples
    +
    +The following example request exchanges the response that was returned from the 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]
    +--------------------------------------------------
    +POST /_security/oidc/authenticate
    +{
    +  "redirect_uri" : "https://oidc-kibana.elastic.co:5603/api/security/v1/oidc?code=jtI3Ntt8v3_XvcLzCFGq&state=4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I",
    +  "state" : "4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I",
    +  "nonce" : "WaBPH0KqPVdG5HHdSxPRjfoZbXMCicm5v1OiAj0DUFM"
    +}
    +--------------------------------------------------
    +// CONSOLE
    +// TEST[skip:These are properly tested in the OpenIDConnectIT suite]
    +
    +The following example output contains the access token that was generated in response, the amount of time (in
    +seconds) that the token expires in, the type, and the refresh token:
    +
    +[source,js]
    +--------------------------------------------------
    +{
    +  "access_token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==",
    +  "type" : "Bearer",
    +  "expires_in" : 1200,
    +  "refresh_token": "vLBPvmAB6KvwvJZr27cS"
    +}
    +--------------------------------------------------
    +// NOTCONSOLE
    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
    new file mode 100644
    index 0000000000000..6f5288a135f2a
    --- /dev/null
    +++ b/x-pack/docs/en/rest-api/security/oidc-logout-api.asciidoc
    @@ -0,0 +1,53 @@
    +[role="xpack"]
    +[[security-api-oidc-logout]]
    +
    +=== OpenID Connect Logout API
    +
    +Submits a request to invalidate a refresh token and an access token that was generated as a response to a call to
    +`/_security/oidc/authenticate`. If the OpenID Connect authentication realm in {es} is accordingly configured, the
    +response to this call will contain a URI pointing to the End Session Endpoint of the OpenID Connect Provider in
    +order to perform Single Logout
    +
    +{es} exposes all the necessary OpenID Connect related functionality via the OpenID Connect APIs. These APIs
    +are used internally by {kib} in order to provide OpenID Connect based authentication, but can also be used by other,
    +custom web applications or other clients. See also <>
    +and <>
    +
    +==== Request
    +
    +`POST /_security/oidc/logout`
    +
    +==== Request Body
    +
    +`access_token`::
    +The value of the access token to be invalidated as part of the logout.
    +
    +`refresh_token`::
    +The value of the refresh token to be invalidated as part of the logout. (Optional)
    +
    +
    +==== Examples
    +
    +The following example performs logout
    +
    +[source,js]
    +--------------------------------------------------
    +POST /_security/oidc/logout
    +{
    +  "token" : "dGhpcyBpcyBub3QgYSByZWFsIHRva2VuIGJ1dCBpdCBpcyBvbmx5IHRlc3QgZGF0YS4gZG8gbm90IHRyeSB0byByZWFkIHRva2VuIQ==",
    +  "refresh_token": "vLBPvmAB6KvwvJZr27cS"
    +}
    +--------------------------------------------------
    +// CONSOLE
    +// TEST[skip:These are properly tested in the OpenIDConnectIT suite]
    +
    +The following example output of the response contains the URI pointing to the End Session Endpoint of the
    +OpenID Connect Provider with all the parameters of the Logout Request, as HTTP GET parameters
    +
    +[source,js]
    +--------------------------------------------------
    +{
    +  "redirect" : "https://op-provider.org/logout?id_token_hint=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiIxMjM0NTY3ODkwIiwibmFtZSI6IkpvaG4gRG9lIiwiaWF0IjoxNTE2MjM5MDIyfQ.SflKxwRJSMeKKF2QT4fwpMeJf36POk6yJV_adQssw5c&post_logout_redirect_uri=http%3A%2F%2Foidc-kibana.elastic.co%2Floggedout&state=lGYK0EcSLjqH6pkT5EVZjC6eIW5YCGgywj2sxROO"
    +}
    +--------------------------------------------------
    +// NOTCONSOLE
    \ No newline at end of file
    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
    new file mode 100644
    index 0000000000000..aeb400ce97ef1
    --- /dev/null
    +++ b/x-pack/docs/en/rest-api/security/oidc-prepare-authentication-api.asciidoc
    @@ -0,0 +1,128 @@
    +[role="xpack"]
    +[[security-api-oidc-prepare-authentication]]
    +
    +=== OpenID Connect Prepare Authentication API
    +
    +Creates an oAuth 2.0 authentication request as a URL string based on the configuration of the respective
    +OpenID Connect authentication realm in {es}. The response of this API is a URL pointing to the Authorization Endpoint
    +of the configured OpenID Connect Provider and can be used to redirect the browser of the user in order to continue
    +the authentication process.
    +
    +{es} exposes all the necessary OpenID Connect related functionality via the OpenID Connect APIs. These APIs
    +are used internally by {kib} in order to provide OpenID Connect based authentication, but can also be used by other,
    +custom web applications or other clients. See also <>
    +and <>
    +
    +==== Request
    +
    +`POST /_security/oidc/prepare`
    +
    +
    +==== Request Body
    +
    +The following parameters can be specified in the body of the request:
    +
    +`realm`::
    +The name of the OpenID Connect realm in {es} the configuration of which should be used in order to
    +generate the authentication request. Cannot be specified when `iss` is specified.
    +
    +`state`::
    +String value used to maintain state between the authentication request and the response, typically used
    +as a Cross-Site Request Forgery mitigation. If the caller of the API doesn't provide a value, {es} will
    +generate one with sufficient entropy itself and return it in the response.
    +
    +`nonce`::
    +String value used to associate a Client session with an ID Token, and to mitigate replay attacks.
    +If the caller of the API doesn't provide a value, {es} will generate one with sufficient entropy itself
    +and return it in the response.
    +
    +`issuer`::
    +In the case of a 3rd Party initiated Single Sign On, this is the Issuer Identifier for the OP that the RP is
    +to send the Authentication Request to. Cannot be specified when `realm` is specified.
    +
    +`login_hint`::
    +In the case of a 3rd Party initiated Single Sign On, a string value to be included in the authentication
    +request, as the `login_hint` parameter. This parameter is not valid when `realm` is specified
    +
    +
    +==== Examples
    +
    +The following example generates an authentication request for the OpenID Connect Realm `oidc1`
    +
    +[source,js]
    +--------------------------------------------------
    +POST /_security/oidc/prepare
    +{
    +  "realm" : "oidc1"
    +}
    +--------------------------------------------------
    +// CONSOLE
    +// TEST[skip:These are properly tested in the OpenIDConnectIT suite]
    +
    +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]
    +--------------------------------------------------
    +{
    +  "redirect" : "https://op-provider.org/login?scope=openid&response_type=code&redirect_uri=http%3A%2F%2Foidc-kibana.elastic.co%3A5603%2Fkmi%2Fapi%2Fsecurity%2Fv1%2Foidc&state=4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I&nonce=WaBPH0KqPVdG5HHdSxPRjfoZbXMCicm5v1OiAj0DUFM&client_id=0o43gasov3TxMWJOt839",
    +  "state" : "4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I",
    +  "nonce" : "WaBPH0KqPVdG5HHdSxPRjfoZbXMCicm5v1OiAj0DUFM"
    +}
    +--------------------------------------------------
    +// NOTCONSOLE
    +
    +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]
    +--------------------------------------------------
    +POST /_security/oidc/prepare
    +{
    +  "realm" : "oidc1",
    +  "state" : "lGYK0EcSLjqH6pkT5EVZjC6eIW5YCGgywj2sxROO",
    +  "nonce" : "zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5"
    +}
    +--------------------------------------------------
    +// CONSOLE
    +// TEST[skip:These are properly tested in the OpenIDConnectIT suite]
    +
    +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]
    +--------------------------------------------------
    +{
    +  "redirect" : "https://op-provider.org/login?scope=openid&response_type=code&redirect_uri=http%3A%2F%2Foidc-kibana.elastic.co%3A5603%2Fkmi%2Fapi%2Fsecurity%2Fv1%2Foidc&state=lGYK0EcSLjqH6pkT5EVZjC6eIW5YCGgywj2sxROO&nonce=zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5&client_id=0o43gasov3TxMWJOt839",
    +  "state" : "lGYK0EcSLjqH6pkT5EVZjC6eIW5YCGgywj2sxROO",
    +  "nonce" : "zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5"
    +}
    +--------------------------------------------------
    +// NOTCONSOLE
    +
    +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]
    +--------------------------------------------------
    +POST /_security/oidc/prepare
    +{
    +  "issuer" : "https://op-issuer.org:8800",
    +  "login_hint": "this_is_an_opaque_string"
    +}
    +--------------------------------------------------
    +// CONSOLE
    +// TEST[skip:These are properly tested in the OpenIDConnectIT suite]
    +
    +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]
    +--------------------------------------------------
    +{
    +  "redirect" : "https://op-provider.org/login?scope=openid&response_type=code&redirect_uri=http%3A%2F%2Foidc-kibana.elastic.co%3A5603%2Fkmi%2Fapi%2Fsecurity%2Fv1%2Foidc&state=lGYK0EcSLjqH6pkT5EVZjC6eIW5YCGgywj2sxROO&nonce=zOBXLJGUooRrbLbQk5YCcyC8AXw3iloynvluYhZ5&client_id=0o43gasov3TxMWJOt839&login_hint=this_is_an_opaque_string",
    +  "state" : "4dbrihtIAt3wBTwo6DxK-vdk-sSyDBV8Yf0AjdkdT5I",
    +  "nonce" : "WaBPH0KqPVdG5HHdSxPRjfoZbXMCicm5v1OiAj0DUFM"
    +}
    +--------------------------------------------------
    +// NOTCONSOLE
    \ No newline at end of file
    
    From 9476a7866ac5340a76c45414aa2c74080e88112b Mon Sep 17 00:00:00 2001
    From: James Rodewig 
    Date: Mon, 22 Apr 2019 05:39:04 -0700
    Subject: [PATCH 138/260] [DOCS] Replace nested open block for Asciidoctor
     migration (#41168)
    
    * [DOCS] Fix nested open blocks for Asciidoctor migration
    
    * [DOCS] Reformat table to definitions
    ---
     docs/reference/index-modules.asciidoc | 29 +++++++++------------------
     1 file changed, 10 insertions(+), 19 deletions(-)
    
    diff --git a/docs/reference/index-modules.asciidoc b/docs/reference/index-modules.asciidoc
    index 7848a48fa58f9..96ccc9ab6a8a4 100644
    --- a/docs/reference/index-modules.asciidoc
    +++ b/docs/reference/index-modules.asciidoc
    @@ -45,26 +45,17 @@ specific index module:
         part of the cluster.
     
     `index.shard.check_on_startup`::
    -+
    ---
    -Whether or not shards should be checked for corruption before opening. When
    -corruption is detected, it will prevent the shard from being opened. Accepts:
    -
    -`false`::
    -
    -    (default) Don't check for corruption when opening a shard.
    -
    -`checksum`::
     
    -    Check for physical corruption.
    -
    -`true`::
    -
    -    Check for both physical and logical corruption. This is much more
    -    expensive in terms of CPU and memory usage.
    -
    -WARNING: Expert only. Checking shards may take a lot of time on large indices.
    ---
    +Whether or not shards should be checked for corruption before opening. When
    +corruption is detected, it will prevent the shard from being opened.
    +Accepts:
    +`false`::: (default) Don't check for corruption when opening a shard.
    +`checksum`::: Check for physical corruption.
    +`true`::: Check for both physical and logical corruption. This is much more
    +expensive in terms of CPU and memory usage.
    ++
    +WARNING: Expert only. Checking shards may take a lot of time on large
    +indices.
     
     [[index-codec]] `index.codec`::
     
    
    From 25d81c149b16f777ab629f9db409f2afd6f23350 Mon Sep 17 00:00:00 2001
    From: James Rodewig 
    Date: Mon, 22 Apr 2019 06:07:53 -0700
    Subject: [PATCH 139/260] [DOCS] Remove inline callouts for Asciidoctor
     migration (#41266)
    
    ---
     docs/reference/sql/endpoints/jdbc.asciidoc | 19 +++++++++++++------
     1 file changed, 13 insertions(+), 6 deletions(-)
    
    diff --git a/docs/reference/sql/endpoints/jdbc.asciidoc b/docs/reference/sql/endpoints/jdbc.asciidoc
    index 7b1169d34d323..9014d3f5719b5 100644
    --- a/docs/reference/sql/endpoints/jdbc.asciidoc
    +++ b/docs/reference/sql/endpoints/jdbc.asciidoc
    @@ -51,14 +51,21 @@ Once registered, the driver understands the following syntax as an URL:
     
     ["source","text",subs="attributes"]
     ----
    -jdbc:es://<1>[[http|https]://]*<2>[host[:port]]*<3>/[prefix]*<4>[?[option=value]&<5>]*
    +jdbc:es://[[http|https]://]*[host[:port]]*/[prefix]*<[?[option=value]&]*
     ----
    +`jdbc:es://`:: Prefix. Mandatory.
     
    -<1> `jdbc:es://` prefix. Mandatory.
    -<2> type of HTTP connection to make - `http` (default) or `https`. Optional.
    -<3> host (`localhost` by default) and port (`9200` by default). Optional.
    -<4> prefix (empty by default). Typically used when hosting {es} under a certain path. Optional.
    -<5> Properties for the JDBC driver. Empty by default. Optional.
    +`[[http|https]://]`:: Type of HTTP connection to make. Possible values are
    +`http` (default) or `https`. Optional.
    +
    +`[host[:port]]`:: Host (`localhost` by default) and port (`9200` by default).
    +Optional.
    +
    +`[prefix]`:: Prefix (empty by default). Typically used when hosting {es} under
    +a certain path. Optional.
    +
    +`[option=value]`:: Properties for the JDBC driver. Empty by default.
    +Optional.
     
     The driver recognized the following properties:
     
    
    From d042f263ce70f8be635cae960d7eb3a7c0bd2265 Mon Sep 17 00:00:00 2001
    From: James Rodewig 
    Date: Mon, 22 Apr 2019 06:22:41 -0700
    Subject: [PATCH 140/260] [DOCS] Remove inline callouts in SQL Command docs for
     Asciidoctor migration (#41276)
    
    ---
     .../sql/language/syntax/commands/describe-table.asciidoc  | 8 ++++++--
     .../sql/language/syntax/commands/show-columns.asciidoc    | 4 +++-
     .../sql/language/syntax/commands/show-functions.asciidoc  | 2 +-
     .../sql/language/syntax/commands/show-tables.asciidoc     | 4 +++-
     4 files changed, 13 insertions(+), 5 deletions(-)
    
    diff --git a/docs/reference/sql/language/syntax/commands/describe-table.asciidoc b/docs/reference/sql/language/syntax/commands/describe-table.asciidoc
    index 81f2d386c26ed..da02f1fa23817 100644
    --- a/docs/reference/sql/language/syntax/commands/describe-table.asciidoc
    +++ b/docs/reference/sql/language/syntax/commands/describe-table.asciidoc
    @@ -6,7 +6,9 @@
     .Synopsis
     [source, sql]
     ----
    -DESCRIBE [table identifier<1> | [LIKE pattern<2>]]
    +DESCRIBE
    +    [table identifier | <1>
    +    [LIKE pattern]]     <2>
     ----
     
     <1> single table identifier or double quoted es multi index
    @@ -16,7 +18,9 @@ or
     
     [source, sql]
     ----
    -DESC [table identifier<1>|[LIKE pattern<2>]]
    +DESC 
    +    [table identifier | <1>
    +    [LIKE pattern]]     <2>
     ----
     
     <1> single table identifier or double quoted es multi index
    diff --git a/docs/reference/sql/language/syntax/commands/show-columns.asciidoc b/docs/reference/sql/language/syntax/commands/show-columns.asciidoc
    index 6b6acc0c4ac5e..b21c02358e526 100644
    --- a/docs/reference/sql/language/syntax/commands/show-columns.asciidoc
    +++ b/docs/reference/sql/language/syntax/commands/show-columns.asciidoc
    @@ -6,7 +6,9 @@
     .Synopsis
     [source, sql]
     ----
    -SHOW COLUMNS [ FROM | IN ]? [ table identifier<1> | [ LIKE pattern<2> ] ]
    +SHOW COLUMNS [ FROM | IN ]?
    +    [table identifier | <1>
    +    [LIKE pattern] ]    <2>
     ----
     
     <1> single table identifier or double quoted es multi index
    diff --git a/docs/reference/sql/language/syntax/commands/show-functions.asciidoc b/docs/reference/sql/language/syntax/commands/show-functions.asciidoc
    index e12c4efed590a..47c000e81d9fd 100644
    --- a/docs/reference/sql/language/syntax/commands/show-functions.asciidoc
    +++ b/docs/reference/sql/language/syntax/commands/show-functions.asciidoc
    @@ -6,7 +6,7 @@
     .Synopsis
     [source, sql]
     ----
    -SHOW FUNCTIONS [ LIKE pattern<1>? ]?
    +SHOW FUNCTIONS [LIKE pattern?]? <1>
     ----
     
     <1> SQL match pattern
    diff --git a/docs/reference/sql/language/syntax/commands/show-tables.asciidoc b/docs/reference/sql/language/syntax/commands/show-tables.asciidoc
    index 28b5ad4c75d5c..691d328aa4bdd 100644
    --- a/docs/reference/sql/language/syntax/commands/show-tables.asciidoc
    +++ b/docs/reference/sql/language/syntax/commands/show-tables.asciidoc
    @@ -6,7 +6,9 @@
     .Synopsis
     [source, sql]
     ----
    -SHOW TABLES [ table identifier<1> | [ LIKE pattern<2> ] ]?
    +SHOW TABLES
    +    [table identifier | <1>
    +    [LIKE pattern ]]?   <2>
     ----
     
     <1> single table identifier or double quoted es multi index
    
    From 9982888f4e38c3572d2e567cab6f6e01680cc88c Mon Sep 17 00:00:00 2001
    From: James Rodewig 
    Date: Mon, 22 Apr 2019 06:33:55 -0700
    Subject: [PATCH 141/260] [DOCS] Remove inline callouts for Asciidoctor
     migration (#41309)
    
    ---
     docs/reference/sql/functions/aggs.asciidoc    | 42 +++++++------
     .../sql/functions/date-time.asciidoc          | 38 ++++++------
     .../reference/sql/functions/grouping.asciidoc |  9 ++-
     .../sql/functions/like-rlike.asciidoc         |  6 +-
     docs/reference/sql/functions/math.asciidoc    | 60 +++++++++++--------
     docs/reference/sql/functions/search.asciidoc  |  9 ++-
     6 files changed, 97 insertions(+), 67 deletions(-)
    
    diff --git a/docs/reference/sql/functions/aggs.asciidoc b/docs/reference/sql/functions/aggs.asciidoc
    index cc0f06cb3bb5e..2beef2c65ea71 100644
    --- a/docs/reference/sql/functions/aggs.asciidoc
    +++ b/docs/reference/sql/functions/aggs.asciidoc
    @@ -16,7 +16,7 @@ Functions for computing a _single_ result from a set of input values.
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -AVG(numeric_field<1>)
    +AVG(numeric_field) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -40,7 +40,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggAvg]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -COUNT(expression<1>)
    +COUNT(expression) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -70,7 +70,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggCountStar]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -COUNT(ALL field_name<1>)
    +COUNT(ALL field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -95,7 +95,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggCountAll]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -COUNT(DISTINCT field_name<1>)
    +COUNT(DISTINCT field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -119,7 +119,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggCountDistinct]
     .Synopsis:
     [source, sql]
     ----------------------------------------------
    -FIRST(field_name<1>[, ordering_field_name]<2>)
    +FIRST(
    +    field_name               <1>
    +    [, ordering_field_name]) <2>
     ----------------------------------------------
     
     *Input*:
    @@ -214,7 +216,9 @@ the field is also <>.
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -LAST(field_name<1>[, ordering_field_name]<2>)
    +LAST(
    +    field_name               <1>
    +    [, ordering_field_name]) <2>
     --------------------------------------------------
     
     *Input*:
    @@ -309,7 +313,7 @@ the field is also <>.
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MAX(field_name<1>)
    +MAX(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -337,7 +341,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggMax]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MIN(field_name<1>)
    +MIN(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -365,7 +369,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggMin]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SUM(field_name<1>)
    +SUM(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -393,7 +397,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggSum]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -KURTOSIS(field_name<1>)
    +KURTOSIS(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -417,7 +421,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggKurtosis]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MAD(field_name<1>)
    +MAD(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -441,7 +445,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggMad]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -PERCENTILE(field_name<1>, numeric_exp<2>)
    +PERCENTILE(
    +    field_name,  <1>
    +    numeric_exp) <2>
     --------------------------------------------------
     
     *Input*:
    @@ -467,7 +473,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggPercentile]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -PERCENTILE_RANK(field_name<1>, numeric_exp<2>)
    +PERCENTILE_RANK(
    +    field_name,  <1>
    +    numeric_exp) <2>
     --------------------------------------------------
     
     *Input*:
    @@ -493,7 +501,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggPercentileRank]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SKEWNESS(field_name<1>)
    +SKEWNESS(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -517,7 +525,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggSkewness]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -STDDEV_POP(field_name<1>)
    +STDDEV_POP(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -541,7 +549,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggStddevPop]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SUM_OF_SQUARES(field_name<1>)
    +SUM_OF_SQUARES(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -565,7 +573,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[aggSumOfSquares]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -VAR_POP(field_name<1>)
    +VAR_POP(field_name) <1>
     --------------------------------------------------
     
     *Input*:
    diff --git a/docs/reference/sql/functions/date-time.asciidoc b/docs/reference/sql/functions/date-time.asciidoc
    index 782ea15f62741..d9d5e7bcf14e5 100644
    --- a/docs/reference/sql/functions/date-time.asciidoc
    +++ b/docs/reference/sql/functions/date-time.asciidoc
    @@ -146,7 +146,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[filterToday]
     [source, sql]
     --------------------------------------------------
     CURRENT_TIME
    -CURRENT_TIME([precision <1>])
    +CURRENT_TIME([precision]) <1>
     CURTIME
     --------------------------------------------------
     
    @@ -203,7 +203,7 @@ function as the maximum number of second fractional digits returned is 3 (millis
     [source, sql]
     --------------------------------------------------
     CURRENT_TIMESTAMP
    -CURRENT_TIMESTAMP([precision <1>])
    +CURRENT_TIMESTAMP([precision]) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -254,7 +254,7 @@ function as the maximum number of second fractional digits returned is 3 (millis
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -DAY_OF_MONTH(datetime_exp<1>)
    +DAY_OF_MONTH(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -278,7 +278,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfMonth]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -DAY_OF_WEEK(datetime_exp<1>)
    +DAY_OF_WEEK(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -302,7 +302,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfWeek]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -DAY_OF_YEAR(datetime_exp<1>)
    +DAY_OF_YEAR(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -326,7 +326,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[dayOfYear]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -DAY_NAME(datetime_exp<1>)
    +DAY_NAME(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -350,7 +350,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[dayName]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -HOUR_OF_DAY(datetime_exp<1>)
    +HOUR_OF_DAY(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -374,7 +374,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[hourOfDay]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ISO_DAY_OF_WEEK(datetime_exp<1>)
    +ISO_DAY_OF_WEEK(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -399,7 +399,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[isoDayOfWeek]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ISO_WEEK_OF_YEAR(datetime_exp<1>)
    +ISO_WEEK_OF_YEAR(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -424,7 +424,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[isoWeekOfYear]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MINUTE_OF_DAY(datetime_exp<1>)
    +MINUTE_OF_DAY(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -448,7 +448,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[minuteOfDay]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MINUTE_OF_HOUR(datetime_exp<1>)
    +MINUTE_OF_HOUR(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -472,7 +472,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[minuteOfHour]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MONTH(datetime_exp<1>)
    +MONTH(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -496,7 +496,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[monthOfYear]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MONTH_NAME(datetime_exp<1>)
    +MONTH_NAME(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -552,7 +552,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[filterNow]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SECOND_OF_MINUTE(datetime_exp<1>)
    +SECOND_OF_MINUTE(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -576,7 +576,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[secondOfMinute]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -QUARTER(datetime_exp<1>)
    +QUARTER(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -632,7 +632,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[filterToday]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -WEEK_OF_YEAR(datetime_exp<1>)
    +WEEK_OF_YEAR(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -656,7 +656,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[weekOfYear]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -YEAR(datetime_exp<1>)
    +YEAR(datetime_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -680,7 +680,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[year]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -EXTRACT(datetime_function<1> FROM datetime_exp<2>)
    +EXTRACT(
    +    datetime_function  <1>
    +    FROM datetime_exp) <2>
     --------------------------------------------------
     
     *Input*:
    diff --git a/docs/reference/sql/functions/grouping.asciidoc b/docs/reference/sql/functions/grouping.asciidoc
    index 742f072dbd039..0a498a1aacef0 100644
    --- a/docs/reference/sql/functions/grouping.asciidoc
    +++ b/docs/reference/sql/functions/grouping.asciidoc
    @@ -12,8 +12,13 @@ as part of the <>.
     .Synopsis:
     [source, sql]
     ----
    -HISTOGRAM(numeric_exp<1>, numeric_interval<2>)
    -HISTOGRAM(date_exp<3>, date_time_interval<4>)
    +HISTOGRAM(
    +    numeric_exp,        <1>
    +    numeric_interval)   <2>
    +
    +HISTOGRAM(
    +    date_exp,           <3>
    +    date_time_interval) <4>
     ----
     
     *Input*:
    diff --git a/docs/reference/sql/functions/like-rlike.asciidoc b/docs/reference/sql/functions/like-rlike.asciidoc
    index c38f62ae7d7c1..73212bc113542 100644
    --- a/docs/reference/sql/functions/like-rlike.asciidoc
    +++ b/docs/reference/sql/functions/like-rlike.asciidoc
    @@ -20,7 +20,8 @@ or has an exact sub-field, it will use it as is, or it will automatically use th
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -expression<1> LIKE constant_exp<2>
    +expression        <1>
    +LIKE constant_exp <2>
     --------------------------------------------------
     
     <1> typically a field, or a constant expression
    @@ -59,7 +60,8 @@ IMPORTANT: Even though `LIKE` is a valid option when searching or filtering in {
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -expression<1> RLIKE constant_exp<2>
    +expression         <1>
    +RLIKE constant_exp <2>
     --------------------------------------------------
     
     <1> typically a field, or a constant expression
    diff --git a/docs/reference/sql/functions/math.asciidoc b/docs/reference/sql/functions/math.asciidoc
    index f77b6975604d2..ebef8a305bcc6 100644
    --- a/docs/reference/sql/functions/math.asciidoc
    +++ b/docs/reference/sql/functions/math.asciidoc
    @@ -16,7 +16,7 @@ to be numeric.
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ABS(numeric_exp<1>)
    +ABS(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -40,7 +40,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[abs]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -CBRT(numeric_exp<1>)
    +CBRT(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -64,7 +64,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineCbrtWithNegativeValue]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -CEIL(numeric_exp<1>)
    +CEIL(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -110,7 +110,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathEulersNumber]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -EXP(numeric_exp<1>)
    +EXP(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -134,7 +134,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathExpInline]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -EXPM1(numeric_exp<1>)
    +EXPM1(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -158,7 +158,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathExpm1Inline]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -FLOOR(numeric_exp<1>)
    +FLOOR(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -182,7 +182,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineFloor]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -LOG(numeric_exp<1>)
    +LOG(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -206,7 +206,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineLog]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -LOG10(numeric_exp<1>)
    +LOG10(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -252,7 +252,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathPINumber]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -POWER(numeric_exp<1>, integer_exp<2>)
    +POWER(
    +    numeric_exp, <1>
    +    integer_exp) <2>
     --------------------------------------------------
     
     *Input*:
    @@ -282,7 +284,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlinePowerNegative]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -RANDOM(seed<1>)
    +RANDOM(seed) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -306,7 +308,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathRandom]
     .Synopsis:
     [source, sql]
     ----
    -ROUND(numeric_exp<1>[, integer_exp<2>])
    +ROUND(
    +    numeric_exp      <1>
    +    [, integer_exp]) <2>
     ----
     *Input*:
     
    @@ -337,7 +341,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathRoundWithNegativeParameter]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SIGN(numeric_exp<1>)
    +SIGN(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -362,7 +366,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineSign]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SQRT(numeric_exp<1>)
    +SQRT(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -386,7 +390,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineSqrt]
     .Synopsis:
     [source, sql]
     ----
    -TRUNCATE(numeric_exp<1>[, integer_exp<2>])
    +TRUNCATE(
    +    numeric_exp      <1>
    +    [, integer_exp]) <2>
     ----
     *Input*:
     
    @@ -421,7 +427,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathTruncateWithNegativeParameter
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ACOS(numeric_exp<1>)
    +ACOS(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -445,7 +451,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineAcos]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ASIN(numeric_exp<1>)
    +ASIN(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -469,7 +475,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineAsin]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ATAN(numeric_exp<1>)
    +ATAN(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -493,7 +499,9 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineAtan]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -ATAN2(ordinate<1>, abscisa<2>)
    +ATAN2(
    +    ordinate, <1>
    +    abscisa)  <2>
     --------------------------------------------------
     
     *Input*:
    @@ -518,7 +526,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineAtan2]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -COS(numeric_exp<1>)
    +COS(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -542,7 +550,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineCosine]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -COSH(numeric_exp<1>)
    +COSH(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -566,7 +574,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineCosh]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -COT(numeric_exp<1>)
    +COT(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -590,7 +598,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineCotangent]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -DEGREES(numeric_exp<1>)
    +DEGREES(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -615,7 +623,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineDegrees]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -RADIANS(numeric_exp<1>)
    +RADIANS(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -640,7 +648,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineRadians]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SIN(numeric_exp<1>)
    +SIN(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -664,7 +672,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineSine]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -SINH(numeric_exp<1>)
    +SINH(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    @@ -688,7 +696,7 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[mathInlineSinh]
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -TAN(numeric_exp<1>)
    +TAN(numeric_exp) <1>
     --------------------------------------------------
     
     *Input*:
    diff --git a/docs/reference/sql/functions/search.asciidoc b/docs/reference/sql/functions/search.asciidoc
    index 0534271caa91f..6990f6669d69c 100644
    --- a/docs/reference/sql/functions/search.asciidoc
    +++ b/docs/reference/sql/functions/search.asciidoc
    @@ -14,7 +14,10 @@ such as `0` or `NULL`.
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -MATCH(field_exp<1>, constant_exp<2>[, options]<3>)
    +MATCH(
    +    field_exp,   <1>
    +    constant_exp <2>
    +    [, options]) <3>
     --------------------------------------------------
     
     *Input*:
    @@ -75,7 +78,9 @@ NOTE: The allowed optional parameters for a multi-field `MATCH()` variant (for t
     .Synopsis:
     [source, sql]
     --------------------------------------------------
    -QUERY(constant_exp<1>[, options]<2>)
    +QUERY(
    +    constant_exp <1>
    +    [, options]) <2>
     --------------------------------------------------
     
     *Input*:
    
    From 8b2577406f47ae60d15803058921d128390af0b6 Mon Sep 17 00:00:00 2001
    From: Marios Trivyzas 
    Date: Mon, 22 Apr 2019 19:26:15 +0300
    Subject: [PATCH 142/260] SQL: Implement CASE... WHEN... THEN... ELSE... END
     (#41349)
    
    Implement the ANSI SQL CASE expression which provides the if/else
    functionality common to most programming languages.
    
    The CASE expression can have multiple WHEN branches and becomes a
    powerful tool for SQL queries as it can be used in SELECT, WHERE,
    GROUP BY, HAVING and ORDER BY clauses.
    
    Closes: #36200
    ---
     .../sql/functions/conditional.asciidoc        |   95 +-
     docs/reference/sql/functions/index.asciidoc   |    1 +
     .../qa/src/main/resources/command.csv-spec    |    1 +
     .../src/main/resources/conditionals.csv-spec  |  181 ++
     .../qa/src/main/resources/docs/docs.csv-spec  |   74 +-
     .../sql/qa/src/main/resources/null.csv-spec   |   36 -
     .../sql/qa/src/main/resources/null.sql-spec   |   17 +-
     x-pack/plugin/sql/src/main/antlr/SqlBase.g4   |   30 +-
     .../plugin/sql/src/main/antlr/SqlBase.tokens  |  446 ++--
     .../sql/src/main/antlr/SqlBaseLexer.tokens    |  444 ++--
     .../expression/function/FunctionRegistry.java |    4 +-
     .../function/scalar/Processors.java           |    4 +-
     .../whitelist/InternalSqlScriptUtils.java     |    7 +-
     .../predicate/conditional/Case.java           |  164 ++
     .../predicate/conditional/CasePipe.java       |   38 +
     .../predicate/conditional/CaseProcessor.java  |   77 +
     .../conditional/ConditionalFunction.java      |   18 +
     .../predicate/conditional/IfConditional.java  |   92 +
     .../xpack/sql/optimizer/Optimizer.java        |   37 +
     .../xpack/sql/parser/ExpressionBuilder.java   |   22 +
     .../xpack/sql/parser/SqlBaseBaseListener.java |   24 +
     .../xpack/sql/parser/SqlBaseBaseVisitor.java  |   14 +
     .../xpack/sql/parser/SqlBaseLexer.java        |  859 ++++----
     .../xpack/sql/parser/SqlBaseListener.java     |   22 +
     .../xpack/sql/parser/SqlBaseParser.java       | 1863 +++++++++--------
     .../xpack/sql/parser/SqlBaseVisitor.java      |   13 +
     .../xpack/sql/plugin/sql_whitelist.txt        |    3 +-
     .../analyzer/VerifierErrorMessagesTests.java  |   20 +
     .../predicate/conditional/CaseTests.java      |  100 +
     .../xpack/sql/optimizer/OptimizerTests.java   |   94 +-
     .../xpack/sql/parser/ExpressionTests.java     |   62 +
     .../sql/planner/QueryTranslatorTests.java     |   15 +
     .../xpack/sql/tree/NodeSubclassTests.java     |    6 +-
     33 files changed, 3114 insertions(+), 1769 deletions(-)
     create mode 100644 x-pack/plugin/sql/qa/src/main/resources/conditionals.csv-spec
     create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Case.java
     create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CasePipe.java
     create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseProcessor.java
     create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IfConditional.java
     create mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java
    
    diff --git a/docs/reference/sql/functions/conditional.asciidoc b/docs/reference/sql/functions/conditional.asciidoc
    index 135381a1c93bd..cf15504bbe379 100644
    --- a/docs/reference/sql/functions/conditional.asciidoc
    +++ b/docs/reference/sql/functions/conditional.asciidoc
    @@ -1,10 +1,103 @@
     [role="xpack"]
     [testenv="basic"]
     [[sql-functions-conditional]]
    -=== Conditional Functions
    +=== Conditional Functions And Expressions
     
     Functions that return one of their arguments by evaluating in an if-else manner.
     
    +[[sql-functions-conditional-case]]
    +==== `CASE`
    +
    +.Synopsis:
    +[source, sql]
    +----
    +CASE WHEN condition THEN result
    +    [WHEN ...]
    +    [ELSE default_result]
    +END
    +----
    +
    +*Input*:
    +
    +One or multiple _WHEN *condition* THEN *result_* clauses are used and the expression can optionally have
    +an _ELSE *default_result_* clause. Every *condition* should be a boolean expression.
    +
    +*Output*: one of the *result* expressions if the corresponding _WHEN *condition_* evaluates to `true` or
    +the *default_result* if all _WHEN *condition_* clauses evaluate to `false`. If the optional _ELSE *default_result_*
    +clause is missing and all _WHEN *condition_* clauses evaluate to `false` then `null` is returned.
    +
    +.Description
    +
    +The CASE expression is a generic conditional expression which simulates if/else statements of other programming languages
    +If the condition’s result is true, the value of the result expression that follows the condition will be the returned
    +the subsequent when clauses will be skipped and not processed.
    +
    +
    +["source","sql",subs="attributes,callouts,macros"]
    +----
    +include-tagged::{sql-specs}/docs/docs.csv-spec[case]
    +----
    +
    +["source","sql",subs="attributes,callouts,macros"]
    +----
    +include-tagged::{sql-specs}/docs/docs.csv-spec[caseReturnNull]
    +----
    +
    +["source","sql",subs="attributes,callouts,macros"]
    +----
    +include-tagged::{sql-specs}/docs/docs.csv-spec[caseWithElse]
    +----
    +
    +
    +As a variant, a case expression can be expressed with a syntax similar to *switch-case* of other programming languages:
    +[source, sql]
    +----
    +CASE expression
    +     WHEN value1 THEN result1
    +    [WHEN value2 THEN result2]
    +    [WHEN ...]
    +    [ELSE default_result]
    +END
    +----
    +
    +In this case it's transformed internally to:
    +[source, sql]
    +----
    +CASE WHEN expression = value1 THEN result1
    +    [WHEN expression = value2 THEN result2]
    +    [WHEN ...]
    +    [ELSE default_result]
    +END
    +----
    +
    +["source","sql",subs="attributes,callouts,macros"]
    +----
    +include-tagged::{sql-specs}/docs/docs.csv-spec[caseWithOperand]
    +----
    +
    +["source","sql",subs="attributes,callouts,macros"]
    +----
    +include-tagged::{sql-specs}/docs/docs.csv-spec[caseWithOperandAndElse]
    +----
    +
    +[NOTE]
    +===============================
    +All result expressions must be of compatible data types. More specifically all result
    +expressions should have a compatible data type with the 1st _non-null_ result expression.
    +E.g.:
    +
    +for the following query:
    +
    +[source, sql]
    +CASE WHEN a = 1 THEN null
    +     WHEN a > 2 THEN 10
    +     WHEN a > 5 THEN 'foo'
    +END
    +
    +an error message would be returned, mentioning that *'foo'* is of data type *keyword*,
    +which does not match the expected data type *integer* (based on result *10*).
    +===============================
    +
     [[sql-functions-conditional-coalesce]]
     ==== `COALESCE`
     
    diff --git a/docs/reference/sql/functions/index.asciidoc b/docs/reference/sql/functions/index.asciidoc
    index 1fa2778ee05f4..831a9d0a4e75b 100644
    --- a/docs/reference/sql/functions/index.asciidoc
    +++ b/docs/reference/sql/functions/index.asciidoc
    @@ -127,6 +127,7 @@
     ** <>
     ** <>
     * <>
    +** <>
     ** <>
     ** <>
     ** <>
    diff --git a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec
    index ea7cfb2c6a2ae..73ef70a96a4ed 100644
    --- a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec
    +++ b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec
    @@ -25,6 +25,7 @@ STDDEV_POP       |AGGREGATE
     SUM_OF_SQUARES   |AGGREGATE      
     VAR_POP          |AGGREGATE      
     HISTOGRAM        |GROUPING        
    +CASE             |CONDITIONAL
     COALESCE         |CONDITIONAL
     GREATEST         |CONDITIONAL
     IFNULL           |CONDITIONAL
    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
    new file mode 100644
    index 0000000000000..2a4eaa501a191
    --- /dev/null
    +++ b/x-pack/plugin/sql/qa/src/main/resources/conditionals.csv-spec
    @@ -0,0 +1,181 @@
    +caseField
    +SELECT emp_no, CASE WHEN emp_no - 10000 < 10 THEN 'First 10' ELSE 'Second 10' END as "case" FROM test_emp WHERE emp_no >= 10005
    +ORDER BY emp_no LIMIT 10;
    +
    + emp_no |   case
    +--------+-----------
    +10005   | First 10
    +10006   | First 10
    +10007   | First 10
    +10008   | First 10
    +10009   | First 10
    +10010   | Second 10
    +10011   | Second 10
    +10012   | Second 10
    +10013   | Second 10
    +10014   | Second 10
    +;
    +
    +caseFieldWithoutAlias
    +SELECT emp_no, CASE WHEN emp_no - 10000 < 10 THEN emp_no ELSE emp_no % 10 END FROM test_emp WHERE emp_no >= 10005
    +ORDER BY emp_no LIMIT 10;
    +
    + emp_no | CASE WHEN emp_no - 10000 < 10 THEN emp_no ELSE emp_no % 10 END
    +--------+----------------------------------------------------------------
    +10005   | 10005
    +10006   | 10006
    +10007   | 10007
    +10008   | 10008
    +10009   | 10009
    +10010   | 0
    +10011   | 1
    +10012   | 2
    +10013   | 3
    +10014   | 4
    +;
    +
    +caseFieldNoElse
    +SELECT emp_no, CASE WHEN emp_no - 10000 < 10 THEN 'First 10' END as "case" FROM test_emp WHERE emp_no >= 10005
    +ORDER BY emp_no LIMIT 10;
    +
    + emp_no |   case
    +--------+----------
    +10005   | First 10
    +10006   | First 10
    +10007   | First 10
    +10008   | First 10
    +10009   | First 10
    +10010   | null
    +10011   | null
    +10012   | null
    +10013   | null
    +10014   | null
    +;
    +
    +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;
    +
    + last_name
    +-----------
    +Facello
    +Bamford
    +Koblick
    +Maliniak
    +Preusig
    +Zielinski
    +Kalloufi
    +Piveteau
    +Bridgland
    +Nooteboom
    +;
    +
    +caseWhereNoElse
    +SELECT last_name FROM test_emp WHERE CASE WHEN LENGTH(last_name) < 7 THEN 'ShortName' END IS NOT NULL
    +ORDER BY emp_no LIMIT 10;
    +
    + last_name
    +-----------
    +Simmel
    +Peac
    +Sluis
    +Terkki
    +Genin
    +Peha
    +Erde
    +Famili
    +Pettey
    +Heyers
    +;
    +
    +caseOrderBy
    +schema::last_name:s|languages:byte|emp_no:i
    +SELECT last_name, languages, emp_no FROM test_emp WHERE emp_no BETWEEN 10005 AND 10015
    +ORDER BY CASE WHEN languages >= 3 THEN 'first' ELSE 'second' END, emp_no LIMIT 10;
    +
    + last_name | languages | emp_no
    +-----------+-----------+--------
    + Preusig   | 3         | 10006
    + Zielinski | 4         | 10007
    + Piveteau  | 4         | 10010
    + Sluis     | 5         | 10011
    + Bridgland | 5         | 10012
    + Genin     | 5         | 10014
    + Nooteboom | 5         | 10015
    + Maliniak  | 1         | 10005
    + Kalloufi  | 2         | 10008
    + Peac      | 1         | 10009
    +;
    +
    +caseOrderByNoElse
    +schema::last_name:s|languages:byte|emp_no:i
    +SELECT last_name, languages, emp_no FROM test_emp WHERE emp_no BETWEEN 10005 AND 10015
    +ORDER BY CASE WHEN languages >= 3 THEN 'first' END NULLS FIRST, emp_no LIMIT 10;
    +
    + last_name | languages | emp_no
    +-----------+-----------+--------
    +Maliniak   | 1         | 10005
    +Kalloufi   | 2         | 10008
    +Peac       | 1         | 10009
    +Terkki     | 1         | 10013
    +Preusig    | 3         | 10006
    +Zielinski  | 4         | 10007
    +Piveteau   | 4         | 10010
    +Sluis      | 5         | 10011
    +Bridgland  | 5         | 10012
    +Genin      | 5         | 10014
    +;
    +
    +caseGroupBy
    +schema::count:l|lang_skills:s
    +SELECT count(*) AS count, CASE WHEN NVL(languages, 0) <= 1  THEN 'zero-to-one' ELSE 'multilingual' END as lang_skills
    +FROM test_emp GROUP BY lang_skills ORDER BY lang_skills;
    +
    + count | lang_skills
    +-------+--------------
    +75     | multilingual
    +25     | zero-to-one
    +;
    +
    +caseGroupByNoElse
    +schema::count:l|lang_skills:s
    +SELECT count(*) AS count, CASE WHEN NVL(languages, 0) <= 1  THEN 'zero-to-one' END as lang_skills
    +FROM test_emp GROUP BY lang_skills ORDER BY lang_skills;
    +
    + count | lang_skills
    +-------+-------------
    +75     | null
    +25     | zero-to-one
    +;
    +
    +caseGroupByComplexNested
    +schema::count:l|lang_skills:s
    +SELECT count(*) AS count,
    +CASE WHEN NVL(languages, 0) = 0 THEN 'zero'
    +     WHEN languages = 1 THEN 'one'
    +     WHEN languages = 2 THEN 'bilingual'
    +     WHEN languages = 3 THEN 'trilingual'
    +     ELSE 'multilingual'
    +END as lang_skills FROM test_emp GROUP BY lang_skills ORDER BY 2;
    +
    + count | lang_skills
    +-------+--------------
    +19     | bilingual
    +39     | multilingual
    +15     | one
    +17     | trilingual
    +10     | zero
    +;
    +
    +caseGroupByAndHaving
    +schema::count:l|gender:s|languages:byte
    +SELECT count(*) AS count, gender, languages FROM test_emp
    +GROUP BY 2, 3 HAVING CASE WHEN count(*) > 10 THEN 'many' ELSE 'a few' END = 'many'
    +ORDER BY 2, 3;
    +
    +count     |    gender   |   languages
    +----------+-------------+---------------
    +11        | M           | 2
    +11        | M           | 3
    +11        | M           | 4
    +;
    diff --git a/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec
    index d7a60200a5ed3..bc7180de4d619 100644
    --- a/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec
    +++ b/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec
    @@ -202,6 +202,7 @@ STDDEV_POP       |AGGREGATE
     SUM_OF_SQUARES   |AGGREGATE      
     VAR_POP          |AGGREGATE      
     HISTOGRAM        |GROUPING
    +CASE             |CONDITIONAL
     COALESCE         |CONDITIONAL
     GREATEST         |CONDITIONAL
     IFNULL           |CONDITIONAL
    @@ -1987,10 +1988,81 @@ SELECT TRUNCATE(-345.153, 1) AS trimmed;
     
     ///////////////////////////////
     //
    -// Null handling
    +// Conditional
     //
     ///////////////////////////////
     
    +case
    +schema::case:s
    +// tag::case
    +SELECT CASE WHEN 1 > 2 THEN 'elastic'
    +            WHEN 2 <= 3 THEN 'search'
    +       END AS "case";
    +
    +    case
    +---------------
    +search
    +// end::case
    +;
    +
    +caseReturnNull
    +schema::case:s
    +// tag::caseReturnNull
    +SELECT CASE WHEN 1 > 2 THEN 'elastic'
    +            WHEN 2 > 10 THEN 'search'
    +       END AS "case";
    +
    +    case
    +---------------
    +null
    +// end::caseReturnNull
    +;
    +
    +caseWithElse
    +schema::case:s
    +// tag::caseWithElse
    +SELECT CASE WHEN 1 > 2 THEN 'elastic'
    +            WHEN 2 > 10 THEN 'search'
    +            ELSE 'default'
    +       END AS "case";
    +
    +    case
    +---------------
    +default
    +// end::caseWithElse
    +;
    +
    +caseWithOperand
    +schema::case:s
    +// tag::caseWithOperand
    +SELECT CASE 5
    +            WHEN 1 THEN 'elastic'
    +            WHEN 2 THEN 'search'
    +            WHEN 5 THEN 'elasticsearch'
    +       END AS "case";
    +
    +    case
    +---------------
    +elasticsearch
    +// end::caseWithOperand
    +;
    +
    +caseWithOperandAndElse
    +schema::case:s
    +// tag::caseWithOperandAndElse
    +SELECT CASE 5
    +            WHEN 1 THEN 'elastic'
    +            WHEN 2 THEN 'search'
    +            WHEN 3 THEN 'elasticsearch'
    +            ELSE 'default'
    +       END AS "case";
    +
    +    case
    +---------------
    +default
    +// end::caseWithOperandAndElse
    +;
    +
     coalesceReturnNonNull
     // tag::coalesceReturnNonNull
     SELECT COALESCE(null, 'elastic', 'search') AS "coalesce";
    diff --git a/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec
    index 610217b233314..df76250694e7c 100644
    --- a/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec
    +++ b/x-pack/plugin/sql/qa/src/main/resources/null.csv-spec
    @@ -38,45 +38,9 @@ c
     null
     ;
     
    -coalesceFirstNotNull
    -SELECT COALESCE(123) AS c;
    -
    -c
    -123
    -;
    -
    -
    -coalesceWithFirstNullOfString
    -SELECT COALESCE(null, 'first') AS c;
    -
    -c:s
    -first
    -;
    -
    -coalesceWithFirstNullOfNumber
    -SELECT COALESCE(null, 123) AS c;
    -
    -c:i
    -123
    -;
    -
     coalesceMixed
     SELECT COALESCE(null, 123, null, 321);
     
     COALESCE(null, 123, null, 321):i
     123
     ;
    -
    -coalesceMixedWithAlias
    -SELECT COALESCE(null, 123, null, 321) AS c;
    -
    -c:i
    -123
    -;
    -
    -coalesceScalar
    -SELECT COALESCE(null, ABS(123) + 1) AS c;
    -
    -c:i
    -124
    -;
    diff --git a/x-pack/plugin/sql/qa/src/main/resources/null.sql-spec b/x-pack/plugin/sql/qa/src/main/resources/null.sql-spec
    index d5a21262ca594..fce74d7569e4c 100644
    --- a/x-pack/plugin/sql/qa/src/main/resources/null.sql-spec
    +++ b/x-pack/plugin/sql/qa/src/main/resources/null.sql-spec
    @@ -1,7 +1,22 @@
     //
    -// Null expressions
    +// Null conditionals
     //
     
    +coalesceFirstNotNull
    +SELECT COALESCE(123) AS c;
    +
    +coalesceWithFirstNullOfString
    +SELECT COALESCE(null, 'first') AS c;
    +
    +coalesceWithFirstNullOfNumber
    +SELECT COALESCE(null, 123) AS c;
    +
    +coalesceMixedWithAlias
    +SELECT COALESCE(null, 123, null, 321) AS c;
    +
    +coalesceScalar
    +SELECT COALESCE(null, ABS(123) + 1) AS c;
    +
     coalesceField
     SELECT COALESCE(null, ABS(emp_no) + 1) AS c FROM test_emp ORDER BY emp_no LIMIT 5;
     
    diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4
    index d814e9e60f246..f60610fc75119 100644
    --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4
    +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4
    @@ -213,16 +213,17 @@ valueExpression
         ;
     
     primaryExpression
    -    : castExpression                                                                 #cast
    -    | primaryExpression CAST_OP dataType                                             #castOperatorExpression
    -    | extractExpression                                                              #extract
    -    | builtinDateTimeFunction                                                        #currentDateTimeFunction
    -    | constant                                                                       #constantDefault
    -    | (qualifiedName DOT)? ASTERISK                                                  #star
    -    | functionExpression                                                             #function
    -    | '(' query ')'                                                                  #subqueryExpression
    -    | qualifiedName                                                                  #dereference
    -    | '(' expression ')'                                                             #parenthesizedExpression
    +    : castExpression                                                                           #cast
    +    | primaryExpression CAST_OP dataType                                                       #castOperatorExpression
    +    | extractExpression                                                                        #extract
    +    | builtinDateTimeFunction                                                                  #currentDateTimeFunction
    +    | constant                                                                                 #constantDefault
    +    | (qualifiedName DOT)? ASTERISK                                                            #star
    +    | functionExpression                                                                       #function
    +    | '(' query ')'                                                                            #subqueryExpression
    +    | qualifiedName                                                                            #dereference
    +    | '(' expression ')'                                                                       #parenthesizedExpression
    +    | CASE (operand=booleanExpression)? whenClause+ (ELSE elseClause=booleanExpression)? END   #case
         ;
     
     builtinDateTimeFunction
    @@ -337,6 +338,10 @@ string
         | STRING
         ;
     
    +whenClause
    +    : WHEN condition=expression THEN result=expression
    +    ;
    +
     // http://developer.mimer.se/validator/sql-reserved-words.tml
     nonReserved
         : ANALYZE | ANALYZED 
    @@ -368,6 +373,7 @@ AS: 'AS';
     ASC: 'ASC';
     BETWEEN: 'BETWEEN';
     BY: 'BY';
    +CASE: 'CASE';
     CAST: 'CAST';
     CATALOG: 'CATALOG';
     CATALOGS: 'CATALOGS';
    @@ -382,6 +388,8 @@ DEBUG: 'DEBUG';
     DESC: 'DESC';
     DESCRIBE: 'DESCRIBE';
     DISTINCT: 'DISTINCT';
    +ELSE: 'ELSE';
    +END: 'END';
     ESCAPE: 'ESCAPE';
     EXECUTABLE: 'EXECUTABLE';
     EXISTS: 'EXISTS';
    @@ -437,12 +445,14 @@ SYS: 'SYS';
     TABLE: 'TABLE';
     TABLES: 'TABLES';
     TEXT: 'TEXT';
    +THEN: 'THEN';
     TRUE: 'TRUE';
     TO: 'TO';
     TYPE: 'TYPE';
     TYPES: 'TYPES';
     USING: 'USING';
     VERIFY: 'VERIFY';
    +WHEN: 'WHEN';
     WHERE: 'WHERE';
     WITH: 'WITH';
     YEAR: 'YEAR';
    diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens b/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens
    index 71b981ebfa3aa..0b24423bbee54 100644
    --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens
    +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.tokens
    @@ -11,122 +11,127 @@ AS=10
     ASC=11
     BETWEEN=12
     BY=13
    -CAST=14
    -CATALOG=15
    -CATALOGS=16
    -COLUMNS=17
    -CONVERT=18
    -CURRENT_DATE=19
    -CURRENT_TIME=20
    -CURRENT_TIMESTAMP=21
    -DAY=22
    -DAYS=23
    -DEBUG=24
    -DESC=25
    -DESCRIBE=26
    -DISTINCT=27
    -ESCAPE=28
    -EXECUTABLE=29
    -EXISTS=30
    -EXPLAIN=31
    -EXTRACT=32
    -FALSE=33
    -FIRST=34
    -FORMAT=35
    -FROM=36
    -FULL=37
    -FUNCTIONS=38
    -GRAPHVIZ=39
    -GROUP=40
    -HAVING=41
    -HOUR=42
    -HOURS=43
    -IN=44
    -INNER=45
    -INTERVAL=46
    -IS=47
    -JOIN=48
    -LAST=49
    -LEFT=50
    -LIKE=51
    -LIMIT=52
    -MAPPED=53
    -MATCH=54
    -MINUTE=55
    -MINUTES=56
    -MONTH=57
    -MONTHS=58
    -NATURAL=59
    -NOT=60
    -NULL=61
    -NULLS=62
    -ON=63
    -OPTIMIZED=64
    -OR=65
    -ORDER=66
    -OUTER=67
    -PARSED=68
    -PHYSICAL=69
    -PLAN=70
    -RIGHT=71
    -RLIKE=72
    -QUERY=73
    -SCHEMAS=74
    -SECOND=75
    -SECONDS=76
    -SELECT=77
    -SHOW=78
    -SYS=79
    -TABLE=80
    -TABLES=81
    -TEXT=82
    -TRUE=83
    -TO=84
    -TYPE=85
    -TYPES=86
    -USING=87
    -VERIFY=88
    -WHERE=89
    -WITH=90
    -YEAR=91
    -YEARS=92
    -ESCAPE_ESC=93
    -FUNCTION_ESC=94
    -LIMIT_ESC=95
    -DATE_ESC=96
    -TIME_ESC=97
    -TIMESTAMP_ESC=98
    -GUID_ESC=99
    -ESC_END=100
    -EQ=101
    -NULLEQ=102
    -NEQ=103
    -LT=104
    -LTE=105
    -GT=106
    -GTE=107
    -PLUS=108
    -MINUS=109
    -ASTERISK=110
    -SLASH=111
    -PERCENT=112
    -CAST_OP=113
    -CONCAT=114
    -DOT=115
    -PARAM=116
    -STRING=117
    -INTEGER_VALUE=118
    -DECIMAL_VALUE=119
    -IDENTIFIER=120
    -DIGIT_IDENTIFIER=121
    -TABLE_IDENTIFIER=122
    -QUOTED_IDENTIFIER=123
    -BACKQUOTED_IDENTIFIER=124
    -SIMPLE_COMMENT=125
    -BRACKETED_COMMENT=126
    -WS=127
    -UNRECOGNIZED=128
    -DELIMITER=129
    +CASE=14
    +CAST=15
    +CATALOG=16
    +CATALOGS=17
    +COLUMNS=18
    +CONVERT=19
    +CURRENT_DATE=20
    +CURRENT_TIME=21
    +CURRENT_TIMESTAMP=22
    +DAY=23
    +DAYS=24
    +DEBUG=25
    +DESC=26
    +DESCRIBE=27
    +DISTINCT=28
    +ELSE=29
    +END=30
    +ESCAPE=31
    +EXECUTABLE=32
    +EXISTS=33
    +EXPLAIN=34
    +EXTRACT=35
    +FALSE=36
    +FIRST=37
    +FORMAT=38
    +FROM=39
    +FULL=40
    +FUNCTIONS=41
    +GRAPHVIZ=42
    +GROUP=43
    +HAVING=44
    +HOUR=45
    +HOURS=46
    +IN=47
    +INNER=48
    +INTERVAL=49
    +IS=50
    +JOIN=51
    +LAST=52
    +LEFT=53
    +LIKE=54
    +LIMIT=55
    +MAPPED=56
    +MATCH=57
    +MINUTE=58
    +MINUTES=59
    +MONTH=60
    +MONTHS=61
    +NATURAL=62
    +NOT=63
    +NULL=64
    +NULLS=65
    +ON=66
    +OPTIMIZED=67
    +OR=68
    +ORDER=69
    +OUTER=70
    +PARSED=71
    +PHYSICAL=72
    +PLAN=73
    +RIGHT=74
    +RLIKE=75
    +QUERY=76
    +SCHEMAS=77
    +SECOND=78
    +SECONDS=79
    +SELECT=80
    +SHOW=81
    +SYS=82
    +TABLE=83
    +TABLES=84
    +TEXT=85
    +THEN=86
    +TRUE=87
    +TO=88
    +TYPE=89
    +TYPES=90
    +USING=91
    +VERIFY=92
    +WHEN=93
    +WHERE=94
    +WITH=95
    +YEAR=96
    +YEARS=97
    +ESCAPE_ESC=98
    +FUNCTION_ESC=99
    +LIMIT_ESC=100
    +DATE_ESC=101
    +TIME_ESC=102
    +TIMESTAMP_ESC=103
    +GUID_ESC=104
    +ESC_END=105
    +EQ=106
    +NULLEQ=107
    +NEQ=108
    +LT=109
    +LTE=110
    +GT=111
    +GTE=112
    +PLUS=113
    +MINUS=114
    +ASTERISK=115
    +SLASH=116
    +PERCENT=117
    +CAST_OP=118
    +CONCAT=119
    +DOT=120
    +PARAM=121
    +STRING=122
    +INTEGER_VALUE=123
    +DECIMAL_VALUE=124
    +IDENTIFIER=125
    +DIGIT_IDENTIFIER=126
    +TABLE_IDENTIFIER=127
    +QUOTED_IDENTIFIER=128
    +BACKQUOTED_IDENTIFIER=129
    +SIMPLE_COMMENT=130
    +BRACKETED_COMMENT=131
    +WS=132
    +UNRECOGNIZED=133
    +DELIMITER=134
     '('=1
     ')'=2
     ','=3
    @@ -140,105 +145,110 @@ DELIMITER=129
     'ASC'=11
     'BETWEEN'=12
     'BY'=13
    -'CAST'=14
    -'CATALOG'=15
    -'CATALOGS'=16
    -'COLUMNS'=17
    -'CONVERT'=18
    -'CURRENT_DATE'=19
    -'CURRENT_TIME'=20
    -'CURRENT_TIMESTAMP'=21
    -'DAY'=22
    -'DAYS'=23
    -'DEBUG'=24
    -'DESC'=25
    -'DESCRIBE'=26
    -'DISTINCT'=27
    -'ESCAPE'=28
    -'EXECUTABLE'=29
    -'EXISTS'=30
    -'EXPLAIN'=31
    -'EXTRACT'=32
    -'FALSE'=33
    -'FIRST'=34
    -'FORMAT'=35
    -'FROM'=36
    -'FULL'=37
    -'FUNCTIONS'=38
    -'GRAPHVIZ'=39
    -'GROUP'=40
    -'HAVING'=41
    -'HOUR'=42
    -'HOURS'=43
    -'IN'=44
    -'INNER'=45
    -'INTERVAL'=46
    -'IS'=47
    -'JOIN'=48
    -'LAST'=49
    -'LEFT'=50
    -'LIKE'=51
    -'LIMIT'=52
    -'MAPPED'=53
    -'MATCH'=54
    -'MINUTE'=55
    -'MINUTES'=56
    -'MONTH'=57
    -'MONTHS'=58
    -'NATURAL'=59
    -'NOT'=60
    -'NULL'=61
    -'NULLS'=62
    -'ON'=63
    -'OPTIMIZED'=64
    -'OR'=65
    -'ORDER'=66
    -'OUTER'=67
    -'PARSED'=68
    -'PHYSICAL'=69
    -'PLAN'=70
    -'RIGHT'=71
    -'RLIKE'=72
    -'QUERY'=73
    -'SCHEMAS'=74
    -'SECOND'=75
    -'SECONDS'=76
    -'SELECT'=77
    -'SHOW'=78
    -'SYS'=79
    -'TABLE'=80
    -'TABLES'=81
    -'TEXT'=82
    -'TRUE'=83
    -'TO'=84
    -'TYPE'=85
    -'TYPES'=86
    -'USING'=87
    -'VERIFY'=88
    -'WHERE'=89
    -'WITH'=90
    -'YEAR'=91
    -'YEARS'=92
    -'{ESCAPE'=93
    -'{FN'=94
    -'{LIMIT'=95
    -'{D'=96
    -'{T'=97
    -'{TS'=98
    -'{GUID'=99
    -'}'=100
    -'='=101
    -'<=>'=102
    -'<'=104
    -'<='=105
    -'>'=106
    -'>='=107
    -'+'=108
    -'-'=109
    -'*'=110
    -'/'=111
    -'%'=112
    -'::'=113
    -'||'=114
    -'.'=115
    -'?'=116
    +'CASE'=14
    +'CAST'=15
    +'CATALOG'=16
    +'CATALOGS'=17
    +'COLUMNS'=18
    +'CONVERT'=19
    +'CURRENT_DATE'=20
    +'CURRENT_TIME'=21
    +'CURRENT_TIMESTAMP'=22
    +'DAY'=23
    +'DAYS'=24
    +'DEBUG'=25
    +'DESC'=26
    +'DESCRIBE'=27
    +'DISTINCT'=28
    +'ELSE'=29
    +'END'=30
    +'ESCAPE'=31
    +'EXECUTABLE'=32
    +'EXISTS'=33
    +'EXPLAIN'=34
    +'EXTRACT'=35
    +'FALSE'=36
    +'FIRST'=37
    +'FORMAT'=38
    +'FROM'=39
    +'FULL'=40
    +'FUNCTIONS'=41
    +'GRAPHVIZ'=42
    +'GROUP'=43
    +'HAVING'=44
    +'HOUR'=45
    +'HOURS'=46
    +'IN'=47
    +'INNER'=48
    +'INTERVAL'=49
    +'IS'=50
    +'JOIN'=51
    +'LAST'=52
    +'LEFT'=53
    +'LIKE'=54
    +'LIMIT'=55
    +'MAPPED'=56
    +'MATCH'=57
    +'MINUTE'=58
    +'MINUTES'=59
    +'MONTH'=60
    +'MONTHS'=61
    +'NATURAL'=62
    +'NOT'=63
    +'NULL'=64
    +'NULLS'=65
    +'ON'=66
    +'OPTIMIZED'=67
    +'OR'=68
    +'ORDER'=69
    +'OUTER'=70
    +'PARSED'=71
    +'PHYSICAL'=72
    +'PLAN'=73
    +'RIGHT'=74
    +'RLIKE'=75
    +'QUERY'=76
    +'SCHEMAS'=77
    +'SECOND'=78
    +'SECONDS'=79
    +'SELECT'=80
    +'SHOW'=81
    +'SYS'=82
    +'TABLE'=83
    +'TABLES'=84
    +'TEXT'=85
    +'THEN'=86
    +'TRUE'=87
    +'TO'=88
    +'TYPE'=89
    +'TYPES'=90
    +'USING'=91
    +'VERIFY'=92
    +'WHEN'=93
    +'WHERE'=94
    +'WITH'=95
    +'YEAR'=96
    +'YEARS'=97
    +'{ESCAPE'=98
    +'{FN'=99
    +'{LIMIT'=100
    +'{D'=101
    +'{T'=102
    +'{TS'=103
    +'{GUID'=104
    +'}'=105
    +'='=106
    +'<=>'=107
    +'<'=109
    +'<='=110
    +'>'=111
    +'>='=112
    +'+'=113
    +'-'=114
    +'*'=115
    +'/'=116
    +'%'=117
    +'::'=118
    +'||'=119
    +'.'=120
    +'?'=121
    diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens b/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens
    index 44cff1d65ad3d..21925952a2e34 100644
    --- a/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens
    +++ b/x-pack/plugin/sql/src/main/antlr/SqlBaseLexer.tokens
    @@ -11,121 +11,126 @@ AS=10
     ASC=11
     BETWEEN=12
     BY=13
    -CAST=14
    -CATALOG=15
    -CATALOGS=16
    -COLUMNS=17
    -CONVERT=18
    -CURRENT_DATE=19
    -CURRENT_TIME=20
    -CURRENT_TIMESTAMP=21
    -DAY=22
    -DAYS=23
    -DEBUG=24
    -DESC=25
    -DESCRIBE=26
    -DISTINCT=27
    -ESCAPE=28
    -EXECUTABLE=29
    -EXISTS=30
    -EXPLAIN=31
    -EXTRACT=32
    -FALSE=33
    -FIRST=34
    -FORMAT=35
    -FROM=36
    -FULL=37
    -FUNCTIONS=38
    -GRAPHVIZ=39
    -GROUP=40
    -HAVING=41
    -HOUR=42
    -HOURS=43
    -IN=44
    -INNER=45
    -INTERVAL=46
    -IS=47
    -JOIN=48
    -LAST=49
    -LEFT=50
    -LIKE=51
    -LIMIT=52
    -MAPPED=53
    -MATCH=54
    -MINUTE=55
    -MINUTES=56
    -MONTH=57
    -MONTHS=58
    -NATURAL=59
    -NOT=60
    -NULL=61
    -NULLS=62
    -ON=63
    -OPTIMIZED=64
    -OR=65
    -ORDER=66
    -OUTER=67
    -PARSED=68
    -PHYSICAL=69
    -PLAN=70
    -RIGHT=71
    -RLIKE=72
    -QUERY=73
    -SCHEMAS=74
    -SECOND=75
    -SECONDS=76
    -SELECT=77
    -SHOW=78
    -SYS=79
    -TABLE=80
    -TABLES=81
    -TEXT=82
    -TRUE=83
    -TO=84
    -TYPE=85
    -TYPES=86
    -USING=87
    -VERIFY=88
    -WHERE=89
    -WITH=90
    -YEAR=91
    -YEARS=92
    -ESCAPE_ESC=93
    -FUNCTION_ESC=94
    -LIMIT_ESC=95
    -DATE_ESC=96
    -TIME_ESC=97
    -TIMESTAMP_ESC=98
    -GUID_ESC=99
    -ESC_END=100
    -EQ=101
    -NULLEQ=102
    -NEQ=103
    -LT=104
    -LTE=105
    -GT=106
    -GTE=107
    -PLUS=108
    -MINUS=109
    -ASTERISK=110
    -SLASH=111
    -PERCENT=112
    -CAST_OP=113
    -CONCAT=114
    -DOT=115
    -PARAM=116
    -STRING=117
    -INTEGER_VALUE=118
    -DECIMAL_VALUE=119
    -IDENTIFIER=120
    -DIGIT_IDENTIFIER=121
    -TABLE_IDENTIFIER=122
    -QUOTED_IDENTIFIER=123
    -BACKQUOTED_IDENTIFIER=124
    -SIMPLE_COMMENT=125
    -BRACKETED_COMMENT=126
    -WS=127
    -UNRECOGNIZED=128
    +CASE=14
    +CAST=15
    +CATALOG=16
    +CATALOGS=17
    +COLUMNS=18
    +CONVERT=19
    +CURRENT_DATE=20
    +CURRENT_TIME=21
    +CURRENT_TIMESTAMP=22
    +DAY=23
    +DAYS=24
    +DEBUG=25
    +DESC=26
    +DESCRIBE=27
    +DISTINCT=28
    +ELSE=29
    +END=30
    +ESCAPE=31
    +EXECUTABLE=32
    +EXISTS=33
    +EXPLAIN=34
    +EXTRACT=35
    +FALSE=36
    +FIRST=37
    +FORMAT=38
    +FROM=39
    +FULL=40
    +FUNCTIONS=41
    +GRAPHVIZ=42
    +GROUP=43
    +HAVING=44
    +HOUR=45
    +HOURS=46
    +IN=47
    +INNER=48
    +INTERVAL=49
    +IS=50
    +JOIN=51
    +LAST=52
    +LEFT=53
    +LIKE=54
    +LIMIT=55
    +MAPPED=56
    +MATCH=57
    +MINUTE=58
    +MINUTES=59
    +MONTH=60
    +MONTHS=61
    +NATURAL=62
    +NOT=63
    +NULL=64
    +NULLS=65
    +ON=66
    +OPTIMIZED=67
    +OR=68
    +ORDER=69
    +OUTER=70
    +PARSED=71
    +PHYSICAL=72
    +PLAN=73
    +RIGHT=74
    +RLIKE=75
    +QUERY=76
    +SCHEMAS=77
    +SECOND=78
    +SECONDS=79
    +SELECT=80
    +SHOW=81
    +SYS=82
    +TABLE=83
    +TABLES=84
    +TEXT=85
    +THEN=86
    +TRUE=87
    +TO=88
    +TYPE=89
    +TYPES=90
    +USING=91
    +VERIFY=92
    +WHEN=93
    +WHERE=94
    +WITH=95
    +YEAR=96
    +YEARS=97
    +ESCAPE_ESC=98
    +FUNCTION_ESC=99
    +LIMIT_ESC=100
    +DATE_ESC=101
    +TIME_ESC=102
    +TIMESTAMP_ESC=103
    +GUID_ESC=104
    +ESC_END=105
    +EQ=106
    +NULLEQ=107
    +NEQ=108
    +LT=109
    +LTE=110
    +GT=111
    +GTE=112
    +PLUS=113
    +MINUS=114
    +ASTERISK=115
    +SLASH=116
    +PERCENT=117
    +CAST_OP=118
    +CONCAT=119
    +DOT=120
    +PARAM=121
    +STRING=122
    +INTEGER_VALUE=123
    +DECIMAL_VALUE=124
    +IDENTIFIER=125
    +DIGIT_IDENTIFIER=126
    +TABLE_IDENTIFIER=127
    +QUOTED_IDENTIFIER=128
    +BACKQUOTED_IDENTIFIER=129
    +SIMPLE_COMMENT=130
    +BRACKETED_COMMENT=131
    +WS=132
    +UNRECOGNIZED=133
     '('=1
     ')'=2
     ','=3
    @@ -139,105 +144,110 @@ UNRECOGNIZED=128
     'ASC'=11
     'BETWEEN'=12
     'BY'=13
    -'CAST'=14
    -'CATALOG'=15
    -'CATALOGS'=16
    -'COLUMNS'=17
    -'CONVERT'=18
    -'CURRENT_DATE'=19
    -'CURRENT_TIME'=20
    -'CURRENT_TIMESTAMP'=21
    -'DAY'=22
    -'DAYS'=23
    -'DEBUG'=24
    -'DESC'=25
    -'DESCRIBE'=26
    -'DISTINCT'=27
    -'ESCAPE'=28
    -'EXECUTABLE'=29
    -'EXISTS'=30
    -'EXPLAIN'=31
    -'EXTRACT'=32
    -'FALSE'=33
    -'FIRST'=34
    -'FORMAT'=35
    -'FROM'=36
    -'FULL'=37
    -'FUNCTIONS'=38
    -'GRAPHVIZ'=39
    -'GROUP'=40
    -'HAVING'=41
    -'HOUR'=42
    -'HOURS'=43
    -'IN'=44
    -'INNER'=45
    -'INTERVAL'=46
    -'IS'=47
    -'JOIN'=48
    -'LAST'=49
    -'LEFT'=50
    -'LIKE'=51
    -'LIMIT'=52
    -'MAPPED'=53
    -'MATCH'=54
    -'MINUTE'=55
    -'MINUTES'=56
    -'MONTH'=57
    -'MONTHS'=58
    -'NATURAL'=59
    -'NOT'=60
    -'NULL'=61
    -'NULLS'=62
    -'ON'=63
    -'OPTIMIZED'=64
    -'OR'=65
    -'ORDER'=66
    -'OUTER'=67
    -'PARSED'=68
    -'PHYSICAL'=69
    -'PLAN'=70
    -'RIGHT'=71
    -'RLIKE'=72
    -'QUERY'=73
    -'SCHEMAS'=74
    -'SECOND'=75
    -'SECONDS'=76
    -'SELECT'=77
    -'SHOW'=78
    -'SYS'=79
    -'TABLE'=80
    -'TABLES'=81
    -'TEXT'=82
    -'TRUE'=83
    -'TO'=84
    -'TYPE'=85
    -'TYPES'=86
    -'USING'=87
    -'VERIFY'=88
    -'WHERE'=89
    -'WITH'=90
    -'YEAR'=91
    -'YEARS'=92
    -'{ESCAPE'=93
    -'{FN'=94
    -'{LIMIT'=95
    -'{D'=96
    -'{T'=97
    -'{TS'=98
    -'{GUID'=99
    -'}'=100
    -'='=101
    -'<=>'=102
    -'<'=104
    -'<='=105
    -'>'=106
    -'>='=107
    -'+'=108
    -'-'=109
    -'*'=110
    -'/'=111
    -'%'=112
    -'::'=113
    -'||'=114
    -'.'=115
    -'?'=116
    +'CASE'=14
    +'CAST'=15
    +'CATALOG'=16
    +'CATALOGS'=17
    +'COLUMNS'=18
    +'CONVERT'=19
    +'CURRENT_DATE'=20
    +'CURRENT_TIME'=21
    +'CURRENT_TIMESTAMP'=22
    +'DAY'=23
    +'DAYS'=24
    +'DEBUG'=25
    +'DESC'=26
    +'DESCRIBE'=27
    +'DISTINCT'=28
    +'ELSE'=29
    +'END'=30
    +'ESCAPE'=31
    +'EXECUTABLE'=32
    +'EXISTS'=33
    +'EXPLAIN'=34
    +'EXTRACT'=35
    +'FALSE'=36
    +'FIRST'=37
    +'FORMAT'=38
    +'FROM'=39
    +'FULL'=40
    +'FUNCTIONS'=41
    +'GRAPHVIZ'=42
    +'GROUP'=43
    +'HAVING'=44
    +'HOUR'=45
    +'HOURS'=46
    +'IN'=47
    +'INNER'=48
    +'INTERVAL'=49
    +'IS'=50
    +'JOIN'=51
    +'LAST'=52
    +'LEFT'=53
    +'LIKE'=54
    +'LIMIT'=55
    +'MAPPED'=56
    +'MATCH'=57
    +'MINUTE'=58
    +'MINUTES'=59
    +'MONTH'=60
    +'MONTHS'=61
    +'NATURAL'=62
    +'NOT'=63
    +'NULL'=64
    +'NULLS'=65
    +'ON'=66
    +'OPTIMIZED'=67
    +'OR'=68
    +'ORDER'=69
    +'OUTER'=70
    +'PARSED'=71
    +'PHYSICAL'=72
    +'PLAN'=73
    +'RIGHT'=74
    +'RLIKE'=75
    +'QUERY'=76
    +'SCHEMAS'=77
    +'SECOND'=78
    +'SECONDS'=79
    +'SELECT'=80
    +'SHOW'=81
    +'SYS'=82
    +'TABLE'=83
    +'TABLES'=84
    +'TEXT'=85
    +'THEN'=86
    +'TRUE'=87
    +'TO'=88
    +'TYPE'=89
    +'TYPES'=90
    +'USING'=91
    +'VERIFY'=92
    +'WHEN'=93
    +'WHERE'=94
    +'WITH'=95
    +'YEAR'=96
    +'YEARS'=97
    +'{ESCAPE'=98
    +'{FN'=99
    +'{LIMIT'=100
    +'{D'=101
    +'{T'=102
    +'{TS'=103
    +'{GUID'=104
    +'}'=105
    +'='=106
    +'<=>'=107
    +'<'=109
    +'<='=110
    +'>'=111
    +'>='=112
    +'+'=113
    +'-'=114
    +'*'=115
    +'/'=116
    +'%'=117
    +'::'=118
    +'||'=119
    +'.'=120
    +'?'=121
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java
    index a29b19e4128fd..883dd0e076a05 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java
    @@ -94,6 +94,7 @@
     import org.elasticsearch.xpack.sql.expression.function.scalar.string.Space;
     import org.elasticsearch.xpack.sql.expression.function.scalar.string.Substring;
     import org.elasticsearch.xpack.sql.expression.function.scalar.string.UCase;
    +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.Greatest;
     import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfNull;
    @@ -169,7 +170,8 @@ private void defineDefaultFunctions() {
             addToMap(def(Histogram.class, Histogram::new, "HISTOGRAM"));
             // Scalar functions
             // Conditional
    -        addToMap(def(Coalesce.class, Coalesce::new, "COALESCE"),
    +        addToMap(def(Case.class, Case::new, "CASE"),
    +                def(Coalesce.class, Coalesce::new, "COALESCE"),
                     def(IfNull.class, IfNull::new, "IFNULL", "ISNULL", "NVL"),
                     def(NullIf.class, NullIf::new, "NULLIF"),
                     def(Greatest.class, Greatest::new, "GREATEST"),
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Processors.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Processors.java
    index f5ceb26696065..d14aeea507f47 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Processors.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Processors.java
    @@ -28,6 +28,7 @@
     import org.elasticsearch.xpack.sql.expression.gen.processor.ConstantProcessor;
     import org.elasticsearch.xpack.sql.expression.gen.processor.HitExtractorProcessor;
     import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
    +import org.elasticsearch.xpack.sql.expression.predicate.conditional.CaseProcessor;
     import org.elasticsearch.xpack.sql.expression.predicate.conditional.ConditionalProcessor;
     import org.elasticsearch.xpack.sql.expression.predicate.conditional.NullIfProcessor;
     import org.elasticsearch.xpack.sql.expression.predicate.logical.BinaryLogicProcessor;
    @@ -62,7 +63,8 @@ public static List getNamedWriteables() {
             // logical
             entries.add(new Entry(Processor.class, BinaryLogicProcessor.NAME, BinaryLogicProcessor::new));
             entries.add(new Entry(Processor.class, NotProcessor.NAME, NotProcessor::new));
    -        // null
    +        // conditionals
    +        entries.add(new Entry(Processor.class, CaseProcessor.NAME, CaseProcessor::new));
             entries.add(new Entry(Processor.class, CheckNullProcessor.NAME, CheckNullProcessor::new));
             entries.add(new Entry(Processor.class, ConditionalProcessor.NAME, ConditionalProcessor::new));
             entries.add(new Entry(Processor.class, NullIfProcessor.NAME, NullIfProcessor::new));
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/whitelist/InternalSqlScriptUtils.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/whitelist/InternalSqlScriptUtils.java
    index 570154026b807..6a4ec411fe1cf 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/whitelist/InternalSqlScriptUtils.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/whitelist/InternalSqlScriptUtils.java
    @@ -26,6 +26,7 @@
     import org.elasticsearch.xpack.sql.expression.function.scalar.string.SubstringFunctionProcessor;
     import org.elasticsearch.xpack.sql.expression.literal.IntervalDayTime;
     import org.elasticsearch.xpack.sql.expression.literal.IntervalYearMonth;
    +import org.elasticsearch.xpack.sql.expression.predicate.conditional.CaseProcessor;
     import org.elasticsearch.xpack.sql.expression.predicate.conditional.ConditionalProcessor.ConditionalOperation;
     import org.elasticsearch.xpack.sql.expression.predicate.conditional.NullIfProcessor;
     import org.elasticsearch.xpack.sql.expression.predicate.logical.BinaryLogicProcessor.BinaryLogicOperation;
    @@ -146,8 +147,12 @@ public static Boolean in(Object value, List values) {
         }
     
         //
    -    // Null
    +    // Conditional
         //
    +    public static Object caseFunction(List expressions) {
    +        return CaseProcessor.apply(expressions);
    +    }
    +
         public static Object coalesce(List expressions) {
             return ConditionalOperation.COALESCE.apply(expressions);
         }
    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
    new file mode 100644
    index 0000000000000..772e75f414a27
    --- /dev/null
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Case.java
    @@ -0,0 +1,164 @@
    +/*
    + * 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.sql.expression.predicate.conditional;
    +
    +import org.elasticsearch.xpack.sql.expression.Expression;
    +import org.elasticsearch.xpack.sql.expression.Expressions;
    +import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe;
    +import org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder;
    +import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate;
    +import org.elasticsearch.xpack.sql.optimizer.Optimizer;
    +import org.elasticsearch.xpack.sql.tree.NodeInfo;
    +import org.elasticsearch.xpack.sql.tree.Source;
    +import org.elasticsearch.xpack.sql.type.DataType;
    +import org.elasticsearch.xpack.sql.type.DataTypeConversion;
    +import org.elasticsearch.xpack.sql.type.DataTypes;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringJoiner;
    +
    +import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
    +import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder;
    +
    +/**
    + * Implements the CASE WHEN ... THEN ... ELSE ... END expression
    + */
    +public class Case extends ConditionalFunction {
    +
    +    private final List conditions;
    +    private final Expression defaultElse;
    +
    +    @SuppressWarnings("unchecked")
    +    public Case(Source source, List expressions) {
    +        super(source, expressions);
    +        this.conditions = (List) (List) expressions.subList(0, expressions.size() - 1);
    +        this.defaultElse = expressions.get(expressions.size() - 1);
    +    }
    +
    +    public List conditions() {
    +        return conditions;
    +    }
    +
    +    public Expression defaultElse() {
    +        return defaultElse;
    +    }
    +
    +    @Override
    +    public DataType dataType() {
    +        if (dataType == null) {
    +            if (conditions.isEmpty()) {
    +                dataType = defaultElse().dataType();
    +            } else {
    +                dataType = DataType.NULL;
    +
    +                for (IfConditional conditional : conditions) {
    +                    dataType = DataTypeConversion.commonType(dataType, conditional.dataType());
    +                }
    +            }
    +        }
    +        return dataType;
    +    }
    +
    +    @Override
    +    public Expression replaceChildren(List newChildren) {
    +        return new Case(source(), newChildren);
    +    }
    +
    +    @Override
    +    protected NodeInfo info() {
    +        return NodeInfo.create(this, Case::new, children());
    +    }
    +
    +    @Override
    +    protected TypeResolution resolveType() {
    +        DataType expectedResultDataType = null;
    +        for (IfConditional ifConditional : conditions) {
    +            if (DataTypes.isNull(ifConditional.result().dataType()) == false) {
    +                expectedResultDataType = ifConditional.result().dataType();
    +                break;
    +            }
    +        }
    +        if (expectedResultDataType == null) {
    +            expectedResultDataType = defaultElse().dataType();
    +        }
    +
    +        for (IfConditional conditional : conditions) {
    +            if (conditional.condition().dataType() != DataType.BOOLEAN) {
    +                return new TypeResolution(format(null, "condition of [{}] must be [boolean], found value [{}] type [{}]",
    +                    conditional.sourceText(),
    +                    Expressions.name(conditional.condition()),
    +                    conditional.condition().dataType().typeName));
    +            }
    +            if (DataTypes.areTypesCompatible(expectedResultDataType, conditional.dataType()) == false) {
    +                return new TypeResolution(format(null, "result of [{}] must be [{}], found value [{}] type [{}]",
    +                    conditional.sourceText(),
    +                    expectedResultDataType.typeName,
    +                    Expressions.name(conditional.result()),
    +                    conditional.dataType().typeName));
    +            }
    +        }
    +
    +        if (DataTypes.areTypesCompatible(expectedResultDataType, defaultElse.dataType()) == false) {
    +            return new TypeResolution(format(null, "ELSE clause of [{}] must be [{}], found value [{}] type [{}]",
    +                defaultElse.sourceText(),
    +                expectedResultDataType.typeName,
    +                Expressions.name(defaultElse),
    +                defaultElse.dataType().typeName));
    +        }
    +
    +        return TypeResolution.TYPE_RESOLVED;
    +    }
    +
    +    /**
    +     * All foldable conditions that fold to FALSE should have
    +     * been removed by the {@link Optimizer}.
    +     */
    +    @Override
    +    public boolean foldable() {
    +        return (conditions.isEmpty() && defaultElse.foldable()) ||
    +            (conditions.size() == 1 && conditions.get(0).condition().foldable() && conditions.get(0).result().foldable());
    +    }
    +
    +    @Override
    +    public Object fold() {
    +        if (conditions.isEmpty() == false && conditions.get(0).condition().fold() == Boolean.TRUE) {
    +            return conditions.get(0).result().fold();
    +        }
    +        return defaultElse.fold();
    +    }
    +
    +    @Override
    +    protected Pipe makePipe() {
    +        List pipes = new ArrayList<>(conditions.size() + 1);
    +        for (IfConditional ifConditional : conditions) {
    +            pipes.add(Expressions.pipe(ifConditional.condition()));
    +            pipes.add(Expressions.pipe(ifConditional.result()));
    +        }
    +        pipes.add(Expressions.pipe(defaultElse));
    +        return new CasePipe(source(), this, pipes);
    +    }
    +
    +    @Override
    +    public ScriptTemplate asScript() {
    +        List templates = new ArrayList<>();
    +        for (IfConditional ifConditional : conditions) {
    +            templates.add(asScript(ifConditional.condition()));
    +            templates.add(asScript(ifConditional.result()));
    +        }
    +        templates.add(asScript(defaultElse));
    +
    +        StringJoiner template = new StringJoiner(",", "{sql}.caseFunction([", "])");
    +        ParamsBuilder params = paramsBuilder();
    +
    +        for (ScriptTemplate scriptTemplate : templates) {
    +            template.add(scriptTemplate.template());
    +            params.script(scriptTemplate.params());
    +        }
    +
    +        return new ScriptTemplate(formatTemplate(template.toString()), params.build(), dataType());
    +    }
    +}
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CasePipe.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CasePipe.java
    new file mode 100644
    index 0000000000000..f3beb755c347b
    --- /dev/null
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CasePipe.java
    @@ -0,0 +1,38 @@
    +/*
    + * 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.sql.expression.predicate.conditional;
    +
    +import org.elasticsearch.xpack.sql.expression.Expression;
    +import org.elasticsearch.xpack.sql.expression.gen.pipeline.MultiPipe;
    +import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe;
    +import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
    +import org.elasticsearch.xpack.sql.tree.NodeInfo;
    +import org.elasticsearch.xpack.sql.tree.Source;
    +
    +import java.util.List;
    +
    +public class CasePipe extends MultiPipe {
    +
    +    public CasePipe(Source source, Expression expression, List children) {
    +        super(source, expression, children);
    +    }
    +
    +    @Override
    +    protected NodeInfo info() {
    +        return NodeInfo.create(this, CasePipe::new, expression(), children());
    +    }
    +
    +    @Override
    +    public Pipe replaceChildren(List newChildren) {
    +        return new CasePipe(source(), expression(), newChildren);
    +    }
    +
    +    @Override
    +    public Processor asProcessor(List procs) {
    +        return new CaseProcessor(procs);
    +    }
    +}
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseProcessor.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseProcessor.java
    new file mode 100644
    index 0000000000000..634e83401fe63
    --- /dev/null
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseProcessor.java
    @@ -0,0 +1,77 @@
    +/*
    + * 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.sql.expression.predicate.conditional;
    +
    +import org.elasticsearch.common.io.stream.StreamInput;
    +import org.elasticsearch.common.io.stream.StreamOutput;
    +import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Objects;
    +
    +public class CaseProcessor implements Processor {
    +
    +    public static final String NAME = "case";
    +
    +    private final List processors;
    +
    +    public CaseProcessor(List processors) {
    +        this.processors = processors;
    +    }
    +
    +    public CaseProcessor(StreamInput in) throws IOException {
    +        processors = in.readNamedWriteableList(Processor.class);
    +    }
    +
    +    @Override
    +    public String getWriteableName() {
    +        return NAME;
    +    }
    +
    +    @Override
    +    public void writeTo(StreamOutput out) throws IOException {
    +        out.writeNamedWriteableList(processors);
    +    }
    +
    +    @Override
    +    public Object process(Object input) {
    +        List objects = new ArrayList<>(processors.size());
    +        for (Processor processor : processors) {
    +            objects.add(processor.process(input));
    +        }
    +        return apply(objects);
    +    }
    +
    +    public static Object apply(List objects) {
    +        for (int i = 0; i < objects.size() - 2; i += 2) {
    +            if (objects.get(i) == Boolean.TRUE) {
    +                return objects.get(i + 1);
    +            }
    +        }
    +        // resort to default value
    +        return objects.get(objects.size() - 1);
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o == null || getClass() != o.getClass()) {
    +            return false;
    +        }
    +
    +        CaseProcessor that = (CaseProcessor) o;
    +        return Objects.equals(processors, that.processors);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hash(processors);
    +    }
    +}
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java
    index b3841f09e825e..79c3f906b4045 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/ConditionalFunction.java
    @@ -15,6 +15,7 @@
     import org.elasticsearch.xpack.sql.type.DataTypeConversion;
     
     import java.util.List;
    +import java.util.Objects;
     
     import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
     import static org.elasticsearch.xpack.sql.type.DataTypes.areTypesCompatible;
    @@ -75,4 +76,21 @@ protected TypeResolution resolveType() {
         public Nullability nullable() {
             return Nullability.UNKNOWN;
         }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o == null || getClass() != o.getClass()) {
    +            return false;
    +        }
    +        ConditionalFunction that = (ConditionalFunction) o;
    +        return children().equals(that.children());
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hash(children());
    +    }
     }
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IfConditional.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IfConditional.java
    new file mode 100644
    index 0000000000000..298f7d67329b2
    --- /dev/null
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IfConditional.java
    @@ -0,0 +1,92 @@
    +/*
    + * 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.sql.expression.predicate.conditional;
    +
    +import org.elasticsearch.xpack.sql.expression.Expression;
    +import org.elasticsearch.xpack.sql.expression.Nullability;
    +import org.elasticsearch.xpack.sql.tree.NodeInfo;
    +import org.elasticsearch.xpack.sql.tree.Source;
    +import org.elasticsearch.xpack.sql.type.DataType;
    +
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.Objects;
    +
    +/**
    + * Helper expression (cannot be created directly from a query) to model a
    + * {@code WHEN  ELSE } clause of {@link Case} expression
    + */
    +public class IfConditional extends Expression {
    +
    +    private final Expression condition;
    +    private final Expression result;
    +
    +    public IfConditional(Source source, Expression condition, Expression result) {
    +        super(source, Arrays.asList(condition, result));
    +        this.condition = condition;
    +        this.result = result;
    +    }
    +
    +    public Expression condition() {
    +        return condition;
    +    }
    +
    +    public Expression result() {
    +        return result;
    +    }
    +
    +    @Override
    +    public Nullability nullable() {
    +        return Nullability.UNKNOWN;
    +    }
    +
    +    @Override
    +    public DataType dataType() {
    +        return result.dataType();
    +    }
    +
    +    @Override
    +    public Expression replaceChildren(List newChildren) {
    +        if (newChildren.size() < 2) {
    +            throw new IllegalArgumentException("expected at least [2] children but received [" + newChildren.size() + "]");
    +        }
    +        return new IfConditional(source(), newChildren.get(0), newChildren.get(1));
    +    }
    +
    +    @Override
    +    protected NodeInfo info() {
    +        return NodeInfo.create(this, IfConditional::new, condition, result);
    +    }
    +
    +    @Override
    +    protected TypeResolution resolveType() {
    +        // Verification takes place is Case function to be
    +        // able to generate more accurate error messages
    +        return TypeResolution.TYPE_RESOLVED;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) {
    +            return true;
    +        }
    +        if (o == null || getClass() != o.getClass()) {
    +            return false;
    +        }
    +        if (!super.equals(o)) {
    +            return false;
    +        }
    +
    +        IfConditional that = (IfConditional) o;
    +        return Objects.equals(condition, that.condition) &&
    +            Objects.equals(result, that.result);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hash(condition, result);
    +    }
    +}
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
    index eafdf21b11930..78cc16470ca56 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
    @@ -48,7 +48,9 @@
     import org.elasticsearch.xpack.sql.expression.predicate.Predicates;
     import org.elasticsearch.xpack.sql.expression.predicate.Range;
     import org.elasticsearch.xpack.sql.expression.predicate.conditional.ArbitraryConditionalFunction;
    +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.IfConditional;
     import org.elasticsearch.xpack.sql.expression.predicate.logical.And;
     import org.elasticsearch.xpack.sql.expression.predicate.logical.Not;
     import org.elasticsearch.xpack.sql.expression.predicate.logical.Or;
    @@ -127,6 +129,7 @@ protected Iterable.Batch> batches() {
                     new FoldNull(),
                     new ConstantFolding(),
                     new SimplifyConditional(),
    +                new SimplifyCase(),
                     // boolean
                     new BooleanSimplification(),
                     new BooleanLiteralsOnTheRight(),
    @@ -1227,6 +1230,40 @@ protected Expression rule(Expression e) {
             }
         }
     
    +    static class SimplifyCase extends OptimizerExpressionRule {
    +
    +        SimplifyCase() {
    +            super(TransformDirection.DOWN);
    +        }
    +
    +        @Override
    +        protected Expression rule(Expression e) {
    +            if (e instanceof Case) {
    +                Case c = (Case) e;
    +
    +                // Remove or foldable conditions that fold to FALSE
    +                // Stop at the 1st foldable condition that folds to TRUE
    +                List newConditions = new ArrayList<>();
    +                for (IfConditional conditional : c.conditions()) {
    +                    if (conditional.condition().foldable()) {
    +                        Boolean res = (Boolean) conditional.condition().fold();
    +                        if (res == Boolean.TRUE) {
    +                            newConditions.add(conditional);
    +                            break;
    +                        }
    +                    } else {
    +                        newConditions.add(conditional);
    +                    }
    +                }
    +
    +                if (newConditions.size() < c.children().size()) {
    +                    return c.replaceChildren(combine(newConditions, c.defaultElse()));
    +                }
    +            }
    +
    +            return e;
    +        }
    +    }
     
         static class BooleanSimplification extends OptimizerExpressionRule {
     
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java
    index ad2539ab99b6d..88b4cefd9b982 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java
    @@ -32,6 +32,8 @@
     import org.elasticsearch.xpack.sql.expression.literal.Intervals;
     import org.elasticsearch.xpack.sql.expression.literal.Intervals.TimeUnit;
     import org.elasticsearch.xpack.sql.expression.predicate.Range;
    +import org.elasticsearch.xpack.sql.expression.predicate.conditional.Case;
    +import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfConditional;
     import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MatchQueryPredicate;
     import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MultiMatchQueryPredicate;
     import org.elasticsearch.xpack.sql.expression.predicate.fulltext.StringQueryPredicate;
    @@ -116,6 +118,7 @@
     import java.time.Period;
     import java.time.format.DateTimeParseException;
     import java.time.temporal.TemporalAmount;
    +import java.util.ArrayList;
     import java.util.EnumSet;
     import java.util.List;
     import java.util.Locale;
    @@ -459,6 +462,25 @@ public Expression visitSubqueryExpression(SubqueryExpressionContext ctx) {
             return new ScalarSubquery(source(ctx), plan(ctx.query()));
         }
     
    +    @Override
    +    public Object visitCase(SqlBaseParser.CaseContext ctx) {
    +        List expressions = new ArrayList<>(ctx.whenClause().size());
    +        for (SqlBaseParser.WhenClauseContext when : ctx.whenClause()) {
    +            if (ctx.operand != null) {
    +                expressions.add(new IfConditional(source(when),
    +                    new Equals(source(when), expression(ctx.operand), expression(when.condition)), expression(when.result)));
    +            } else {
    +                expressions.add(new IfConditional(source(when), expression(when.condition), expression(when.result)));
    +            }
    +        }
    +        if (ctx.elseClause != null) {
    +            expressions.add(expression(ctx.elseClause));
    +        } else {
    +            expressions.add(Literal.NULL);
    +        }
    +        return new Case(source(ctx), expressions);
    +    }
    +
         @Override
         public Expression visitParenthesizedExpression(ParenthesizedExpressionContext ctx) {
             return expression(ctx.expression());
    diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseListener.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseListener.java
    index bf5055ae21615..9e8dd6cd6af4f 100644
    --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseListener.java
    +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseListener.java
    @@ -743,6 +743,18 @@ class SqlBaseBaseListener implements SqlBaseListener {
        * 

    The default implementation does nothing.

    */ @Override public void exitSubqueryExpression(SqlBaseParser.SubqueryExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

    The default implementation does nothing.

    + */ + @Override public void enterCase(SqlBaseParser.CaseContext ctx) { } + /** + * {@inheritDoc} + * + *

    The default implementation does nothing.

    + */ + @Override public void exitCase(SqlBaseParser.CaseContext ctx) { } /** * {@inheritDoc} * @@ -1151,6 +1163,18 @@ class SqlBaseBaseListener implements SqlBaseListener { *

    The default implementation does nothing.

    */ @Override public void exitString(SqlBaseParser.StringContext ctx) { } + /** + * {@inheritDoc} + * + *

    The default implementation does nothing.

    + */ + @Override public void enterWhenClause(SqlBaseParser.WhenClauseContext ctx) { } + /** + * {@inheritDoc} + * + *

    The default implementation does nothing.

    + */ + @Override public void exitWhenClause(SqlBaseParser.WhenClauseContext ctx) { } /** * {@inheritDoc} * diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseVisitor.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseVisitor.java index 02d92832149af..199fb407698d1 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseVisitor.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseBaseVisitor.java @@ -438,6 +438,13 @@ class SqlBaseBaseVisitor extends AbstractParseTreeVisitor implements SqlBa * {@link #visitChildren} on {@code ctx}.

    */ @Override public T visitSubqueryExpression(SqlBaseParser.SubqueryExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

    The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

    + */ + @Override public T visitCase(SqlBaseParser.CaseContext ctx) { return visitChildren(ctx); } /** * {@inheritDoc} * @@ -676,6 +683,13 @@ class SqlBaseBaseVisitor extends AbstractParseTreeVisitor implements SqlBa * {@link #visitChildren} on {@code ctx}.

    */ @Override public T visitString(SqlBaseParser.StringContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

    The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

    + */ + @Override public T visitWhenClause(SqlBaseParser.WhenClauseContext ctx) { return visitChildren(ctx); } /** * {@inheritDoc} * diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java index 132675e417889..a537f9f369e92 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseLexer.java @@ -1,13 +1,16 @@ // ANTLR GENERATED CODE: DO NOT EDIT package org.elasticsearch.xpack.sql.parser; -import org.antlr.v4.runtime.Lexer; + import org.antlr.v4.runtime.CharStream; -import org.antlr.v4.runtime.Token; -import org.antlr.v4.runtime.TokenStream; -import org.antlr.v4.runtime.*; -import org.antlr.v4.runtime.atn.*; +import org.antlr.v4.runtime.Lexer; +import org.antlr.v4.runtime.RuntimeMetaData; +import org.antlr.v4.runtime.Vocabulary; +import org.antlr.v4.runtime.VocabularyImpl; +import org.antlr.v4.runtime.atn.ATN; +import org.antlr.v4.runtime.atn.ATNDeserializer; +import org.antlr.v4.runtime.atn.LexerATNSimulator; +import org.antlr.v4.runtime.atn.PredictionContextCache; import org.antlr.v4.runtime.dfa.DFA; -import org.antlr.v4.runtime.misc.*; @SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"}) class SqlBaseLexer extends Lexer { @@ -18,88 +21,90 @@ class SqlBaseLexer extends Lexer { new PredictionContextCache(); public static final int T__0=1, T__1=2, T__2=3, T__3=4, ALL=5, ANALYZE=6, ANALYZED=7, AND=8, ANY=9, - AS=10, ASC=11, BETWEEN=12, BY=13, CAST=14, CATALOG=15, CATALOGS=16, COLUMNS=17, - CONVERT=18, CURRENT_DATE=19, CURRENT_TIME=20, CURRENT_TIMESTAMP=21, DAY=22, - DAYS=23, DEBUG=24, DESC=25, DESCRIBE=26, DISTINCT=27, ESCAPE=28, EXECUTABLE=29, - EXISTS=30, EXPLAIN=31, EXTRACT=32, FALSE=33, FIRST=34, FORMAT=35, FROM=36, - FULL=37, FUNCTIONS=38, GRAPHVIZ=39, GROUP=40, HAVING=41, HOUR=42, HOURS=43, - IN=44, INNER=45, INTERVAL=46, IS=47, JOIN=48, LAST=49, LEFT=50, LIKE=51, - LIMIT=52, MAPPED=53, MATCH=54, MINUTE=55, MINUTES=56, MONTH=57, MONTHS=58, - NATURAL=59, NOT=60, NULL=61, NULLS=62, ON=63, OPTIMIZED=64, OR=65, ORDER=66, - OUTER=67, PARSED=68, PHYSICAL=69, PLAN=70, RIGHT=71, RLIKE=72, QUERY=73, - SCHEMAS=74, SECOND=75, SECONDS=76, SELECT=77, SHOW=78, SYS=79, TABLE=80, - TABLES=81, TEXT=82, TRUE=83, TO=84, TYPE=85, TYPES=86, USING=87, VERIFY=88, - WHERE=89, WITH=90, YEAR=91, YEARS=92, ESCAPE_ESC=93, FUNCTION_ESC=94, - LIMIT_ESC=95, DATE_ESC=96, TIME_ESC=97, TIMESTAMP_ESC=98, GUID_ESC=99, - ESC_END=100, EQ=101, NULLEQ=102, NEQ=103, LT=104, LTE=105, GT=106, GTE=107, - PLUS=108, MINUS=109, ASTERISK=110, SLASH=111, PERCENT=112, CAST_OP=113, - CONCAT=114, DOT=115, PARAM=116, STRING=117, INTEGER_VALUE=118, DECIMAL_VALUE=119, - IDENTIFIER=120, DIGIT_IDENTIFIER=121, TABLE_IDENTIFIER=122, QUOTED_IDENTIFIER=123, - BACKQUOTED_IDENTIFIER=124, SIMPLE_COMMENT=125, BRACKETED_COMMENT=126, - WS=127, UNRECOGNIZED=128; + AS=10, ASC=11, BETWEEN=12, BY=13, CASE=14, CAST=15, CATALOG=16, CATALOGS=17, + COLUMNS=18, CONVERT=19, CURRENT_DATE=20, CURRENT_TIME=21, CURRENT_TIMESTAMP=22, + DAY=23, DAYS=24, DEBUG=25, DESC=26, DESCRIBE=27, DISTINCT=28, ELSE=29, + END=30, ESCAPE=31, EXECUTABLE=32, EXISTS=33, EXPLAIN=34, EXTRACT=35, FALSE=36, + FIRST=37, FORMAT=38, FROM=39, FULL=40, FUNCTIONS=41, GRAPHVIZ=42, GROUP=43, + HAVING=44, HOUR=45, HOURS=46, IN=47, INNER=48, INTERVAL=49, IS=50, JOIN=51, + LAST=52, LEFT=53, LIKE=54, LIMIT=55, MAPPED=56, MATCH=57, MINUTE=58, MINUTES=59, + MONTH=60, MONTHS=61, NATURAL=62, NOT=63, NULL=64, NULLS=65, ON=66, OPTIMIZED=67, + OR=68, ORDER=69, OUTER=70, PARSED=71, PHYSICAL=72, PLAN=73, RIGHT=74, + RLIKE=75, QUERY=76, SCHEMAS=77, SECOND=78, SECONDS=79, SELECT=80, SHOW=81, + SYS=82, TABLE=83, TABLES=84, TEXT=85, THEN=86, TRUE=87, TO=88, TYPE=89, + TYPES=90, USING=91, VERIFY=92, WHEN=93, WHERE=94, WITH=95, YEAR=96, YEARS=97, + ESCAPE_ESC=98, FUNCTION_ESC=99, LIMIT_ESC=100, DATE_ESC=101, TIME_ESC=102, + TIMESTAMP_ESC=103, GUID_ESC=104, ESC_END=105, EQ=106, NULLEQ=107, NEQ=108, + LT=109, LTE=110, GT=111, GTE=112, PLUS=113, MINUS=114, ASTERISK=115, SLASH=116, + PERCENT=117, CAST_OP=118, CONCAT=119, DOT=120, PARAM=121, STRING=122, + INTEGER_VALUE=123, DECIMAL_VALUE=124, IDENTIFIER=125, DIGIT_IDENTIFIER=126, + TABLE_IDENTIFIER=127, QUOTED_IDENTIFIER=128, BACKQUOTED_IDENTIFIER=129, + SIMPLE_COMMENT=130, BRACKETED_COMMENT=131, WS=132, UNRECOGNIZED=133; public static String[] modeNames = { "DEFAULT_MODE" }; public static final String[] ruleNames = { "T__0", "T__1", "T__2", "T__3", "ALL", "ANALYZE", "ANALYZED", "AND", "ANY", - "AS", "ASC", "BETWEEN", "BY", "CAST", "CATALOG", "CATALOGS", "COLUMNS", + "AS", "ASC", "BETWEEN", "BY", "CASE", "CAST", "CATALOG", "CATALOGS", "COLUMNS", "CONVERT", "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "DAY", - "DAYS", "DEBUG", "DESC", "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", - "EXISTS", "EXPLAIN", "EXTRACT", "FALSE", "FIRST", "FORMAT", "FROM", "FULL", - "FUNCTIONS", "GRAPHVIZ", "GROUP", "HAVING", "HOUR", "HOURS", "IN", "INNER", - "INTERVAL", "IS", "JOIN", "LAST", "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", - "MINUTE", "MINUTES", "MONTH", "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", - "ON", "OPTIMIZED", "OR", "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", - "RIGHT", "RLIKE", "QUERY", "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", - "SYS", "TABLE", "TABLES", "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", - "VERIFY", "WHERE", "WITH", "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", - "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", "GUID_ESC", "ESC_END", - "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", - "SLASH", "PERCENT", "CAST_OP", "CONCAT", "DOT", "PARAM", "STRING", "INTEGER_VALUE", - "DECIMAL_VALUE", "IDENTIFIER", "DIGIT_IDENTIFIER", "TABLE_IDENTIFIER", - "QUOTED_IDENTIFIER", "BACKQUOTED_IDENTIFIER", "EXPONENT", "DIGIT", "LETTER", - "SIMPLE_COMMENT", "BRACKETED_COMMENT", "WS", "UNRECOGNIZED" + "DAYS", "DEBUG", "DESC", "DESCRIBE", "DISTINCT", "ELSE", "END", "ESCAPE", + "EXECUTABLE", "EXISTS", "EXPLAIN", "EXTRACT", "FALSE", "FIRST", "FORMAT", + "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", "GROUP", "HAVING", "HOUR", "HOURS", + "IN", "INNER", "INTERVAL", "IS", "JOIN", "LAST", "LEFT", "LIKE", "LIMIT", + "MAPPED", "MATCH", "MINUTE", "MINUTES", "MONTH", "MONTHS", "NATURAL", + "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", "OR", "ORDER", "OUTER", "PARSED", + "PHYSICAL", "PLAN", "RIGHT", "RLIKE", "QUERY", "SCHEMAS", "SECOND", "SECONDS", + "SELECT", "SHOW", "SYS", "TABLE", "TABLES", "TEXT", "THEN", "TRUE", "TO", + "TYPE", "TYPES", "USING", "VERIFY", "WHEN", "WHERE", "WITH", "YEAR", "YEARS", + "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", + "GUID_ESC", "ESC_END", "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", + "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CAST_OP", "CONCAT", + "DOT", "PARAM", "STRING", "INTEGER_VALUE", "DECIMAL_VALUE", "IDENTIFIER", + "DIGIT_IDENTIFIER", "TABLE_IDENTIFIER", "QUOTED_IDENTIFIER", "BACKQUOTED_IDENTIFIER", + "EXPONENT", "DIGIT", "LETTER", "SIMPLE_COMMENT", "BRACKETED_COMMENT", + "WS", "UNRECOGNIZED" }; private static final String[] _LITERAL_NAMES = { null, "'('", "')'", "','", "':'", "'ALL'", "'ANALYZE'", "'ANALYZED'", - "'AND'", "'ANY'", "'AS'", "'ASC'", "'BETWEEN'", "'BY'", "'CAST'", "'CATALOG'", - "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT_DATE'", "'CURRENT_TIME'", - "'CURRENT_TIMESTAMP'", "'DAY'", "'DAYS'", "'DEBUG'", "'DESC'", "'DESCRIBE'", - "'DISTINCT'", "'ESCAPE'", "'EXECUTABLE'", "'EXISTS'", "'EXPLAIN'", "'EXTRACT'", - "'FALSE'", "'FIRST'", "'FORMAT'", "'FROM'", "'FULL'", "'FUNCTIONS'", "'GRAPHVIZ'", - "'GROUP'", "'HAVING'", "'HOUR'", "'HOURS'", "'IN'", "'INNER'", "'INTERVAL'", - "'IS'", "'JOIN'", "'LAST'", "'LEFT'", "'LIKE'", "'LIMIT'", "'MAPPED'", - "'MATCH'", "'MINUTE'", "'MINUTES'", "'MONTH'", "'MONTHS'", "'NATURAL'", - "'NOT'", "'NULL'", "'NULLS'", "'ON'", "'OPTIMIZED'", "'OR'", "'ORDER'", - "'OUTER'", "'PARSED'", "'PHYSICAL'", "'PLAN'", "'RIGHT'", "'RLIKE'", "'QUERY'", - "'SCHEMAS'", "'SECOND'", "'SECONDS'", "'SELECT'", "'SHOW'", "'SYS'", "'TABLE'", - "'TABLES'", "'TEXT'", "'TRUE'", "'TO'", "'TYPE'", "'TYPES'", "'USING'", - "'VERIFY'", "'WHERE'", "'WITH'", "'YEAR'", "'YEARS'", "'{ESCAPE'", "'{FN'", - "'{LIMIT'", "'{D'", "'{T'", "'{TS'", "'{GUID'", "'}'", "'='", "'<=>'", - null, "'<'", "'<='", "'>'", "'>='", "'+'", "'-'", "'*'", "'/'", "'%'", - "'::'", "'||'", "'.'", "'?'" + "'AND'", "'ANY'", "'AS'", "'ASC'", "'BETWEEN'", "'BY'", "'CASE'", "'CAST'", + "'CATALOG'", "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT_DATE'", + "'CURRENT_TIME'", "'CURRENT_TIMESTAMP'", "'DAY'", "'DAYS'", "'DEBUG'", + "'DESC'", "'DESCRIBE'", "'DISTINCT'", "'ELSE'", "'END'", "'ESCAPE'", "'EXECUTABLE'", + "'EXISTS'", "'EXPLAIN'", "'EXTRACT'", "'FALSE'", "'FIRST'", "'FORMAT'", + "'FROM'", "'FULL'", "'FUNCTIONS'", "'GRAPHVIZ'", "'GROUP'", "'HAVING'", + "'HOUR'", "'HOURS'", "'IN'", "'INNER'", "'INTERVAL'", "'IS'", "'JOIN'", + "'LAST'", "'LEFT'", "'LIKE'", "'LIMIT'", "'MAPPED'", "'MATCH'", "'MINUTE'", + "'MINUTES'", "'MONTH'", "'MONTHS'", "'NATURAL'", "'NOT'", "'NULL'", "'NULLS'", + "'ON'", "'OPTIMIZED'", "'OR'", "'ORDER'", "'OUTER'", "'PARSED'", "'PHYSICAL'", + "'PLAN'", "'RIGHT'", "'RLIKE'", "'QUERY'", "'SCHEMAS'", "'SECOND'", "'SECONDS'", + "'SELECT'", "'SHOW'", "'SYS'", "'TABLE'", "'TABLES'", "'TEXT'", "'THEN'", + "'TRUE'", "'TO'", "'TYPE'", "'TYPES'", "'USING'", "'VERIFY'", "'WHEN'", + "'WHERE'", "'WITH'", "'YEAR'", "'YEARS'", "'{ESCAPE'", "'{FN'", "'{LIMIT'", + "'{D'", "'{T'", "'{TS'", "'{GUID'", "'}'", "'='", "'<=>'", null, "'<'", + "'<='", "'>'", "'>='", "'+'", "'-'", "'*'", "'/'", "'%'", "'::'", "'||'", + "'.'", "'?'" }; private static final String[] _SYMBOLIC_NAMES = { null, null, null, null, null, "ALL", "ANALYZE", "ANALYZED", "AND", "ANY", - "AS", "ASC", "BETWEEN", "BY", "CAST", "CATALOG", "CATALOGS", "COLUMNS", + "AS", "ASC", "BETWEEN", "BY", "CASE", "CAST", "CATALOG", "CATALOGS", "COLUMNS", "CONVERT", "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "DAY", - "DAYS", "DEBUG", "DESC", "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", - "EXISTS", "EXPLAIN", "EXTRACT", "FALSE", "FIRST", "FORMAT", "FROM", "FULL", - "FUNCTIONS", "GRAPHVIZ", "GROUP", "HAVING", "HOUR", "HOURS", "IN", "INNER", - "INTERVAL", "IS", "JOIN", "LAST", "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", - "MINUTE", "MINUTES", "MONTH", "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", - "ON", "OPTIMIZED", "OR", "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", - "RIGHT", "RLIKE", "QUERY", "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", - "SYS", "TABLE", "TABLES", "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", - "VERIFY", "WHERE", "WITH", "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", - "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", "GUID_ESC", "ESC_END", - "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", - "SLASH", "PERCENT", "CAST_OP", "CONCAT", "DOT", "PARAM", "STRING", "INTEGER_VALUE", - "DECIMAL_VALUE", "IDENTIFIER", "DIGIT_IDENTIFIER", "TABLE_IDENTIFIER", - "QUOTED_IDENTIFIER", "BACKQUOTED_IDENTIFIER", "SIMPLE_COMMENT", "BRACKETED_COMMENT", - "WS", "UNRECOGNIZED" + "DAYS", "DEBUG", "DESC", "DESCRIBE", "DISTINCT", "ELSE", "END", "ESCAPE", + "EXECUTABLE", "EXISTS", "EXPLAIN", "EXTRACT", "FALSE", "FIRST", "FORMAT", + "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", "GROUP", "HAVING", "HOUR", "HOURS", + "IN", "INNER", "INTERVAL", "IS", "JOIN", "LAST", "LEFT", "LIKE", "LIMIT", + "MAPPED", "MATCH", "MINUTE", "MINUTES", "MONTH", "MONTHS", "NATURAL", + "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", "OR", "ORDER", "OUTER", "PARSED", + "PHYSICAL", "PLAN", "RIGHT", "RLIKE", "QUERY", "SCHEMAS", "SECOND", "SECONDS", + "SELECT", "SHOW", "SYS", "TABLE", "TABLES", "TEXT", "THEN", "TRUE", "TO", + "TYPE", "TYPES", "USING", "VERIFY", "WHEN", "WHERE", "WITH", "YEAR", "YEARS", + "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", + "GUID_ESC", "ESC_END", "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", + "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CAST_OP", "CONCAT", + "DOT", "PARAM", "STRING", "INTEGER_VALUE", "DECIMAL_VALUE", "IDENTIFIER", + "DIGIT_IDENTIFIER", "TABLE_IDENTIFIER", "QUOTED_IDENTIFIER", "BACKQUOTED_IDENTIFIER", + "SIMPLE_COMMENT", "BRACKETED_COMMENT", "WS", "UNRECOGNIZED" }; public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES); @@ -156,7 +161,7 @@ public SqlBaseLexer(CharStream input) { public ATN getATN() { return _ATN; } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\2\u0082\u043c\b\1\4"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\2\u0087\u045e\b\1\4"+ "\2\t\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n"+ "\4\13\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22"+ "\t\22\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31"+ @@ -171,55 +176,60 @@ public SqlBaseLexer(CharStream input) { "\tk\4l\tl\4m\tm\4n\tn\4o\to\4p\tp\4q\tq\4r\tr\4s\ts\4t\tt\4u\tu\4v\tv"+ "\4w\tw\4x\tx\4y\ty\4z\tz\4{\t{\4|\t|\4}\t}\4~\t~\4\177\t\177\4\u0080\t"+ "\u0080\4\u0081\t\u0081\4\u0082\t\u0082\4\u0083\t\u0083\4\u0084\t\u0084"+ - "\3\2\3\2\3\3\3\3\3\4\3\4\3\5\3\5\3\6\3\6\3\6\3\6\3\7\3\7\3\7\3\7\3\7\3"+ - "\7\3\7\3\7\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\t\3\t\3\t\3\t\3\n\3\n"+ - "\3\n\3\n\3\13\3\13\3\13\3\f\3\f\3\f\3\f\3\r\3\r\3\r\3\r\3\r\3\r\3\r\3"+ - "\r\3\16\3\16\3\16\3\17\3\17\3\17\3\17\3\17\3\20\3\20\3\20\3\20\3\20\3"+ - "\20\3\20\3\20\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\22\3\22\3"+ - "\22\3\22\3\22\3\22\3\22\3\22\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3"+ - "\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\25\3"+ - "\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\26\3\26\3"+ - "\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3"+ - "\26\3\26\3\27\3\27\3\27\3\27\3\30\3\30\3\30\3\30\3\30\3\31\3\31\3\31\3"+ - "\31\3\31\3\31\3\32\3\32\3\32\3\32\3\32\3\33\3\33\3\33\3\33\3\33\3\33\3"+ - "\33\3\33\3\33\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\35\3\35\3"+ - "\35\3\35\3\35\3\35\3\35\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3"+ - "\36\3\36\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3 \3 \3 \3 \3 \3 \3 \3 \3"+ - "!\3!\3!\3!\3!\3!\3!\3!\3\"\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\3#\3#\3#\3$\3"+ - "$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\3\'"+ - "\3\'\3\'\3\'\3\'\3(\3(\3(\3(\3(\3(\3(\3(\3(\3)\3)\3)\3)\3)\3)\3*\3*\3"+ - "*\3*\3*\3*\3*\3+\3+\3+\3+\3+\3,\3,\3,\3,\3,\3,\3-\3-\3-\3.\3.\3.\3.\3"+ - ".\3.\3/\3/\3/\3/\3/\3/\3/\3/\3/\3\60\3\60\3\60\3\61\3\61\3\61\3\61\3\61"+ - "\3\62\3\62\3\62\3\62\3\62\3\63\3\63\3\63\3\63\3\63\3\64\3\64\3\64\3\64"+ - "\3\64\3\65\3\65\3\65\3\65\3\65\3\65\3\66\3\66\3\66\3\66\3\66\3\66\3\66"+ - "\3\67\3\67\3\67\3\67\3\67\3\67\38\38\38\38\38\38\38\39\39\39\39\39\39"+ - "\39\39\3:\3:\3:\3:\3:\3:\3;\3;\3;\3;\3;\3;\3;\3<\3<\3<\3<\3<\3<\3<\3<"+ - "\3=\3=\3=\3=\3>\3>\3>\3>\3>\3?\3?\3?\3?\3?\3?\3@\3@\3@\3A\3A\3A\3A\3A"+ - "\3A\3A\3A\3A\3A\3B\3B\3B\3C\3C\3C\3C\3C\3C\3D\3D\3D\3D\3D\3D\3E\3E\3E"+ - "\3E\3E\3E\3E\3F\3F\3F\3F\3F\3F\3F\3F\3F\3G\3G\3G\3G\3G\3H\3H\3H\3H\3H"+ - "\3H\3I\3I\3I\3I\3I\3I\3J\3J\3J\3J\3J\3J\3K\3K\3K\3K\3K\3K\3K\3K\3L\3L"+ - "\3L\3L\3L\3L\3L\3M\3M\3M\3M\3M\3M\3M\3M\3N\3N\3N\3N\3N\3N\3N\3O\3O\3O"+ - "\3O\3O\3P\3P\3P\3P\3Q\3Q\3Q\3Q\3Q\3Q\3R\3R\3R\3R\3R\3R\3R\3S\3S\3S\3S"+ - "\3S\3T\3T\3T\3T\3T\3U\3U\3U\3V\3V\3V\3V\3V\3W\3W\3W\3W\3W\3W\3X\3X\3X"+ - "\3X\3X\3X\3Y\3Y\3Y\3Y\3Y\3Y\3Y\3Z\3Z\3Z\3Z\3Z\3Z\3[\3[\3[\3[\3[\3\\\3"+ - "\\\3\\\3\\\3\\\3]\3]\3]\3]\3]\3]\3^\3^\3^\3^\3^\3^\3^\3^\3_\3_\3_\3_\3"+ - "`\3`\3`\3`\3`\3`\3`\3a\3a\3a\3b\3b\3b\3c\3c\3c\3c\3d\3d\3d\3d\3d\3d\3"+ - "e\3e\3f\3f\3g\3g\3g\3g\3h\3h\3h\3h\5h\u037a\nh\3i\3i\3j\3j\3j\3k\3k\3"+ - "l\3l\3l\3m\3m\3n\3n\3o\3o\3p\3p\3q\3q\3r\3r\3r\3s\3s\3s\3t\3t\3u\3u\3"+ - "v\3v\3v\3v\7v\u039e\nv\fv\16v\u03a1\13v\3v\3v\3w\6w\u03a6\nw\rw\16w\u03a7"+ - "\3x\6x\u03ab\nx\rx\16x\u03ac\3x\3x\7x\u03b1\nx\fx\16x\u03b4\13x\3x\3x"+ - "\6x\u03b8\nx\rx\16x\u03b9\3x\6x\u03bd\nx\rx\16x\u03be\3x\3x\7x\u03c3\n"+ - "x\fx\16x\u03c6\13x\5x\u03c8\nx\3x\3x\3x\3x\6x\u03ce\nx\rx\16x\u03cf\3"+ - "x\3x\5x\u03d4\nx\3y\3y\5y\u03d8\ny\3y\3y\3y\7y\u03dd\ny\fy\16y\u03e0\13"+ - "y\3z\3z\3z\3z\6z\u03e6\nz\rz\16z\u03e7\3{\3{\3{\6{\u03ed\n{\r{\16{\u03ee"+ - "\3|\3|\3|\3|\7|\u03f5\n|\f|\16|\u03f8\13|\3|\3|\3}\3}\3}\3}\7}\u0400\n"+ - "}\f}\16}\u0403\13}\3}\3}\3~\3~\5~\u0409\n~\3~\6~\u040c\n~\r~\16~\u040d"+ - "\3\177\3\177\3\u0080\3\u0080\3\u0081\3\u0081\3\u0081\3\u0081\7\u0081\u0418"+ - "\n\u0081\f\u0081\16\u0081\u041b\13\u0081\3\u0081\5\u0081\u041e\n\u0081"+ - "\3\u0081\5\u0081\u0421\n\u0081\3\u0081\3\u0081\3\u0082\3\u0082\3\u0082"+ - "\3\u0082\3\u0082\7\u0082\u042a\n\u0082\f\u0082\16\u0082\u042d\13\u0082"+ - "\3\u0082\3\u0082\3\u0082\3\u0082\3\u0082\3\u0083\6\u0083\u0435\n\u0083"+ - "\r\u0083\16\u0083\u0436\3\u0083\3\u0083\3\u0084\3\u0084\3\u042b\2\u0085"+ + "\4\u0085\t\u0085\4\u0086\t\u0086\4\u0087\t\u0087\4\u0088\t\u0088\4\u0089"+ + "\t\u0089\3\2\3\2\3\3\3\3\3\4\3\4\3\5\3\5\3\6\3\6\3\6\3\6\3\7\3\7\3\7\3"+ + "\7\3\7\3\7\3\7\3\7\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\t\3\t\3\t\3\t"+ + "\3\n\3\n\3\n\3\n\3\13\3\13\3\13\3\f\3\f\3\f\3\f\3\r\3\r\3\r\3\r\3\r\3"+ + "\r\3\r\3\r\3\16\3\16\3\16\3\17\3\17\3\17\3\17\3\17\3\20\3\20\3\20\3\20"+ + "\3\20\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\22\3\22\3\22\3\22\3\22"+ + "\3\22\3\22\3\22\3\22\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\24\3\24"+ + "\3\24\3\24\3\24\3\24\3\24\3\24\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25"+ + "\3\25\3\25\3\25\3\25\3\25\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26"+ + "\3\26\3\26\3\26\3\26\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27"+ + "\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\30\3\30\3\30\3\30\3\31\3\31"+ + "\3\31\3\31\3\31\3\32\3\32\3\32\3\32\3\32\3\32\3\33\3\33\3\33\3\33\3\33"+ + "\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\35\3\35\3\35\3\35\3\35"+ + "\3\35\3\35\3\35\3\35\3\36\3\36\3\36\3\36\3\36\3\37\3\37\3\37\3\37\3 \3"+ + " \3 \3 \3 \3 \3 \3!\3!\3!\3!\3!\3!\3!\3!\3!\3!\3!\3\"\3\"\3\"\3\"\3\""+ + "\3\"\3\"\3#\3#\3#\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3"+ + "%\3%\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3(\3(\3(\3(\3(\3)\3"+ + ")\3)\3)\3)\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3+\3+\3+\3+\3+\3+\3+\3+\3+\3"+ + ",\3,\3,\3,\3,\3,\3-\3-\3-\3-\3-\3-\3-\3.\3.\3.\3.\3.\3/\3/\3/\3/\3/\3"+ + "/\3\60\3\60\3\60\3\61\3\61\3\61\3\61\3\61\3\61\3\62\3\62\3\62\3\62\3\62"+ + "\3\62\3\62\3\62\3\62\3\63\3\63\3\63\3\64\3\64\3\64\3\64\3\64\3\65\3\65"+ + "\3\65\3\65\3\65\3\66\3\66\3\66\3\66\3\66\3\67\3\67\3\67\3\67\3\67\38\3"+ + "8\38\38\38\38\39\39\39\39\39\39\39\3:\3:\3:\3:\3:\3:\3;\3;\3;\3;\3;\3"+ + ";\3;\3<\3<\3<\3<\3<\3<\3<\3<\3=\3=\3=\3=\3=\3=\3>\3>\3>\3>\3>\3>\3>\3"+ + "?\3?\3?\3?\3?\3?\3?\3?\3@\3@\3@\3@\3A\3A\3A\3A\3A\3B\3B\3B\3B\3B\3B\3"+ + "C\3C\3C\3D\3D\3D\3D\3D\3D\3D\3D\3D\3D\3E\3E\3E\3F\3F\3F\3F\3F\3F\3G\3"+ + "G\3G\3G\3G\3G\3H\3H\3H\3H\3H\3H\3H\3I\3I\3I\3I\3I\3I\3I\3I\3I\3J\3J\3"+ + "J\3J\3J\3K\3K\3K\3K\3K\3K\3L\3L\3L\3L\3L\3L\3M\3M\3M\3M\3M\3M\3N\3N\3"+ + "N\3N\3N\3N\3N\3N\3O\3O\3O\3O\3O\3O\3O\3P\3P\3P\3P\3P\3P\3P\3P\3Q\3Q\3"+ + "Q\3Q\3Q\3Q\3Q\3R\3R\3R\3R\3R\3S\3S\3S\3S\3T\3T\3T\3T\3T\3T\3U\3U\3U\3"+ + "U\3U\3U\3U\3V\3V\3V\3V\3V\3W\3W\3W\3W\3W\3X\3X\3X\3X\3X\3Y\3Y\3Y\3Z\3"+ + "Z\3Z\3Z\3Z\3[\3[\3[\3[\3[\3[\3\\\3\\\3\\\3\\\3\\\3\\\3]\3]\3]\3]\3]\3"+ + "]\3]\3^\3^\3^\3^\3^\3_\3_\3_\3_\3_\3_\3`\3`\3`\3`\3`\3a\3a\3a\3a\3a\3"+ + "b\3b\3b\3b\3b\3b\3c\3c\3c\3c\3c\3c\3c\3c\3d\3d\3d\3d\3e\3e\3e\3e\3e\3"+ + "e\3e\3f\3f\3f\3g\3g\3g\3h\3h\3h\3h\3i\3i\3i\3i\3i\3i\3j\3j\3k\3k\3l\3"+ + "l\3l\3l\3m\3m\3m\3m\5m\u039c\nm\3n\3n\3o\3o\3o\3p\3p\3q\3q\3q\3r\3r\3"+ + "s\3s\3t\3t\3u\3u\3v\3v\3w\3w\3w\3x\3x\3x\3y\3y\3z\3z\3{\3{\3{\3{\7{\u03c0"+ + "\n{\f{\16{\u03c3\13{\3{\3{\3|\6|\u03c8\n|\r|\16|\u03c9\3}\6}\u03cd\n}"+ + "\r}\16}\u03ce\3}\3}\7}\u03d3\n}\f}\16}\u03d6\13}\3}\3}\6}\u03da\n}\r}"+ + "\16}\u03db\3}\6}\u03df\n}\r}\16}\u03e0\3}\3}\7}\u03e5\n}\f}\16}\u03e8"+ + "\13}\5}\u03ea\n}\3}\3}\3}\3}\6}\u03f0\n}\r}\16}\u03f1\3}\3}\5}\u03f6\n"+ + "}\3~\3~\5~\u03fa\n~\3~\3~\3~\7~\u03ff\n~\f~\16~\u0402\13~\3\177\3\177"+ + "\3\177\3\177\6\177\u0408\n\177\r\177\16\177\u0409\3\u0080\3\u0080\3\u0080"+ + "\6\u0080\u040f\n\u0080\r\u0080\16\u0080\u0410\3\u0081\3\u0081\3\u0081"+ + "\3\u0081\7\u0081\u0417\n\u0081\f\u0081\16\u0081\u041a\13\u0081\3\u0081"+ + "\3\u0081\3\u0082\3\u0082\3\u0082\3\u0082\7\u0082\u0422\n\u0082\f\u0082"+ + "\16\u0082\u0425\13\u0082\3\u0082\3\u0082\3\u0083\3\u0083\5\u0083\u042b"+ + "\n\u0083\3\u0083\6\u0083\u042e\n\u0083\r\u0083\16\u0083\u042f\3\u0084"+ + "\3\u0084\3\u0085\3\u0085\3\u0086\3\u0086\3\u0086\3\u0086\7\u0086\u043a"+ + "\n\u0086\f\u0086\16\u0086\u043d\13\u0086\3\u0086\5\u0086\u0440\n\u0086"+ + "\3\u0086\5\u0086\u0443\n\u0086\3\u0086\3\u0086\3\u0087\3\u0087\3\u0087"+ + "\3\u0087\3\u0087\7\u0087\u044c\n\u0087\f\u0087\16\u0087\u044f\13\u0087"+ + "\3\u0087\3\u0087\3\u0087\3\u0087\3\u0087\3\u0088\6\u0088\u0457\n\u0088"+ + "\r\u0088\16\u0088\u0458\3\u0088\3\u0088\3\u0089\3\u0089\3\u044d\2\u008a"+ "\3\3\5\4\7\5\t\6\13\7\r\b\17\t\21\n\23\13\25\f\27\r\31\16\33\17\35\20"+ "\37\21!\22#\23%\24\'\25)\26+\27-\30/\31\61\32\63\33\65\34\67\359\36;\37"+ "= ?!A\"C#E$G%I&K\'M(O)Q*S+U,W-Y.[/]\60_\61a\62c\63e\64g\65i\66k\67m8o"+ @@ -229,302 +239,313 @@ public SqlBaseLexer(CharStream input) { "\\\u00b7]\u00b9^\u00bb_\u00bd`\u00bfa\u00c1b\u00c3c\u00c5d\u00c7e\u00c9"+ "f\u00cbg\u00cdh\u00cfi\u00d1j\u00d3k\u00d5l\u00d7m\u00d9n\u00dbo\u00dd"+ "p\u00dfq\u00e1r\u00e3s\u00e5t\u00e7u\u00e9v\u00ebw\u00edx\u00efy\u00f1"+ - "z\u00f3{\u00f5|\u00f7}\u00f9~\u00fb\2\u00fd\2\u00ff\2\u0101\177\u0103"+ - "\u0080\u0105\u0081\u0107\u0082\3\2\13\3\2))\4\2BBaa\3\2$$\3\2bb\4\2--"+ - "//\3\2\62;\3\2C\\\4\2\f\f\17\17\5\2\13\f\17\17\"\"\u045c\2\3\3\2\2\2\2"+ - "\5\3\2\2\2\2\7\3\2\2\2\2\t\3\2\2\2\2\13\3\2\2\2\2\r\3\2\2\2\2\17\3\2\2"+ - "\2\2\21\3\2\2\2\2\23\3\2\2\2\2\25\3\2\2\2\2\27\3\2\2\2\2\31\3\2\2\2\2"+ - "\33\3\2\2\2\2\35\3\2\2\2\2\37\3\2\2\2\2!\3\2\2\2\2#\3\2\2\2\2%\3\2\2\2"+ - "\2\'\3\2\2\2\2)\3\2\2\2\2+\3\2\2\2\2-\3\2\2\2\2/\3\2\2\2\2\61\3\2\2\2"+ - "\2\63\3\2\2\2\2\65\3\2\2\2\2\67\3\2\2\2\29\3\2\2\2\2;\3\2\2\2\2=\3\2\2"+ - "\2\2?\3\2\2\2\2A\3\2\2\2\2C\3\2\2\2\2E\3\2\2\2\2G\3\2\2\2\2I\3\2\2\2\2"+ - "K\3\2\2\2\2M\3\2\2\2\2O\3\2\2\2\2Q\3\2\2\2\2S\3\2\2\2\2U\3\2\2\2\2W\3"+ - "\2\2\2\2Y\3\2\2\2\2[\3\2\2\2\2]\3\2\2\2\2_\3\2\2\2\2a\3\2\2\2\2c\3\2\2"+ - "\2\2e\3\2\2\2\2g\3\2\2\2\2i\3\2\2\2\2k\3\2\2\2\2m\3\2\2\2\2o\3\2\2\2\2"+ - "q\3\2\2\2\2s\3\2\2\2\2u\3\2\2\2\2w\3\2\2\2\2y\3\2\2\2\2{\3\2\2\2\2}\3"+ - "\2\2\2\2\177\3\2\2\2\2\u0081\3\2\2\2\2\u0083\3\2\2\2\2\u0085\3\2\2\2\2"+ - "\u0087\3\2\2\2\2\u0089\3\2\2\2\2\u008b\3\2\2\2\2\u008d\3\2\2\2\2\u008f"+ - "\3\2\2\2\2\u0091\3\2\2\2\2\u0093\3\2\2\2\2\u0095\3\2\2\2\2\u0097\3\2\2"+ - "\2\2\u0099\3\2\2\2\2\u009b\3\2\2\2\2\u009d\3\2\2\2\2\u009f\3\2\2\2\2\u00a1"+ - "\3\2\2\2\2\u00a3\3\2\2\2\2\u00a5\3\2\2\2\2\u00a7\3\2\2\2\2\u00a9\3\2\2"+ - "\2\2\u00ab\3\2\2\2\2\u00ad\3\2\2\2\2\u00af\3\2\2\2\2\u00b1\3\2\2\2\2\u00b3"+ - "\3\2\2\2\2\u00b5\3\2\2\2\2\u00b7\3\2\2\2\2\u00b9\3\2\2\2\2\u00bb\3\2\2"+ - "\2\2\u00bd\3\2\2\2\2\u00bf\3\2\2\2\2\u00c1\3\2\2\2\2\u00c3\3\2\2\2\2\u00c5"+ - "\3\2\2\2\2\u00c7\3\2\2\2\2\u00c9\3\2\2\2\2\u00cb\3\2\2\2\2\u00cd\3\2\2"+ - "\2\2\u00cf\3\2\2\2\2\u00d1\3\2\2\2\2\u00d3\3\2\2\2\2\u00d5\3\2\2\2\2\u00d7"+ - "\3\2\2\2\2\u00d9\3\2\2\2\2\u00db\3\2\2\2\2\u00dd\3\2\2\2\2\u00df\3\2\2"+ - "\2\2\u00e1\3\2\2\2\2\u00e3\3\2\2\2\2\u00e5\3\2\2\2\2\u00e7\3\2\2\2\2\u00e9"+ - "\3\2\2\2\2\u00eb\3\2\2\2\2\u00ed\3\2\2\2\2\u00ef\3\2\2\2\2\u00f1\3\2\2"+ - "\2\2\u00f3\3\2\2\2\2\u00f5\3\2\2\2\2\u00f7\3\2\2\2\2\u00f9\3\2\2\2\2\u0101"+ - "\3\2\2\2\2\u0103\3\2\2\2\2\u0105\3\2\2\2\2\u0107\3\2\2\2\3\u0109\3\2\2"+ - "\2\5\u010b\3\2\2\2\7\u010d\3\2\2\2\t\u010f\3\2\2\2\13\u0111\3\2\2\2\r"+ - "\u0115\3\2\2\2\17\u011d\3\2\2\2\21\u0126\3\2\2\2\23\u012a\3\2\2\2\25\u012e"+ - "\3\2\2\2\27\u0131\3\2\2\2\31\u0135\3\2\2\2\33\u013d\3\2\2\2\35\u0140\3"+ - "\2\2\2\37\u0145\3\2\2\2!\u014d\3\2\2\2#\u0156\3\2\2\2%\u015e\3\2\2\2\'"+ - "\u0166\3\2\2\2)\u0173\3\2\2\2+\u0180\3\2\2\2-\u0192\3\2\2\2/\u0196\3\2"+ - "\2\2\61\u019b\3\2\2\2\63\u01a1\3\2\2\2\65\u01a6\3\2\2\2\67\u01af\3\2\2"+ - "\29\u01b8\3\2\2\2;\u01bf\3\2\2\2=\u01ca\3\2\2\2?\u01d1\3\2\2\2A\u01d9"+ - "\3\2\2\2C\u01e1\3\2\2\2E\u01e7\3\2\2\2G\u01ed\3\2\2\2I\u01f4\3\2\2\2K"+ - "\u01f9\3\2\2\2M\u01fe\3\2\2\2O\u0208\3\2\2\2Q\u0211\3\2\2\2S\u0217\3\2"+ - "\2\2U\u021e\3\2\2\2W\u0223\3\2\2\2Y\u0229\3\2\2\2[\u022c\3\2\2\2]\u0232"+ - "\3\2\2\2_\u023b\3\2\2\2a\u023e\3\2\2\2c\u0243\3\2\2\2e\u0248\3\2\2\2g"+ - "\u024d\3\2\2\2i\u0252\3\2\2\2k\u0258\3\2\2\2m\u025f\3\2\2\2o\u0265\3\2"+ - "\2\2q\u026c\3\2\2\2s\u0274\3\2\2\2u\u027a\3\2\2\2w\u0281\3\2\2\2y\u0289"+ - "\3\2\2\2{\u028d\3\2\2\2}\u0292\3\2\2\2\177\u0298\3\2\2\2\u0081\u029b\3"+ - "\2\2\2\u0083\u02a5\3\2\2\2\u0085\u02a8\3\2\2\2\u0087\u02ae\3\2\2\2\u0089"+ - "\u02b4\3\2\2\2\u008b\u02bb\3\2\2\2\u008d\u02c4\3\2\2\2\u008f\u02c9\3\2"+ - "\2\2\u0091\u02cf\3\2\2\2\u0093\u02d5\3\2\2\2\u0095\u02db\3\2\2\2\u0097"+ - "\u02e3\3\2\2\2\u0099\u02ea\3\2\2\2\u009b\u02f2\3\2\2\2\u009d\u02f9\3\2"+ - "\2\2\u009f\u02fe\3\2\2\2\u00a1\u0302\3\2\2\2\u00a3\u0308\3\2\2\2\u00a5"+ - "\u030f\3\2\2\2\u00a7\u0314\3\2\2\2\u00a9\u0319\3\2\2\2\u00ab\u031c\3\2"+ - "\2\2\u00ad\u0321\3\2\2\2\u00af\u0327\3\2\2\2\u00b1\u032d\3\2\2\2\u00b3"+ - "\u0334\3\2\2\2\u00b5\u033a\3\2\2\2\u00b7\u033f\3\2\2\2\u00b9\u0344\3\2"+ - "\2\2\u00bb\u034a\3\2\2\2\u00bd\u0352\3\2\2\2\u00bf\u0356\3\2\2\2\u00c1"+ - "\u035d\3\2\2\2\u00c3\u0360\3\2\2\2\u00c5\u0363\3\2\2\2\u00c7\u0367\3\2"+ - "\2\2\u00c9\u036d\3\2\2\2\u00cb\u036f\3\2\2\2\u00cd\u0371\3\2\2\2\u00cf"+ - "\u0379\3\2\2\2\u00d1\u037b\3\2\2\2\u00d3\u037d\3\2\2\2\u00d5\u0380\3\2"+ - "\2\2\u00d7\u0382\3\2\2\2\u00d9\u0385\3\2\2\2\u00db\u0387\3\2\2\2\u00dd"+ - "\u0389\3\2\2\2\u00df\u038b\3\2\2\2\u00e1\u038d\3\2\2\2\u00e3\u038f\3\2"+ - "\2\2\u00e5\u0392\3\2\2\2\u00e7\u0395\3\2\2\2\u00e9\u0397\3\2\2\2\u00eb"+ - "\u0399\3\2\2\2\u00ed\u03a5\3\2\2\2\u00ef\u03d3\3\2\2\2\u00f1\u03d7\3\2"+ - "\2\2\u00f3\u03e1\3\2\2\2\u00f5\u03ec\3\2\2\2\u00f7\u03f0\3\2\2\2\u00f9"+ - "\u03fb\3\2\2\2\u00fb\u0406\3\2\2\2\u00fd\u040f\3\2\2\2\u00ff\u0411\3\2"+ - "\2\2\u0101\u0413\3\2\2\2\u0103\u0424\3\2\2\2\u0105\u0434\3\2\2\2\u0107"+ - "\u043a\3\2\2\2\u0109\u010a\7*\2\2\u010a\4\3\2\2\2\u010b\u010c\7+\2\2\u010c"+ - "\6\3\2\2\2\u010d\u010e\7.\2\2\u010e\b\3\2\2\2\u010f\u0110\7<\2\2\u0110"+ - "\n\3\2\2\2\u0111\u0112\7C\2\2\u0112\u0113\7N\2\2\u0113\u0114\7N\2\2\u0114"+ - "\f\3\2\2\2\u0115\u0116\7C\2\2\u0116\u0117\7P\2\2\u0117\u0118\7C\2\2\u0118"+ - "\u0119\7N\2\2\u0119\u011a\7[\2\2\u011a\u011b\7\\\2\2\u011b\u011c\7G\2"+ - "\2\u011c\16\3\2\2\2\u011d\u011e\7C\2\2\u011e\u011f\7P\2\2\u011f\u0120"+ - "\7C\2\2\u0120\u0121\7N\2\2\u0121\u0122\7[\2\2\u0122\u0123\7\\\2\2\u0123"+ - "\u0124\7G\2\2\u0124\u0125\7F\2\2\u0125\20\3\2\2\2\u0126\u0127\7C\2\2\u0127"+ - "\u0128\7P\2\2\u0128\u0129\7F\2\2\u0129\22\3\2\2\2\u012a\u012b\7C\2\2\u012b"+ - "\u012c\7P\2\2\u012c\u012d\7[\2\2\u012d\24\3\2\2\2\u012e\u012f\7C\2\2\u012f"+ - "\u0130\7U\2\2\u0130\26\3\2\2\2\u0131\u0132\7C\2\2\u0132\u0133\7U\2\2\u0133"+ - "\u0134\7E\2\2\u0134\30\3\2\2\2\u0135\u0136\7D\2\2\u0136\u0137\7G\2\2\u0137"+ - "\u0138\7V\2\2\u0138\u0139\7Y\2\2\u0139\u013a\7G\2\2\u013a\u013b\7G\2\2"+ - "\u013b\u013c\7P\2\2\u013c\32\3\2\2\2\u013d\u013e\7D\2\2\u013e\u013f\7"+ - "[\2\2\u013f\34\3\2\2\2\u0140\u0141\7E\2\2\u0141\u0142\7C\2\2\u0142\u0143"+ - "\7U\2\2\u0143\u0144\7V\2\2\u0144\36\3\2\2\2\u0145\u0146\7E\2\2\u0146\u0147"+ - "\7C\2\2\u0147\u0148\7V\2\2\u0148\u0149\7C\2\2\u0149\u014a\7N\2\2\u014a"+ - "\u014b\7Q\2\2\u014b\u014c\7I\2\2\u014c \3\2\2\2\u014d\u014e\7E\2\2\u014e"+ - "\u014f\7C\2\2\u014f\u0150\7V\2\2\u0150\u0151\7C\2\2\u0151\u0152\7N\2\2"+ - "\u0152\u0153\7Q\2\2\u0153\u0154\7I\2\2\u0154\u0155\7U\2\2\u0155\"\3\2"+ - "\2\2\u0156\u0157\7E\2\2\u0157\u0158\7Q\2\2\u0158\u0159\7N\2\2\u0159\u015a"+ - "\7W\2\2\u015a\u015b\7O\2\2\u015b\u015c\7P\2\2\u015c\u015d\7U\2\2\u015d"+ - "$\3\2\2\2\u015e\u015f\7E\2\2\u015f\u0160\7Q\2\2\u0160\u0161\7P\2\2\u0161"+ - "\u0162\7X\2\2\u0162\u0163\7G\2\2\u0163\u0164\7T\2\2\u0164\u0165\7V\2\2"+ - "\u0165&\3\2\2\2\u0166\u0167\7E\2\2\u0167\u0168\7W\2\2\u0168\u0169\7T\2"+ - "\2\u0169\u016a\7T\2\2\u016a\u016b\7G\2\2\u016b\u016c\7P\2\2\u016c\u016d"+ - "\7V\2\2\u016d\u016e\7a\2\2\u016e\u016f\7F\2\2\u016f\u0170\7C\2\2\u0170"+ - "\u0171\7V\2\2\u0171\u0172\7G\2\2\u0172(\3\2\2\2\u0173\u0174\7E\2\2\u0174"+ - "\u0175\7W\2\2\u0175\u0176\7T\2\2\u0176\u0177\7T\2\2\u0177\u0178\7G\2\2"+ - "\u0178\u0179\7P\2\2\u0179\u017a\7V\2\2\u017a\u017b\7a\2\2\u017b\u017c"+ - "\7V\2\2\u017c\u017d\7K\2\2\u017d\u017e\7O\2\2\u017e\u017f\7G\2\2\u017f"+ - "*\3\2\2\2\u0180\u0181\7E\2\2\u0181\u0182\7W\2\2\u0182\u0183\7T\2\2\u0183"+ - "\u0184\7T\2\2\u0184\u0185\7G\2\2\u0185\u0186\7P\2\2\u0186\u0187\7V\2\2"+ - "\u0187\u0188\7a\2\2\u0188\u0189\7V\2\2\u0189\u018a\7K\2\2\u018a\u018b"+ - "\7O\2\2\u018b\u018c\7G\2\2\u018c\u018d\7U\2\2\u018d\u018e\7V\2\2\u018e"+ - "\u018f\7C\2\2\u018f\u0190\7O\2\2\u0190\u0191\7R\2\2\u0191,\3\2\2\2\u0192"+ - "\u0193\7F\2\2\u0193\u0194\7C\2\2\u0194\u0195\7[\2\2\u0195.\3\2\2\2\u0196"+ - "\u0197\7F\2\2\u0197\u0198\7C\2\2\u0198\u0199\7[\2\2\u0199\u019a\7U\2\2"+ - "\u019a\60\3\2\2\2\u019b\u019c\7F\2\2\u019c\u019d\7G\2\2\u019d\u019e\7"+ - "D\2\2\u019e\u019f\7W\2\2\u019f\u01a0\7I\2\2\u01a0\62\3\2\2\2\u01a1\u01a2"+ - "\7F\2\2\u01a2\u01a3\7G\2\2\u01a3\u01a4\7U\2\2\u01a4\u01a5\7E\2\2\u01a5"+ - "\64\3\2\2\2\u01a6\u01a7\7F\2\2\u01a7\u01a8\7G\2\2\u01a8\u01a9\7U\2\2\u01a9"+ - "\u01aa\7E\2\2\u01aa\u01ab\7T\2\2\u01ab\u01ac\7K\2\2\u01ac\u01ad\7D\2\2"+ - "\u01ad\u01ae\7G\2\2\u01ae\66\3\2\2\2\u01af\u01b0\7F\2\2\u01b0\u01b1\7"+ - "K\2\2\u01b1\u01b2\7U\2\2\u01b2\u01b3\7V\2\2\u01b3\u01b4\7K\2\2\u01b4\u01b5"+ - "\7P\2\2\u01b5\u01b6\7E\2\2\u01b6\u01b7\7V\2\2\u01b78\3\2\2\2\u01b8\u01b9"+ - "\7G\2\2\u01b9\u01ba\7U\2\2\u01ba\u01bb\7E\2\2\u01bb\u01bc\7C\2\2\u01bc"+ - "\u01bd\7R\2\2\u01bd\u01be\7G\2\2\u01be:\3\2\2\2\u01bf\u01c0\7G\2\2\u01c0"+ - "\u01c1\7Z\2\2\u01c1\u01c2\7G\2\2\u01c2\u01c3\7E\2\2\u01c3\u01c4\7W\2\2"+ - "\u01c4\u01c5\7V\2\2\u01c5\u01c6\7C\2\2\u01c6\u01c7\7D\2\2\u01c7\u01c8"+ - "\7N\2\2\u01c8\u01c9\7G\2\2\u01c9<\3\2\2\2\u01ca\u01cb\7G\2\2\u01cb\u01cc"+ - "\7Z\2\2\u01cc\u01cd\7K\2\2\u01cd\u01ce\7U\2\2\u01ce\u01cf\7V\2\2\u01cf"+ - "\u01d0\7U\2\2\u01d0>\3\2\2\2\u01d1\u01d2\7G\2\2\u01d2\u01d3\7Z\2\2\u01d3"+ - "\u01d4\7R\2\2\u01d4\u01d5\7N\2\2\u01d5\u01d6\7C\2\2\u01d6\u01d7\7K\2\2"+ - "\u01d7\u01d8\7P\2\2\u01d8@\3\2\2\2\u01d9\u01da\7G\2\2\u01da\u01db\7Z\2"+ - "\2\u01db\u01dc\7V\2\2\u01dc\u01dd\7T\2\2\u01dd\u01de\7C\2\2\u01de\u01df"+ - "\7E\2\2\u01df\u01e0\7V\2\2\u01e0B\3\2\2\2\u01e1\u01e2\7H\2\2\u01e2\u01e3"+ - "\7C\2\2\u01e3\u01e4\7N\2\2\u01e4\u01e5\7U\2\2\u01e5\u01e6\7G\2\2\u01e6"+ - "D\3\2\2\2\u01e7\u01e8\7H\2\2\u01e8\u01e9\7K\2\2\u01e9\u01ea\7T\2\2\u01ea"+ - "\u01eb\7U\2\2\u01eb\u01ec\7V\2\2\u01ecF\3\2\2\2\u01ed\u01ee\7H\2\2\u01ee"+ - "\u01ef\7Q\2\2\u01ef\u01f0\7T\2\2\u01f0\u01f1\7O\2\2\u01f1\u01f2\7C\2\2"+ - "\u01f2\u01f3\7V\2\2\u01f3H\3\2\2\2\u01f4\u01f5\7H\2\2\u01f5\u01f6\7T\2"+ - "\2\u01f6\u01f7\7Q\2\2\u01f7\u01f8\7O\2\2\u01f8J\3\2\2\2\u01f9\u01fa\7"+ - "H\2\2\u01fa\u01fb\7W\2\2\u01fb\u01fc\7N\2\2\u01fc\u01fd\7N\2\2\u01fdL"+ - "\3\2\2\2\u01fe\u01ff\7H\2\2\u01ff\u0200\7W\2\2\u0200\u0201\7P\2\2\u0201"+ - "\u0202\7E\2\2\u0202\u0203\7V\2\2\u0203\u0204\7K\2\2\u0204\u0205\7Q\2\2"+ - "\u0205\u0206\7P\2\2\u0206\u0207\7U\2\2\u0207N\3\2\2\2\u0208\u0209\7I\2"+ - "\2\u0209\u020a\7T\2\2\u020a\u020b\7C\2\2\u020b\u020c\7R\2\2\u020c\u020d"+ - "\7J\2\2\u020d\u020e\7X\2\2\u020e\u020f\7K\2\2\u020f\u0210\7\\\2\2\u0210"+ - "P\3\2\2\2\u0211\u0212\7I\2\2\u0212\u0213\7T\2\2\u0213\u0214\7Q\2\2\u0214"+ - "\u0215\7W\2\2\u0215\u0216\7R\2\2\u0216R\3\2\2\2\u0217\u0218\7J\2\2\u0218"+ - "\u0219\7C\2\2\u0219\u021a\7X\2\2\u021a\u021b\7K\2\2\u021b\u021c\7P\2\2"+ - "\u021c\u021d\7I\2\2\u021dT\3\2\2\2\u021e\u021f\7J\2\2\u021f\u0220\7Q\2"+ - "\2\u0220\u0221\7W\2\2\u0221\u0222\7T\2\2\u0222V\3\2\2\2\u0223\u0224\7"+ - "J\2\2\u0224\u0225\7Q\2\2\u0225\u0226\7W\2\2\u0226\u0227\7T\2\2\u0227\u0228"+ - "\7U\2\2\u0228X\3\2\2\2\u0229\u022a\7K\2\2\u022a\u022b\7P\2\2\u022bZ\3"+ - "\2\2\2\u022c\u022d\7K\2\2\u022d\u022e\7P\2\2\u022e\u022f\7P\2\2\u022f"+ - "\u0230\7G\2\2\u0230\u0231\7T\2\2\u0231\\\3\2\2\2\u0232\u0233\7K\2\2\u0233"+ - "\u0234\7P\2\2\u0234\u0235\7V\2\2\u0235\u0236\7G\2\2\u0236\u0237\7T\2\2"+ - "\u0237\u0238\7X\2\2\u0238\u0239\7C\2\2\u0239\u023a\7N\2\2\u023a^\3\2\2"+ - "\2\u023b\u023c\7K\2\2\u023c\u023d\7U\2\2\u023d`\3\2\2\2\u023e\u023f\7"+ - "L\2\2\u023f\u0240\7Q\2\2\u0240\u0241\7K\2\2\u0241\u0242\7P\2\2\u0242b"+ - "\3\2\2\2\u0243\u0244\7N\2\2\u0244\u0245\7C\2\2\u0245\u0246\7U\2\2\u0246"+ - "\u0247\7V\2\2\u0247d\3\2\2\2\u0248\u0249\7N\2\2\u0249\u024a\7G\2\2\u024a"+ - "\u024b\7H\2\2\u024b\u024c\7V\2\2\u024cf\3\2\2\2\u024d\u024e\7N\2\2\u024e"+ - "\u024f\7K\2\2\u024f\u0250\7M\2\2\u0250\u0251\7G\2\2\u0251h\3\2\2\2\u0252"+ - "\u0253\7N\2\2\u0253\u0254\7K\2\2\u0254\u0255\7O\2\2\u0255\u0256\7K\2\2"+ - "\u0256\u0257\7V\2\2\u0257j\3\2\2\2\u0258\u0259\7O\2\2\u0259\u025a\7C\2"+ - "\2\u025a\u025b\7R\2\2\u025b\u025c\7R\2\2\u025c\u025d\7G\2\2\u025d\u025e"+ - "\7F\2\2\u025el\3\2\2\2\u025f\u0260\7O\2\2\u0260\u0261\7C\2\2\u0261\u0262"+ - "\7V\2\2\u0262\u0263\7E\2\2\u0263\u0264\7J\2\2\u0264n\3\2\2\2\u0265\u0266"+ - "\7O\2\2\u0266\u0267\7K\2\2\u0267\u0268\7P\2\2\u0268\u0269\7W\2\2\u0269"+ - "\u026a\7V\2\2\u026a\u026b\7G\2\2\u026bp\3\2\2\2\u026c\u026d\7O\2\2\u026d"+ - "\u026e\7K\2\2\u026e\u026f\7P\2\2\u026f\u0270\7W\2\2\u0270\u0271\7V\2\2"+ - "\u0271\u0272\7G\2\2\u0272\u0273\7U\2\2\u0273r\3\2\2\2\u0274\u0275\7O\2"+ - "\2\u0275\u0276\7Q\2\2\u0276\u0277\7P\2\2\u0277\u0278\7V\2\2\u0278\u0279"+ - "\7J\2\2\u0279t\3\2\2\2\u027a\u027b\7O\2\2\u027b\u027c\7Q\2\2\u027c\u027d"+ - "\7P\2\2\u027d\u027e\7V\2\2\u027e\u027f\7J\2\2\u027f\u0280\7U\2\2\u0280"+ - "v\3\2\2\2\u0281\u0282\7P\2\2\u0282\u0283\7C\2\2\u0283\u0284\7V\2\2\u0284"+ - "\u0285\7W\2\2\u0285\u0286\7T\2\2\u0286\u0287\7C\2\2\u0287\u0288\7N\2\2"+ - "\u0288x\3\2\2\2\u0289\u028a\7P\2\2\u028a\u028b\7Q\2\2\u028b\u028c\7V\2"+ - "\2\u028cz\3\2\2\2\u028d\u028e\7P\2\2\u028e\u028f\7W\2\2\u028f\u0290\7"+ - "N\2\2\u0290\u0291\7N\2\2\u0291|\3\2\2\2\u0292\u0293\7P\2\2\u0293\u0294"+ - "\7W\2\2\u0294\u0295\7N\2\2\u0295\u0296\7N\2\2\u0296\u0297\7U\2\2\u0297"+ - "~\3\2\2\2\u0298\u0299\7Q\2\2\u0299\u029a\7P\2\2\u029a\u0080\3\2\2\2\u029b"+ - "\u029c\7Q\2\2\u029c\u029d\7R\2\2\u029d\u029e\7V\2\2\u029e\u029f\7K\2\2"+ - "\u029f\u02a0\7O\2\2\u02a0\u02a1\7K\2\2\u02a1\u02a2\7\\\2\2\u02a2\u02a3"+ - "\7G\2\2\u02a3\u02a4\7F\2\2\u02a4\u0082\3\2\2\2\u02a5\u02a6\7Q\2\2\u02a6"+ - "\u02a7\7T\2\2\u02a7\u0084\3\2\2\2\u02a8\u02a9\7Q\2\2\u02a9\u02aa\7T\2"+ - "\2\u02aa\u02ab\7F\2\2\u02ab\u02ac\7G\2\2\u02ac\u02ad\7T\2\2\u02ad\u0086"+ - "\3\2\2\2\u02ae\u02af\7Q\2\2\u02af\u02b0\7W\2\2\u02b0\u02b1\7V\2\2\u02b1"+ - "\u02b2\7G\2\2\u02b2\u02b3\7T\2\2\u02b3\u0088\3\2\2\2\u02b4\u02b5\7R\2"+ - "\2\u02b5\u02b6\7C\2\2\u02b6\u02b7\7T\2\2\u02b7\u02b8\7U\2\2\u02b8\u02b9"+ - "\7G\2\2\u02b9\u02ba\7F\2\2\u02ba\u008a\3\2\2\2\u02bb\u02bc\7R\2\2\u02bc"+ - "\u02bd\7J\2\2\u02bd\u02be\7[\2\2\u02be\u02bf\7U\2\2\u02bf\u02c0\7K\2\2"+ - "\u02c0\u02c1\7E\2\2\u02c1\u02c2\7C\2\2\u02c2\u02c3\7N\2\2\u02c3\u008c"+ - "\3\2\2\2\u02c4\u02c5\7R\2\2\u02c5\u02c6\7N\2\2\u02c6\u02c7\7C\2\2\u02c7"+ - "\u02c8\7P\2\2\u02c8\u008e\3\2\2\2\u02c9\u02ca\7T\2\2\u02ca\u02cb\7K\2"+ - "\2\u02cb\u02cc\7I\2\2\u02cc\u02cd\7J\2\2\u02cd\u02ce\7V\2\2\u02ce\u0090"+ - "\3\2\2\2\u02cf\u02d0\7T\2\2\u02d0\u02d1\7N\2\2\u02d1\u02d2\7K\2\2\u02d2"+ - "\u02d3\7M\2\2\u02d3\u02d4\7G\2\2\u02d4\u0092\3\2\2\2\u02d5\u02d6\7S\2"+ - "\2\u02d6\u02d7\7W\2\2\u02d7\u02d8\7G\2\2\u02d8\u02d9\7T\2\2\u02d9\u02da"+ - "\7[\2\2\u02da\u0094\3\2\2\2\u02db\u02dc\7U\2\2\u02dc\u02dd\7E\2\2\u02dd"+ - "\u02de\7J\2\2\u02de\u02df\7G\2\2\u02df\u02e0\7O\2\2\u02e0\u02e1\7C\2\2"+ - "\u02e1\u02e2\7U\2\2\u02e2\u0096\3\2\2\2\u02e3\u02e4\7U\2\2\u02e4\u02e5"+ - "\7G\2\2\u02e5\u02e6\7E\2\2\u02e6\u02e7\7Q\2\2\u02e7\u02e8\7P\2\2\u02e8"+ - "\u02e9\7F\2\2\u02e9\u0098\3\2\2\2\u02ea\u02eb\7U\2\2\u02eb\u02ec\7G\2"+ - "\2\u02ec\u02ed\7E\2\2\u02ed\u02ee\7Q\2\2\u02ee\u02ef\7P\2\2\u02ef\u02f0"+ - "\7F\2\2\u02f0\u02f1\7U\2\2\u02f1\u009a\3\2\2\2\u02f2\u02f3\7U\2\2\u02f3"+ - "\u02f4\7G\2\2\u02f4\u02f5\7N\2\2\u02f5\u02f6\7G\2\2\u02f6\u02f7\7E\2\2"+ - "\u02f7\u02f8\7V\2\2\u02f8\u009c\3\2\2\2\u02f9\u02fa\7U\2\2\u02fa\u02fb"+ - "\7J\2\2\u02fb\u02fc\7Q\2\2\u02fc\u02fd\7Y\2\2\u02fd\u009e\3\2\2\2\u02fe"+ - "\u02ff\7U\2\2\u02ff\u0300\7[\2\2\u0300\u0301\7U\2\2\u0301\u00a0\3\2\2"+ - "\2\u0302\u0303\7V\2\2\u0303\u0304\7C\2\2\u0304\u0305\7D\2\2\u0305\u0306"+ - "\7N\2\2\u0306\u0307\7G\2\2\u0307\u00a2\3\2\2\2\u0308\u0309\7V\2\2\u0309"+ - "\u030a\7C\2\2\u030a\u030b\7D\2\2\u030b\u030c\7N\2\2\u030c\u030d\7G\2\2"+ - "\u030d\u030e\7U\2\2\u030e\u00a4\3\2\2\2\u030f\u0310\7V\2\2\u0310\u0311"+ - "\7G\2\2\u0311\u0312\7Z\2\2\u0312\u0313\7V\2\2\u0313\u00a6\3\2\2\2\u0314"+ - "\u0315\7V\2\2\u0315\u0316\7T\2\2\u0316\u0317\7W\2\2\u0317\u0318\7G\2\2"+ - "\u0318\u00a8\3\2\2\2\u0319\u031a\7V\2\2\u031a\u031b\7Q\2\2\u031b\u00aa"+ - "\3\2\2\2\u031c\u031d\7V\2\2\u031d\u031e\7[\2\2\u031e\u031f\7R\2\2\u031f"+ - "\u0320\7G\2\2\u0320\u00ac\3\2\2\2\u0321\u0322\7V\2\2\u0322\u0323\7[\2"+ - "\2\u0323\u0324\7R\2\2\u0324\u0325\7G\2\2\u0325\u0326\7U\2\2\u0326\u00ae"+ - "\3\2\2\2\u0327\u0328\7W\2\2\u0328\u0329\7U\2\2\u0329\u032a\7K\2\2\u032a"+ - "\u032b\7P\2\2\u032b\u032c\7I\2\2\u032c\u00b0\3\2\2\2\u032d\u032e\7X\2"+ - "\2\u032e\u032f\7G\2\2\u032f\u0330\7T\2\2\u0330\u0331\7K\2\2\u0331\u0332"+ - "\7H\2\2\u0332\u0333\7[\2\2\u0333\u00b2\3\2\2\2\u0334\u0335\7Y\2\2\u0335"+ - "\u0336\7J\2\2\u0336\u0337\7G\2\2\u0337\u0338\7T\2\2\u0338\u0339\7G\2\2"+ - "\u0339\u00b4\3\2\2\2\u033a\u033b\7Y\2\2\u033b\u033c\7K\2\2\u033c\u033d"+ - "\7V\2\2\u033d\u033e\7J\2\2\u033e\u00b6\3\2\2\2\u033f\u0340\7[\2\2\u0340"+ - "\u0341\7G\2\2\u0341\u0342\7C\2\2\u0342\u0343\7T\2\2\u0343\u00b8\3\2\2"+ - "\2\u0344\u0345\7[\2\2\u0345\u0346\7G\2\2\u0346\u0347\7C\2\2\u0347\u0348"+ - "\7T\2\2\u0348\u0349\7U\2\2\u0349\u00ba\3\2\2\2\u034a\u034b\7}\2\2\u034b"+ - "\u034c\7G\2\2\u034c\u034d\7U\2\2\u034d\u034e\7E\2\2\u034e\u034f\7C\2\2"+ - "\u034f\u0350\7R\2\2\u0350\u0351\7G\2\2\u0351\u00bc\3\2\2\2\u0352\u0353"+ - "\7}\2\2\u0353\u0354\7H\2\2\u0354\u0355\7P\2\2\u0355\u00be\3\2\2\2\u0356"+ - "\u0357\7}\2\2\u0357\u0358\7N\2\2\u0358\u0359\7K\2\2\u0359\u035a\7O\2\2"+ - "\u035a\u035b\7K\2\2\u035b\u035c\7V\2\2\u035c\u00c0\3\2\2\2\u035d\u035e"+ - "\7}\2\2\u035e\u035f\7F\2\2\u035f\u00c2\3\2\2\2\u0360\u0361\7}\2\2\u0361"+ - "\u0362\7V\2\2\u0362\u00c4\3\2\2\2\u0363\u0364\7}\2\2\u0364\u0365\7V\2"+ - "\2\u0365\u0366\7U\2\2\u0366\u00c6\3\2\2\2\u0367\u0368\7}\2\2\u0368\u0369"+ - "\7I\2\2\u0369\u036a\7W\2\2\u036a\u036b\7K\2\2\u036b\u036c\7F\2\2\u036c"+ - "\u00c8\3\2\2\2\u036d\u036e\7\177\2\2\u036e\u00ca\3\2\2\2\u036f\u0370\7"+ - "?\2\2\u0370\u00cc\3\2\2\2\u0371\u0372\7>\2\2\u0372\u0373\7?\2\2\u0373"+ - "\u0374\7@\2\2\u0374\u00ce\3\2\2\2\u0375\u0376\7>\2\2\u0376\u037a\7@\2"+ - "\2\u0377\u0378\7#\2\2\u0378\u037a\7?\2\2\u0379\u0375\3\2\2\2\u0379\u0377"+ - "\3\2\2\2\u037a\u00d0\3\2\2\2\u037b\u037c\7>\2\2\u037c\u00d2\3\2\2\2\u037d"+ - "\u037e\7>\2\2\u037e\u037f\7?\2\2\u037f\u00d4\3\2\2\2\u0380\u0381\7@\2"+ - "\2\u0381\u00d6\3\2\2\2\u0382\u0383\7@\2\2\u0383\u0384\7?\2\2\u0384\u00d8"+ - "\3\2\2\2\u0385\u0386\7-\2\2\u0386\u00da\3\2\2\2\u0387\u0388\7/\2\2\u0388"+ - "\u00dc\3\2\2\2\u0389\u038a\7,\2\2\u038a\u00de\3\2\2\2\u038b\u038c\7\61"+ - "\2\2\u038c\u00e0\3\2\2\2\u038d\u038e\7\'\2\2\u038e\u00e2\3\2\2\2\u038f"+ - "\u0390\7<\2\2\u0390\u0391\7<\2\2\u0391\u00e4\3\2\2\2\u0392\u0393\7~\2"+ - "\2\u0393\u0394\7~\2\2\u0394\u00e6\3\2\2\2\u0395\u0396\7\60\2\2\u0396\u00e8"+ - "\3\2\2\2\u0397\u0398\7A\2\2\u0398\u00ea\3\2\2\2\u0399\u039f\7)\2\2\u039a"+ - "\u039e\n\2\2\2\u039b\u039c\7)\2\2\u039c\u039e\7)\2\2\u039d\u039a\3\2\2"+ - "\2\u039d\u039b\3\2\2\2\u039e\u03a1\3\2\2\2\u039f\u039d\3\2\2\2\u039f\u03a0"+ - "\3\2\2\2\u03a0\u03a2\3\2\2\2\u03a1\u039f\3\2\2\2\u03a2\u03a3\7)\2\2\u03a3"+ - "\u00ec\3\2\2\2\u03a4\u03a6\5\u00fd\177\2\u03a5\u03a4\3\2\2\2\u03a6\u03a7"+ - "\3\2\2\2\u03a7\u03a5\3\2\2\2\u03a7\u03a8\3\2\2\2\u03a8\u00ee\3\2\2\2\u03a9"+ - "\u03ab\5\u00fd\177\2\u03aa\u03a9\3\2\2\2\u03ab\u03ac\3\2\2\2\u03ac\u03aa"+ - "\3\2\2\2\u03ac\u03ad\3\2\2\2\u03ad\u03ae\3\2\2\2\u03ae\u03b2\5\u00e7t"+ - "\2\u03af\u03b1\5\u00fd\177\2\u03b0\u03af\3\2\2\2\u03b1\u03b4\3\2\2\2\u03b2"+ - "\u03b0\3\2\2\2\u03b2\u03b3\3\2\2\2\u03b3\u03d4\3\2\2\2\u03b4\u03b2\3\2"+ - "\2\2\u03b5\u03b7\5\u00e7t\2\u03b6\u03b8\5\u00fd\177\2\u03b7\u03b6\3\2"+ - "\2\2\u03b8\u03b9\3\2\2\2\u03b9\u03b7\3\2\2\2\u03b9\u03ba\3\2\2\2\u03ba"+ - "\u03d4\3\2\2\2\u03bb\u03bd\5\u00fd\177\2\u03bc\u03bb\3\2\2\2\u03bd\u03be"+ - "\3\2\2\2\u03be\u03bc\3\2\2\2\u03be\u03bf\3\2\2\2\u03bf\u03c7\3\2\2\2\u03c0"+ - "\u03c4\5\u00e7t\2\u03c1\u03c3\5\u00fd\177\2\u03c2\u03c1\3\2\2\2\u03c3"+ - "\u03c6\3\2\2\2\u03c4\u03c2\3\2\2\2\u03c4\u03c5\3\2\2\2\u03c5\u03c8\3\2"+ - "\2\2\u03c6\u03c4\3\2\2\2\u03c7\u03c0\3\2\2\2\u03c7\u03c8\3\2\2\2\u03c8"+ - "\u03c9\3\2\2\2\u03c9\u03ca\5\u00fb~\2\u03ca\u03d4\3\2\2\2\u03cb\u03cd"+ - "\5\u00e7t\2\u03cc\u03ce\5\u00fd\177\2\u03cd\u03cc\3\2\2\2\u03ce\u03cf"+ - "\3\2\2\2\u03cf\u03cd\3\2\2\2\u03cf\u03d0\3\2\2\2\u03d0\u03d1\3\2\2\2\u03d1"+ - "\u03d2\5\u00fb~\2\u03d2\u03d4\3\2\2\2\u03d3\u03aa\3\2\2\2\u03d3\u03b5"+ - "\3\2\2\2\u03d3\u03bc\3\2\2\2\u03d3\u03cb\3\2\2\2\u03d4\u00f0\3\2\2\2\u03d5"+ - "\u03d8\5\u00ff\u0080\2\u03d6\u03d8\7a\2\2\u03d7\u03d5\3\2\2\2\u03d7\u03d6"+ - "\3\2\2\2\u03d8\u03de\3\2\2\2\u03d9\u03dd\5\u00ff\u0080\2\u03da\u03dd\5"+ - "\u00fd\177\2\u03db\u03dd\t\3\2\2\u03dc\u03d9\3\2\2\2\u03dc\u03da\3\2\2"+ - "\2\u03dc\u03db\3\2\2\2\u03dd\u03e0\3\2\2\2\u03de\u03dc\3\2\2\2\u03de\u03df"+ - "\3\2\2\2\u03df\u00f2\3\2\2\2\u03e0\u03de\3\2\2\2\u03e1\u03e5\5\u00fd\177"+ - "\2\u03e2\u03e6\5\u00ff\u0080\2\u03e3\u03e6\5\u00fd\177\2\u03e4\u03e6\t"+ - "\3\2\2\u03e5\u03e2\3\2\2\2\u03e5\u03e3\3\2\2\2\u03e5\u03e4\3\2\2\2\u03e6"+ - "\u03e7\3\2\2\2\u03e7\u03e5\3\2\2\2\u03e7\u03e8\3\2\2\2\u03e8\u00f4\3\2"+ - "\2\2\u03e9\u03ed\5\u00ff\u0080\2\u03ea\u03ed\5\u00fd\177\2\u03eb\u03ed"+ - "\7a\2\2\u03ec\u03e9\3\2\2\2\u03ec\u03ea\3\2\2\2\u03ec\u03eb\3\2\2\2\u03ed"+ - "\u03ee\3\2\2\2\u03ee\u03ec\3\2\2\2\u03ee\u03ef\3\2\2\2\u03ef\u00f6\3\2"+ - "\2\2\u03f0\u03f6\7$\2\2\u03f1\u03f5\n\4\2\2\u03f2\u03f3\7$\2\2\u03f3\u03f5"+ - "\7$\2\2\u03f4\u03f1\3\2\2\2\u03f4\u03f2\3\2\2\2\u03f5\u03f8\3\2\2\2\u03f6"+ - "\u03f4\3\2\2\2\u03f6\u03f7\3\2\2\2\u03f7\u03f9\3\2\2\2\u03f8\u03f6\3\2"+ - "\2\2\u03f9\u03fa\7$\2\2\u03fa\u00f8\3\2\2\2\u03fb\u0401\7b\2\2\u03fc\u0400"+ - "\n\5\2\2\u03fd\u03fe\7b\2\2\u03fe\u0400\7b\2\2\u03ff\u03fc\3\2\2\2\u03ff"+ - "\u03fd\3\2\2\2\u0400\u0403\3\2\2\2\u0401\u03ff\3\2\2\2\u0401\u0402\3\2"+ - "\2\2\u0402\u0404\3\2\2\2\u0403\u0401\3\2\2\2\u0404\u0405\7b\2\2\u0405"+ - "\u00fa\3\2\2\2\u0406\u0408\7G\2\2\u0407\u0409\t\6\2\2\u0408\u0407\3\2"+ - "\2\2\u0408\u0409\3\2\2\2\u0409\u040b\3\2\2\2\u040a\u040c\5\u00fd\177\2"+ - "\u040b\u040a\3\2\2\2\u040c\u040d\3\2\2\2\u040d\u040b\3\2\2\2\u040d\u040e"+ - "\3\2\2\2\u040e\u00fc\3\2\2\2\u040f\u0410\t\7\2\2\u0410\u00fe\3\2\2\2\u0411"+ - "\u0412\t\b\2\2\u0412\u0100\3\2\2\2\u0413\u0414\7/\2\2\u0414\u0415\7/\2"+ - "\2\u0415\u0419\3\2\2\2\u0416\u0418\n\t\2\2\u0417\u0416\3\2\2\2\u0418\u041b"+ - "\3\2\2\2\u0419\u0417\3\2\2\2\u0419\u041a\3\2\2\2\u041a\u041d\3\2\2\2\u041b"+ - "\u0419\3\2\2\2\u041c\u041e\7\17\2\2\u041d\u041c\3\2\2\2\u041d\u041e\3"+ - "\2\2\2\u041e\u0420\3\2\2\2\u041f\u0421\7\f\2\2\u0420\u041f\3\2\2\2\u0420"+ - "\u0421\3\2\2\2\u0421\u0422\3\2\2\2\u0422\u0423\b\u0081\2\2\u0423\u0102"+ - "\3\2\2\2\u0424\u0425\7\61\2\2\u0425\u0426\7,\2\2\u0426\u042b\3\2\2\2\u0427"+ - "\u042a\5\u0103\u0082\2\u0428\u042a\13\2\2\2\u0429\u0427\3\2\2\2\u0429"+ - "\u0428\3\2\2\2\u042a\u042d\3\2\2\2\u042b\u042c\3\2\2\2\u042b\u0429\3\2"+ - "\2\2\u042c\u042e\3\2\2\2\u042d\u042b\3\2\2\2\u042e\u042f\7,\2\2\u042f"+ - "\u0430\7\61\2\2\u0430\u0431\3\2\2\2\u0431\u0432\b\u0082\2\2\u0432\u0104"+ - "\3\2\2\2\u0433\u0435\t\n\2\2\u0434\u0433\3\2\2\2\u0435\u0436\3\2\2\2\u0436"+ - "\u0434\3\2\2\2\u0436\u0437\3\2\2\2\u0437\u0438\3\2\2\2\u0438\u0439\b\u0083"+ - "\2\2\u0439\u0106\3\2\2\2\u043a\u043b\13\2\2\2\u043b\u0108\3\2\2\2\"\2"+ - "\u0379\u039d\u039f\u03a7\u03ac\u03b2\u03b9\u03be\u03c4\u03c7\u03cf\u03d3"+ - "\u03d7\u03dc\u03de\u03e5\u03e7\u03ec\u03ee\u03f4\u03f6\u03ff\u0401\u0408"+ - "\u040d\u0419\u041d\u0420\u0429\u042b\u0436\3\2\3\2"; + "z\u00f3{\u00f5|\u00f7}\u00f9~\u00fb\177\u00fd\u0080\u00ff\u0081\u0101"+ + "\u0082\u0103\u0083\u0105\2\u0107\2\u0109\2\u010b\u0084\u010d\u0085\u010f"+ + "\u0086\u0111\u0087\3\2\13\3\2))\4\2BBaa\3\2$$\3\2bb\4\2--//\3\2\62;\3"+ + "\2C\\\4\2\f\f\17\17\5\2\13\f\17\17\"\"\u047e\2\3\3\2\2\2\2\5\3\2\2\2\2"+ + "\7\3\2\2\2\2\t\3\2\2\2\2\13\3\2\2\2\2\r\3\2\2\2\2\17\3\2\2\2\2\21\3\2"+ + "\2\2\2\23\3\2\2\2\2\25\3\2\2\2\2\27\3\2\2\2\2\31\3\2\2\2\2\33\3\2\2\2"+ + "\2\35\3\2\2\2\2\37\3\2\2\2\2!\3\2\2\2\2#\3\2\2\2\2%\3\2\2\2\2\'\3\2\2"+ + "\2\2)\3\2\2\2\2+\3\2\2\2\2-\3\2\2\2\2/\3\2\2\2\2\61\3\2\2\2\2\63\3\2\2"+ + "\2\2\65\3\2\2\2\2\67\3\2\2\2\29\3\2\2\2\2;\3\2\2\2\2=\3\2\2\2\2?\3\2\2"+ + "\2\2A\3\2\2\2\2C\3\2\2\2\2E\3\2\2\2\2G\3\2\2\2\2I\3\2\2\2\2K\3\2\2\2\2"+ + "M\3\2\2\2\2O\3\2\2\2\2Q\3\2\2\2\2S\3\2\2\2\2U\3\2\2\2\2W\3\2\2\2\2Y\3"+ + "\2\2\2\2[\3\2\2\2\2]\3\2\2\2\2_\3\2\2\2\2a\3\2\2\2\2c\3\2\2\2\2e\3\2\2"+ + "\2\2g\3\2\2\2\2i\3\2\2\2\2k\3\2\2\2\2m\3\2\2\2\2o\3\2\2\2\2q\3\2\2\2\2"+ + "s\3\2\2\2\2u\3\2\2\2\2w\3\2\2\2\2y\3\2\2\2\2{\3\2\2\2\2}\3\2\2\2\2\177"+ + "\3\2\2\2\2\u0081\3\2\2\2\2\u0083\3\2\2\2\2\u0085\3\2\2\2\2\u0087\3\2\2"+ + "\2\2\u0089\3\2\2\2\2\u008b\3\2\2\2\2\u008d\3\2\2\2\2\u008f\3\2\2\2\2\u0091"+ + "\3\2\2\2\2\u0093\3\2\2\2\2\u0095\3\2\2\2\2\u0097\3\2\2\2\2\u0099\3\2\2"+ + "\2\2\u009b\3\2\2\2\2\u009d\3\2\2\2\2\u009f\3\2\2\2\2\u00a1\3\2\2\2\2\u00a3"+ + "\3\2\2\2\2\u00a5\3\2\2\2\2\u00a7\3\2\2\2\2\u00a9\3\2\2\2\2\u00ab\3\2\2"+ + "\2\2\u00ad\3\2\2\2\2\u00af\3\2\2\2\2\u00b1\3\2\2\2\2\u00b3\3\2\2\2\2\u00b5"+ + "\3\2\2\2\2\u00b7\3\2\2\2\2\u00b9\3\2\2\2\2\u00bb\3\2\2\2\2\u00bd\3\2\2"+ + "\2\2\u00bf\3\2\2\2\2\u00c1\3\2\2\2\2\u00c3\3\2\2\2\2\u00c5\3\2\2\2\2\u00c7"+ + "\3\2\2\2\2\u00c9\3\2\2\2\2\u00cb\3\2\2\2\2\u00cd\3\2\2\2\2\u00cf\3\2\2"+ + "\2\2\u00d1\3\2\2\2\2\u00d3\3\2\2\2\2\u00d5\3\2\2\2\2\u00d7\3\2\2\2\2\u00d9"+ + "\3\2\2\2\2\u00db\3\2\2\2\2\u00dd\3\2\2\2\2\u00df\3\2\2\2\2\u00e1\3\2\2"+ + "\2\2\u00e3\3\2\2\2\2\u00e5\3\2\2\2\2\u00e7\3\2\2\2\2\u00e9\3\2\2\2\2\u00eb"+ + "\3\2\2\2\2\u00ed\3\2\2\2\2\u00ef\3\2\2\2\2\u00f1\3\2\2\2\2\u00f3\3\2\2"+ + "\2\2\u00f5\3\2\2\2\2\u00f7\3\2\2\2\2\u00f9\3\2\2\2\2\u00fb\3\2\2\2\2\u00fd"+ + "\3\2\2\2\2\u00ff\3\2\2\2\2\u0101\3\2\2\2\2\u0103\3\2\2\2\2\u010b\3\2\2"+ + "\2\2\u010d\3\2\2\2\2\u010f\3\2\2\2\2\u0111\3\2\2\2\3\u0113\3\2\2\2\5\u0115"+ + "\3\2\2\2\7\u0117\3\2\2\2\t\u0119\3\2\2\2\13\u011b\3\2\2\2\r\u011f\3\2"+ + "\2\2\17\u0127\3\2\2\2\21\u0130\3\2\2\2\23\u0134\3\2\2\2\25\u0138\3\2\2"+ + "\2\27\u013b\3\2\2\2\31\u013f\3\2\2\2\33\u0147\3\2\2\2\35\u014a\3\2\2\2"+ + "\37\u014f\3\2\2\2!\u0154\3\2\2\2#\u015c\3\2\2\2%\u0165\3\2\2\2\'\u016d"+ + "\3\2\2\2)\u0175\3\2\2\2+\u0182\3\2\2\2-\u018f\3\2\2\2/\u01a1\3\2\2\2\61"+ + "\u01a5\3\2\2\2\63\u01aa\3\2\2\2\65\u01b0\3\2\2\2\67\u01b5\3\2\2\29\u01be"+ + "\3\2\2\2;\u01c7\3\2\2\2=\u01cc\3\2\2\2?\u01d0\3\2\2\2A\u01d7\3\2\2\2C"+ + "\u01e2\3\2\2\2E\u01e9\3\2\2\2G\u01f1\3\2\2\2I\u01f9\3\2\2\2K\u01ff\3\2"+ + "\2\2M\u0205\3\2\2\2O\u020c\3\2\2\2Q\u0211\3\2\2\2S\u0216\3\2\2\2U\u0220"+ + "\3\2\2\2W\u0229\3\2\2\2Y\u022f\3\2\2\2[\u0236\3\2\2\2]\u023b\3\2\2\2_"+ + "\u0241\3\2\2\2a\u0244\3\2\2\2c\u024a\3\2\2\2e\u0253\3\2\2\2g\u0256\3\2"+ + "\2\2i\u025b\3\2\2\2k\u0260\3\2\2\2m\u0265\3\2\2\2o\u026a\3\2\2\2q\u0270"+ + "\3\2\2\2s\u0277\3\2\2\2u\u027d\3\2\2\2w\u0284\3\2\2\2y\u028c\3\2\2\2{"+ + "\u0292\3\2\2\2}\u0299\3\2\2\2\177\u02a1\3\2\2\2\u0081\u02a5\3\2\2\2\u0083"+ + "\u02aa\3\2\2\2\u0085\u02b0\3\2\2\2\u0087\u02b3\3\2\2\2\u0089\u02bd\3\2"+ + "\2\2\u008b\u02c0\3\2\2\2\u008d\u02c6\3\2\2\2\u008f\u02cc\3\2\2\2\u0091"+ + "\u02d3\3\2\2\2\u0093\u02dc\3\2\2\2\u0095\u02e1\3\2\2\2\u0097\u02e7\3\2"+ + "\2\2\u0099\u02ed\3\2\2\2\u009b\u02f3\3\2\2\2\u009d\u02fb\3\2\2\2\u009f"+ + "\u0302\3\2\2\2\u00a1\u030a\3\2\2\2\u00a3\u0311\3\2\2\2\u00a5\u0316\3\2"+ + "\2\2\u00a7\u031a\3\2\2\2\u00a9\u0320\3\2\2\2\u00ab\u0327\3\2\2\2\u00ad"+ + "\u032c\3\2\2\2\u00af\u0331\3\2\2\2\u00b1\u0336\3\2\2\2\u00b3\u0339\3\2"+ + "\2\2\u00b5\u033e\3\2\2\2\u00b7\u0344\3\2\2\2\u00b9\u034a\3\2\2\2\u00bb"+ + "\u0351\3\2\2\2\u00bd\u0356\3\2\2\2\u00bf\u035c\3\2\2\2\u00c1\u0361\3\2"+ + "\2\2\u00c3\u0366\3\2\2\2\u00c5\u036c\3\2\2\2\u00c7\u0374\3\2\2\2\u00c9"+ + "\u0378\3\2\2\2\u00cb\u037f\3\2\2\2\u00cd\u0382\3\2\2\2\u00cf\u0385\3\2"+ + "\2\2\u00d1\u0389\3\2\2\2\u00d3\u038f\3\2\2\2\u00d5\u0391\3\2\2\2\u00d7"+ + "\u0393\3\2\2\2\u00d9\u039b\3\2\2\2\u00db\u039d\3\2\2\2\u00dd\u039f\3\2"+ + "\2\2\u00df\u03a2\3\2\2\2\u00e1\u03a4\3\2\2\2\u00e3\u03a7\3\2\2\2\u00e5"+ + "\u03a9\3\2\2\2\u00e7\u03ab\3\2\2\2\u00e9\u03ad\3\2\2\2\u00eb\u03af\3\2"+ + "\2\2\u00ed\u03b1\3\2\2\2\u00ef\u03b4\3\2\2\2\u00f1\u03b7\3\2\2\2\u00f3"+ + "\u03b9\3\2\2\2\u00f5\u03bb\3\2\2\2\u00f7\u03c7\3\2\2\2\u00f9\u03f5\3\2"+ + "\2\2\u00fb\u03f9\3\2\2\2\u00fd\u0403\3\2\2\2\u00ff\u040e\3\2\2\2\u0101"+ + "\u0412\3\2\2\2\u0103\u041d\3\2\2\2\u0105\u0428\3\2\2\2\u0107\u0431\3\2"+ + "\2\2\u0109\u0433\3\2\2\2\u010b\u0435\3\2\2\2\u010d\u0446\3\2\2\2\u010f"+ + "\u0456\3\2\2\2\u0111\u045c\3\2\2\2\u0113\u0114\7*\2\2\u0114\4\3\2\2\2"+ + "\u0115\u0116\7+\2\2\u0116\6\3\2\2\2\u0117\u0118\7.\2\2\u0118\b\3\2\2\2"+ + "\u0119\u011a\7<\2\2\u011a\n\3\2\2\2\u011b\u011c\7C\2\2\u011c\u011d\7N"+ + "\2\2\u011d\u011e\7N\2\2\u011e\f\3\2\2\2\u011f\u0120\7C\2\2\u0120\u0121"+ + "\7P\2\2\u0121\u0122\7C\2\2\u0122\u0123\7N\2\2\u0123\u0124\7[\2\2\u0124"+ + "\u0125\7\\\2\2\u0125\u0126\7G\2\2\u0126\16\3\2\2\2\u0127\u0128\7C\2\2"+ + "\u0128\u0129\7P\2\2\u0129\u012a\7C\2\2\u012a\u012b\7N\2\2\u012b\u012c"+ + "\7[\2\2\u012c\u012d\7\\\2\2\u012d\u012e\7G\2\2\u012e\u012f\7F\2\2\u012f"+ + "\20\3\2\2\2\u0130\u0131\7C\2\2\u0131\u0132\7P\2\2\u0132\u0133\7F\2\2\u0133"+ + "\22\3\2\2\2\u0134\u0135\7C\2\2\u0135\u0136\7P\2\2\u0136\u0137\7[\2\2\u0137"+ + "\24\3\2\2\2\u0138\u0139\7C\2\2\u0139\u013a\7U\2\2\u013a\26\3\2\2\2\u013b"+ + "\u013c\7C\2\2\u013c\u013d\7U\2\2\u013d\u013e\7E\2\2\u013e\30\3\2\2\2\u013f"+ + "\u0140\7D\2\2\u0140\u0141\7G\2\2\u0141\u0142\7V\2\2\u0142\u0143\7Y\2\2"+ + "\u0143\u0144\7G\2\2\u0144\u0145\7G\2\2\u0145\u0146\7P\2\2\u0146\32\3\2"+ + "\2\2\u0147\u0148\7D\2\2\u0148\u0149\7[\2\2\u0149\34\3\2\2\2\u014a\u014b"+ + "\7E\2\2\u014b\u014c\7C\2\2\u014c\u014d\7U\2\2\u014d\u014e\7G\2\2\u014e"+ + "\36\3\2\2\2\u014f\u0150\7E\2\2\u0150\u0151\7C\2\2\u0151\u0152\7U\2\2\u0152"+ + "\u0153\7V\2\2\u0153 \3\2\2\2\u0154\u0155\7E\2\2\u0155\u0156\7C\2\2\u0156"+ + "\u0157\7V\2\2\u0157\u0158\7C\2\2\u0158\u0159\7N\2\2\u0159\u015a\7Q\2\2"+ + "\u015a\u015b\7I\2\2\u015b\"\3\2\2\2\u015c\u015d\7E\2\2\u015d\u015e\7C"+ + "\2\2\u015e\u015f\7V\2\2\u015f\u0160\7C\2\2\u0160\u0161\7N\2\2\u0161\u0162"+ + "\7Q\2\2\u0162\u0163\7I\2\2\u0163\u0164\7U\2\2\u0164$\3\2\2\2\u0165\u0166"+ + "\7E\2\2\u0166\u0167\7Q\2\2\u0167\u0168\7N\2\2\u0168\u0169\7W\2\2\u0169"+ + "\u016a\7O\2\2\u016a\u016b\7P\2\2\u016b\u016c\7U\2\2\u016c&\3\2\2\2\u016d"+ + "\u016e\7E\2\2\u016e\u016f\7Q\2\2\u016f\u0170\7P\2\2\u0170\u0171\7X\2\2"+ + "\u0171\u0172\7G\2\2\u0172\u0173\7T\2\2\u0173\u0174\7V\2\2\u0174(\3\2\2"+ + "\2\u0175\u0176\7E\2\2\u0176\u0177\7W\2\2\u0177\u0178\7T\2\2\u0178\u0179"+ + "\7T\2\2\u0179\u017a\7G\2\2\u017a\u017b\7P\2\2\u017b\u017c\7V\2\2\u017c"+ + "\u017d\7a\2\2\u017d\u017e\7F\2\2\u017e\u017f\7C\2\2\u017f\u0180\7V\2\2"+ + "\u0180\u0181\7G\2\2\u0181*\3\2\2\2\u0182\u0183\7E\2\2\u0183\u0184\7W\2"+ + "\2\u0184\u0185\7T\2\2\u0185\u0186\7T\2\2\u0186\u0187\7G\2\2\u0187\u0188"+ + "\7P\2\2\u0188\u0189\7V\2\2\u0189\u018a\7a\2\2\u018a\u018b\7V\2\2\u018b"+ + "\u018c\7K\2\2\u018c\u018d\7O\2\2\u018d\u018e\7G\2\2\u018e,\3\2\2\2\u018f"+ + "\u0190\7E\2\2\u0190\u0191\7W\2\2\u0191\u0192\7T\2\2\u0192\u0193\7T\2\2"+ + "\u0193\u0194\7G\2\2\u0194\u0195\7P\2\2\u0195\u0196\7V\2\2\u0196\u0197"+ + "\7a\2\2\u0197\u0198\7V\2\2\u0198\u0199\7K\2\2\u0199\u019a\7O\2\2\u019a"+ + "\u019b\7G\2\2\u019b\u019c\7U\2\2\u019c\u019d\7V\2\2\u019d\u019e\7C\2\2"+ + "\u019e\u019f\7O\2\2\u019f\u01a0\7R\2\2\u01a0.\3\2\2\2\u01a1\u01a2\7F\2"+ + "\2\u01a2\u01a3\7C\2\2\u01a3\u01a4\7[\2\2\u01a4\60\3\2\2\2\u01a5\u01a6"+ + "\7F\2\2\u01a6\u01a7\7C\2\2\u01a7\u01a8\7[\2\2\u01a8\u01a9\7U\2\2\u01a9"+ + "\62\3\2\2\2\u01aa\u01ab\7F\2\2\u01ab\u01ac\7G\2\2\u01ac\u01ad\7D\2\2\u01ad"+ + "\u01ae\7W\2\2\u01ae\u01af\7I\2\2\u01af\64\3\2\2\2\u01b0\u01b1\7F\2\2\u01b1"+ + "\u01b2\7G\2\2\u01b2\u01b3\7U\2\2\u01b3\u01b4\7E\2\2\u01b4\66\3\2\2\2\u01b5"+ + "\u01b6\7F\2\2\u01b6\u01b7\7G\2\2\u01b7\u01b8\7U\2\2\u01b8\u01b9\7E\2\2"+ + "\u01b9\u01ba\7T\2\2\u01ba\u01bb\7K\2\2\u01bb\u01bc\7D\2\2\u01bc\u01bd"+ + "\7G\2\2\u01bd8\3\2\2\2\u01be\u01bf\7F\2\2\u01bf\u01c0\7K\2\2\u01c0\u01c1"+ + "\7U\2\2\u01c1\u01c2\7V\2\2\u01c2\u01c3\7K\2\2\u01c3\u01c4\7P\2\2\u01c4"+ + "\u01c5\7E\2\2\u01c5\u01c6\7V\2\2\u01c6:\3\2\2\2\u01c7\u01c8\7G\2\2\u01c8"+ + "\u01c9\7N\2\2\u01c9\u01ca\7U\2\2\u01ca\u01cb\7G\2\2\u01cb<\3\2\2\2\u01cc"+ + "\u01cd\7G\2\2\u01cd\u01ce\7P\2\2\u01ce\u01cf\7F\2\2\u01cf>\3\2\2\2\u01d0"+ + "\u01d1\7G\2\2\u01d1\u01d2\7U\2\2\u01d2\u01d3\7E\2\2\u01d3\u01d4\7C\2\2"+ + "\u01d4\u01d5\7R\2\2\u01d5\u01d6\7G\2\2\u01d6@\3\2\2\2\u01d7\u01d8\7G\2"+ + "\2\u01d8\u01d9\7Z\2\2\u01d9\u01da\7G\2\2\u01da\u01db\7E\2\2\u01db\u01dc"+ + "\7W\2\2\u01dc\u01dd\7V\2\2\u01dd\u01de\7C\2\2\u01de\u01df\7D\2\2\u01df"+ + "\u01e0\7N\2\2\u01e0\u01e1\7G\2\2\u01e1B\3\2\2\2\u01e2\u01e3\7G\2\2\u01e3"+ + "\u01e4\7Z\2\2\u01e4\u01e5\7K\2\2\u01e5\u01e6\7U\2\2\u01e6\u01e7\7V\2\2"+ + "\u01e7\u01e8\7U\2\2\u01e8D\3\2\2\2\u01e9\u01ea\7G\2\2\u01ea\u01eb\7Z\2"+ + "\2\u01eb\u01ec\7R\2\2\u01ec\u01ed\7N\2\2\u01ed\u01ee\7C\2\2\u01ee\u01ef"+ + "\7K\2\2\u01ef\u01f0\7P\2\2\u01f0F\3\2\2\2\u01f1\u01f2\7G\2\2\u01f2\u01f3"+ + "\7Z\2\2\u01f3\u01f4\7V\2\2\u01f4\u01f5\7T\2\2\u01f5\u01f6\7C\2\2\u01f6"+ + "\u01f7\7E\2\2\u01f7\u01f8\7V\2\2\u01f8H\3\2\2\2\u01f9\u01fa\7H\2\2\u01fa"+ + "\u01fb\7C\2\2\u01fb\u01fc\7N\2\2\u01fc\u01fd\7U\2\2\u01fd\u01fe\7G\2\2"+ + "\u01feJ\3\2\2\2\u01ff\u0200\7H\2\2\u0200\u0201\7K\2\2\u0201\u0202\7T\2"+ + "\2\u0202\u0203\7U\2\2\u0203\u0204\7V\2\2\u0204L\3\2\2\2\u0205\u0206\7"+ + "H\2\2\u0206\u0207\7Q\2\2\u0207\u0208\7T\2\2\u0208\u0209\7O\2\2\u0209\u020a"+ + "\7C\2\2\u020a\u020b\7V\2\2\u020bN\3\2\2\2\u020c\u020d\7H\2\2\u020d\u020e"+ + "\7T\2\2\u020e\u020f\7Q\2\2\u020f\u0210\7O\2\2\u0210P\3\2\2\2\u0211\u0212"+ + "\7H\2\2\u0212\u0213\7W\2\2\u0213\u0214\7N\2\2\u0214\u0215\7N\2\2\u0215"+ + "R\3\2\2\2\u0216\u0217\7H\2\2\u0217\u0218\7W\2\2\u0218\u0219\7P\2\2\u0219"+ + "\u021a\7E\2\2\u021a\u021b\7V\2\2\u021b\u021c\7K\2\2\u021c\u021d\7Q\2\2"+ + "\u021d\u021e\7P\2\2\u021e\u021f\7U\2\2\u021fT\3\2\2\2\u0220\u0221\7I\2"+ + "\2\u0221\u0222\7T\2\2\u0222\u0223\7C\2\2\u0223\u0224\7R\2\2\u0224\u0225"+ + "\7J\2\2\u0225\u0226\7X\2\2\u0226\u0227\7K\2\2\u0227\u0228\7\\\2\2\u0228"+ + "V\3\2\2\2\u0229\u022a\7I\2\2\u022a\u022b\7T\2\2\u022b\u022c\7Q\2\2\u022c"+ + "\u022d\7W\2\2\u022d\u022e\7R\2\2\u022eX\3\2\2\2\u022f\u0230\7J\2\2\u0230"+ + "\u0231\7C\2\2\u0231\u0232\7X\2\2\u0232\u0233\7K\2\2\u0233\u0234\7P\2\2"+ + "\u0234\u0235\7I\2\2\u0235Z\3\2\2\2\u0236\u0237\7J\2\2\u0237\u0238\7Q\2"+ + "\2\u0238\u0239\7W\2\2\u0239\u023a\7T\2\2\u023a\\\3\2\2\2\u023b\u023c\7"+ + "J\2\2\u023c\u023d\7Q\2\2\u023d\u023e\7W\2\2\u023e\u023f\7T\2\2\u023f\u0240"+ + "\7U\2\2\u0240^\3\2\2\2\u0241\u0242\7K\2\2\u0242\u0243\7P\2\2\u0243`\3"+ + "\2\2\2\u0244\u0245\7K\2\2\u0245\u0246\7P\2\2\u0246\u0247\7P\2\2\u0247"+ + "\u0248\7G\2\2\u0248\u0249\7T\2\2\u0249b\3\2\2\2\u024a\u024b\7K\2\2\u024b"+ + "\u024c\7P\2\2\u024c\u024d\7V\2\2\u024d\u024e\7G\2\2\u024e\u024f\7T\2\2"+ + "\u024f\u0250\7X\2\2\u0250\u0251\7C\2\2\u0251\u0252\7N\2\2\u0252d\3\2\2"+ + "\2\u0253\u0254\7K\2\2\u0254\u0255\7U\2\2\u0255f\3\2\2\2\u0256\u0257\7"+ + "L\2\2\u0257\u0258\7Q\2\2\u0258\u0259\7K\2\2\u0259\u025a\7P\2\2\u025ah"+ + "\3\2\2\2\u025b\u025c\7N\2\2\u025c\u025d\7C\2\2\u025d\u025e\7U\2\2\u025e"+ + "\u025f\7V\2\2\u025fj\3\2\2\2\u0260\u0261\7N\2\2\u0261\u0262\7G\2\2\u0262"+ + "\u0263\7H\2\2\u0263\u0264\7V\2\2\u0264l\3\2\2\2\u0265\u0266\7N\2\2\u0266"+ + "\u0267\7K\2\2\u0267\u0268\7M\2\2\u0268\u0269\7G\2\2\u0269n\3\2\2\2\u026a"+ + "\u026b\7N\2\2\u026b\u026c\7K\2\2\u026c\u026d\7O\2\2\u026d\u026e\7K\2\2"+ + "\u026e\u026f\7V\2\2\u026fp\3\2\2\2\u0270\u0271\7O\2\2\u0271\u0272\7C\2"+ + "\2\u0272\u0273\7R\2\2\u0273\u0274\7R\2\2\u0274\u0275\7G\2\2\u0275\u0276"+ + "\7F\2\2\u0276r\3\2\2\2\u0277\u0278\7O\2\2\u0278\u0279\7C\2\2\u0279\u027a"+ + "\7V\2\2\u027a\u027b\7E\2\2\u027b\u027c\7J\2\2\u027ct\3\2\2\2\u027d\u027e"+ + "\7O\2\2\u027e\u027f\7K\2\2\u027f\u0280\7P\2\2\u0280\u0281\7W\2\2\u0281"+ + "\u0282\7V\2\2\u0282\u0283\7G\2\2\u0283v\3\2\2\2\u0284\u0285\7O\2\2\u0285"+ + "\u0286\7K\2\2\u0286\u0287\7P\2\2\u0287\u0288\7W\2\2\u0288\u0289\7V\2\2"+ + "\u0289\u028a\7G\2\2\u028a\u028b\7U\2\2\u028bx\3\2\2\2\u028c\u028d\7O\2"+ + "\2\u028d\u028e\7Q\2\2\u028e\u028f\7P\2\2\u028f\u0290\7V\2\2\u0290\u0291"+ + "\7J\2\2\u0291z\3\2\2\2\u0292\u0293\7O\2\2\u0293\u0294\7Q\2\2\u0294\u0295"+ + "\7P\2\2\u0295\u0296\7V\2\2\u0296\u0297\7J\2\2\u0297\u0298\7U\2\2\u0298"+ + "|\3\2\2\2\u0299\u029a\7P\2\2\u029a\u029b\7C\2\2\u029b\u029c\7V\2\2\u029c"+ + "\u029d\7W\2\2\u029d\u029e\7T\2\2\u029e\u029f\7C\2\2\u029f\u02a0\7N\2\2"+ + "\u02a0~\3\2\2\2\u02a1\u02a2\7P\2\2\u02a2\u02a3\7Q\2\2\u02a3\u02a4\7V\2"+ + "\2\u02a4\u0080\3\2\2\2\u02a5\u02a6\7P\2\2\u02a6\u02a7\7W\2\2\u02a7\u02a8"+ + "\7N\2\2\u02a8\u02a9\7N\2\2\u02a9\u0082\3\2\2\2\u02aa\u02ab\7P\2\2\u02ab"+ + "\u02ac\7W\2\2\u02ac\u02ad\7N\2\2\u02ad\u02ae\7N\2\2\u02ae\u02af\7U\2\2"+ + "\u02af\u0084\3\2\2\2\u02b0\u02b1\7Q\2\2\u02b1\u02b2\7P\2\2\u02b2\u0086"+ + "\3\2\2\2\u02b3\u02b4\7Q\2\2\u02b4\u02b5\7R\2\2\u02b5\u02b6\7V\2\2\u02b6"+ + "\u02b7\7K\2\2\u02b7\u02b8\7O\2\2\u02b8\u02b9\7K\2\2\u02b9\u02ba\7\\\2"+ + "\2\u02ba\u02bb\7G\2\2\u02bb\u02bc\7F\2\2\u02bc\u0088\3\2\2\2\u02bd\u02be"+ + "\7Q\2\2\u02be\u02bf\7T\2\2\u02bf\u008a\3\2\2\2\u02c0\u02c1\7Q\2\2\u02c1"+ + "\u02c2\7T\2\2\u02c2\u02c3\7F\2\2\u02c3\u02c4\7G\2\2\u02c4\u02c5\7T\2\2"+ + "\u02c5\u008c\3\2\2\2\u02c6\u02c7\7Q\2\2\u02c7\u02c8\7W\2\2\u02c8\u02c9"+ + "\7V\2\2\u02c9\u02ca\7G\2\2\u02ca\u02cb\7T\2\2\u02cb\u008e\3\2\2\2\u02cc"+ + "\u02cd\7R\2\2\u02cd\u02ce\7C\2\2\u02ce\u02cf\7T\2\2\u02cf\u02d0\7U\2\2"+ + "\u02d0\u02d1\7G\2\2\u02d1\u02d2\7F\2\2\u02d2\u0090\3\2\2\2\u02d3\u02d4"+ + "\7R\2\2\u02d4\u02d5\7J\2\2\u02d5\u02d6\7[\2\2\u02d6\u02d7\7U\2\2\u02d7"+ + "\u02d8\7K\2\2\u02d8\u02d9\7E\2\2\u02d9\u02da\7C\2\2\u02da\u02db\7N\2\2"+ + "\u02db\u0092\3\2\2\2\u02dc\u02dd\7R\2\2\u02dd\u02de\7N\2\2\u02de\u02df"+ + "\7C\2\2\u02df\u02e0\7P\2\2\u02e0\u0094\3\2\2\2\u02e1\u02e2\7T\2\2\u02e2"+ + "\u02e3\7K\2\2\u02e3\u02e4\7I\2\2\u02e4\u02e5\7J\2\2\u02e5\u02e6\7V\2\2"+ + "\u02e6\u0096\3\2\2\2\u02e7\u02e8\7T\2\2\u02e8\u02e9\7N\2\2\u02e9\u02ea"+ + "\7K\2\2\u02ea\u02eb\7M\2\2\u02eb\u02ec\7G\2\2\u02ec\u0098\3\2\2\2\u02ed"+ + "\u02ee\7S\2\2\u02ee\u02ef\7W\2\2\u02ef\u02f0\7G\2\2\u02f0\u02f1\7T\2\2"+ + "\u02f1\u02f2\7[\2\2\u02f2\u009a\3\2\2\2\u02f3\u02f4\7U\2\2\u02f4\u02f5"+ + "\7E\2\2\u02f5\u02f6\7J\2\2\u02f6\u02f7\7G\2\2\u02f7\u02f8\7O\2\2\u02f8"+ + "\u02f9\7C\2\2\u02f9\u02fa\7U\2\2\u02fa\u009c\3\2\2\2\u02fb\u02fc\7U\2"+ + "\2\u02fc\u02fd\7G\2\2\u02fd\u02fe\7E\2\2\u02fe\u02ff\7Q\2\2\u02ff\u0300"+ + "\7P\2\2\u0300\u0301\7F\2\2\u0301\u009e\3\2\2\2\u0302\u0303\7U\2\2\u0303"+ + "\u0304\7G\2\2\u0304\u0305\7E\2\2\u0305\u0306\7Q\2\2\u0306\u0307\7P\2\2"+ + "\u0307\u0308\7F\2\2\u0308\u0309\7U\2\2\u0309\u00a0\3\2\2\2\u030a\u030b"+ + "\7U\2\2\u030b\u030c\7G\2\2\u030c\u030d\7N\2\2\u030d\u030e\7G\2\2\u030e"+ + "\u030f\7E\2\2\u030f\u0310\7V\2\2\u0310\u00a2\3\2\2\2\u0311\u0312\7U\2"+ + "\2\u0312\u0313\7J\2\2\u0313\u0314\7Q\2\2\u0314\u0315\7Y\2\2\u0315\u00a4"+ + "\3\2\2\2\u0316\u0317\7U\2\2\u0317\u0318\7[\2\2\u0318\u0319\7U\2\2\u0319"+ + "\u00a6\3\2\2\2\u031a\u031b\7V\2\2\u031b\u031c\7C\2\2\u031c\u031d\7D\2"+ + "\2\u031d\u031e\7N\2\2\u031e\u031f\7G\2\2\u031f\u00a8\3\2\2\2\u0320\u0321"+ + "\7V\2\2\u0321\u0322\7C\2\2\u0322\u0323\7D\2\2\u0323\u0324\7N\2\2\u0324"+ + "\u0325\7G\2\2\u0325\u0326\7U\2\2\u0326\u00aa\3\2\2\2\u0327\u0328\7V\2"+ + "\2\u0328\u0329\7G\2\2\u0329\u032a\7Z\2\2\u032a\u032b\7V\2\2\u032b\u00ac"+ + "\3\2\2\2\u032c\u032d\7V\2\2\u032d\u032e\7J\2\2\u032e\u032f\7G\2\2\u032f"+ + "\u0330\7P\2\2\u0330\u00ae\3\2\2\2\u0331\u0332\7V\2\2\u0332\u0333\7T\2"+ + "\2\u0333\u0334\7W\2\2\u0334\u0335\7G\2\2\u0335\u00b0\3\2\2\2\u0336\u0337"+ + "\7V\2\2\u0337\u0338\7Q\2\2\u0338\u00b2\3\2\2\2\u0339\u033a\7V\2\2\u033a"+ + "\u033b\7[\2\2\u033b\u033c\7R\2\2\u033c\u033d\7G\2\2\u033d\u00b4\3\2\2"+ + "\2\u033e\u033f\7V\2\2\u033f\u0340\7[\2\2\u0340\u0341\7R\2\2\u0341\u0342"+ + "\7G\2\2\u0342\u0343\7U\2\2\u0343\u00b6\3\2\2\2\u0344\u0345\7W\2\2\u0345"+ + "\u0346\7U\2\2\u0346\u0347\7K\2\2\u0347\u0348\7P\2\2\u0348\u0349\7I\2\2"+ + "\u0349\u00b8\3\2\2\2\u034a\u034b\7X\2\2\u034b\u034c\7G\2\2\u034c\u034d"+ + "\7T\2\2\u034d\u034e\7K\2\2\u034e\u034f\7H\2\2\u034f\u0350\7[\2\2\u0350"+ + "\u00ba\3\2\2\2\u0351\u0352\7Y\2\2\u0352\u0353\7J\2\2\u0353\u0354\7G\2"+ + "\2\u0354\u0355\7P\2\2\u0355\u00bc\3\2\2\2\u0356\u0357\7Y\2\2\u0357\u0358"+ + "\7J\2\2\u0358\u0359\7G\2\2\u0359\u035a\7T\2\2\u035a\u035b\7G\2\2\u035b"+ + "\u00be\3\2\2\2\u035c\u035d\7Y\2\2\u035d\u035e\7K\2\2\u035e\u035f\7V\2"+ + "\2\u035f\u0360\7J\2\2\u0360\u00c0\3\2\2\2\u0361\u0362\7[\2\2\u0362\u0363"+ + "\7G\2\2\u0363\u0364\7C\2\2\u0364\u0365\7T\2\2\u0365\u00c2\3\2\2\2\u0366"+ + "\u0367\7[\2\2\u0367\u0368\7G\2\2\u0368\u0369\7C\2\2\u0369\u036a\7T\2\2"+ + "\u036a\u036b\7U\2\2\u036b\u00c4\3\2\2\2\u036c\u036d\7}\2\2\u036d\u036e"+ + "\7G\2\2\u036e\u036f\7U\2\2\u036f\u0370\7E\2\2\u0370\u0371\7C\2\2\u0371"+ + "\u0372\7R\2\2\u0372\u0373\7G\2\2\u0373\u00c6\3\2\2\2\u0374\u0375\7}\2"+ + "\2\u0375\u0376\7H\2\2\u0376\u0377\7P\2\2\u0377\u00c8\3\2\2\2\u0378\u0379"+ + "\7}\2\2\u0379\u037a\7N\2\2\u037a\u037b\7K\2\2\u037b\u037c\7O\2\2\u037c"+ + "\u037d\7K\2\2\u037d\u037e\7V\2\2\u037e\u00ca\3\2\2\2\u037f\u0380\7}\2"+ + "\2\u0380\u0381\7F\2\2\u0381\u00cc\3\2\2\2\u0382\u0383\7}\2\2\u0383\u0384"+ + "\7V\2\2\u0384\u00ce\3\2\2\2\u0385\u0386\7}\2\2\u0386\u0387\7V\2\2\u0387"+ + "\u0388\7U\2\2\u0388\u00d0\3\2\2\2\u0389\u038a\7}\2\2\u038a\u038b\7I\2"+ + "\2\u038b\u038c\7W\2\2\u038c\u038d\7K\2\2\u038d\u038e\7F\2\2\u038e\u00d2"+ + "\3\2\2\2\u038f\u0390\7\177\2\2\u0390\u00d4\3\2\2\2\u0391\u0392\7?\2\2"+ + "\u0392\u00d6\3\2\2\2\u0393\u0394\7>\2\2\u0394\u0395\7?\2\2\u0395\u0396"+ + "\7@\2\2\u0396\u00d8\3\2\2\2\u0397\u0398\7>\2\2\u0398\u039c\7@\2\2\u0399"+ + "\u039a\7#\2\2\u039a\u039c\7?\2\2\u039b\u0397\3\2\2\2\u039b\u0399\3\2\2"+ + "\2\u039c\u00da\3\2\2\2\u039d\u039e\7>\2\2\u039e\u00dc\3\2\2\2\u039f\u03a0"+ + "\7>\2\2\u03a0\u03a1\7?\2\2\u03a1\u00de\3\2\2\2\u03a2\u03a3\7@\2\2\u03a3"+ + "\u00e0\3\2\2\2\u03a4\u03a5\7@\2\2\u03a5\u03a6\7?\2\2\u03a6\u00e2\3\2\2"+ + "\2\u03a7\u03a8\7-\2\2\u03a8\u00e4\3\2\2\2\u03a9\u03aa\7/\2\2\u03aa\u00e6"+ + "\3\2\2\2\u03ab\u03ac\7,\2\2\u03ac\u00e8\3\2\2\2\u03ad\u03ae\7\61\2\2\u03ae"+ + "\u00ea\3\2\2\2\u03af\u03b0\7\'\2\2\u03b0\u00ec\3\2\2\2\u03b1\u03b2\7<"+ + "\2\2\u03b2\u03b3\7<\2\2\u03b3\u00ee\3\2\2\2\u03b4\u03b5\7~\2\2\u03b5\u03b6"+ + "\7~\2\2\u03b6\u00f0\3\2\2\2\u03b7\u03b8\7\60\2\2\u03b8\u00f2\3\2\2\2\u03b9"+ + "\u03ba\7A\2\2\u03ba\u00f4\3\2\2\2\u03bb\u03c1\7)\2\2\u03bc\u03c0\n\2\2"+ + "\2\u03bd\u03be\7)\2\2\u03be\u03c0\7)\2\2\u03bf\u03bc\3\2\2\2\u03bf\u03bd"+ + "\3\2\2\2\u03c0\u03c3\3\2\2\2\u03c1\u03bf\3\2\2\2\u03c1\u03c2\3\2\2\2\u03c2"+ + "\u03c4\3\2\2\2\u03c3\u03c1\3\2\2\2\u03c4\u03c5\7)\2\2\u03c5\u00f6\3\2"+ + "\2\2\u03c6\u03c8\5\u0107\u0084\2\u03c7\u03c6\3\2\2\2\u03c8\u03c9\3\2\2"+ + "\2\u03c9\u03c7\3\2\2\2\u03c9\u03ca\3\2\2\2\u03ca\u00f8\3\2\2\2\u03cb\u03cd"+ + "\5\u0107\u0084\2\u03cc\u03cb\3\2\2\2\u03cd\u03ce\3\2\2\2\u03ce\u03cc\3"+ + "\2\2\2\u03ce\u03cf\3\2\2\2\u03cf\u03d0\3\2\2\2\u03d0\u03d4\5\u00f1y\2"+ + "\u03d1\u03d3\5\u0107\u0084\2\u03d2\u03d1\3\2\2\2\u03d3\u03d6\3\2\2\2\u03d4"+ + "\u03d2\3\2\2\2\u03d4\u03d5\3\2\2\2\u03d5\u03f6\3\2\2\2\u03d6\u03d4\3\2"+ + "\2\2\u03d7\u03d9\5\u00f1y\2\u03d8\u03da\5\u0107\u0084\2\u03d9\u03d8\3"+ + "\2\2\2\u03da\u03db\3\2\2\2\u03db\u03d9\3\2\2\2\u03db\u03dc\3\2\2\2\u03dc"+ + "\u03f6\3\2\2\2\u03dd\u03df\5\u0107\u0084\2\u03de\u03dd\3\2\2\2\u03df\u03e0"+ + "\3\2\2\2\u03e0\u03de\3\2\2\2\u03e0\u03e1\3\2\2\2\u03e1\u03e9\3\2\2\2\u03e2"+ + "\u03e6\5\u00f1y\2\u03e3\u03e5\5\u0107\u0084\2\u03e4\u03e3\3\2\2\2\u03e5"+ + "\u03e8\3\2\2\2\u03e6\u03e4\3\2\2\2\u03e6\u03e7\3\2\2\2\u03e7\u03ea\3\2"+ + "\2\2\u03e8\u03e6\3\2\2\2\u03e9\u03e2\3\2\2\2\u03e9\u03ea\3\2\2\2\u03ea"+ + "\u03eb\3\2\2\2\u03eb\u03ec\5\u0105\u0083\2\u03ec\u03f6\3\2\2\2\u03ed\u03ef"+ + "\5\u00f1y\2\u03ee\u03f0\5\u0107\u0084\2\u03ef\u03ee\3\2\2\2\u03f0\u03f1"+ + "\3\2\2\2\u03f1\u03ef\3\2\2\2\u03f1\u03f2\3\2\2\2\u03f2\u03f3\3\2\2\2\u03f3"+ + "\u03f4\5\u0105\u0083\2\u03f4\u03f6\3\2\2\2\u03f5\u03cc\3\2\2\2\u03f5\u03d7"+ + "\3\2\2\2\u03f5\u03de\3\2\2\2\u03f5\u03ed\3\2\2\2\u03f6\u00fa\3\2\2\2\u03f7"+ + "\u03fa\5\u0109\u0085\2\u03f8\u03fa\7a\2\2\u03f9\u03f7\3\2\2\2\u03f9\u03f8"+ + "\3\2\2\2\u03fa\u0400\3\2\2\2\u03fb\u03ff\5\u0109\u0085\2\u03fc\u03ff\5"+ + "\u0107\u0084\2\u03fd\u03ff\t\3\2\2\u03fe\u03fb\3\2\2\2\u03fe\u03fc\3\2"+ + "\2\2\u03fe\u03fd\3\2\2\2\u03ff\u0402\3\2\2\2\u0400\u03fe\3\2\2\2\u0400"+ + "\u0401\3\2\2\2\u0401\u00fc\3\2\2\2\u0402\u0400\3\2\2\2\u0403\u0407\5\u0107"+ + "\u0084\2\u0404\u0408\5\u0109\u0085\2\u0405\u0408\5\u0107\u0084\2\u0406"+ + "\u0408\t\3\2\2\u0407\u0404\3\2\2\2\u0407\u0405\3\2\2\2\u0407\u0406\3\2"+ + "\2\2\u0408\u0409\3\2\2\2\u0409\u0407\3\2\2\2\u0409\u040a\3\2\2\2\u040a"+ + "\u00fe\3\2\2\2\u040b\u040f\5\u0109\u0085\2\u040c\u040f\5\u0107\u0084\2"+ + "\u040d\u040f\7a\2\2\u040e\u040b\3\2\2\2\u040e\u040c\3\2\2\2\u040e\u040d"+ + "\3\2\2\2\u040f\u0410\3\2\2\2\u0410\u040e\3\2\2\2\u0410\u0411\3\2\2\2\u0411"+ + "\u0100\3\2\2\2\u0412\u0418\7$\2\2\u0413\u0417\n\4\2\2\u0414\u0415\7$\2"+ + "\2\u0415\u0417\7$\2\2\u0416\u0413\3\2\2\2\u0416\u0414\3\2\2\2\u0417\u041a"+ + "\3\2\2\2\u0418\u0416\3\2\2\2\u0418\u0419\3\2\2\2\u0419\u041b\3\2\2\2\u041a"+ + "\u0418\3\2\2\2\u041b\u041c\7$\2\2\u041c\u0102\3\2\2\2\u041d\u0423\7b\2"+ + "\2\u041e\u0422\n\5\2\2\u041f\u0420\7b\2\2\u0420\u0422\7b\2\2\u0421\u041e"+ + "\3\2\2\2\u0421\u041f\3\2\2\2\u0422\u0425\3\2\2\2\u0423\u0421\3\2\2\2\u0423"+ + "\u0424\3\2\2\2\u0424\u0426\3\2\2\2\u0425\u0423\3\2\2\2\u0426\u0427\7b"+ + "\2\2\u0427\u0104\3\2\2\2\u0428\u042a\7G\2\2\u0429\u042b\t\6\2\2\u042a"+ + "\u0429\3\2\2\2\u042a\u042b\3\2\2\2\u042b\u042d\3\2\2\2\u042c\u042e\5\u0107"+ + "\u0084\2\u042d\u042c\3\2\2\2\u042e\u042f\3\2\2\2\u042f\u042d\3\2\2\2\u042f"+ + "\u0430\3\2\2\2\u0430\u0106\3\2\2\2\u0431\u0432\t\7\2\2\u0432\u0108\3\2"+ + "\2\2\u0433\u0434\t\b\2\2\u0434\u010a\3\2\2\2\u0435\u0436\7/\2\2\u0436"+ + "\u0437\7/\2\2\u0437\u043b\3\2\2\2\u0438\u043a\n\t\2\2\u0439\u0438\3\2"+ + "\2\2\u043a\u043d\3\2\2\2\u043b\u0439\3\2\2\2\u043b\u043c\3\2\2\2\u043c"+ + "\u043f\3\2\2\2\u043d\u043b\3\2\2\2\u043e\u0440\7\17\2\2\u043f\u043e\3"+ + "\2\2\2\u043f\u0440\3\2\2\2\u0440\u0442\3\2\2\2\u0441\u0443\7\f\2\2\u0442"+ + "\u0441\3\2\2\2\u0442\u0443\3\2\2\2\u0443\u0444\3\2\2\2\u0444\u0445\b\u0086"+ + "\2\2\u0445\u010c\3\2\2\2\u0446\u0447\7\61\2\2\u0447\u0448\7,\2\2\u0448"+ + "\u044d\3\2\2\2\u0449\u044c\5\u010d\u0087\2\u044a\u044c\13\2\2\2\u044b"+ + "\u0449\3\2\2\2\u044b\u044a\3\2\2\2\u044c\u044f\3\2\2\2\u044d\u044e\3\2"+ + "\2\2\u044d\u044b\3\2\2\2\u044e\u0450\3\2\2\2\u044f\u044d\3\2\2\2\u0450"+ + "\u0451\7,\2\2\u0451\u0452\7\61\2\2\u0452\u0453\3\2\2\2\u0453\u0454\b\u0087"+ + "\2\2\u0454\u010e\3\2\2\2\u0455\u0457\t\n\2\2\u0456\u0455\3\2\2\2\u0457"+ + "\u0458\3\2\2\2\u0458\u0456\3\2\2\2\u0458\u0459\3\2\2\2\u0459\u045a\3\2"+ + "\2\2\u045a\u045b\b\u0088\2\2\u045b\u0110\3\2\2\2\u045c\u045d\13\2\2\2"+ + "\u045d\u0112\3\2\2\2\"\2\u039b\u03bf\u03c1\u03c9\u03ce\u03d4\u03db\u03e0"+ + "\u03e6\u03e9\u03f1\u03f5\u03f9\u03fe\u0400\u0407\u0409\u040e\u0410\u0416"+ + "\u0418\u0421\u0423\u042a\u042f\u043b\u043f\u0442\u044b\u044d\u0458\3\2"+ + "\3\2"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseListener.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseListener.java index 9d9bc1a23c080..c0845b7adb564 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseListener.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseListener.java @@ -693,6 +693,18 @@ interface SqlBaseListener extends ParseTreeListener { * @param ctx the parse tree */ void exitSubqueryExpression(SqlBaseParser.SubqueryExpressionContext ctx); + /** + * Enter a parse tree produced by the {@code case} + * labeled alternative in {@link SqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void enterCase(SqlBaseParser.CaseContext ctx); + /** + * Exit a parse tree produced by the {@code case} + * labeled alternative in {@link SqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void exitCase(SqlBaseParser.CaseContext ctx); /** * Enter a parse tree produced by {@link SqlBaseParser#builtinDateTimeFunction}. * @param ctx the parse tree @@ -1067,6 +1079,16 @@ interface SqlBaseListener extends ParseTreeListener { * @param ctx the parse tree */ void exitString(SqlBaseParser.StringContext ctx); + /** + * Enter a parse tree produced by {@link SqlBaseParser#whenClause}. + * @param ctx the parse tree + */ + void enterWhenClause(SqlBaseParser.WhenClauseContext ctx); + /** + * Exit a parse tree produced by {@link SqlBaseParser#whenClause}. + * @param ctx the parse tree + */ + void exitWhenClause(SqlBaseParser.WhenClauseContext ctx); /** * Enter a parse tree produced by {@link SqlBaseParser#nonReserved}. * @param ctx the parse tree diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java index 6769cc88695d6..1ae317552268e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java @@ -32,25 +32,25 @@ class SqlBaseParser extends Parser { new PredictionContextCache(); public static final int T__0=1, T__1=2, T__2=3, T__3=4, ALL=5, ANALYZE=6, ANALYZED=7, AND=8, ANY=9, - AS=10, ASC=11, BETWEEN=12, BY=13, CAST=14, CATALOG=15, CATALOGS=16, COLUMNS=17, - CONVERT=18, CURRENT_DATE=19, CURRENT_TIME=20, CURRENT_TIMESTAMP=21, DAY=22, - DAYS=23, DEBUG=24, DESC=25, DESCRIBE=26, DISTINCT=27, ESCAPE=28, EXECUTABLE=29, - EXISTS=30, EXPLAIN=31, EXTRACT=32, FALSE=33, FIRST=34, FORMAT=35, FROM=36, - FULL=37, FUNCTIONS=38, GRAPHVIZ=39, GROUP=40, HAVING=41, HOUR=42, HOURS=43, - IN=44, INNER=45, INTERVAL=46, IS=47, JOIN=48, LAST=49, LEFT=50, LIKE=51, - LIMIT=52, MAPPED=53, MATCH=54, MINUTE=55, MINUTES=56, MONTH=57, MONTHS=58, - NATURAL=59, NOT=60, NULL=61, NULLS=62, ON=63, OPTIMIZED=64, OR=65, ORDER=66, - OUTER=67, PARSED=68, PHYSICAL=69, PLAN=70, RIGHT=71, RLIKE=72, QUERY=73, - SCHEMAS=74, SECOND=75, SECONDS=76, SELECT=77, SHOW=78, SYS=79, TABLE=80, - TABLES=81, TEXT=82, TRUE=83, TO=84, TYPE=85, TYPES=86, USING=87, VERIFY=88, - WHERE=89, WITH=90, YEAR=91, YEARS=92, ESCAPE_ESC=93, FUNCTION_ESC=94, - LIMIT_ESC=95, DATE_ESC=96, TIME_ESC=97, TIMESTAMP_ESC=98, GUID_ESC=99, - ESC_END=100, EQ=101, NULLEQ=102, NEQ=103, LT=104, LTE=105, GT=106, GTE=107, - PLUS=108, MINUS=109, ASTERISK=110, SLASH=111, PERCENT=112, CAST_OP=113, - CONCAT=114, DOT=115, PARAM=116, STRING=117, INTEGER_VALUE=118, DECIMAL_VALUE=119, - IDENTIFIER=120, DIGIT_IDENTIFIER=121, TABLE_IDENTIFIER=122, QUOTED_IDENTIFIER=123, - BACKQUOTED_IDENTIFIER=124, SIMPLE_COMMENT=125, BRACKETED_COMMENT=126, - WS=127, UNRECOGNIZED=128, DELIMITER=129; + AS=10, ASC=11, BETWEEN=12, BY=13, CASE=14, CAST=15, CATALOG=16, CATALOGS=17, + COLUMNS=18, CONVERT=19, CURRENT_DATE=20, CURRENT_TIME=21, CURRENT_TIMESTAMP=22, + DAY=23, DAYS=24, DEBUG=25, DESC=26, DESCRIBE=27, DISTINCT=28, ELSE=29, + END=30, ESCAPE=31, EXECUTABLE=32, EXISTS=33, EXPLAIN=34, EXTRACT=35, FALSE=36, + FIRST=37, FORMAT=38, FROM=39, FULL=40, FUNCTIONS=41, GRAPHVIZ=42, GROUP=43, + HAVING=44, HOUR=45, HOURS=46, IN=47, INNER=48, INTERVAL=49, IS=50, JOIN=51, + LAST=52, LEFT=53, LIKE=54, LIMIT=55, MAPPED=56, MATCH=57, MINUTE=58, MINUTES=59, + MONTH=60, MONTHS=61, NATURAL=62, NOT=63, NULL=64, NULLS=65, ON=66, OPTIMIZED=67, + OR=68, ORDER=69, OUTER=70, PARSED=71, PHYSICAL=72, PLAN=73, RIGHT=74, + RLIKE=75, QUERY=76, SCHEMAS=77, SECOND=78, SECONDS=79, SELECT=80, SHOW=81, + SYS=82, TABLE=83, TABLES=84, TEXT=85, THEN=86, TRUE=87, TO=88, TYPE=89, + TYPES=90, USING=91, VERIFY=92, WHEN=93, WHERE=94, WITH=95, YEAR=96, YEARS=97, + ESCAPE_ESC=98, FUNCTION_ESC=99, LIMIT_ESC=100, DATE_ESC=101, TIME_ESC=102, + TIMESTAMP_ESC=103, GUID_ESC=104, ESC_END=105, EQ=106, NULLEQ=107, NEQ=108, + LT=109, LTE=110, GT=111, GTE=112, PLUS=113, MINUS=114, ASTERISK=115, SLASH=116, + PERCENT=117, CAST_OP=118, CONCAT=119, DOT=120, PARAM=121, STRING=122, + INTEGER_VALUE=123, DECIMAL_VALUE=124, IDENTIFIER=125, DIGIT_IDENTIFIER=126, + TABLE_IDENTIFIER=127, QUOTED_IDENTIFIER=128, BACKQUOTED_IDENTIFIER=129, + SIMPLE_COMMENT=130, BRACKETED_COMMENT=131, WS=132, UNRECOGNIZED=133, DELIMITER=134; public static final int RULE_singleStatement = 0, RULE_singleExpression = 1, RULE_statement = 2, RULE_query = 3, RULE_queryNoWith = 4, RULE_limitClause = 5, RULE_queryTerm = 6, @@ -67,7 +67,8 @@ class SqlBaseParser extends Parser { RULE_constant = 40, RULE_comparisonOperator = 41, RULE_booleanValue = 42, RULE_interval = 43, RULE_intervalField = 44, RULE_dataType = 45, RULE_qualifiedName = 46, RULE_identifier = 47, RULE_tableIdentifier = 48, RULE_quoteIdentifier = 49, - RULE_unquoteIdentifier = 50, RULE_number = 51, RULE_string = 52, RULE_nonReserved = 53; + RULE_unquoteIdentifier = 50, RULE_number = 51, RULE_string = 52, RULE_whenClause = 53, + RULE_nonReserved = 54; public static final String[] ruleNames = { "singleStatement", "singleExpression", "statement", "query", "queryNoWith", "limitClause", "queryTerm", "orderBy", "querySpecification", "fromClause", @@ -80,47 +81,48 @@ class SqlBaseParser extends Parser { "functionExpression", "functionTemplate", "functionName", "constant", "comparisonOperator", "booleanValue", "interval", "intervalField", "dataType", "qualifiedName", "identifier", "tableIdentifier", "quoteIdentifier", "unquoteIdentifier", - "number", "string", "nonReserved" + "number", "string", "whenClause", "nonReserved" }; private static final String[] _LITERAL_NAMES = { null, "'('", "')'", "','", "':'", "'ALL'", "'ANALYZE'", "'ANALYZED'", - "'AND'", "'ANY'", "'AS'", "'ASC'", "'BETWEEN'", "'BY'", "'CAST'", "'CATALOG'", - "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT_DATE'", "'CURRENT_TIME'", - "'CURRENT_TIMESTAMP'", "'DAY'", "'DAYS'", "'DEBUG'", "'DESC'", "'DESCRIBE'", - "'DISTINCT'", "'ESCAPE'", "'EXECUTABLE'", "'EXISTS'", "'EXPLAIN'", "'EXTRACT'", - "'FALSE'", "'FIRST'", "'FORMAT'", "'FROM'", "'FULL'", "'FUNCTIONS'", "'GRAPHVIZ'", - "'GROUP'", "'HAVING'", "'HOUR'", "'HOURS'", "'IN'", "'INNER'", "'INTERVAL'", - "'IS'", "'JOIN'", "'LAST'", "'LEFT'", "'LIKE'", "'LIMIT'", "'MAPPED'", - "'MATCH'", "'MINUTE'", "'MINUTES'", "'MONTH'", "'MONTHS'", "'NATURAL'", - "'NOT'", "'NULL'", "'NULLS'", "'ON'", "'OPTIMIZED'", "'OR'", "'ORDER'", - "'OUTER'", "'PARSED'", "'PHYSICAL'", "'PLAN'", "'RIGHT'", "'RLIKE'", "'QUERY'", - "'SCHEMAS'", "'SECOND'", "'SECONDS'", "'SELECT'", "'SHOW'", "'SYS'", "'TABLE'", - "'TABLES'", "'TEXT'", "'TRUE'", "'TO'", "'TYPE'", "'TYPES'", "'USING'", - "'VERIFY'", "'WHERE'", "'WITH'", "'YEAR'", "'YEARS'", "'{ESCAPE'", "'{FN'", - "'{LIMIT'", "'{D'", "'{T'", "'{TS'", "'{GUID'", "'}'", "'='", "'<=>'", - null, "'<'", "'<='", "'>'", "'>='", "'+'", "'-'", "'*'", "'/'", "'%'", - "'::'", "'||'", "'.'", "'?'" + "'AND'", "'ANY'", "'AS'", "'ASC'", "'BETWEEN'", "'BY'", "'CASE'", "'CAST'", + "'CATALOG'", "'CATALOGS'", "'COLUMNS'", "'CONVERT'", "'CURRENT_DATE'", + "'CURRENT_TIME'", "'CURRENT_TIMESTAMP'", "'DAY'", "'DAYS'", "'DEBUG'", + "'DESC'", "'DESCRIBE'", "'DISTINCT'", "'ELSE'", "'END'", "'ESCAPE'", "'EXECUTABLE'", + "'EXISTS'", "'EXPLAIN'", "'EXTRACT'", "'FALSE'", "'FIRST'", "'FORMAT'", + "'FROM'", "'FULL'", "'FUNCTIONS'", "'GRAPHVIZ'", "'GROUP'", "'HAVING'", + "'HOUR'", "'HOURS'", "'IN'", "'INNER'", "'INTERVAL'", "'IS'", "'JOIN'", + "'LAST'", "'LEFT'", "'LIKE'", "'LIMIT'", "'MAPPED'", "'MATCH'", "'MINUTE'", + "'MINUTES'", "'MONTH'", "'MONTHS'", "'NATURAL'", "'NOT'", "'NULL'", "'NULLS'", + "'ON'", "'OPTIMIZED'", "'OR'", "'ORDER'", "'OUTER'", "'PARSED'", "'PHYSICAL'", + "'PLAN'", "'RIGHT'", "'RLIKE'", "'QUERY'", "'SCHEMAS'", "'SECOND'", "'SECONDS'", + "'SELECT'", "'SHOW'", "'SYS'", "'TABLE'", "'TABLES'", "'TEXT'", "'THEN'", + "'TRUE'", "'TO'", "'TYPE'", "'TYPES'", "'USING'", "'VERIFY'", "'WHEN'", + "'WHERE'", "'WITH'", "'YEAR'", "'YEARS'", "'{ESCAPE'", "'{FN'", "'{LIMIT'", + "'{D'", "'{T'", "'{TS'", "'{GUID'", "'}'", "'='", "'<=>'", null, "'<'", + "'<='", "'>'", "'>='", "'+'", "'-'", "'*'", "'/'", "'%'", "'::'", "'||'", + "'.'", "'?'" }; private static final String[] _SYMBOLIC_NAMES = { null, null, null, null, null, "ALL", "ANALYZE", "ANALYZED", "AND", "ANY", - "AS", "ASC", "BETWEEN", "BY", "CAST", "CATALOG", "CATALOGS", "COLUMNS", + "AS", "ASC", "BETWEEN", "BY", "CASE", "CAST", "CATALOG", "CATALOGS", "COLUMNS", "CONVERT", "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "DAY", - "DAYS", "DEBUG", "DESC", "DESCRIBE", "DISTINCT", "ESCAPE", "EXECUTABLE", - "EXISTS", "EXPLAIN", "EXTRACT", "FALSE", "FIRST", "FORMAT", "FROM", "FULL", - "FUNCTIONS", "GRAPHVIZ", "GROUP", "HAVING", "HOUR", "HOURS", "IN", "INNER", - "INTERVAL", "IS", "JOIN", "LAST", "LEFT", "LIKE", "LIMIT", "MAPPED", "MATCH", - "MINUTE", "MINUTES", "MONTH", "MONTHS", "NATURAL", "NOT", "NULL", "NULLS", - "ON", "OPTIMIZED", "OR", "ORDER", "OUTER", "PARSED", "PHYSICAL", "PLAN", - "RIGHT", "RLIKE", "QUERY", "SCHEMAS", "SECOND", "SECONDS", "SELECT", "SHOW", - "SYS", "TABLE", "TABLES", "TEXT", "TRUE", "TO", "TYPE", "TYPES", "USING", - "VERIFY", "WHERE", "WITH", "YEAR", "YEARS", "ESCAPE_ESC", "FUNCTION_ESC", - "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", "GUID_ESC", "ESC_END", - "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", - "SLASH", "PERCENT", "CAST_OP", "CONCAT", "DOT", "PARAM", "STRING", "INTEGER_VALUE", - "DECIMAL_VALUE", "IDENTIFIER", "DIGIT_IDENTIFIER", "TABLE_IDENTIFIER", - "QUOTED_IDENTIFIER", "BACKQUOTED_IDENTIFIER", "SIMPLE_COMMENT", "BRACKETED_COMMENT", - "WS", "UNRECOGNIZED", "DELIMITER" + "DAYS", "DEBUG", "DESC", "DESCRIBE", "DISTINCT", "ELSE", "END", "ESCAPE", + "EXECUTABLE", "EXISTS", "EXPLAIN", "EXTRACT", "FALSE", "FIRST", "FORMAT", + "FROM", "FULL", "FUNCTIONS", "GRAPHVIZ", "GROUP", "HAVING", "HOUR", "HOURS", + "IN", "INNER", "INTERVAL", "IS", "JOIN", "LAST", "LEFT", "LIKE", "LIMIT", + "MAPPED", "MATCH", "MINUTE", "MINUTES", "MONTH", "MONTHS", "NATURAL", + "NOT", "NULL", "NULLS", "ON", "OPTIMIZED", "OR", "ORDER", "OUTER", "PARSED", + "PHYSICAL", "PLAN", "RIGHT", "RLIKE", "QUERY", "SCHEMAS", "SECOND", "SECONDS", + "SELECT", "SHOW", "SYS", "TABLE", "TABLES", "TEXT", "THEN", "TRUE", "TO", + "TYPE", "TYPES", "USING", "VERIFY", "WHEN", "WHERE", "WITH", "YEAR", "YEARS", + "ESCAPE_ESC", "FUNCTION_ESC", "LIMIT_ESC", "DATE_ESC", "TIME_ESC", "TIMESTAMP_ESC", + "GUID_ESC", "ESC_END", "EQ", "NULLEQ", "NEQ", "LT", "LTE", "GT", "GTE", + "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CAST_OP", "CONCAT", + "DOT", "PARAM", "STRING", "INTEGER_VALUE", "DECIMAL_VALUE", "IDENTIFIER", + "DIGIT_IDENTIFIER", "TABLE_IDENTIFIER", "QUOTED_IDENTIFIER", "BACKQUOTED_IDENTIFIER", + "SIMPLE_COMMENT", "BRACKETED_COMMENT", "WS", "UNRECOGNIZED", "DELIMITER" }; public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES); @@ -201,9 +203,9 @@ public final SingleStatementContext singleStatement() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(108); + setState(110); statement(); - setState(109); + setState(111); match(EOF); } } @@ -248,9 +250,9 @@ public final SingleExpressionContext singleExpression() throws RecognitionExcept try { enterOuterAlt(_localctx, 1); { - setState(111); + setState(113); expression(); - setState(112); + setState(114); match(EOF); } } @@ -613,14 +615,14 @@ public final StatementContext statement() throws RecognitionException { enterRule(_localctx, 4, RULE_statement); int _la; try { - setState(215); + setState(217); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,19,_ctx) ) { case 1: _localctx = new StatementDefaultContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(114); + setState(116); query(); } break; @@ -628,30 +630,30 @@ public final StatementContext statement() throws RecognitionException { _localctx = new ExplainContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(115); + setState(117); match(EXPLAIN); - setState(129); + setState(131); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,2,_ctx) ) { case 1: { - setState(116); + setState(118); match(T__0); - setState(125); + setState(127); _errHandler.sync(this); _la = _input.LA(1); - while (((((_la - 35)) & ~0x3f) == 0 && ((1L << (_la - 35)) & ((1L << (FORMAT - 35)) | (1L << (PLAN - 35)) | (1L << (VERIFY - 35)))) != 0)) { + while (((((_la - 38)) & ~0x3f) == 0 && ((1L << (_la - 38)) & ((1L << (FORMAT - 38)) | (1L << (PLAN - 38)) | (1L << (VERIFY - 38)))) != 0)) { { - setState(123); + setState(125); switch (_input.LA(1)) { case PLAN: { - setState(117); + setState(119); match(PLAN); - setState(118); + setState(120); ((ExplainContext)_localctx).type = _input.LT(1); _la = _input.LA(1); - if ( !(((((_la - 5)) & ~0x3f) == 0 && ((1L << (_la - 5)) & ((1L << (ALL - 5)) | (1L << (ANALYZED - 5)) | (1L << (EXECUTABLE - 5)) | (1L << (MAPPED - 5)) | (1L << (OPTIMIZED - 5)) | (1L << (PARSED - 5)))) != 0)) ) { + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ALL) | (1L << ANALYZED) | (1L << EXECUTABLE) | (1L << MAPPED))) != 0) || _la==OPTIMIZED || _la==PARSED) ) { ((ExplainContext)_localctx).type = (Token)_errHandler.recoverInline(this); } else { consume(); @@ -660,9 +662,9 @@ public final StatementContext statement() throws RecognitionException { break; case FORMAT: { - setState(119); + setState(121); match(FORMAT); - setState(120); + setState(122); ((ExplainContext)_localctx).format = _input.LT(1); _la = _input.LA(1); if ( !(_la==GRAPHVIZ || _la==TEXT) ) { @@ -674,9 +676,9 @@ public final StatementContext statement() throws RecognitionException { break; case VERIFY: { - setState(121); + setState(123); match(VERIFY); - setState(122); + setState(124); ((ExplainContext)_localctx).verify = booleanValue(); } break; @@ -684,16 +686,16 @@ public final StatementContext statement() throws RecognitionException { throw new NoViableAltException(this); } } - setState(127); + setState(129); _errHandler.sync(this); _la = _input.LA(1); } - setState(128); + setState(130); match(T__1); } break; } - setState(131); + setState(133); statement(); } break; @@ -701,27 +703,27 @@ public final StatementContext statement() throws RecognitionException { _localctx = new DebugContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(132); + setState(134); match(DEBUG); - setState(144); + setState(146); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,5,_ctx) ) { case 1: { - setState(133); + setState(135); match(T__0); - setState(140); + setState(142); _errHandler.sync(this); _la = _input.LA(1); while (_la==FORMAT || _la==PLAN) { { - setState(138); + setState(140); switch (_input.LA(1)) { case PLAN: { - setState(134); + setState(136); match(PLAN); - setState(135); + setState(137); ((DebugContext)_localctx).type = _input.LT(1); _la = _input.LA(1); if ( !(_la==ANALYZED || _la==OPTIMIZED) ) { @@ -733,9 +735,9 @@ public final StatementContext statement() throws RecognitionException { break; case FORMAT: { - setState(136); + setState(138); match(FORMAT); - setState(137); + setState(139); ((DebugContext)_localctx).format = _input.LT(1); _la = _input.LA(1); if ( !(_la==GRAPHVIZ || _la==TEXT) ) { @@ -749,16 +751,16 @@ public final StatementContext statement() throws RecognitionException { throw new NoViableAltException(this); } } - setState(142); + setState(144); _errHandler.sync(this); _la = _input.LA(1); } - setState(143); + setState(145); match(T__1); } break; } - setState(146); + setState(148); statement(); } break; @@ -766,15 +768,15 @@ public final StatementContext statement() throws RecognitionException { _localctx = new ShowTablesContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(147); + setState(149); match(SHOW); - setState(148); + setState(150); match(TABLES); - setState(151); + setState(153); switch (_input.LA(1)) { case LIKE: { - setState(149); + setState(151); ((ShowTablesContext)_localctx).tableLike = likePattern(); } break; @@ -823,7 +825,7 @@ public final StatementContext statement() throws RecognitionException { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: { - setState(150); + setState(152); ((ShowTablesContext)_localctx).tableIdent = tableIdentifier(); } break; @@ -838,22 +840,22 @@ public final StatementContext statement() throws RecognitionException { _localctx = new ShowColumnsContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(153); + setState(155); match(SHOW); - setState(154); + setState(156); match(COLUMNS); - setState(155); + setState(157); _la = _input.LA(1); if ( !(_la==FROM || _la==IN) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(158); + setState(160); switch (_input.LA(1)) { case LIKE: { - setState(156); + setState(158); ((ShowColumnsContext)_localctx).tableLike = likePattern(); } break; @@ -902,7 +904,7 @@ public final StatementContext statement() throws RecognitionException { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: { - setState(157); + setState(159); ((ShowColumnsContext)_localctx).tableIdent = tableIdentifier(); } break; @@ -915,18 +917,18 @@ public final StatementContext statement() throws RecognitionException { _localctx = new ShowColumnsContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(160); + setState(162); _la = _input.LA(1); if ( !(_la==DESC || _la==DESCRIBE) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(163); + setState(165); switch (_input.LA(1)) { case LIKE: { - setState(161); + setState(163); ((ShowColumnsContext)_localctx).tableLike = likePattern(); } break; @@ -975,7 +977,7 @@ public final StatementContext statement() throws RecognitionException { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: { - setState(162); + setState(164); ((ShowColumnsContext)_localctx).tableIdent = tableIdentifier(); } break; @@ -988,15 +990,15 @@ public final StatementContext statement() throws RecognitionException { _localctx = new ShowFunctionsContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(165); + setState(167); match(SHOW); - setState(166); - match(FUNCTIONS); setState(168); + match(FUNCTIONS); + setState(170); _la = _input.LA(1); if (_la==LIKE) { { - setState(167); + setState(169); likePattern(); } } @@ -1007,9 +1009,9 @@ public final StatementContext statement() throws RecognitionException { _localctx = new ShowSchemasContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(170); + setState(172); match(SHOW); - setState(171); + setState(173); match(SCHEMAS); } break; @@ -1017,58 +1019,58 @@ public final StatementContext statement() throws RecognitionException { _localctx = new SysTablesContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(172); + setState(174); match(SYS); - setState(173); + setState(175); match(TABLES); - setState(176); + setState(178); _la = _input.LA(1); if (_la==CATALOG) { { - setState(174); + setState(176); match(CATALOG); - setState(175); + setState(177); ((SysTablesContext)_localctx).clusterLike = likePattern(); } } - setState(180); + setState(182); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,11,_ctx) ) { case 1: { - setState(178); + setState(180); ((SysTablesContext)_localctx).tableLike = likePattern(); } break; case 2: { - setState(179); + setState(181); ((SysTablesContext)_localctx).tableIdent = tableIdentifier(); } break; } - setState(191); + setState(193); _la = _input.LA(1); if (_la==TYPE) { { - setState(182); + setState(184); match(TYPE); - setState(183); + setState(185); string(); - setState(188); + setState(190); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(184); + setState(186); match(T__2); - setState(185); + setState(187); string(); } } - setState(190); + setState(192); _errHandler.sync(this); _la = _input.LA(1); } @@ -1081,28 +1083,28 @@ public final StatementContext statement() throws RecognitionException { _localctx = new SysColumnsContext(_localctx); enterOuterAlt(_localctx, 10); { - setState(193); + setState(195); match(SYS); - setState(194); + setState(196); match(COLUMNS); - setState(197); + setState(199); _la = _input.LA(1); if (_la==CATALOG) { { - setState(195); + setState(197); match(CATALOG); - setState(196); + setState(198); ((SysColumnsContext)_localctx).cluster = string(); } } - setState(202); + setState(204); switch (_input.LA(1)) { case TABLE: { - setState(199); + setState(201); match(TABLE); - setState(200); + setState(202); ((SysColumnsContext)_localctx).tableLike = likePattern(); } break; @@ -1151,7 +1153,7 @@ public final StatementContext statement() throws RecognitionException { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: { - setState(201); + setState(203); ((SysColumnsContext)_localctx).tableIdent = tableIdentifier(); } break; @@ -1161,11 +1163,11 @@ public final StatementContext statement() throws RecognitionException { default: throw new NoViableAltException(this); } - setState(205); + setState(207); _la = _input.LA(1); if (_la==LIKE) { { - setState(204); + setState(206); ((SysColumnsContext)_localctx).columnPattern = likePattern(); } } @@ -1176,19 +1178,19 @@ public final StatementContext statement() throws RecognitionException { _localctx = new SysTypesContext(_localctx); enterOuterAlt(_localctx, 11); { - setState(207); + setState(209); match(SYS); - setState(208); + setState(210); match(TYPES); - setState(213); + setState(215); _la = _input.LA(1); - if (((((_la - 108)) & ~0x3f) == 0 && ((1L << (_la - 108)) & ((1L << (PLUS - 108)) | (1L << (MINUS - 108)) | (1L << (INTEGER_VALUE - 108)) | (1L << (DECIMAL_VALUE - 108)))) != 0)) { + if (((((_la - 113)) & ~0x3f) == 0 && ((1L << (_la - 113)) & ((1L << (PLUS - 113)) | (1L << (MINUS - 113)) | (1L << (INTEGER_VALUE - 113)) | (1L << (DECIMAL_VALUE - 113)))) != 0)) { { - setState(210); + setState(212); _la = _input.LA(1); if (_la==PLUS || _la==MINUS) { { - setState(209); + setState(211); _la = _input.LA(1); if ( !(_la==PLUS || _la==MINUS) ) { _errHandler.recoverInline(this); @@ -1198,7 +1200,7 @@ public final StatementContext statement() throws RecognitionException { } } - setState(212); + setState(214); ((SysTypesContext)_localctx).type = number(); } } @@ -1255,34 +1257,34 @@ public final QueryContext query() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(226); + setState(228); _la = _input.LA(1); if (_la==WITH) { { - setState(217); + setState(219); match(WITH); - setState(218); + setState(220); namedQuery(); - setState(223); + setState(225); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(219); + setState(221); match(T__2); - setState(220); + setState(222); namedQuery(); } } - setState(225); + setState(227); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(228); + setState(230); queryNoWith(); } } @@ -1338,42 +1340,42 @@ public final QueryNoWithContext queryNoWith() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(230); + setState(232); queryTerm(); - setState(241); + setState(243); _la = _input.LA(1); if (_la==ORDER) { { - setState(231); + setState(233); match(ORDER); - setState(232); + setState(234); match(BY); - setState(233); + setState(235); orderBy(); - setState(238); + setState(240); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(234); + setState(236); match(T__2); - setState(235); + setState(237); orderBy(); } } - setState(240); + setState(242); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(244); + setState(246); _la = _input.LA(1); if (_la==LIMIT || _la==LIMIT_ESC) { { - setState(243); + setState(245); limitClause(); } } @@ -1422,14 +1424,14 @@ public final LimitClauseContext limitClause() throws RecognitionException { enterRule(_localctx, 10, RULE_limitClause); int _la; try { - setState(251); + setState(253); switch (_input.LA(1)) { case LIMIT: enterOuterAlt(_localctx, 1); { - setState(246); + setState(248); match(LIMIT); - setState(247); + setState(249); ((LimitClauseContext)_localctx).limit = _input.LT(1); _la = _input.LA(1); if ( !(_la==ALL || _la==INTEGER_VALUE) ) { @@ -1442,9 +1444,9 @@ public final LimitClauseContext limitClause() throws RecognitionException { case LIMIT_ESC: enterOuterAlt(_localctx, 2); { - setState(248); + setState(250); match(LIMIT_ESC); - setState(249); + setState(251); ((LimitClauseContext)_localctx).limit = _input.LT(1); _la = _input.LA(1); if ( !(_la==ALL || _la==INTEGER_VALUE) ) { @@ -1452,7 +1454,7 @@ public final LimitClauseContext limitClause() throws RecognitionException { } else { consume(); } - setState(250); + setState(252); match(ESC_END); } break; @@ -1525,13 +1527,13 @@ public final QueryTermContext queryTerm() throws RecognitionException { QueryTermContext _localctx = new QueryTermContext(_ctx, getState()); enterRule(_localctx, 12, RULE_queryTerm); try { - setState(258); + setState(260); switch (_input.LA(1)) { case SELECT: _localctx = new QueryPrimaryDefaultContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(253); + setState(255); querySpecification(); } break; @@ -1539,11 +1541,11 @@ public final QueryTermContext queryTerm() throws RecognitionException { _localctx = new SubqueryContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(254); + setState(256); match(T__0); - setState(255); + setState(257); queryNoWith(); - setState(256); + setState(258); match(T__1); } break; @@ -1599,13 +1601,13 @@ public final OrderByContext orderBy() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(260); - expression(); setState(262); + expression(); + setState(264); _la = _input.LA(1); if (_la==ASC || _la==DESC) { { - setState(261); + setState(263); ((OrderByContext)_localctx).ordering = _input.LT(1); _la = _input.LA(1); if ( !(_la==ASC || _la==DESC) ) { @@ -1616,13 +1618,13 @@ public final OrderByContext orderBy() throws RecognitionException { } } - setState(266); + setState(268); _la = _input.LA(1); if (_la==NULLS) { { - setState(264); + setState(266); match(NULLS); - setState(265); + setState(267); ((OrderByContext)_localctx).nullOrdering = _input.LT(1); _la = _input.LA(1); if ( !(_la==FIRST || _la==LAST) ) { @@ -1701,75 +1703,75 @@ public final QuerySpecificationContext querySpecification() throws RecognitionEx try { enterOuterAlt(_localctx, 1); { - setState(268); - match(SELECT); setState(270); + match(SELECT); + setState(272); _la = _input.LA(1); if (_la==ALL || _la==DISTINCT) { { - setState(269); + setState(271); setQuantifier(); } } - setState(272); + setState(274); selectItem(); - setState(277); + setState(279); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(273); + setState(275); match(T__2); - setState(274); + setState(276); selectItem(); } } - setState(279); + setState(281); _errHandler.sync(this); _la = _input.LA(1); } - setState(281); + setState(283); _la = _input.LA(1); if (_la==FROM) { { - setState(280); + setState(282); fromClause(); } } - setState(285); + setState(287); _la = _input.LA(1); if (_la==WHERE) { { - setState(283); + setState(285); match(WHERE); - setState(284); + setState(286); ((QuerySpecificationContext)_localctx).where = booleanExpression(0); } } - setState(290); + setState(292); _la = _input.LA(1); if (_la==GROUP) { { - setState(287); + setState(289); match(GROUP); - setState(288); + setState(290); match(BY); - setState(289); + setState(291); groupBy(); } } - setState(294); + setState(296); _la = _input.LA(1); if (_la==HAVING) { { - setState(292); + setState(294); match(HAVING); - setState(293); + setState(295); ((QuerySpecificationContext)_localctx).having = booleanExpression(0); } } @@ -1821,23 +1823,23 @@ public final FromClauseContext fromClause() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(296); + setState(298); match(FROM); - setState(297); + setState(299); relation(); - setState(302); + setState(304); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(298); + setState(300); match(T__2); - setState(299); + setState(301); relation(); } } - setState(304); + setState(306); _errHandler.sync(this); _la = _input.LA(1); } @@ -1890,30 +1892,30 @@ public final GroupByContext groupBy() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(306); + setState(308); _la = _input.LA(1); if (_la==ALL || _la==DISTINCT) { { - setState(305); + setState(307); setQuantifier(); } } - setState(308); + setState(310); groupingElement(); - setState(313); + setState(315); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(309); + setState(311); match(T__2); - setState(310); + setState(312); groupingElement(); } } - setState(315); + setState(317); _errHandler.sync(this); _la = _input.LA(1); } @@ -1968,7 +1970,7 @@ public final GroupingElementContext groupingElement() throws RecognitionExceptio _localctx = new SingleGroupingSetContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(316); + setState(318); groupingExpressions(); } } @@ -2014,47 +2016,47 @@ public final GroupingExpressionsContext groupingExpressions() throws Recognition enterRule(_localctx, 24, RULE_groupingExpressions); int _la; try { - setState(331); + setState(333); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,40,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(318); + setState(320); match(T__0); - setState(327); + setState(329); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RIGHT - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TRUE - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (FUNCTION_ESC - 64)) | (1L << (DATE_ESC - 64)) | (1L << (TIME_ESC - 64)) | (1L << (TIMESTAMP_ESC - 64)) | (1L << (GUID_ESC - 64)) | (1L << (PLUS - 64)) | (1L << (MINUS - 64)) | (1L << (ASTERISK - 64)) | (1L << (PARAM - 64)) | (1L << (STRING - 64)) | (1L << (INTEGER_VALUE - 64)) | (1L << (DECIMAL_VALUE - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { + if (((((_la - 1)) & ~0x3f) == 0 && ((1L << (_la - 1)) & ((1L << (T__0 - 1)) | (1L << (ANALYZE - 1)) | (1L << (ANALYZED - 1)) | (1L << (CASE - 1)) | (1L << (CAST - 1)) | (1L << (CATALOGS - 1)) | (1L << (COLUMNS - 1)) | (1L << (CONVERT - 1)) | (1L << (CURRENT_DATE - 1)) | (1L << (CURRENT_TIME - 1)) | (1L << (CURRENT_TIMESTAMP - 1)) | (1L << (DAY - 1)) | (1L << (DEBUG - 1)) | (1L << (EXECUTABLE - 1)) | (1L << (EXISTS - 1)) | (1L << (EXPLAIN - 1)) | (1L << (EXTRACT - 1)) | (1L << (FALSE - 1)) | (1L << (FIRST - 1)) | (1L << (FORMAT - 1)) | (1L << (FULL - 1)) | (1L << (FUNCTIONS - 1)) | (1L << (GRAPHVIZ - 1)) | (1L << (HOUR - 1)) | (1L << (INTERVAL - 1)) | (1L << (LAST - 1)) | (1L << (LEFT - 1)) | (1L << (LIMIT - 1)) | (1L << (MAPPED - 1)) | (1L << (MATCH - 1)) | (1L << (MINUTE - 1)) | (1L << (MONTH - 1)) | (1L << (NOT - 1)) | (1L << (NULL - 1)))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (OPTIMIZED - 67)) | (1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { - setState(319); + setState(321); expression(); - setState(324); + setState(326); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(320); + setState(322); match(T__2); - setState(321); + setState(323); expression(); } } - setState(326); + setState(328); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(329); + setState(331); match(T__1); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(330); + setState(332); expression(); } break; @@ -2105,15 +2107,15 @@ public final NamedQueryContext namedQuery() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(333); + setState(335); ((NamedQueryContext)_localctx).name = identifier(); - setState(334); + setState(336); match(AS); - setState(335); + setState(337); match(T__0); - setState(336); + setState(338); queryNoWith(); - setState(337); + setState(339); match(T__1); } } @@ -2157,7 +2159,7 @@ public final SetQuantifierContext setQuantifier() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(339); + setState(341); _la = _input.LA(1); if ( !(_la==ALL || _la==DISTINCT) ) { _errHandler.recoverInline(this); @@ -2220,23 +2222,23 @@ public final SelectItemContext selectItem() throws RecognitionException { _localctx = new SelectExpressionContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(341); + setState(343); expression(); - setState(346); + setState(348); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,42,_ctx) ) { case 1: { - setState(343); + setState(345); _la = _input.LA(1); if (_la==AS) { { - setState(342); + setState(344); match(AS); } } - setState(345); + setState(347); identifier(); } break; @@ -2290,19 +2292,19 @@ public final RelationContext relation() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(348); + setState(350); relationPrimary(); - setState(352); + setState(354); _errHandler.sync(this); _la = _input.LA(1); - while (((((_la - 37)) & ~0x3f) == 0 && ((1L << (_la - 37)) & ((1L << (FULL - 37)) | (1L << (INNER - 37)) | (1L << (JOIN - 37)) | (1L << (LEFT - 37)) | (1L << (NATURAL - 37)) | (1L << (RIGHT - 37)))) != 0)) { + while (((((_la - 40)) & ~0x3f) == 0 && ((1L << (_la - 40)) & ((1L << (FULL - 40)) | (1L << (INNER - 40)) | (1L << (JOIN - 40)) | (1L << (LEFT - 40)) | (1L << (NATURAL - 40)) | (1L << (RIGHT - 40)))) != 0)) { { { - setState(349); + setState(351); joinRelation(); } } - setState(354); + setState(356); _errHandler.sync(this); _la = _input.LA(1); } @@ -2356,7 +2358,7 @@ public final JoinRelationContext joinRelation() throws RecognitionException { enterRule(_localctx, 34, RULE_joinRelation); int _la; try { - setState(366); + setState(368); switch (_input.LA(1)) { case FULL: case INNER: @@ -2366,18 +2368,18 @@ public final JoinRelationContext joinRelation() throws RecognitionException { enterOuterAlt(_localctx, 1); { { - setState(355); + setState(357); joinType(); } - setState(356); + setState(358); match(JOIN); - setState(357); - ((JoinRelationContext)_localctx).right = relationPrimary(); setState(359); + ((JoinRelationContext)_localctx).right = relationPrimary(); + setState(361); _la = _input.LA(1); if (_la==ON || _la==USING) { { - setState(358); + setState(360); joinCriteria(); } } @@ -2387,13 +2389,13 @@ public final JoinRelationContext joinRelation() throws RecognitionException { case NATURAL: enterOuterAlt(_localctx, 2); { - setState(361); + setState(363); match(NATURAL); - setState(362); + setState(364); joinType(); - setState(363); + setState(365); match(JOIN); - setState(364); + setState(366); ((JoinRelationContext)_localctx).right = relationPrimary(); } break; @@ -2442,17 +2444,17 @@ public final JoinTypeContext joinType() throws RecognitionException { enterRule(_localctx, 36, RULE_joinType); int _la; try { - setState(383); + setState(385); switch (_input.LA(1)) { case INNER: case JOIN: enterOuterAlt(_localctx, 1); { - setState(369); + setState(371); _la = _input.LA(1); if (_la==INNER) { { - setState(368); + setState(370); match(INNER); } } @@ -2462,13 +2464,13 @@ public final JoinTypeContext joinType() throws RecognitionException { case LEFT: enterOuterAlt(_localctx, 2); { - setState(371); - match(LEFT); setState(373); + match(LEFT); + setState(375); _la = _input.LA(1); if (_la==OUTER) { { - setState(372); + setState(374); match(OUTER); } } @@ -2478,13 +2480,13 @@ public final JoinTypeContext joinType() throws RecognitionException { case RIGHT: enterOuterAlt(_localctx, 3); { - setState(375); - match(RIGHT); setState(377); + match(RIGHT); + setState(379); _la = _input.LA(1); if (_la==OUTER) { { - setState(376); + setState(378); match(OUTER); } } @@ -2494,13 +2496,13 @@ public final JoinTypeContext joinType() throws RecognitionException { case FULL: enterOuterAlt(_localctx, 4); { - setState(379); - match(FULL); setState(381); + match(FULL); + setState(383); _la = _input.LA(1); if (_la==OUTER) { { - setState(380); + setState(382); match(OUTER); } } @@ -2558,43 +2560,43 @@ public final JoinCriteriaContext joinCriteria() throws RecognitionException { enterRule(_localctx, 38, RULE_joinCriteria); int _la; try { - setState(399); + setState(401); switch (_input.LA(1)) { case ON: enterOuterAlt(_localctx, 1); { - setState(385); + setState(387); match(ON); - setState(386); + setState(388); booleanExpression(0); } break; case USING: enterOuterAlt(_localctx, 2); { - setState(387); + setState(389); match(USING); - setState(388); + setState(390); match(T__0); - setState(389); + setState(391); identifier(); - setState(394); + setState(396); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(390); + setState(392); match(T__2); - setState(391); + setState(393); identifier(); } } - setState(396); + setState(398); _errHandler.sync(this); _la = _input.LA(1); } - setState(397); + setState(399); match(T__1); } break; @@ -2699,30 +2701,30 @@ public final RelationPrimaryContext relationPrimary() throws RecognitionExceptio enterRule(_localctx, 40, RULE_relationPrimary); int _la; try { - setState(426); + setState(428); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,59,_ctx) ) { case 1: _localctx = new TableNameContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(401); + setState(403); tableIdentifier(); - setState(406); + setState(408); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,54,_ctx) ) { case 1: { - setState(403); + setState(405); _la = _input.LA(1); if (_la==AS) { { - setState(402); + setState(404); match(AS); } } - setState(405); + setState(407); qualifiedName(); } break; @@ -2733,27 +2735,27 @@ public final RelationPrimaryContext relationPrimary() throws RecognitionExceptio _localctx = new AliasedQueryContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(408); + setState(410); match(T__0); - setState(409); + setState(411); queryNoWith(); - setState(410); + setState(412); match(T__1); - setState(415); + setState(417); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,56,_ctx) ) { case 1: { - setState(412); + setState(414); _la = _input.LA(1); if (_la==AS) { { - setState(411); + setState(413); match(AS); } } - setState(414); + setState(416); qualifiedName(); } break; @@ -2764,27 +2766,27 @@ public final RelationPrimaryContext relationPrimary() throws RecognitionExceptio _localctx = new AliasedRelationContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(417); + setState(419); match(T__0); - setState(418); + setState(420); relation(); - setState(419); + setState(421); match(T__1); - setState(424); + setState(426); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,58,_ctx) ) { case 1: { - setState(421); + setState(423); _la = _input.LA(1); if (_la==AS) { { - setState(420); + setState(422); match(AS); } } - setState(423); + setState(425); qualifiedName(); } break; @@ -2833,7 +2835,7 @@ public final ExpressionContext expression() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(428); + setState(430); booleanExpression(0); } } @@ -3041,7 +3043,7 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc int _alt; enterOuterAlt(_localctx, 1); { - setState(461); + setState(463); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,60,_ctx) ) { case 1: @@ -3050,9 +3052,9 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _ctx = _localctx; _prevctx = _localctx; - setState(431); + setState(433); match(NOT); - setState(432); + setState(434); booleanExpression(8); } break; @@ -3061,13 +3063,13 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new ExistsContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(433); + setState(435); match(EXISTS); - setState(434); + setState(436); match(T__0); - setState(435); + setState(437); query(); - setState(436); + setState(438); match(T__1); } break; @@ -3076,15 +3078,15 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new StringQueryContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(438); + setState(440); match(QUERY); - setState(439); + setState(441); match(T__0); - setState(440); + setState(442); ((StringQueryContext)_localctx).queryString = string(); - setState(441); + setState(443); matchQueryOptions(); - setState(442); + setState(444); match(T__1); } break; @@ -3093,19 +3095,19 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new MatchQueryContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(444); + setState(446); match(MATCH); - setState(445); + setState(447); match(T__0); - setState(446); + setState(448); ((MatchQueryContext)_localctx).singleField = qualifiedName(); - setState(447); + setState(449); match(T__2); - setState(448); + setState(450); ((MatchQueryContext)_localctx).queryString = string(); - setState(449); + setState(451); matchQueryOptions(); - setState(450); + setState(452); match(T__1); } break; @@ -3114,19 +3116,19 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new MultiMatchQueryContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(452); + setState(454); match(MATCH); - setState(453); + setState(455); match(T__0); - setState(454); + setState(456); ((MultiMatchQueryContext)_localctx).multiFields = string(); - setState(455); + setState(457); match(T__2); - setState(456); + setState(458); ((MultiMatchQueryContext)_localctx).queryString = string(); - setState(457); + setState(459); matchQueryOptions(); - setState(458); + setState(460); match(T__1); } break; @@ -3135,13 +3137,13 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new BooleanDefaultContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(460); + setState(462); predicated(); } break; } _ctx.stop = _input.LT(-1); - setState(471); + setState(473); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,62,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { @@ -3149,7 +3151,7 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc if ( _parseListeners!=null ) triggerExitRuleEvent(); _prevctx = _localctx; { - setState(469); + setState(471); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,61,_ctx) ) { case 1: @@ -3157,11 +3159,11 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState)); ((LogicalBinaryContext)_localctx).left = _prevctx; pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression); - setState(463); + setState(465); if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(464); + setState(466); ((LogicalBinaryContext)_localctx).operator = match(AND); - setState(465); + setState(467); ((LogicalBinaryContext)_localctx).right = booleanExpression(3); } break; @@ -3170,18 +3172,18 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc _localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState)); ((LogicalBinaryContext)_localctx).left = _prevctx; pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression); - setState(466); + setState(468); if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(467); + setState(469); ((LogicalBinaryContext)_localctx).operator = match(OR); - setState(468); + setState(470); ((LogicalBinaryContext)_localctx).right = booleanExpression(2); } break; } } } - setState(473); + setState(475); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,62,_ctx); } @@ -3231,19 +3233,19 @@ public final MatchQueryOptionsContext matchQueryOptions() throws RecognitionExce try { enterOuterAlt(_localctx, 1); { - setState(478); + setState(480); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(474); + setState(476); match(T__2); - setState(475); + setState(477); string(); } } - setState(480); + setState(482); _errHandler.sync(this); _la = _input.LA(1); } @@ -3292,14 +3294,14 @@ public final PredicatedContext predicated() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(481); - valueExpression(0); setState(483); + valueExpression(0); + setState(485); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,64,_ctx) ) { case 1: { - setState(482); + setState(484); predicate(); } break; @@ -3369,142 +3371,142 @@ public final PredicateContext predicate() throws RecognitionException { enterRule(_localctx, 50, RULE_predicate); int _la; try { - setState(531); + setState(533); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,72,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(486); + setState(488); _la = _input.LA(1); if (_la==NOT) { { - setState(485); + setState(487); match(NOT); } } - setState(488); + setState(490); ((PredicateContext)_localctx).kind = match(BETWEEN); - setState(489); + setState(491); ((PredicateContext)_localctx).lower = valueExpression(0); - setState(490); + setState(492); match(AND); - setState(491); + setState(493); ((PredicateContext)_localctx).upper = valueExpression(0); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(494); + setState(496); _la = _input.LA(1); if (_la==NOT) { { - setState(493); + setState(495); match(NOT); } } - setState(496); + setState(498); ((PredicateContext)_localctx).kind = match(IN); - setState(497); + setState(499); match(T__0); - setState(498); + setState(500); valueExpression(0); - setState(503); + setState(505); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(499); + setState(501); match(T__2); - setState(500); + setState(502); valueExpression(0); } } - setState(505); + setState(507); _errHandler.sync(this); _la = _input.LA(1); } - setState(506); + setState(508); match(T__1); } break; case 3: enterOuterAlt(_localctx, 3); { - setState(509); + setState(511); _la = _input.LA(1); if (_la==NOT) { { - setState(508); + setState(510); match(NOT); } } - setState(511); + setState(513); ((PredicateContext)_localctx).kind = match(IN); - setState(512); + setState(514); match(T__0); - setState(513); + setState(515); query(); - setState(514); + setState(516); match(T__1); } break; case 4: enterOuterAlt(_localctx, 4); { - setState(517); + setState(519); _la = _input.LA(1); if (_la==NOT) { { - setState(516); + setState(518); match(NOT); } } - setState(519); + setState(521); ((PredicateContext)_localctx).kind = match(LIKE); - setState(520); + setState(522); pattern(); } break; case 5: enterOuterAlt(_localctx, 5); { - setState(522); + setState(524); _la = _input.LA(1); if (_la==NOT) { { - setState(521); + setState(523); match(NOT); } } - setState(524); + setState(526); ((PredicateContext)_localctx).kind = match(RLIKE); - setState(525); + setState(527); ((PredicateContext)_localctx).regex = string(); } break; case 6: enterOuterAlt(_localctx, 6); { - setState(526); - match(IS); setState(528); + match(IS); + setState(530); _la = _input.LA(1); if (_la==NOT) { { - setState(527); + setState(529); match(NOT); } } - setState(530); + setState(532); ((PredicateContext)_localctx).kind = match(NULL); } break; @@ -3551,9 +3553,9 @@ public final LikePatternContext likePattern() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(533); + setState(535); match(LIKE); - setState(534); + setState(536); pattern(); } } @@ -3601,14 +3603,14 @@ public final PatternContext pattern() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(536); - ((PatternContext)_localctx).value = string(); setState(538); + ((PatternContext)_localctx).value = string(); + setState(540); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,73,_ctx) ) { case 1: { - setState(537); + setState(539); patternEscape(); } break; @@ -3656,25 +3658,25 @@ public final PatternEscapeContext patternEscape() throws RecognitionException { PatternEscapeContext _localctx = new PatternEscapeContext(_ctx, getState()); enterRule(_localctx, 56, RULE_patternEscape); try { - setState(546); + setState(548); switch (_input.LA(1)) { case ESCAPE: enterOuterAlt(_localctx, 1); { - setState(540); + setState(542); match(ESCAPE); - setState(541); + setState(543); ((PatternEscapeContext)_localctx).escape = string(); } break; case ESCAPE_ESC: enterOuterAlt(_localctx, 2); { - setState(542); + setState(544); match(ESCAPE_ESC); - setState(543); + setState(545); ((PatternEscapeContext)_localctx).escape = string(); - setState(544); + setState(546); match(ESC_END); } break; @@ -3819,11 +3821,12 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti int _alt; enterOuterAlt(_localctx, 1); { - setState(552); + setState(554); switch (_input.LA(1)) { case T__0: case ANALYZE: case ANALYZED: + case CASE: case CAST: case CATALOGS: case COLUMNS: @@ -3888,7 +3891,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti _ctx = _localctx; _prevctx = _localctx; - setState(549); + setState(551); primaryExpression(0); } break; @@ -3898,7 +3901,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti _localctx = new ArithmeticUnaryContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(550); + setState(552); ((ArithmeticUnaryContext)_localctx).operator = _input.LT(1); _la = _input.LA(1); if ( !(_la==PLUS || _la==MINUS) ) { @@ -3906,7 +3909,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti } else { consume(); } - setState(551); + setState(553); valueExpression(4); } break; @@ -3914,7 +3917,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti throw new NoViableAltException(this); } _ctx.stop = _input.LT(-1); - setState(566); + setState(568); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,77,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { @@ -3922,7 +3925,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti if ( _parseListeners!=null ) triggerExitRuleEvent(); _prevctx = _localctx; { - setState(564); + setState(566); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,76,_ctx) ) { case 1: @@ -3930,17 +3933,17 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); ((ArithmeticBinaryContext)_localctx).left = _prevctx; pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(554); + setState(556); if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(555); + setState(557); ((ArithmeticBinaryContext)_localctx).operator = _input.LT(1); _la = _input.LA(1); - if ( !(((((_la - 110)) & ~0x3f) == 0 && ((1L << (_la - 110)) & ((1L << (ASTERISK - 110)) | (1L << (SLASH - 110)) | (1L << (PERCENT - 110)))) != 0)) ) { + if ( !(((((_la - 115)) & ~0x3f) == 0 && ((1L << (_la - 115)) & ((1L << (ASTERISK - 115)) | (1L << (SLASH - 115)) | (1L << (PERCENT - 115)))) != 0)) ) { ((ArithmeticBinaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); } else { consume(); } - setState(556); + setState(558); ((ArithmeticBinaryContext)_localctx).right = valueExpression(4); } break; @@ -3949,9 +3952,9 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti _localctx = new ArithmeticBinaryContext(new ValueExpressionContext(_parentctx, _parentState)); ((ArithmeticBinaryContext)_localctx).left = _prevctx; pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(557); + setState(559); if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(558); + setState(560); ((ArithmeticBinaryContext)_localctx).operator = _input.LT(1); _la = _input.LA(1); if ( !(_la==PLUS || _la==MINUS) ) { @@ -3959,7 +3962,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti } else { consume(); } - setState(559); + setState(561); ((ArithmeticBinaryContext)_localctx).right = valueExpression(3); } break; @@ -3968,18 +3971,18 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti _localctx = new ComparisonContext(new ValueExpressionContext(_parentctx, _parentState)); ((ComparisonContext)_localctx).left = _prevctx; pushNewRecursionContext(_localctx, _startState, RULE_valueExpression); - setState(560); + setState(562); if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); - setState(561); + setState(563); comparisonOperator(); - setState(562); + setState(564); ((ComparisonContext)_localctx).right = valueExpression(2); } break; } } } - setState(568); + setState(570); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,77,_ctx); } @@ -4203,6 +4206,39 @@ public T accept(ParseTreeVisitor visitor) { else return visitor.visitChildren(this); } } + public static class CaseContext extends PrimaryExpressionContext { + public BooleanExpressionContext operand; + public BooleanExpressionContext elseClause; + public TerminalNode CASE() { return getToken(SqlBaseParser.CASE, 0); } + public TerminalNode END() { return getToken(SqlBaseParser.END, 0); } + public List whenClause() { + return getRuleContexts(WhenClauseContext.class); + } + public WhenClauseContext whenClause(int i) { + return getRuleContext(WhenClauseContext.class,i); + } + public TerminalNode ELSE() { return getToken(SqlBaseParser.ELSE, 0); } + public List booleanExpression() { + return getRuleContexts(BooleanExpressionContext.class); + } + public BooleanExpressionContext booleanExpression(int i) { + return getRuleContext(BooleanExpressionContext.class,i); + } + public CaseContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof SqlBaseListener ) ((SqlBaseListener)listener).enterCase(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof SqlBaseListener ) ((SqlBaseListener)listener).exitCase(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof SqlBaseVisitor ) return ((SqlBaseVisitor)visitor).visitCase(this); + else return visitor.visitChildren(this); + } + } public final PrimaryExpressionContext primaryExpression() throws RecognitionException { return primaryExpression(0); @@ -4220,16 +4256,16 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc int _alt; enterOuterAlt(_localctx, 1); { - setState(590); + setState(607); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,79,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,82,_ctx) ) { case 1: { _localctx = new CastContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(570); + setState(572); castExpression(); } break; @@ -4238,7 +4274,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new ExtractContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(571); + setState(573); extractExpression(); } break; @@ -4247,7 +4283,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new CurrentDateTimeFunctionContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(572); + setState(574); builtinDateTimeFunction(); } break; @@ -4256,7 +4292,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new ConstantDefaultContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(573); + setState(575); constant(); } break; @@ -4265,18 +4301,18 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new StarContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(577); + setState(579); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (OPTIMIZED - 67)) | (1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { - setState(574); + setState(576); qualifiedName(); - setState(575); + setState(577); match(DOT); } } - setState(579); + setState(581); match(ASTERISK); } break; @@ -4285,7 +4321,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new FunctionContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(580); + setState(582); functionExpression(); } break; @@ -4294,11 +4330,11 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new SubqueryExpressionContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(581); + setState(583); match(T__0); - setState(582); + setState(584); query(); - setState(583); + setState(585); match(T__1); } break; @@ -4307,7 +4343,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new DereferenceContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(585); + setState(587); qualifiedName(); } break; @@ -4316,19 +4352,64 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc _localctx = new ParenthesizedExpressionContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(586); + setState(588); match(T__0); - setState(587); + setState(589); expression(); - setState(588); + setState(590); match(T__1); } break; + case 10: + { + _localctx = new CaseContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + setState(592); + match(CASE); + setState(594); + _la = _input.LA(1); + if (((((_la - 1)) & ~0x3f) == 0 && ((1L << (_la - 1)) & ((1L << (T__0 - 1)) | (1L << (ANALYZE - 1)) | (1L << (ANALYZED - 1)) | (1L << (CASE - 1)) | (1L << (CAST - 1)) | (1L << (CATALOGS - 1)) | (1L << (COLUMNS - 1)) | (1L << (CONVERT - 1)) | (1L << (CURRENT_DATE - 1)) | (1L << (CURRENT_TIME - 1)) | (1L << (CURRENT_TIMESTAMP - 1)) | (1L << (DAY - 1)) | (1L << (DEBUG - 1)) | (1L << (EXECUTABLE - 1)) | (1L << (EXISTS - 1)) | (1L << (EXPLAIN - 1)) | (1L << (EXTRACT - 1)) | (1L << (FALSE - 1)) | (1L << (FIRST - 1)) | (1L << (FORMAT - 1)) | (1L << (FULL - 1)) | (1L << (FUNCTIONS - 1)) | (1L << (GRAPHVIZ - 1)) | (1L << (HOUR - 1)) | (1L << (INTERVAL - 1)) | (1L << (LAST - 1)) | (1L << (LEFT - 1)) | (1L << (LIMIT - 1)) | (1L << (MAPPED - 1)) | (1L << (MATCH - 1)) | (1L << (MINUTE - 1)) | (1L << (MONTH - 1)) | (1L << (NOT - 1)) | (1L << (NULL - 1)))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (OPTIMIZED - 67)) | (1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + { + setState(593); + ((CaseContext)_localctx).operand = booleanExpression(0); + } + } + + setState(597); + _errHandler.sync(this); + _la = _input.LA(1); + do { + { + { + setState(596); + whenClause(); + } + } + setState(599); + _errHandler.sync(this); + _la = _input.LA(1); + } while ( _la==WHEN ); + setState(603); + _la = _input.LA(1); + if (_la==ELSE) { + { + setState(601); + match(ELSE); + setState(602); + ((CaseContext)_localctx).elseClause = booleanExpression(0); + } + } + + setState(605); + match(END); + } + break; } _ctx.stop = _input.LT(-1); - setState(597); + setState(614); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,80,_ctx); + _alt = getInterpreter().adaptivePredict(_input,83,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { if ( _parseListeners!=null ) triggerExitRuleEvent(); @@ -4337,18 +4418,18 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc { _localctx = new CastOperatorExpressionContext(new PrimaryExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_primaryExpression); - setState(592); - if (!(precpred(_ctx, 9))) throw new FailedPredicateException(this, "precpred(_ctx, 9)"); - setState(593); + setState(609); + if (!(precpred(_ctx, 10))) throw new FailedPredicateException(this, "precpred(_ctx, 10)"); + setState(610); match(CAST_OP); - setState(594); + setState(611); dataType(); } } } - setState(599); + setState(616); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,80,_ctx); + _alt = getInterpreter().adaptivePredict(_input,83,_ctx); } } } @@ -4391,26 +4472,26 @@ public final BuiltinDateTimeFunctionContext builtinDateTimeFunction() throws Rec BuiltinDateTimeFunctionContext _localctx = new BuiltinDateTimeFunctionContext(_ctx, getState()); enterRule(_localctx, 62, RULE_builtinDateTimeFunction); try { - setState(603); + setState(620); switch (_input.LA(1)) { case CURRENT_TIMESTAMP: enterOuterAlt(_localctx, 1); { - setState(600); + setState(617); ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_TIMESTAMP); } break; case CURRENT_DATE: enterOuterAlt(_localctx, 2); { - setState(601); + setState(618); ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_DATE); } break; case CURRENT_TIME: enterOuterAlt(_localctx, 3); { - setState(602); + setState(619); ((BuiltinDateTimeFunctionContext)_localctx).name = match(CURRENT_TIME); } break; @@ -4461,42 +4542,42 @@ public final CastExpressionContext castExpression() throws RecognitionException CastExpressionContext _localctx = new CastExpressionContext(_ctx, getState()); enterRule(_localctx, 64, RULE_castExpression); try { - setState(615); + setState(632); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,82,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,85,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(605); + setState(622); castTemplate(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(606); + setState(623); match(FUNCTION_ESC); - setState(607); + setState(624); castTemplate(); - setState(608); + setState(625); match(ESC_END); } break; case 3: enterOuterAlt(_localctx, 3); { - setState(610); + setState(627); convertTemplate(); } break; case 4: enterOuterAlt(_localctx, 4); { - setState(611); + setState(628); match(FUNCTION_ESC); - setState(612); + setState(629); convertTemplate(); - setState(613); + setState(630); match(ESC_END); } break; @@ -4547,17 +4628,17 @@ public final CastTemplateContext castTemplate() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(617); + setState(634); match(CAST); - setState(618); + setState(635); match(T__0); - setState(619); + setState(636); expression(); - setState(620); + setState(637); match(AS); - setState(621); + setState(638); dataType(); - setState(622); + setState(639); match(T__1); } } @@ -4605,17 +4686,17 @@ public final ConvertTemplateContext convertTemplate() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(624); + setState(641); match(CONVERT); - setState(625); + setState(642); match(T__0); - setState(626); + setState(643); expression(); - setState(627); + setState(644); match(T__2); - setState(628); + setState(645); dataType(); - setState(629); + setState(646); match(T__1); } } @@ -4659,23 +4740,23 @@ public final ExtractExpressionContext extractExpression() throws RecognitionExce ExtractExpressionContext _localctx = new ExtractExpressionContext(_ctx, getState()); enterRule(_localctx, 70, RULE_extractExpression); try { - setState(636); + setState(653); switch (_input.LA(1)) { case EXTRACT: enterOuterAlt(_localctx, 1); { - setState(631); + setState(648); extractTemplate(); } break; case FUNCTION_ESC: enterOuterAlt(_localctx, 2); { - setState(632); + setState(649); match(FUNCTION_ESC); - setState(633); + setState(650); extractTemplate(); - setState(634); + setState(651); match(ESC_END); } break; @@ -4729,17 +4810,17 @@ public final ExtractTemplateContext extractTemplate() throws RecognitionExceptio try { enterOuterAlt(_localctx, 1); { - setState(638); + setState(655); match(EXTRACT); - setState(639); + setState(656); match(T__0); - setState(640); + setState(657); ((ExtractTemplateContext)_localctx).field = identifier(); - setState(641); + setState(658); match(FROM); - setState(642); + setState(659); valueExpression(0); - setState(643); + setState(660); match(T__1); } } @@ -4782,7 +4863,7 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx FunctionExpressionContext _localctx = new FunctionExpressionContext(_ctx, getState()); enterRule(_localctx, 74, RULE_functionExpression); try { - setState(650); + setState(667); switch (_input.LA(1)) { case ANALYZE: case ANALYZED: @@ -4831,18 +4912,18 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 1); { - setState(645); + setState(662); functionTemplate(); } break; case FUNCTION_ESC: enterOuterAlt(_localctx, 2); { - setState(646); + setState(663); match(FUNCTION_ESC); - setState(647); + setState(664); functionTemplate(); - setState(648); + setState(665); match(ESC_END); } break; @@ -4900,45 +4981,45 @@ public final FunctionTemplateContext functionTemplate() throws RecognitionExcept try { enterOuterAlt(_localctx, 1); { - setState(652); + setState(669); functionName(); - setState(653); + setState(670); match(T__0); - setState(665); + setState(682); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ALL) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << DISTINCT) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RIGHT - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TRUE - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (FUNCTION_ESC - 64)) | (1L << (DATE_ESC - 64)) | (1L << (TIME_ESC - 64)) | (1L << (TIMESTAMP_ESC - 64)) | (1L << (GUID_ESC - 64)) | (1L << (PLUS - 64)) | (1L << (MINUS - 64)) | (1L << (ASTERISK - 64)) | (1L << (PARAM - 64)) | (1L << (STRING - 64)) | (1L << (INTEGER_VALUE - 64)) | (1L << (DECIMAL_VALUE - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { + if (((((_la - 1)) & ~0x3f) == 0 && ((1L << (_la - 1)) & ((1L << (T__0 - 1)) | (1L << (ALL - 1)) | (1L << (ANALYZE - 1)) | (1L << (ANALYZED - 1)) | (1L << (CASE - 1)) | (1L << (CAST - 1)) | (1L << (CATALOGS - 1)) | (1L << (COLUMNS - 1)) | (1L << (CONVERT - 1)) | (1L << (CURRENT_DATE - 1)) | (1L << (CURRENT_TIME - 1)) | (1L << (CURRENT_TIMESTAMP - 1)) | (1L << (DAY - 1)) | (1L << (DEBUG - 1)) | (1L << (DISTINCT - 1)) | (1L << (EXECUTABLE - 1)) | (1L << (EXISTS - 1)) | (1L << (EXPLAIN - 1)) | (1L << (EXTRACT - 1)) | (1L << (FALSE - 1)) | (1L << (FIRST - 1)) | (1L << (FORMAT - 1)) | (1L << (FULL - 1)) | (1L << (FUNCTIONS - 1)) | (1L << (GRAPHVIZ - 1)) | (1L << (HOUR - 1)) | (1L << (INTERVAL - 1)) | (1L << (LAST - 1)) | (1L << (LEFT - 1)) | (1L << (LIMIT - 1)) | (1L << (MAPPED - 1)) | (1L << (MATCH - 1)) | (1L << (MINUTE - 1)) | (1L << (MONTH - 1)) | (1L << (NOT - 1)) | (1L << (NULL - 1)))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (OPTIMIZED - 67)) | (1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { - setState(655); + setState(672); _la = _input.LA(1); if (_la==ALL || _la==DISTINCT) { { - setState(654); + setState(671); setQuantifier(); } } - setState(657); + setState(674); expression(); - setState(662); + setState(679); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__2) { { { - setState(658); + setState(675); match(T__2); - setState(659); + setState(676); expression(); } } - setState(664); + setState(681); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(667); + setState(684); match(T__1); } } @@ -4982,19 +5063,19 @@ public final FunctionNameContext functionName() throws RecognitionException { FunctionNameContext _localctx = new FunctionNameContext(_ctx, getState()); enterRule(_localctx, 78, RULE_functionName); try { - setState(672); + setState(689); switch (_input.LA(1)) { case LEFT: enterOuterAlt(_localctx, 1); { - setState(669); + setState(686); match(LEFT); } break; case RIGHT: enterOuterAlt(_localctx, 2); { - setState(670); + setState(687); match(RIGHT); } break; @@ -5043,7 +5124,7 @@ public final FunctionNameContext functionName() throws RecognitionException { case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 3); { - setState(671); + setState(688); identifier(); } break; @@ -5274,13 +5355,13 @@ public final ConstantContext constant() throws RecognitionException { enterRule(_localctx, 80, RULE_constant); try { int _alt; - setState(700); + setState(717); switch (_input.LA(1)) { case NULL: _localctx = new NullLiteralContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(674); + setState(691); match(NULL); } break; @@ -5288,7 +5369,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new IntervalLiteralContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(675); + setState(692); interval(); } break; @@ -5297,7 +5378,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new NumericLiteralContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(676); + setState(693); number(); } break; @@ -5306,7 +5387,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new BooleanLiteralContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(677); + setState(694); booleanValue(); } break; @@ -5314,7 +5395,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new StringLiteralContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(679); + setState(696); _errHandler.sync(this); _alt = 1; do { @@ -5322,7 +5403,7 @@ public final ConstantContext constant() throws RecognitionException { case 1: { { - setState(678); + setState(695); match(STRING); } } @@ -5330,9 +5411,9 @@ public final ConstantContext constant() throws RecognitionException { default: throw new NoViableAltException(this); } - setState(681); + setState(698); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,89,_ctx); + _alt = getInterpreter().adaptivePredict(_input,92,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); } break; @@ -5340,7 +5421,7 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new ParamLiteralContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(683); + setState(700); match(PARAM); } break; @@ -5348,11 +5429,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new DateEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(684); + setState(701); match(DATE_ESC); - setState(685); + setState(702); string(); - setState(686); + setState(703); match(ESC_END); } break; @@ -5360,11 +5441,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new TimeEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(688); + setState(705); match(TIME_ESC); - setState(689); + setState(706); string(); - setState(690); + setState(707); match(ESC_END); } break; @@ -5372,11 +5453,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new TimestampEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(692); + setState(709); match(TIMESTAMP_ESC); - setState(693); + setState(710); string(); - setState(694); + setState(711); match(ESC_END); } break; @@ -5384,11 +5465,11 @@ public final ConstantContext constant() throws RecognitionException { _localctx = new GuidEscapedLiteralContext(_localctx); enterOuterAlt(_localctx, 10); { - setState(696); + setState(713); match(GUID_ESC); - setState(697); + setState(714); string(); - setState(698); + setState(715); match(ESC_END); } break; @@ -5441,9 +5522,9 @@ public final ComparisonOperatorContext comparisonOperator() throws RecognitionEx try { enterOuterAlt(_localctx, 1); { - setState(702); + setState(719); _la = _input.LA(1); - if ( !(((((_la - 101)) & ~0x3f) == 0 && ((1L << (_la - 101)) & ((1L << (EQ - 101)) | (1L << (NULLEQ - 101)) | (1L << (NEQ - 101)) | (1L << (LT - 101)) | (1L << (LTE - 101)) | (1L << (GT - 101)) | (1L << (GTE - 101)))) != 0)) ) { + if ( !(((((_la - 106)) & ~0x3f) == 0 && ((1L << (_la - 106)) & ((1L << (EQ - 106)) | (1L << (NULLEQ - 106)) | (1L << (NEQ - 106)) | (1L << (LT - 106)) | (1L << (LTE - 106)) | (1L << (GT - 106)) | (1L << (GTE - 106)))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -5490,7 +5571,7 @@ public final BooleanValueContext booleanValue() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(704); + setState(721); _la = _input.LA(1); if ( !(_la==FALSE || _la==TRUE) ) { _errHandler.recoverInline(this); @@ -5558,13 +5639,13 @@ public final IntervalContext interval() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(706); + setState(723); match(INTERVAL); - setState(708); + setState(725); _la = _input.LA(1); if (_la==PLUS || _la==MINUS) { { - setState(707); + setState(724); ((IntervalContext)_localctx).sign = _input.LT(1); _la = _input.LA(1); if ( !(_la==PLUS || _la==MINUS) ) { @@ -5575,35 +5656,35 @@ public final IntervalContext interval() throws RecognitionException { } } - setState(712); + setState(729); switch (_input.LA(1)) { case INTEGER_VALUE: case DECIMAL_VALUE: { - setState(710); + setState(727); ((IntervalContext)_localctx).valueNumeric = number(); } break; case PARAM: case STRING: { - setState(711); + setState(728); ((IntervalContext)_localctx).valuePattern = string(); } break; default: throw new NoViableAltException(this); } - setState(714); + setState(731); ((IntervalContext)_localctx).leading = intervalField(); - setState(717); + setState(734); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,93,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,96,_ctx) ) { case 1: { - setState(715); + setState(732); match(TO); - setState(716); + setState(733); ((IntervalContext)_localctx).trailing = intervalField(); } break; @@ -5660,9 +5741,9 @@ public final IntervalFieldContext intervalField() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(719); + setState(736); _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << DAY) | (1L << DAYS) | (1L << HOUR) | (1L << HOURS) | (1L << MINUTE) | (1L << MINUTES) | (1L << MONTH) | (1L << MONTHS))) != 0) || ((((_la - 75)) & ~0x3f) == 0 && ((1L << (_la - 75)) & ((1L << (SECOND - 75)) | (1L << (SECONDS - 75)) | (1L << (YEAR - 75)) | (1L << (YEARS - 75)))) != 0)) ) { + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << DAY) | (1L << DAYS) | (1L << HOUR) | (1L << HOURS) | (1L << MINUTE) | (1L << MINUTES) | (1L << MONTH) | (1L << MONTHS))) != 0) || ((((_la - 78)) & ~0x3f) == 0 && ((1L << (_la - 78)) & ((1L << (SECOND - 78)) | (1L << (SECONDS - 78)) | (1L << (YEAR - 78)) | (1L << (YEARS - 78)))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -5718,7 +5799,7 @@ public final DataTypeContext dataType() throws RecognitionException { _localctx = new PrimitiveDataTypeContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(721); + setState(738); identifier(); } } @@ -5770,25 +5851,25 @@ public final QualifiedNameContext qualifiedName() throws RecognitionException { int _alt; enterOuterAlt(_localctx, 1); { - setState(728); + setState(745); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,94,_ctx); + _alt = getInterpreter().adaptivePredict(_input,97,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(723); + setState(740); identifier(); - setState(724); + setState(741); match(DOT); } } } - setState(730); + setState(747); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,94,_ctx); + _alt = getInterpreter().adaptivePredict(_input,97,_ctx); } - setState(731); + setState(748); identifier(); } } @@ -5833,13 +5914,13 @@ public final IdentifierContext identifier() throws RecognitionException { IdentifierContext _localctx = new IdentifierContext(_ctx, getState()); enterRule(_localctx, 94, RULE_identifier); try { - setState(735); + setState(752); switch (_input.LA(1)) { case QUOTED_IDENTIFIER: case BACKQUOTED_IDENTIFIER: enterOuterAlt(_localctx, 1); { - setState(733); + setState(750); quoteIdentifier(); } break; @@ -5886,7 +5967,7 @@ public final IdentifierContext identifier() throws RecognitionException { case DIGIT_IDENTIFIER: enterOuterAlt(_localctx, 2); { - setState(734); + setState(751); unquoteIdentifier(); } break; @@ -5939,43 +6020,43 @@ public final TableIdentifierContext tableIdentifier() throws RecognitionExceptio enterRule(_localctx, 96, RULE_tableIdentifier); int _la; try { - setState(749); + setState(766); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,98,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,101,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(740); + setState(757); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)) | (1L << (IDENTIFIER - 64)) | (1L << (DIGIT_IDENTIFIER - 64)) | (1L << (QUOTED_IDENTIFIER - 64)) | (1L << (BACKQUOTED_IDENTIFIER - 64)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (OPTIMIZED - 67)) | (1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { - setState(737); + setState(754); ((TableIdentifierContext)_localctx).catalog = identifier(); - setState(738); + setState(755); match(T__3); } } - setState(742); + setState(759); match(TABLE_IDENTIFIER); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(746); + setState(763); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,97,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,100,_ctx) ) { case 1: { - setState(743); + setState(760); ((TableIdentifierContext)_localctx).catalog = identifier(); - setState(744); + setState(761); match(T__3); } break; } - setState(748); + setState(765); ((TableIdentifierContext)_localctx).name = identifier(); } break; @@ -6042,13 +6123,13 @@ public final QuoteIdentifierContext quoteIdentifier() throws RecognitionExceptio QuoteIdentifierContext _localctx = new QuoteIdentifierContext(_ctx, getState()); enterRule(_localctx, 98, RULE_quoteIdentifier); try { - setState(753); + setState(770); switch (_input.LA(1)) { case QUOTED_IDENTIFIER: _localctx = new QuotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(751); + setState(768); match(QUOTED_IDENTIFIER); } break; @@ -6056,7 +6137,7 @@ public final QuoteIdentifierContext quoteIdentifier() throws RecognitionExceptio _localctx = new BackQuotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(752); + setState(769); match(BACKQUOTED_IDENTIFIER); } break; @@ -6128,13 +6209,13 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce UnquoteIdentifierContext _localctx = new UnquoteIdentifierContext(_ctx, getState()); enterRule(_localctx, 100, RULE_unquoteIdentifier); try { - setState(758); + setState(775); switch (_input.LA(1)) { case IDENTIFIER: _localctx = new UnquotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(755); + setState(772); match(IDENTIFIER); } break; @@ -6180,7 +6261,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce _localctx = new UnquotedIdentifierContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(756); + setState(773); nonReserved(); } break; @@ -6188,7 +6269,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce _localctx = new DigitIdentifierContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(757); + setState(774); match(DIGIT_IDENTIFIER); } break; @@ -6257,13 +6338,13 @@ public final NumberContext number() throws RecognitionException { NumberContext _localctx = new NumberContext(_ctx, getState()); enterRule(_localctx, 102, RULE_number); try { - setState(762); + setState(779); switch (_input.LA(1)) { case DECIMAL_VALUE: _localctx = new DecimalLiteralContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(760); + setState(777); match(DECIMAL_VALUE); } break; @@ -6271,7 +6352,7 @@ public final NumberContext number() throws RecognitionException { _localctx = new IntegerLiteralContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(761); + setState(778); match(INTEGER_VALUE); } break; @@ -6319,7 +6400,7 @@ public final StringContext string() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(764); + setState(781); _la = _input.LA(1); if ( !(_la==PARAM || _la==STRING) ) { _errHandler.recoverInline(this); @@ -6339,6 +6420,63 @@ public final StringContext string() throws RecognitionException { return _localctx; } + public static class WhenClauseContext extends ParserRuleContext { + public ExpressionContext condition; + public ExpressionContext result; + public TerminalNode WHEN() { return getToken(SqlBaseParser.WHEN, 0); } + public TerminalNode THEN() { return getToken(SqlBaseParser.THEN, 0); } + public List expression() { + return getRuleContexts(ExpressionContext.class); + } + public ExpressionContext expression(int i) { + return getRuleContext(ExpressionContext.class,i); + } + public WhenClauseContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_whenClause; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof SqlBaseListener ) ((SqlBaseListener)listener).enterWhenClause(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof SqlBaseListener ) ((SqlBaseListener)listener).exitWhenClause(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof SqlBaseVisitor ) return ((SqlBaseVisitor)visitor).visitWhenClause(this); + else return visitor.visitChildren(this); + } + } + + public final WhenClauseContext whenClause() throws RecognitionException { + WhenClauseContext _localctx = new WhenClauseContext(_ctx, getState()); + enterRule(_localctx, 106, RULE_whenClause); + try { + enterOuterAlt(_localctx, 1); + { + setState(783); + match(WHEN); + setState(784); + ((WhenClauseContext)_localctx).condition = expression(); + setState(785); + match(THEN); + setState(786); + ((WhenClauseContext)_localctx).result = expression(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + public static class NonReservedContext extends ParserRuleContext { public TerminalNode ANALYZE() { return getToken(SqlBaseParser.ANALYZE, 0); } public TerminalNode ANALYZED() { return getToken(SqlBaseParser.ANALYZED, 0); } @@ -6400,14 +6538,14 @@ public T accept(ParseTreeVisitor visitor) { public final NonReservedContext nonReserved() throws RecognitionException { NonReservedContext _localctx = new NonReservedContext(_ctx, getState()); - enterRule(_localctx, 106, RULE_nonReserved); + enterRule(_localctx, 108, RULE_nonReserved); int _la; try { enterOuterAlt(_localctx, 1); { - setState(766); + setState(788); _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OPTIMIZED - 64)) | (1L << (PARSED - 64)) | (1L << (PHYSICAL - 64)) | (1L << (PLAN - 64)) | (1L << (RLIKE - 64)) | (1L << (QUERY - 64)) | (1L << (SCHEMAS - 64)) | (1L << (SECOND - 64)) | (1L << (SHOW - 64)) | (1L << (SYS - 64)) | (1L << (TABLES - 64)) | (1L << (TEXT - 64)) | (1L << (TYPE - 64)) | (1L << (TYPES - 64)) | (1L << (VERIFY - 64)) | (1L << (YEAR - 64)))) != 0)) ) { + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT_DATE) | (1L << CURRENT_TIME) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (OPTIMIZED - 67)) | (1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -6459,317 +6597,328 @@ private boolean valueExpression_sempred(ValueExpressionContext _localctx, int pr private boolean primaryExpression_sempred(PrimaryExpressionContext _localctx, int predIndex) { switch (predIndex) { case 5: - return precpred(_ctx, 9); + return precpred(_ctx, 10); } return true; } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\u0083\u0303\4\2\t"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\u0088\u0319\4\2\t"+ "\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13"+ "\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31\t\31"+ "\4\32\t\32\4\33\t\33\4\34\t\34\4\35\t\35\4\36\t\36\4\37\t\37\4 \t \4!"+ "\t!\4\"\t\"\4#\t#\4$\t$\4%\t%\4&\t&\4\'\t\'\4(\t(\4)\t)\4*\t*\4+\t+\4"+ ",\t,\4-\t-\4.\t.\4/\t/\4\60\t\60\4\61\t\61\4\62\t\62\4\63\t\63\4\64\t"+ - "\64\4\65\t\65\4\66\t\66\4\67\t\67\3\2\3\2\3\2\3\3\3\3\3\3\3\4\3\4\3\4"+ - "\3\4\3\4\3\4\3\4\3\4\3\4\7\4~\n\4\f\4\16\4\u0081\13\4\3\4\5\4\u0084\n"+ - "\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\7\4\u008d\n\4\f\4\16\4\u0090\13\4\3\4\5"+ - "\4\u0093\n\4\3\4\3\4\3\4\3\4\3\4\5\4\u009a\n\4\3\4\3\4\3\4\3\4\3\4\5\4"+ - "\u00a1\n\4\3\4\3\4\3\4\5\4\u00a6\n\4\3\4\3\4\3\4\5\4\u00ab\n\4\3\4\3\4"+ - "\3\4\3\4\3\4\3\4\5\4\u00b3\n\4\3\4\3\4\5\4\u00b7\n\4\3\4\3\4\3\4\3\4\7"+ - "\4\u00bd\n\4\f\4\16\4\u00c0\13\4\5\4\u00c2\n\4\3\4\3\4\3\4\3\4\5\4\u00c8"+ - "\n\4\3\4\3\4\3\4\5\4\u00cd\n\4\3\4\5\4\u00d0\n\4\3\4\3\4\3\4\5\4\u00d5"+ - "\n\4\3\4\5\4\u00d8\n\4\5\4\u00da\n\4\3\5\3\5\3\5\3\5\7\5\u00e0\n\5\f\5"+ - "\16\5\u00e3\13\5\5\5\u00e5\n\5\3\5\3\5\3\6\3\6\3\6\3\6\3\6\3\6\7\6\u00ef"+ - "\n\6\f\6\16\6\u00f2\13\6\5\6\u00f4\n\6\3\6\5\6\u00f7\n\6\3\7\3\7\3\7\3"+ - "\7\3\7\5\7\u00fe\n\7\3\b\3\b\3\b\3\b\3\b\5\b\u0105\n\b\3\t\3\t\5\t\u0109"+ - "\n\t\3\t\3\t\5\t\u010d\n\t\3\n\3\n\5\n\u0111\n\n\3\n\3\n\3\n\7\n\u0116"+ - "\n\n\f\n\16\n\u0119\13\n\3\n\5\n\u011c\n\n\3\n\3\n\5\n\u0120\n\n\3\n\3"+ - "\n\3\n\5\n\u0125\n\n\3\n\3\n\5\n\u0129\n\n\3\13\3\13\3\13\3\13\7\13\u012f"+ - "\n\13\f\13\16\13\u0132\13\13\3\f\5\f\u0135\n\f\3\f\3\f\3\f\7\f\u013a\n"+ - "\f\f\f\16\f\u013d\13\f\3\r\3\r\3\16\3\16\3\16\3\16\7\16\u0145\n\16\f\16"+ - "\16\16\u0148\13\16\5\16\u014a\n\16\3\16\3\16\5\16\u014e\n\16\3\17\3\17"+ - "\3\17\3\17\3\17\3\17\3\20\3\20\3\21\3\21\5\21\u015a\n\21\3\21\5\21\u015d"+ - "\n\21\3\22\3\22\7\22\u0161\n\22\f\22\16\22\u0164\13\22\3\23\3\23\3\23"+ - "\3\23\5\23\u016a\n\23\3\23\3\23\3\23\3\23\3\23\5\23\u0171\n\23\3\24\5"+ - "\24\u0174\n\24\3\24\3\24\5\24\u0178\n\24\3\24\3\24\5\24\u017c\n\24\3\24"+ - "\3\24\5\24\u0180\n\24\5\24\u0182\n\24\3\25\3\25\3\25\3\25\3\25\3\25\3"+ - "\25\7\25\u018b\n\25\f\25\16\25\u018e\13\25\3\25\3\25\5\25\u0192\n\25\3"+ - "\26\3\26\5\26\u0196\n\26\3\26\5\26\u0199\n\26\3\26\3\26\3\26\3\26\5\26"+ - "\u019f\n\26\3\26\5\26\u01a2\n\26\3\26\3\26\3\26\3\26\5\26\u01a8\n\26\3"+ - "\26\5\26\u01ab\n\26\5\26\u01ad\n\26\3\27\3\27\3\30\3\30\3\30\3\30\3\30"+ + "\64\4\65\t\65\4\66\t\66\4\67\t\67\48\t8\3\2\3\2\3\2\3\3\3\3\3\3\3\4\3"+ + "\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\7\4\u0080\n\4\f\4\16\4\u0083\13\4\3\4\5"+ + "\4\u0086\n\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\7\4\u008f\n\4\f\4\16\4\u0092"+ + "\13\4\3\4\5\4\u0095\n\4\3\4\3\4\3\4\3\4\3\4\5\4\u009c\n\4\3\4\3\4\3\4"+ + "\3\4\3\4\5\4\u00a3\n\4\3\4\3\4\3\4\5\4\u00a8\n\4\3\4\3\4\3\4\5\4\u00ad"+ + "\n\4\3\4\3\4\3\4\3\4\3\4\3\4\5\4\u00b5\n\4\3\4\3\4\5\4\u00b9\n\4\3\4\3"+ + "\4\3\4\3\4\7\4\u00bf\n\4\f\4\16\4\u00c2\13\4\5\4\u00c4\n\4\3\4\3\4\3\4"+ + "\3\4\5\4\u00ca\n\4\3\4\3\4\3\4\5\4\u00cf\n\4\3\4\5\4\u00d2\n\4\3\4\3\4"+ + "\3\4\5\4\u00d7\n\4\3\4\5\4\u00da\n\4\5\4\u00dc\n\4\3\5\3\5\3\5\3\5\7\5"+ + "\u00e2\n\5\f\5\16\5\u00e5\13\5\5\5\u00e7\n\5\3\5\3\5\3\6\3\6\3\6\3\6\3"+ + "\6\3\6\7\6\u00f1\n\6\f\6\16\6\u00f4\13\6\5\6\u00f6\n\6\3\6\5\6\u00f9\n"+ + "\6\3\7\3\7\3\7\3\7\3\7\5\7\u0100\n\7\3\b\3\b\3\b\3\b\3\b\5\b\u0107\n\b"+ + "\3\t\3\t\5\t\u010b\n\t\3\t\3\t\5\t\u010f\n\t\3\n\3\n\5\n\u0113\n\n\3\n"+ + "\3\n\3\n\7\n\u0118\n\n\f\n\16\n\u011b\13\n\3\n\5\n\u011e\n\n\3\n\3\n\5"+ + "\n\u0122\n\n\3\n\3\n\3\n\5\n\u0127\n\n\3\n\3\n\5\n\u012b\n\n\3\13\3\13"+ + "\3\13\3\13\7\13\u0131\n\13\f\13\16\13\u0134\13\13\3\f\5\f\u0137\n\f\3"+ + "\f\3\f\3\f\7\f\u013c\n\f\f\f\16\f\u013f\13\f\3\r\3\r\3\16\3\16\3\16\3"+ + "\16\7\16\u0147\n\16\f\16\16\16\u014a\13\16\5\16\u014c\n\16\3\16\3\16\5"+ + "\16\u0150\n\16\3\17\3\17\3\17\3\17\3\17\3\17\3\20\3\20\3\21\3\21\5\21"+ + "\u015c\n\21\3\21\5\21\u015f\n\21\3\22\3\22\7\22\u0163\n\22\f\22\16\22"+ + "\u0166\13\22\3\23\3\23\3\23\3\23\5\23\u016c\n\23\3\23\3\23\3\23\3\23\3"+ + "\23\5\23\u0173\n\23\3\24\5\24\u0176\n\24\3\24\3\24\5\24\u017a\n\24\3\24"+ + "\3\24\5\24\u017e\n\24\3\24\3\24\5\24\u0182\n\24\5\24\u0184\n\24\3\25\3"+ + "\25\3\25\3\25\3\25\3\25\3\25\7\25\u018d\n\25\f\25\16\25\u0190\13\25\3"+ + "\25\3\25\5\25\u0194\n\25\3\26\3\26\5\26\u0198\n\26\3\26\5\26\u019b\n\26"+ + "\3\26\3\26\3\26\3\26\5\26\u01a1\n\26\3\26\5\26\u01a4\n\26\3\26\3\26\3"+ + "\26\3\26\5\26\u01aa\n\26\3\26\5\26\u01ad\n\26\5\26\u01af\n\26\3\27\3\27"+ + "\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30"+ "\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30"+ - "\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\5\30\u01d0"+ - "\n\30\3\30\3\30\3\30\3\30\3\30\3\30\7\30\u01d8\n\30\f\30\16\30\u01db\13"+ - "\30\3\31\3\31\7\31\u01df\n\31\f\31\16\31\u01e2\13\31\3\32\3\32\5\32\u01e6"+ - "\n\32\3\33\5\33\u01e9\n\33\3\33\3\33\3\33\3\33\3\33\3\33\5\33\u01f1\n"+ - "\33\3\33\3\33\3\33\3\33\3\33\7\33\u01f8\n\33\f\33\16\33\u01fb\13\33\3"+ - "\33\3\33\3\33\5\33\u0200\n\33\3\33\3\33\3\33\3\33\3\33\3\33\5\33\u0208"+ - "\n\33\3\33\3\33\3\33\5\33\u020d\n\33\3\33\3\33\3\33\3\33\5\33\u0213\n"+ - "\33\3\33\5\33\u0216\n\33\3\34\3\34\3\34\3\35\3\35\5\35\u021d\n\35\3\36"+ - "\3\36\3\36\3\36\3\36\3\36\5\36\u0225\n\36\3\37\3\37\3\37\3\37\5\37\u022b"+ - "\n\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\7\37\u0237\n\37"+ - "\f\37\16\37\u023a\13\37\3 \3 \3 \3 \3 \3 \3 \3 \5 \u0244\n \3 \3 \3 \3"+ - " \3 \3 \3 \3 \3 \3 \3 \5 \u0251\n \3 \3 \3 \7 \u0256\n \f \16 \u0259\13"+ - " \3!\3!\3!\5!\u025e\n!\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\5\"\u026a"+ - "\n\"\3#\3#\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\5%\u027f"+ - "\n%\3&\3&\3&\3&\3&\3&\3&\3\'\3\'\3\'\3\'\3\'\5\'\u028d\n\'\3(\3(\3(\5"+ - "(\u0292\n(\3(\3(\3(\7(\u0297\n(\f(\16(\u029a\13(\5(\u029c\n(\3(\3(\3)"+ - "\3)\3)\5)\u02a3\n)\3*\3*\3*\3*\3*\6*\u02aa\n*\r*\16*\u02ab\3*\3*\3*\3"+ - "*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\5*\u02bf\n*\3+\3+\3,\3,\3-\3"+ - "-\5-\u02c7\n-\3-\3-\5-\u02cb\n-\3-\3-\3-\5-\u02d0\n-\3.\3.\3/\3/\3\60"+ - "\3\60\3\60\7\60\u02d9\n\60\f\60\16\60\u02dc\13\60\3\60\3\60\3\61\3\61"+ - "\5\61\u02e2\n\61\3\62\3\62\3\62\5\62\u02e7\n\62\3\62\3\62\3\62\3\62\5"+ - "\62\u02ed\n\62\3\62\5\62\u02f0\n\62\3\63\3\63\5\63\u02f4\n\63\3\64\3\64"+ - "\3\64\5\64\u02f9\n\64\3\65\3\65\5\65\u02fd\n\65\3\66\3\66\3\67\3\67\3"+ - "\67\2\5.<>8\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,.\60\62\64\66"+ - "8:<>@BDFHJLNPRTVXZ\\^`bdfhjl\2\22\b\2\7\7\t\t\37\37\67\67BBFF\4\2))TT"+ - "\4\2\t\tBB\4\2&&..\3\2\33\34\3\2no\4\2\7\7xx\4\2\r\r\33\33\4\2$$\63\63"+ - "\4\2\7\7\35\35\3\2pr\3\2gm\4\2##UU\7\2\30\31,-9\u0250\3\2\2\2@\u025d"+ - "\3\2\2\2B\u0269\3\2\2\2D\u026b\3\2\2\2F\u0272\3\2\2\2H\u027e\3\2\2\2J"+ - "\u0280\3\2\2\2L\u028c\3\2\2\2N\u028e\3\2\2\2P\u02a2\3\2\2\2R\u02be\3\2"+ - "\2\2T\u02c0\3\2\2\2V\u02c2\3\2\2\2X\u02c4\3\2\2\2Z\u02d1\3\2\2\2\\\u02d3"+ - "\3\2\2\2^\u02da\3\2\2\2`\u02e1\3\2\2\2b\u02ef\3\2\2\2d\u02f3\3\2\2\2f"+ - "\u02f8\3\2\2\2h\u02fc\3\2\2\2j\u02fe\3\2\2\2l\u0300\3\2\2\2no\5\6\4\2"+ - "op\7\2\2\3p\3\3\2\2\2qr\5,\27\2rs\7\2\2\3s\5\3\2\2\2t\u00da\5\b\5\2u\u0083"+ - "\7!\2\2v\177\7\3\2\2wx\7H\2\2x~\t\2\2\2yz\7%\2\2z~\t\3\2\2{|\7Z\2\2|~"+ - "\5V,\2}w\3\2\2\2}y\3\2\2\2}{\3\2\2\2~\u0081\3\2\2\2\177}\3\2\2\2\177\u0080"+ - "\3\2\2\2\u0080\u0082\3\2\2\2\u0081\177\3\2\2\2\u0082\u0084\7\4\2\2\u0083"+ - "v\3\2\2\2\u0083\u0084\3\2\2\2\u0084\u0085\3\2\2\2\u0085\u00da\5\6\4\2"+ - "\u0086\u0092\7\32\2\2\u0087\u008e\7\3\2\2\u0088\u0089\7H\2\2\u0089\u008d"+ - "\t\4\2\2\u008a\u008b\7%\2\2\u008b\u008d\t\3\2\2\u008c\u0088\3\2\2\2\u008c"+ - "\u008a\3\2\2\2\u008d\u0090\3\2\2\2\u008e\u008c\3\2\2\2\u008e\u008f\3\2"+ - "\2\2\u008f\u0091\3\2\2\2\u0090\u008e\3\2\2\2\u0091\u0093\7\4\2\2\u0092"+ - "\u0087\3\2\2\2\u0092\u0093\3\2\2\2\u0093\u0094\3\2\2\2\u0094\u00da\5\6"+ - "\4\2\u0095\u0096\7P\2\2\u0096\u0099\7S\2\2\u0097\u009a\5\66\34\2\u0098"+ - "\u009a\5b\62\2\u0099\u0097\3\2\2\2\u0099\u0098\3\2\2\2\u0099\u009a\3\2"+ - "\2\2\u009a\u00da\3\2\2\2\u009b\u009c\7P\2\2\u009c\u009d\7\23\2\2\u009d"+ - "\u00a0\t\5\2\2\u009e\u00a1\5\66\34\2\u009f\u00a1\5b\62\2\u00a0\u009e\3"+ - "\2\2\2\u00a0\u009f\3\2\2\2\u00a1\u00da\3\2\2\2\u00a2\u00a5\t\6\2\2\u00a3"+ - "\u00a6\5\66\34\2\u00a4\u00a6\5b\62\2\u00a5\u00a3\3\2\2\2\u00a5\u00a4\3"+ - "\2\2\2\u00a6\u00da\3\2\2\2\u00a7\u00a8\7P\2\2\u00a8\u00aa\7(\2\2\u00a9"+ - "\u00ab\5\66\34\2\u00aa\u00a9\3\2\2\2\u00aa\u00ab\3\2\2\2\u00ab\u00da\3"+ - "\2\2\2\u00ac\u00ad\7P\2\2\u00ad\u00da\7L\2\2\u00ae\u00af\7Q\2\2\u00af"+ - "\u00b2\7S\2\2\u00b0\u00b1\7\21\2\2\u00b1\u00b3\5\66\34\2\u00b2\u00b0\3"+ - "\2\2\2\u00b2\u00b3\3\2\2\2\u00b3\u00b6\3\2\2\2\u00b4\u00b7\5\66\34\2\u00b5"+ - "\u00b7\5b\62\2\u00b6\u00b4\3\2\2\2\u00b6\u00b5\3\2\2\2\u00b6\u00b7\3\2"+ - "\2\2\u00b7\u00c1\3\2\2\2\u00b8\u00b9\7W\2\2\u00b9\u00be\5j\66\2\u00ba"+ - "\u00bb\7\5\2\2\u00bb\u00bd\5j\66\2\u00bc\u00ba\3\2\2\2\u00bd\u00c0\3\2"+ - "\2\2\u00be\u00bc\3\2\2\2\u00be\u00bf\3\2\2\2\u00bf\u00c2\3\2\2\2\u00c0"+ - "\u00be\3\2\2\2\u00c1\u00b8\3\2\2\2\u00c1\u00c2\3\2\2\2\u00c2\u00da\3\2"+ - "\2\2\u00c3\u00c4\7Q\2\2\u00c4\u00c7\7\23\2\2\u00c5\u00c6\7\21\2\2\u00c6"+ - "\u00c8\5j\66\2\u00c7\u00c5\3\2\2\2\u00c7\u00c8\3\2\2\2\u00c8\u00cc\3\2"+ - "\2\2\u00c9\u00ca\7R\2\2\u00ca\u00cd\5\66\34\2\u00cb\u00cd\5b\62\2\u00cc"+ - "\u00c9\3\2\2\2\u00cc\u00cb\3\2\2\2\u00cc\u00cd\3\2\2\2\u00cd\u00cf\3\2"+ - "\2\2\u00ce\u00d0\5\66\34\2\u00cf\u00ce\3\2\2\2\u00cf\u00d0\3\2\2\2\u00d0"+ - "\u00da\3\2\2\2\u00d1\u00d2\7Q\2\2\u00d2\u00d7\7X\2\2\u00d3\u00d5\t\7\2"+ - "\2\u00d4\u00d3\3\2\2\2\u00d4\u00d5\3\2\2\2\u00d5\u00d6\3\2\2\2\u00d6\u00d8"+ - "\5h\65\2\u00d7\u00d4\3\2\2\2\u00d7\u00d8\3\2\2\2\u00d8\u00da\3\2\2\2\u00d9"+ - "t\3\2\2\2\u00d9u\3\2\2\2\u00d9\u0086\3\2\2\2\u00d9\u0095\3\2\2\2\u00d9"+ - "\u009b\3\2\2\2\u00d9\u00a2\3\2\2\2\u00d9\u00a7\3\2\2\2\u00d9\u00ac\3\2"+ - "\2\2\u00d9\u00ae\3\2\2\2\u00d9\u00c3\3\2\2\2\u00d9\u00d1\3\2\2\2\u00da"+ - "\7\3\2\2\2\u00db\u00dc\7\\\2\2\u00dc\u00e1\5\34\17\2\u00dd\u00de\7\5\2"+ - "\2\u00de\u00e0\5\34\17\2\u00df\u00dd\3\2\2\2\u00e0\u00e3\3\2\2\2\u00e1"+ - "\u00df\3\2\2\2\u00e1\u00e2\3\2\2\2\u00e2\u00e5\3\2\2\2\u00e3\u00e1\3\2"+ - "\2\2\u00e4\u00db\3\2\2\2\u00e4\u00e5\3\2\2\2\u00e5\u00e6\3\2\2\2\u00e6"+ - "\u00e7\5\n\6\2\u00e7\t\3\2\2\2\u00e8\u00f3\5\16\b\2\u00e9\u00ea\7D\2\2"+ - "\u00ea\u00eb\7\17\2\2\u00eb\u00f0\5\20\t\2\u00ec\u00ed\7\5\2\2\u00ed\u00ef"+ - "\5\20\t\2\u00ee\u00ec\3\2\2\2\u00ef\u00f2\3\2\2\2\u00f0\u00ee\3\2\2\2"+ - "\u00f0\u00f1\3\2\2\2\u00f1\u00f4\3\2\2\2\u00f2\u00f0\3\2\2\2\u00f3\u00e9"+ - "\3\2\2\2\u00f3\u00f4\3\2\2\2\u00f4\u00f6\3\2\2\2\u00f5\u00f7\5\f\7\2\u00f6"+ - "\u00f5\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7\13\3\2\2\2\u00f8\u00f9\7\66\2"+ - "\2\u00f9\u00fe\t\b\2\2\u00fa\u00fb\7a\2\2\u00fb\u00fc\t\b\2\2\u00fc\u00fe"+ - "\7f\2\2\u00fd\u00f8\3\2\2\2\u00fd\u00fa\3\2\2\2\u00fe\r\3\2\2\2\u00ff"+ - "\u0105\5\22\n\2\u0100\u0101\7\3\2\2\u0101\u0102\5\n\6\2\u0102\u0103\7"+ - "\4\2\2\u0103\u0105\3\2\2\2\u0104\u00ff\3\2\2\2\u0104\u0100\3\2\2\2\u0105"+ - "\17\3\2\2\2\u0106\u0108\5,\27\2\u0107\u0109\t\t\2\2\u0108\u0107\3\2\2"+ - "\2\u0108\u0109\3\2\2\2\u0109\u010c\3\2\2\2\u010a\u010b\7@\2\2\u010b\u010d"+ - "\t\n\2\2\u010c\u010a\3\2\2\2\u010c\u010d\3\2\2\2\u010d\21\3\2\2\2\u010e"+ - "\u0110\7O\2\2\u010f\u0111\5\36\20\2\u0110\u010f\3\2\2\2\u0110\u0111\3"+ - "\2\2\2\u0111\u0112\3\2\2\2\u0112\u0117\5 \21\2\u0113\u0114\7\5\2\2\u0114"+ - "\u0116\5 \21\2\u0115\u0113\3\2\2\2\u0116\u0119\3\2\2\2\u0117\u0115\3\2"+ - "\2\2\u0117\u0118\3\2\2\2\u0118\u011b\3\2\2\2\u0119\u0117\3\2\2\2\u011a"+ - "\u011c\5\24\13\2\u011b\u011a\3\2\2\2\u011b\u011c\3\2\2\2\u011c\u011f\3"+ - "\2\2\2\u011d\u011e\7[\2\2\u011e\u0120\5.\30\2\u011f\u011d\3\2\2\2\u011f"+ - "\u0120\3\2\2\2\u0120\u0124\3\2\2\2\u0121\u0122\7*\2\2\u0122\u0123\7\17"+ - "\2\2\u0123\u0125\5\26\f\2\u0124\u0121\3\2\2\2\u0124\u0125\3\2\2\2\u0125"+ - "\u0128\3\2\2\2\u0126\u0127\7+\2\2\u0127\u0129\5.\30\2\u0128\u0126\3\2"+ - "\2\2\u0128\u0129\3\2\2\2\u0129\23\3\2\2\2\u012a\u012b\7&\2\2\u012b\u0130"+ - "\5\"\22\2\u012c\u012d\7\5\2\2\u012d\u012f\5\"\22\2\u012e\u012c\3\2\2\2"+ - "\u012f\u0132\3\2\2\2\u0130\u012e\3\2\2\2\u0130\u0131\3\2\2\2\u0131\25"+ - "\3\2\2\2\u0132\u0130\3\2\2\2\u0133\u0135\5\36\20\2\u0134\u0133\3\2\2\2"+ - "\u0134\u0135\3\2\2\2\u0135\u0136\3\2\2\2\u0136\u013b\5\30\r\2\u0137\u0138"+ - "\7\5\2\2\u0138\u013a\5\30\r\2\u0139\u0137\3\2\2\2\u013a\u013d\3\2\2\2"+ - "\u013b\u0139\3\2\2\2\u013b\u013c\3\2\2\2\u013c\27\3\2\2\2\u013d\u013b"+ - "\3\2\2\2\u013e\u013f\5\32\16\2\u013f\31\3\2\2\2\u0140\u0149\7\3\2\2\u0141"+ - "\u0146\5,\27\2\u0142\u0143\7\5\2\2\u0143\u0145\5,\27\2\u0144\u0142\3\2"+ - "\2\2\u0145\u0148\3\2\2\2\u0146\u0144\3\2\2\2\u0146\u0147\3\2\2\2\u0147"+ - "\u014a\3\2\2\2\u0148\u0146\3\2\2\2\u0149\u0141\3\2\2\2\u0149\u014a\3\2"+ - "\2\2\u014a\u014b\3\2\2\2\u014b\u014e\7\4\2\2\u014c\u014e\5,\27\2\u014d"+ - "\u0140\3\2\2\2\u014d\u014c\3\2\2\2\u014e\33\3\2\2\2\u014f\u0150\5`\61"+ - "\2\u0150\u0151\7\f\2\2\u0151\u0152\7\3\2\2\u0152\u0153\5\n\6\2\u0153\u0154"+ - "\7\4\2\2\u0154\35\3\2\2\2\u0155\u0156\t\13\2\2\u0156\37\3\2\2\2\u0157"+ - "\u015c\5,\27\2\u0158\u015a\7\f\2\2\u0159\u0158\3\2\2\2\u0159\u015a\3\2"+ - "\2\2\u015a\u015b\3\2\2\2\u015b\u015d\5`\61\2\u015c\u0159\3\2\2\2\u015c"+ - "\u015d\3\2\2\2\u015d!\3\2\2\2\u015e\u0162\5*\26\2\u015f\u0161\5$\23\2"+ - "\u0160\u015f\3\2\2\2\u0161\u0164\3\2\2\2\u0162\u0160\3\2\2\2\u0162\u0163"+ - "\3\2\2\2\u0163#\3\2\2\2\u0164\u0162\3\2\2\2\u0165\u0166\5&\24\2\u0166"+ - "\u0167\7\62\2\2\u0167\u0169\5*\26\2\u0168\u016a\5(\25\2\u0169\u0168\3"+ - "\2\2\2\u0169\u016a\3\2\2\2\u016a\u0171\3\2\2\2\u016b\u016c\7=\2\2\u016c"+ - "\u016d\5&\24\2\u016d\u016e\7\62\2\2\u016e\u016f\5*\26\2\u016f\u0171\3"+ - "\2\2\2\u0170\u0165\3\2\2\2\u0170\u016b\3\2\2\2\u0171%\3\2\2\2\u0172\u0174"+ - "\7/\2\2\u0173\u0172\3\2\2\2\u0173\u0174\3\2\2\2\u0174\u0182\3\2\2\2\u0175"+ - "\u0177\7\64\2\2\u0176\u0178\7E\2\2\u0177\u0176\3\2\2\2\u0177\u0178\3\2"+ - "\2\2\u0178\u0182\3\2\2\2\u0179\u017b\7I\2\2\u017a\u017c\7E\2\2\u017b\u017a"+ - "\3\2\2\2\u017b\u017c\3\2\2\2\u017c\u0182\3\2\2\2\u017d\u017f\7\'\2\2\u017e"+ - "\u0180\7E\2\2\u017f\u017e\3\2\2\2\u017f\u0180\3\2\2\2\u0180\u0182\3\2"+ - "\2\2\u0181\u0173\3\2\2\2\u0181\u0175\3\2\2\2\u0181\u0179\3\2\2\2\u0181"+ - "\u017d\3\2\2\2\u0182\'\3\2\2\2\u0183\u0184\7A\2\2\u0184\u0192\5.\30\2"+ - "\u0185\u0186\7Y\2\2\u0186\u0187\7\3\2\2\u0187\u018c\5`\61\2\u0188\u0189"+ - "\7\5\2\2\u0189\u018b\5`\61\2\u018a\u0188\3\2\2\2\u018b\u018e\3\2\2\2\u018c"+ - "\u018a\3\2\2\2\u018c\u018d\3\2\2\2\u018d\u018f\3\2\2\2\u018e\u018c\3\2"+ - "\2\2\u018f\u0190\7\4\2\2\u0190\u0192\3\2\2\2\u0191\u0183\3\2\2\2\u0191"+ - "\u0185\3\2\2\2\u0192)\3\2\2\2\u0193\u0198\5b\62\2\u0194\u0196\7\f\2\2"+ - "\u0195\u0194\3\2\2\2\u0195\u0196\3\2\2\2\u0196\u0197\3\2\2\2\u0197\u0199"+ - "\5^\60\2\u0198\u0195\3\2\2\2\u0198\u0199\3\2\2\2\u0199\u01ad\3\2\2\2\u019a"+ - "\u019b\7\3\2\2\u019b\u019c\5\n\6\2\u019c\u01a1\7\4\2\2\u019d\u019f\7\f"+ - "\2\2\u019e\u019d\3\2\2\2\u019e\u019f\3\2\2\2\u019f\u01a0\3\2\2\2\u01a0"+ - "\u01a2\5^\60\2\u01a1\u019e\3\2\2\2\u01a1\u01a2\3\2\2\2\u01a2\u01ad\3\2"+ - "\2\2\u01a3\u01a4\7\3\2\2\u01a4\u01a5\5\"\22\2\u01a5\u01aa\7\4\2\2\u01a6"+ - "\u01a8\7\f\2\2\u01a7\u01a6\3\2\2\2\u01a7\u01a8\3\2\2\2\u01a8\u01a9\3\2"+ - "\2\2\u01a9\u01ab\5^\60\2\u01aa\u01a7\3\2\2\2\u01aa\u01ab\3\2\2\2\u01ab"+ - "\u01ad\3\2\2\2\u01ac\u0193\3\2\2\2\u01ac\u019a\3\2\2\2\u01ac\u01a3\3\2"+ - "\2\2\u01ad+\3\2\2\2\u01ae\u01af\5.\30\2\u01af-\3\2\2\2\u01b0\u01b1\b\30"+ - "\1\2\u01b1\u01b2\7>\2\2\u01b2\u01d0\5.\30\n\u01b3\u01b4\7 \2\2\u01b4\u01b5"+ - "\7\3\2\2\u01b5\u01b6\5\b\5\2\u01b6\u01b7\7\4\2\2\u01b7\u01d0\3\2\2\2\u01b8"+ - "\u01b9\7K\2\2\u01b9\u01ba\7\3\2\2\u01ba\u01bb\5j\66\2\u01bb\u01bc\5\60"+ - "\31\2\u01bc\u01bd\7\4\2\2\u01bd\u01d0\3\2\2\2\u01be\u01bf\78\2\2\u01bf"+ - "\u01c0\7\3\2\2\u01c0\u01c1\5^\60\2\u01c1\u01c2\7\5\2\2\u01c2\u01c3\5j"+ - "\66\2\u01c3\u01c4\5\60\31\2\u01c4\u01c5\7\4\2\2\u01c5\u01d0\3\2\2\2\u01c6"+ - "\u01c7\78\2\2\u01c7\u01c8\7\3\2\2\u01c8\u01c9\5j\66\2\u01c9\u01ca\7\5"+ - "\2\2\u01ca\u01cb\5j\66\2\u01cb\u01cc\5\60\31\2\u01cc\u01cd\7\4\2\2\u01cd"+ - "\u01d0\3\2\2\2\u01ce\u01d0\5\62\32\2\u01cf\u01b0\3\2\2\2\u01cf\u01b3\3"+ - "\2\2\2\u01cf\u01b8\3\2\2\2\u01cf\u01be\3\2\2\2\u01cf\u01c6\3\2\2\2\u01cf"+ - "\u01ce\3\2\2\2\u01d0\u01d9\3\2\2\2\u01d1\u01d2\f\4\2\2\u01d2\u01d3\7\n"+ - "\2\2\u01d3\u01d8\5.\30\5\u01d4\u01d5\f\3\2\2\u01d5\u01d6\7C\2\2\u01d6"+ - "\u01d8\5.\30\4\u01d7\u01d1\3\2\2\2\u01d7\u01d4\3\2\2\2\u01d8\u01db\3\2"+ - "\2\2\u01d9\u01d7\3\2\2\2\u01d9\u01da\3\2\2\2\u01da/\3\2\2\2\u01db\u01d9"+ - "\3\2\2\2\u01dc\u01dd\7\5\2\2\u01dd\u01df\5j\66\2\u01de\u01dc\3\2\2\2\u01df"+ - "\u01e2\3\2\2\2\u01e0\u01de\3\2\2\2\u01e0\u01e1\3\2\2\2\u01e1\61\3\2\2"+ - "\2\u01e2\u01e0\3\2\2\2\u01e3\u01e5\5<\37\2\u01e4\u01e6\5\64\33\2\u01e5"+ - "\u01e4\3\2\2\2\u01e5\u01e6\3\2\2\2\u01e6\63\3\2\2\2\u01e7\u01e9\7>\2\2"+ - "\u01e8\u01e7\3\2\2\2\u01e8\u01e9\3\2\2\2\u01e9\u01ea\3\2\2\2\u01ea\u01eb"+ - "\7\16\2\2\u01eb\u01ec\5<\37\2\u01ec\u01ed\7\n\2\2\u01ed\u01ee\5<\37\2"+ - "\u01ee\u0216\3\2\2\2\u01ef\u01f1\7>\2\2\u01f0\u01ef\3\2\2\2\u01f0\u01f1"+ - "\3\2\2\2\u01f1\u01f2\3\2\2\2\u01f2\u01f3\7.\2\2\u01f3\u01f4\7\3\2\2\u01f4"+ - "\u01f9\5<\37\2\u01f5\u01f6\7\5\2\2\u01f6\u01f8\5<\37\2\u01f7\u01f5\3\2"+ - "\2\2\u01f8\u01fb\3\2\2\2\u01f9\u01f7\3\2\2\2\u01f9\u01fa\3\2\2\2\u01fa"+ - "\u01fc\3\2\2\2\u01fb\u01f9\3\2\2\2\u01fc\u01fd\7\4\2\2\u01fd\u0216\3\2"+ - "\2\2\u01fe\u0200\7>\2\2\u01ff\u01fe\3\2\2\2\u01ff\u0200\3\2\2\2\u0200"+ - "\u0201\3\2\2\2\u0201\u0202\7.\2\2\u0202\u0203\7\3\2\2\u0203\u0204\5\b"+ - "\5\2\u0204\u0205\7\4\2\2\u0205\u0216\3\2\2\2\u0206\u0208\7>\2\2\u0207"+ - "\u0206\3\2\2\2\u0207\u0208\3\2\2\2\u0208\u0209\3\2\2\2\u0209\u020a\7\65"+ - "\2\2\u020a\u0216\58\35\2\u020b\u020d\7>\2\2\u020c\u020b\3\2\2\2\u020c"+ - "\u020d\3\2\2\2\u020d\u020e\3\2\2\2\u020e\u020f\7J\2\2\u020f\u0216\5j\66"+ - "\2\u0210\u0212\7\61\2\2\u0211\u0213\7>\2\2\u0212\u0211\3\2\2\2\u0212\u0213"+ - "\3\2\2\2\u0213\u0214\3\2\2\2\u0214\u0216\7?\2\2\u0215\u01e8\3\2\2\2\u0215"+ - "\u01f0\3\2\2\2\u0215\u01ff\3\2\2\2\u0215\u0207\3\2\2\2\u0215\u020c\3\2"+ - "\2\2\u0215\u0210\3\2\2\2\u0216\65\3\2\2\2\u0217\u0218\7\65\2\2\u0218\u0219"+ - "\58\35\2\u0219\67\3\2\2\2\u021a\u021c\5j\66\2\u021b\u021d\5:\36\2\u021c"+ - "\u021b\3\2\2\2\u021c\u021d\3\2\2\2\u021d9\3\2\2\2\u021e\u021f\7\36\2\2"+ - "\u021f\u0225\5j\66\2\u0220\u0221\7_\2\2\u0221\u0222\5j\66\2\u0222\u0223"+ - "\7f\2\2\u0223\u0225\3\2\2\2\u0224\u021e\3\2\2\2\u0224\u0220\3\2\2\2\u0225"+ - ";\3\2\2\2\u0226\u0227\b\37\1\2\u0227\u022b\5> \2\u0228\u0229\t\7\2\2\u0229"+ - "\u022b\5<\37\6\u022a\u0226\3\2\2\2\u022a\u0228\3\2\2\2\u022b\u0238\3\2"+ - "\2\2\u022c\u022d\f\5\2\2\u022d\u022e\t\f\2\2\u022e\u0237\5<\37\6\u022f"+ - "\u0230\f\4\2\2\u0230\u0231\t\7\2\2\u0231\u0237\5<\37\5\u0232\u0233\f\3"+ - "\2\2\u0233\u0234\5T+\2\u0234\u0235\5<\37\4\u0235\u0237\3\2\2\2\u0236\u022c"+ - "\3\2\2\2\u0236\u022f\3\2\2\2\u0236\u0232\3\2\2\2\u0237\u023a\3\2\2\2\u0238"+ - "\u0236\3\2\2\2\u0238\u0239\3\2\2\2\u0239=\3\2\2\2\u023a\u0238\3\2\2\2"+ - "\u023b\u023c\b \1\2\u023c\u0251\5B\"\2\u023d\u0251\5H%\2\u023e\u0251\5"+ - "@!\2\u023f\u0251\5R*\2\u0240\u0241\5^\60\2\u0241\u0242\7u\2\2\u0242\u0244"+ - "\3\2\2\2\u0243\u0240\3\2\2\2\u0243\u0244\3\2\2\2\u0244\u0245\3\2\2\2\u0245"+ - "\u0251\7p\2\2\u0246\u0251\5L\'\2\u0247\u0248\7\3\2\2\u0248\u0249\5\b\5"+ - "\2\u0249\u024a\7\4\2\2\u024a\u0251\3\2\2\2\u024b\u0251\5^\60\2\u024c\u024d"+ - "\7\3\2\2\u024d\u024e\5,\27\2\u024e\u024f\7\4\2\2\u024f\u0251\3\2\2\2\u0250"+ - "\u023b\3\2\2\2\u0250\u023d\3\2\2\2\u0250\u023e\3\2\2\2\u0250\u023f\3\2"+ - "\2\2\u0250\u0243\3\2\2\2\u0250\u0246\3\2\2\2\u0250\u0247\3\2\2\2\u0250"+ - "\u024b\3\2\2\2\u0250\u024c\3\2\2\2\u0251\u0257\3\2\2\2\u0252\u0253\f\13"+ - "\2\2\u0253\u0254\7s\2\2\u0254\u0256\5\\/\2\u0255\u0252\3\2\2\2\u0256\u0259"+ - "\3\2\2\2\u0257\u0255\3\2\2\2\u0257\u0258\3\2\2\2\u0258?\3\2\2\2\u0259"+ - "\u0257\3\2\2\2\u025a\u025e\7\27\2\2\u025b\u025e\7\25\2\2\u025c\u025e\7"+ - "\26\2\2\u025d\u025a\3\2\2\2\u025d\u025b\3\2\2\2\u025d\u025c\3\2\2\2\u025e"+ - "A\3\2\2\2\u025f\u026a\5D#\2\u0260\u0261\7`\2\2\u0261\u0262\5D#\2\u0262"+ - "\u0263\7f\2\2\u0263\u026a\3\2\2\2\u0264\u026a\5F$\2\u0265\u0266\7`\2\2"+ - "\u0266\u0267\5F$\2\u0267\u0268\7f\2\2\u0268\u026a\3\2\2\2\u0269\u025f"+ - "\3\2\2\2\u0269\u0260\3\2\2\2\u0269\u0264\3\2\2\2\u0269\u0265\3\2\2\2\u026a"+ - "C\3\2\2\2\u026b\u026c\7\20\2\2\u026c\u026d\7\3\2\2\u026d\u026e\5,\27\2"+ - "\u026e\u026f\7\f\2\2\u026f\u0270\5\\/\2\u0270\u0271\7\4\2\2\u0271E\3\2"+ - "\2\2\u0272\u0273\7\24\2\2\u0273\u0274\7\3\2\2\u0274\u0275\5,\27\2\u0275"+ - "\u0276\7\5\2\2\u0276\u0277\5\\/\2\u0277\u0278\7\4\2\2\u0278G\3\2\2\2\u0279"+ - "\u027f\5J&\2\u027a\u027b\7`\2\2\u027b\u027c\5J&\2\u027c\u027d\7f\2\2\u027d"+ - "\u027f\3\2\2\2\u027e\u0279\3\2\2\2\u027e\u027a\3\2\2\2\u027fI\3\2\2\2"+ - "\u0280\u0281\7\"\2\2\u0281\u0282\7\3\2\2\u0282\u0283\5`\61\2\u0283\u0284"+ - "\7&\2\2\u0284\u0285\5<\37\2\u0285\u0286\7\4\2\2\u0286K\3\2\2\2\u0287\u028d"+ - "\5N(\2\u0288\u0289\7`\2\2\u0289\u028a\5N(\2\u028a\u028b\7f\2\2\u028b\u028d"+ - "\3\2\2\2\u028c\u0287\3\2\2\2\u028c\u0288\3\2\2\2\u028dM\3\2\2\2\u028e"+ - "\u028f\5P)\2\u028f\u029b\7\3\2\2\u0290\u0292\5\36\20\2\u0291\u0290\3\2"+ - "\2\2\u0291\u0292\3\2\2\2\u0292\u0293\3\2\2\2\u0293\u0298\5,\27\2\u0294"+ - "\u0295\7\5\2\2\u0295\u0297\5,\27\2\u0296\u0294\3\2\2\2\u0297\u029a\3\2"+ - "\2\2\u0298\u0296\3\2\2\2\u0298\u0299\3\2\2\2\u0299\u029c\3\2\2\2\u029a"+ - "\u0298\3\2\2\2\u029b\u0291\3\2\2\2\u029b\u029c\3\2\2\2\u029c\u029d\3\2"+ - "\2\2\u029d\u029e\7\4\2\2\u029eO\3\2\2\2\u029f\u02a3\7\64\2\2\u02a0\u02a3"+ - "\7I\2\2\u02a1\u02a3\5`\61\2\u02a2\u029f\3\2\2\2\u02a2\u02a0\3\2\2\2\u02a2"+ - "\u02a1\3\2\2\2\u02a3Q\3\2\2\2\u02a4\u02bf\7?\2\2\u02a5\u02bf\5X-\2\u02a6"+ - "\u02bf\5h\65\2\u02a7\u02bf\5V,\2\u02a8\u02aa\7w\2\2\u02a9\u02a8\3\2\2"+ - "\2\u02aa\u02ab\3\2\2\2\u02ab\u02a9\3\2\2\2\u02ab\u02ac\3\2\2\2\u02ac\u02bf"+ - "\3\2\2\2\u02ad\u02bf\7v\2\2\u02ae\u02af\7b\2\2\u02af\u02b0\5j\66\2\u02b0"+ - "\u02b1\7f\2\2\u02b1\u02bf\3\2\2\2\u02b2\u02b3\7c\2\2\u02b3\u02b4\5j\66"+ - "\2\u02b4\u02b5\7f\2\2\u02b5\u02bf\3\2\2\2\u02b6\u02b7\7d\2\2\u02b7\u02b8"+ - "\5j\66\2\u02b8\u02b9\7f\2\2\u02b9\u02bf\3\2\2\2\u02ba\u02bb\7e\2\2\u02bb"+ - "\u02bc\5j\66\2\u02bc\u02bd\7f\2\2\u02bd\u02bf\3\2\2\2\u02be\u02a4\3\2"+ - "\2\2\u02be\u02a5\3\2\2\2\u02be\u02a6\3\2\2\2\u02be\u02a7\3\2\2\2\u02be"+ - "\u02a9\3\2\2\2\u02be\u02ad\3\2\2\2\u02be\u02ae\3\2\2\2\u02be\u02b2\3\2"+ - "\2\2\u02be\u02b6\3\2\2\2\u02be\u02ba\3\2\2\2\u02bfS\3\2\2\2\u02c0\u02c1"+ - "\t\r\2\2\u02c1U\3\2\2\2\u02c2\u02c3\t\16\2\2\u02c3W\3\2\2\2\u02c4\u02c6"+ - "\7\60\2\2\u02c5\u02c7\t\7\2\2\u02c6\u02c5\3\2\2\2\u02c6\u02c7\3\2\2\2"+ - "\u02c7\u02ca\3\2\2\2\u02c8\u02cb\5h\65\2\u02c9\u02cb\5j\66\2\u02ca\u02c8"+ - "\3\2\2\2\u02ca\u02c9\3\2\2\2\u02cb\u02cc\3\2\2\2\u02cc\u02cf\5Z.\2\u02cd"+ - "\u02ce\7V\2\2\u02ce\u02d0\5Z.\2\u02cf\u02cd\3\2\2\2\u02cf\u02d0\3\2\2"+ - "\2\u02d0Y\3\2\2\2\u02d1\u02d2\t\17\2\2\u02d2[\3\2\2\2\u02d3\u02d4\5`\61"+ - "\2\u02d4]\3\2\2\2\u02d5\u02d6\5`\61\2\u02d6\u02d7\7u\2\2\u02d7\u02d9\3"+ - "\2\2\2\u02d8\u02d5\3\2\2\2\u02d9\u02dc\3\2\2\2\u02da\u02d8\3\2\2\2\u02da"+ - "\u02db\3\2\2\2\u02db\u02dd\3\2\2\2\u02dc\u02da\3\2\2\2\u02dd\u02de\5`"+ - "\61\2\u02de_\3\2\2\2\u02df\u02e2\5d\63\2\u02e0\u02e2\5f\64\2\u02e1\u02df"+ - "\3\2\2\2\u02e1\u02e0\3\2\2\2\u02e2a\3\2\2\2\u02e3\u02e4\5`\61\2\u02e4"+ - "\u02e5\7\6\2\2\u02e5\u02e7\3\2\2\2\u02e6\u02e3\3\2\2\2\u02e6\u02e7\3\2"+ - "\2\2\u02e7\u02e8\3\2\2\2\u02e8\u02f0\7|\2\2\u02e9\u02ea\5`\61\2\u02ea"+ - "\u02eb\7\6\2\2\u02eb\u02ed\3\2\2\2\u02ec\u02e9\3\2\2\2\u02ec\u02ed\3\2"+ - "\2\2\u02ed\u02ee\3\2\2\2\u02ee\u02f0\5`\61\2\u02ef\u02e6\3\2\2\2\u02ef"+ - "\u02ec\3\2\2\2\u02f0c\3\2\2\2\u02f1\u02f4\7}\2\2\u02f2\u02f4\7~\2\2\u02f3"+ - "\u02f1\3\2\2\2\u02f3\u02f2\3\2\2\2\u02f4e\3\2\2\2\u02f5\u02f9\7z\2\2\u02f6"+ - "\u02f9\5l\67\2\u02f7\u02f9\7{\2\2\u02f8\u02f5\3\2\2\2\u02f8\u02f6\3\2"+ - "\2\2\u02f8\u02f7\3\2\2\2\u02f9g\3\2\2\2\u02fa\u02fd\7y\2\2\u02fb\u02fd"+ - "\7x\2\2\u02fc\u02fa\3\2\2\2\u02fc\u02fb\3\2\2\2\u02fdi\3\2\2\2\u02fe\u02ff"+ - "\t\20\2\2\u02ffk\3\2\2\2\u0300\u0301\t\21\2\2\u0301m\3\2\2\2h}\177\u0083"+ - "\u008c\u008e\u0092\u0099\u00a0\u00a5\u00aa\u00b2\u00b6\u00be\u00c1\u00c7"+ - "\u00cc\u00cf\u00d4\u00d7\u00d9\u00e1\u00e4\u00f0\u00f3\u00f6\u00fd\u0104"+ - "\u0108\u010c\u0110\u0117\u011b\u011f\u0124\u0128\u0130\u0134\u013b\u0146"+ - "\u0149\u014d\u0159\u015c\u0162\u0169\u0170\u0173\u0177\u017b\u017f\u0181"+ - "\u018c\u0191\u0195\u0198\u019e\u01a1\u01a7\u01aa\u01ac\u01cf\u01d7\u01d9"+ - "\u01e0\u01e5\u01e8\u01f0\u01f9\u01ff\u0207\u020c\u0212\u0215\u021c\u0224"+ - "\u022a\u0236\u0238\u0243\u0250\u0257\u025d\u0269\u027e\u028c\u0291\u0298"+ - "\u029b\u02a2\u02ab\u02be\u02c6\u02ca\u02cf\u02da\u02e1\u02e6\u02ec\u02ef"+ - "\u02f3\u02f8\u02fc"; + "\3\30\3\30\3\30\5\30\u01d2\n\30\3\30\3\30\3\30\3\30\3\30\3\30\7\30\u01da"+ + "\n\30\f\30\16\30\u01dd\13\30\3\31\3\31\7\31\u01e1\n\31\f\31\16\31\u01e4"+ + "\13\31\3\32\3\32\5\32\u01e8\n\32\3\33\5\33\u01eb\n\33\3\33\3\33\3\33\3"+ + "\33\3\33\3\33\5\33\u01f3\n\33\3\33\3\33\3\33\3\33\3\33\7\33\u01fa\n\33"+ + "\f\33\16\33\u01fd\13\33\3\33\3\33\3\33\5\33\u0202\n\33\3\33\3\33\3\33"+ + "\3\33\3\33\3\33\5\33\u020a\n\33\3\33\3\33\3\33\5\33\u020f\n\33\3\33\3"+ + "\33\3\33\3\33\5\33\u0215\n\33\3\33\5\33\u0218\n\33\3\34\3\34\3\34\3\35"+ + "\3\35\5\35\u021f\n\35\3\36\3\36\3\36\3\36\3\36\3\36\5\36\u0227\n\36\3"+ + "\37\3\37\3\37\3\37\5\37\u022d\n\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37"+ + "\3\37\3\37\3\37\7\37\u0239\n\37\f\37\16\37\u023c\13\37\3 \3 \3 \3 \3 "+ + "\3 \3 \3 \5 \u0246\n \3 \3 \3 \3 \3 \3 \3 \3 \3 \3 \3 \3 \3 \5 \u0255"+ + "\n \3 \6 \u0258\n \r \16 \u0259\3 \3 \5 \u025e\n \3 \3 \5 \u0262\n \3"+ + " \3 \3 \7 \u0267\n \f \16 \u026a\13 \3!\3!\3!\5!\u026f\n!\3\"\3\"\3\""+ + "\3\"\3\"\3\"\3\"\3\"\3\"\3\"\5\"\u027b\n\"\3#\3#\3#\3#\3#\3#\3#\3$\3$"+ + "\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\5%\u0290\n%\3&\3&\3&\3&\3&\3&\3&\3\'\3"+ + "\'\3\'\3\'\3\'\5\'\u029e\n\'\3(\3(\3(\5(\u02a3\n(\3(\3(\3(\7(\u02a8\n"+ + "(\f(\16(\u02ab\13(\5(\u02ad\n(\3(\3(\3)\3)\3)\5)\u02b4\n)\3*\3*\3*\3*"+ + "\3*\6*\u02bb\n*\r*\16*\u02bc\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3"+ + "*\3*\3*\3*\5*\u02d0\n*\3+\3+\3,\3,\3-\3-\5-\u02d8\n-\3-\3-\5-\u02dc\n"+ + "-\3-\3-\3-\5-\u02e1\n-\3.\3.\3/\3/\3\60\3\60\3\60\7\60\u02ea\n\60\f\60"+ + "\16\60\u02ed\13\60\3\60\3\60\3\61\3\61\5\61\u02f3\n\61\3\62\3\62\3\62"+ + "\5\62\u02f8\n\62\3\62\3\62\3\62\3\62\5\62\u02fe\n\62\3\62\5\62\u0301\n"+ + "\62\3\63\3\63\5\63\u0305\n\63\3\64\3\64\3\64\5\64\u030a\n\64\3\65\3\65"+ + "\5\65\u030e\n\65\3\66\3\66\3\67\3\67\3\67\3\67\3\67\38\38\38\2\5.<>9\2"+ + "\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,.\60\62\64\668:<>@BDFHJL"+ + "NPRTVXZ\\^`bdfhjln\2\22\b\2\7\7\t\t\"\"::EEII\4\2,,WW\4\2\t\tEE\4\2))"+ + "\61\61\3\2\34\35\3\2st\4\2\7\7}}\4\2\r\r\34\34\4\2\'\'\66\66\4\2\7\7\36"+ + "\36\3\2uw\3\2lr\4\2&&YY\7\2\31\32/\60>EEIKMPSTVW[\\^^bb\u0378\2p\3\2\2"+ + "\2\4s\3\2\2\2\6\u00db\3\2\2\2\b\u00e6\3\2\2\2\n\u00ea\3\2\2\2\f\u00ff"+ + "\3\2\2\2\16\u0106\3\2\2\2\20\u0108\3\2\2\2\22\u0110\3\2\2\2\24\u012c\3"+ + "\2\2\2\26\u0136\3\2\2\2\30\u0140\3\2\2\2\32\u014f\3\2\2\2\34\u0151\3\2"+ + "\2\2\36\u0157\3\2\2\2 \u0159\3\2\2\2\"\u0160\3\2\2\2$\u0172\3\2\2\2&\u0183"+ + "\3\2\2\2(\u0193\3\2\2\2*\u01ae\3\2\2\2,\u01b0\3\2\2\2.\u01d1\3\2\2\2\60"+ + "\u01e2\3\2\2\2\62\u01e5\3\2\2\2\64\u0217\3\2\2\2\66\u0219\3\2\2\28\u021c"+ + "\3\2\2\2:\u0226\3\2\2\2<\u022c\3\2\2\2>\u0261\3\2\2\2@\u026e\3\2\2\2B"+ + "\u027a\3\2\2\2D\u027c\3\2\2\2F\u0283\3\2\2\2H\u028f\3\2\2\2J\u0291\3\2"+ + "\2\2L\u029d\3\2\2\2N\u029f\3\2\2\2P\u02b3\3\2\2\2R\u02cf\3\2\2\2T\u02d1"+ + "\3\2\2\2V\u02d3\3\2\2\2X\u02d5\3\2\2\2Z\u02e2\3\2\2\2\\\u02e4\3\2\2\2"+ + "^\u02eb\3\2\2\2`\u02f2\3\2\2\2b\u0300\3\2\2\2d\u0304\3\2\2\2f\u0309\3"+ + "\2\2\2h\u030d\3\2\2\2j\u030f\3\2\2\2l\u0311\3\2\2\2n\u0316\3\2\2\2pq\5"+ + "\6\4\2qr\7\2\2\3r\3\3\2\2\2st\5,\27\2tu\7\2\2\3u\5\3\2\2\2v\u00dc\5\b"+ + "\5\2w\u0085\7$\2\2x\u0081\7\3\2\2yz\7K\2\2z\u0080\t\2\2\2{|\7(\2\2|\u0080"+ + "\t\3\2\2}~\7^\2\2~\u0080\5V,\2\177y\3\2\2\2\177{\3\2\2\2\177}\3\2\2\2"+ + "\u0080\u0083\3\2\2\2\u0081\177\3\2\2\2\u0081\u0082\3\2\2\2\u0082\u0084"+ + "\3\2\2\2\u0083\u0081\3\2\2\2\u0084\u0086\7\4\2\2\u0085x\3\2\2\2\u0085"+ + "\u0086\3\2\2\2\u0086\u0087\3\2\2\2\u0087\u00dc\5\6\4\2\u0088\u0094\7\33"+ + "\2\2\u0089\u0090\7\3\2\2\u008a\u008b\7K\2\2\u008b\u008f\t\4\2\2\u008c"+ + "\u008d\7(\2\2\u008d\u008f\t\3\2\2\u008e\u008a\3\2\2\2\u008e\u008c\3\2"+ + "\2\2\u008f\u0092\3\2\2\2\u0090\u008e\3\2\2\2\u0090\u0091\3\2\2\2\u0091"+ + "\u0093\3\2\2\2\u0092\u0090\3\2\2\2\u0093\u0095\7\4\2\2\u0094\u0089\3\2"+ + "\2\2\u0094\u0095\3\2\2\2\u0095\u0096\3\2\2\2\u0096\u00dc\5\6\4\2\u0097"+ + "\u0098\7S\2\2\u0098\u009b\7V\2\2\u0099\u009c\5\66\34\2\u009a\u009c\5b"+ + "\62\2\u009b\u0099\3\2\2\2\u009b\u009a\3\2\2\2\u009b\u009c\3\2\2\2\u009c"+ + "\u00dc\3\2\2\2\u009d\u009e\7S\2\2\u009e\u009f\7\24\2\2\u009f\u00a2\t\5"+ + "\2\2\u00a0\u00a3\5\66\34\2\u00a1\u00a3\5b\62\2\u00a2\u00a0\3\2\2\2\u00a2"+ + "\u00a1\3\2\2\2\u00a3\u00dc\3\2\2\2\u00a4\u00a7\t\6\2\2\u00a5\u00a8\5\66"+ + "\34\2\u00a6\u00a8\5b\62\2\u00a7\u00a5\3\2\2\2\u00a7\u00a6\3\2\2\2\u00a8"+ + "\u00dc\3\2\2\2\u00a9\u00aa\7S\2\2\u00aa\u00ac\7+\2\2\u00ab\u00ad\5\66"+ + "\34\2\u00ac\u00ab\3\2\2\2\u00ac\u00ad\3\2\2\2\u00ad\u00dc\3\2\2\2\u00ae"+ + "\u00af\7S\2\2\u00af\u00dc\7O\2\2\u00b0\u00b1\7T\2\2\u00b1\u00b4\7V\2\2"+ + "\u00b2\u00b3\7\22\2\2\u00b3\u00b5\5\66\34\2\u00b4\u00b2\3\2\2\2\u00b4"+ + "\u00b5\3\2\2\2\u00b5\u00b8\3\2\2\2\u00b6\u00b9\5\66\34\2\u00b7\u00b9\5"+ + "b\62\2\u00b8\u00b6\3\2\2\2\u00b8\u00b7\3\2\2\2\u00b8\u00b9\3\2\2\2\u00b9"+ + "\u00c3\3\2\2\2\u00ba\u00bb\7[\2\2\u00bb\u00c0\5j\66\2\u00bc\u00bd\7\5"+ + "\2\2\u00bd\u00bf\5j\66\2\u00be\u00bc\3\2\2\2\u00bf\u00c2\3\2\2\2\u00c0"+ + "\u00be\3\2\2\2\u00c0\u00c1\3\2\2\2\u00c1\u00c4\3\2\2\2\u00c2\u00c0\3\2"+ + "\2\2\u00c3\u00ba\3\2\2\2\u00c3\u00c4\3\2\2\2\u00c4\u00dc\3\2\2\2\u00c5"+ + "\u00c6\7T\2\2\u00c6\u00c9\7\24\2\2\u00c7\u00c8\7\22\2\2\u00c8\u00ca\5"+ + "j\66\2\u00c9\u00c7\3\2\2\2\u00c9\u00ca\3\2\2\2\u00ca\u00ce\3\2\2\2\u00cb"+ + "\u00cc\7U\2\2\u00cc\u00cf\5\66\34\2\u00cd\u00cf\5b\62\2\u00ce\u00cb\3"+ + "\2\2\2\u00ce\u00cd\3\2\2\2\u00ce\u00cf\3\2\2\2\u00cf\u00d1\3\2\2\2\u00d0"+ + "\u00d2\5\66\34\2\u00d1\u00d0\3\2\2\2\u00d1\u00d2\3\2\2\2\u00d2\u00dc\3"+ + "\2\2\2\u00d3\u00d4\7T\2\2\u00d4\u00d9\7\\\2\2\u00d5\u00d7\t\7\2\2\u00d6"+ + "\u00d5\3\2\2\2\u00d6\u00d7\3\2\2\2\u00d7\u00d8\3\2\2\2\u00d8\u00da\5h"+ + "\65\2\u00d9\u00d6\3\2\2\2\u00d9\u00da\3\2\2\2\u00da\u00dc\3\2\2\2\u00db"+ + "v\3\2\2\2\u00dbw\3\2\2\2\u00db\u0088\3\2\2\2\u00db\u0097\3\2\2\2\u00db"+ + "\u009d\3\2\2\2\u00db\u00a4\3\2\2\2\u00db\u00a9\3\2\2\2\u00db\u00ae\3\2"+ + "\2\2\u00db\u00b0\3\2\2\2\u00db\u00c5\3\2\2\2\u00db\u00d3\3\2\2\2\u00dc"+ + "\7\3\2\2\2\u00dd\u00de\7a\2\2\u00de\u00e3\5\34\17\2\u00df\u00e0\7\5\2"+ + "\2\u00e0\u00e2\5\34\17\2\u00e1\u00df\3\2\2\2\u00e2\u00e5\3\2\2\2\u00e3"+ + "\u00e1\3\2\2\2\u00e3\u00e4\3\2\2\2\u00e4\u00e7\3\2\2\2\u00e5\u00e3\3\2"+ + "\2\2\u00e6\u00dd\3\2\2\2\u00e6\u00e7\3\2\2\2\u00e7\u00e8\3\2\2\2\u00e8"+ + "\u00e9\5\n\6\2\u00e9\t\3\2\2\2\u00ea\u00f5\5\16\b\2\u00eb\u00ec\7G\2\2"+ + "\u00ec\u00ed\7\17\2\2\u00ed\u00f2\5\20\t\2\u00ee\u00ef\7\5\2\2\u00ef\u00f1"+ + "\5\20\t\2\u00f0\u00ee\3\2\2\2\u00f1\u00f4\3\2\2\2\u00f2\u00f0\3\2\2\2"+ + "\u00f2\u00f3\3\2\2\2\u00f3\u00f6\3\2\2\2\u00f4\u00f2\3\2\2\2\u00f5\u00eb"+ + "\3\2\2\2\u00f5\u00f6\3\2\2\2\u00f6\u00f8\3\2\2\2\u00f7\u00f9\5\f\7\2\u00f8"+ + "\u00f7\3\2\2\2\u00f8\u00f9\3\2\2\2\u00f9\13\3\2\2\2\u00fa\u00fb\79\2\2"+ + "\u00fb\u0100\t\b\2\2\u00fc\u00fd\7f\2\2\u00fd\u00fe\t\b\2\2\u00fe\u0100"+ + "\7k\2\2\u00ff\u00fa\3\2\2\2\u00ff\u00fc\3\2\2\2\u0100\r\3\2\2\2\u0101"+ + "\u0107\5\22\n\2\u0102\u0103\7\3\2\2\u0103\u0104\5\n\6\2\u0104\u0105\7"+ + "\4\2\2\u0105\u0107\3\2\2\2\u0106\u0101\3\2\2\2\u0106\u0102\3\2\2\2\u0107"+ + "\17\3\2\2\2\u0108\u010a\5,\27\2\u0109\u010b\t\t\2\2\u010a\u0109\3\2\2"+ + "\2\u010a\u010b\3\2\2\2\u010b\u010e\3\2\2\2\u010c\u010d\7C\2\2\u010d\u010f"+ + "\t\n\2\2\u010e\u010c\3\2\2\2\u010e\u010f\3\2\2\2\u010f\21\3\2\2\2\u0110"+ + "\u0112\7R\2\2\u0111\u0113\5\36\20\2\u0112\u0111\3\2\2\2\u0112\u0113\3"+ + "\2\2\2\u0113\u0114\3\2\2\2\u0114\u0119\5 \21\2\u0115\u0116\7\5\2\2\u0116"+ + "\u0118\5 \21\2\u0117\u0115\3\2\2\2\u0118\u011b\3\2\2\2\u0119\u0117\3\2"+ + "\2\2\u0119\u011a\3\2\2\2\u011a\u011d\3\2\2\2\u011b\u0119\3\2\2\2\u011c"+ + "\u011e\5\24\13\2\u011d\u011c\3\2\2\2\u011d\u011e\3\2\2\2\u011e\u0121\3"+ + "\2\2\2\u011f\u0120\7`\2\2\u0120\u0122\5.\30\2\u0121\u011f\3\2\2\2\u0121"+ + "\u0122\3\2\2\2\u0122\u0126\3\2\2\2\u0123\u0124\7-\2\2\u0124\u0125\7\17"+ + "\2\2\u0125\u0127\5\26\f\2\u0126\u0123\3\2\2\2\u0126\u0127\3\2\2\2\u0127"+ + "\u012a\3\2\2\2\u0128\u0129\7.\2\2\u0129\u012b\5.\30\2\u012a\u0128\3\2"+ + "\2\2\u012a\u012b\3\2\2\2\u012b\23\3\2\2\2\u012c\u012d\7)\2\2\u012d\u0132"+ + "\5\"\22\2\u012e\u012f\7\5\2\2\u012f\u0131\5\"\22\2\u0130\u012e\3\2\2\2"+ + "\u0131\u0134\3\2\2\2\u0132\u0130\3\2\2\2\u0132\u0133\3\2\2\2\u0133\25"+ + "\3\2\2\2\u0134\u0132\3\2\2\2\u0135\u0137\5\36\20\2\u0136\u0135\3\2\2\2"+ + "\u0136\u0137\3\2\2\2\u0137\u0138\3\2\2\2\u0138\u013d\5\30\r\2\u0139\u013a"+ + "\7\5\2\2\u013a\u013c\5\30\r\2\u013b\u0139\3\2\2\2\u013c\u013f\3\2\2\2"+ + "\u013d\u013b\3\2\2\2\u013d\u013e\3\2\2\2\u013e\27\3\2\2\2\u013f\u013d"+ + "\3\2\2\2\u0140\u0141\5\32\16\2\u0141\31\3\2\2\2\u0142\u014b\7\3\2\2\u0143"+ + "\u0148\5,\27\2\u0144\u0145\7\5\2\2\u0145\u0147\5,\27\2\u0146\u0144\3\2"+ + "\2\2\u0147\u014a\3\2\2\2\u0148\u0146\3\2\2\2\u0148\u0149\3\2\2\2\u0149"+ + "\u014c\3\2\2\2\u014a\u0148\3\2\2\2\u014b\u0143\3\2\2\2\u014b\u014c\3\2"+ + "\2\2\u014c\u014d\3\2\2\2\u014d\u0150\7\4\2\2\u014e\u0150\5,\27\2\u014f"+ + "\u0142\3\2\2\2\u014f\u014e\3\2\2\2\u0150\33\3\2\2\2\u0151\u0152\5`\61"+ + "\2\u0152\u0153\7\f\2\2\u0153\u0154\7\3\2\2\u0154\u0155\5\n\6\2\u0155\u0156"+ + "\7\4\2\2\u0156\35\3\2\2\2\u0157\u0158\t\13\2\2\u0158\37\3\2\2\2\u0159"+ + "\u015e\5,\27\2\u015a\u015c\7\f\2\2\u015b\u015a\3\2\2\2\u015b\u015c\3\2"+ + "\2\2\u015c\u015d\3\2\2\2\u015d\u015f\5`\61\2\u015e\u015b\3\2\2\2\u015e"+ + "\u015f\3\2\2\2\u015f!\3\2\2\2\u0160\u0164\5*\26\2\u0161\u0163\5$\23\2"+ + "\u0162\u0161\3\2\2\2\u0163\u0166\3\2\2\2\u0164\u0162\3\2\2\2\u0164\u0165"+ + "\3\2\2\2\u0165#\3\2\2\2\u0166\u0164\3\2\2\2\u0167\u0168\5&\24\2\u0168"+ + "\u0169\7\65\2\2\u0169\u016b\5*\26\2\u016a\u016c\5(\25\2\u016b\u016a\3"+ + "\2\2\2\u016b\u016c\3\2\2\2\u016c\u0173\3\2\2\2\u016d\u016e\7@\2\2\u016e"+ + "\u016f\5&\24\2\u016f\u0170\7\65\2\2\u0170\u0171\5*\26\2\u0171\u0173\3"+ + "\2\2\2\u0172\u0167\3\2\2\2\u0172\u016d\3\2\2\2\u0173%\3\2\2\2\u0174\u0176"+ + "\7\62\2\2\u0175\u0174\3\2\2\2\u0175\u0176\3\2\2\2\u0176\u0184\3\2\2\2"+ + "\u0177\u0179\7\67\2\2\u0178\u017a\7H\2\2\u0179\u0178\3\2\2\2\u0179\u017a"+ + "\3\2\2\2\u017a\u0184\3\2\2\2\u017b\u017d\7L\2\2\u017c\u017e\7H\2\2\u017d"+ + "\u017c\3\2\2\2\u017d\u017e\3\2\2\2\u017e\u0184\3\2\2\2\u017f\u0181\7*"+ + "\2\2\u0180\u0182\7H\2\2\u0181\u0180\3\2\2\2\u0181\u0182\3\2\2\2\u0182"+ + "\u0184\3\2\2\2\u0183\u0175\3\2\2\2\u0183\u0177\3\2\2\2\u0183\u017b\3\2"+ + "\2\2\u0183\u017f\3\2\2\2\u0184\'\3\2\2\2\u0185\u0186\7D\2\2\u0186\u0194"+ + "\5.\30\2\u0187\u0188\7]\2\2\u0188\u0189\7\3\2\2\u0189\u018e\5`\61\2\u018a"+ + "\u018b\7\5\2\2\u018b\u018d\5`\61\2\u018c\u018a\3\2\2\2\u018d\u0190\3\2"+ + "\2\2\u018e\u018c\3\2\2\2\u018e\u018f\3\2\2\2\u018f\u0191\3\2\2\2\u0190"+ + "\u018e\3\2\2\2\u0191\u0192\7\4\2\2\u0192\u0194\3\2\2\2\u0193\u0185\3\2"+ + "\2\2\u0193\u0187\3\2\2\2\u0194)\3\2\2\2\u0195\u019a\5b\62\2\u0196\u0198"+ + "\7\f\2\2\u0197\u0196\3\2\2\2\u0197\u0198\3\2\2\2\u0198\u0199\3\2\2\2\u0199"+ + "\u019b\5^\60\2\u019a\u0197\3\2\2\2\u019a\u019b\3\2\2\2\u019b\u01af\3\2"+ + "\2\2\u019c\u019d\7\3\2\2\u019d\u019e\5\n\6\2\u019e\u01a3\7\4\2\2\u019f"+ + "\u01a1\7\f\2\2\u01a0\u019f\3\2\2\2\u01a0\u01a1\3\2\2\2\u01a1\u01a2\3\2"+ + "\2\2\u01a2\u01a4\5^\60\2\u01a3\u01a0\3\2\2\2\u01a3\u01a4\3\2\2\2\u01a4"+ + "\u01af\3\2\2\2\u01a5\u01a6\7\3\2\2\u01a6\u01a7\5\"\22\2\u01a7\u01ac\7"+ + "\4\2\2\u01a8\u01aa\7\f\2\2\u01a9\u01a8\3\2\2\2\u01a9\u01aa\3\2\2\2\u01aa"+ + "\u01ab\3\2\2\2\u01ab\u01ad\5^\60\2\u01ac\u01a9\3\2\2\2\u01ac\u01ad\3\2"+ + "\2\2\u01ad\u01af\3\2\2\2\u01ae\u0195\3\2\2\2\u01ae\u019c\3\2\2\2\u01ae"+ + "\u01a5\3\2\2\2\u01af+\3\2\2\2\u01b0\u01b1\5.\30\2\u01b1-\3\2\2\2\u01b2"+ + "\u01b3\b\30\1\2\u01b3\u01b4\7A\2\2\u01b4\u01d2\5.\30\n\u01b5\u01b6\7#"+ + "\2\2\u01b6\u01b7\7\3\2\2\u01b7\u01b8\5\b\5\2\u01b8\u01b9\7\4\2\2\u01b9"+ + "\u01d2\3\2\2\2\u01ba\u01bb\7N\2\2\u01bb\u01bc\7\3\2\2\u01bc\u01bd\5j\66"+ + "\2\u01bd\u01be\5\60\31\2\u01be\u01bf\7\4\2\2\u01bf\u01d2\3\2\2\2\u01c0"+ + "\u01c1\7;\2\2\u01c1\u01c2\7\3\2\2\u01c2\u01c3\5^\60\2\u01c3\u01c4\7\5"+ + "\2\2\u01c4\u01c5\5j\66\2\u01c5\u01c6\5\60\31\2\u01c6\u01c7\7\4\2\2\u01c7"+ + "\u01d2\3\2\2\2\u01c8\u01c9\7;\2\2\u01c9\u01ca\7\3\2\2\u01ca\u01cb\5j\66"+ + "\2\u01cb\u01cc\7\5\2\2\u01cc\u01cd\5j\66\2\u01cd\u01ce\5\60\31\2\u01ce"+ + "\u01cf\7\4\2\2\u01cf\u01d2\3\2\2\2\u01d0\u01d2\5\62\32\2\u01d1\u01b2\3"+ + "\2\2\2\u01d1\u01b5\3\2\2\2\u01d1\u01ba\3\2\2\2\u01d1\u01c0\3\2\2\2\u01d1"+ + "\u01c8\3\2\2\2\u01d1\u01d0\3\2\2\2\u01d2\u01db\3\2\2\2\u01d3\u01d4\f\4"+ + "\2\2\u01d4\u01d5\7\n\2\2\u01d5\u01da\5.\30\5\u01d6\u01d7\f\3\2\2\u01d7"+ + "\u01d8\7F\2\2\u01d8\u01da\5.\30\4\u01d9\u01d3\3\2\2\2\u01d9\u01d6\3\2"+ + "\2\2\u01da\u01dd\3\2\2\2\u01db\u01d9\3\2\2\2\u01db\u01dc\3\2\2\2\u01dc"+ + "/\3\2\2\2\u01dd\u01db\3\2\2\2\u01de\u01df\7\5\2\2\u01df\u01e1\5j\66\2"+ + "\u01e0\u01de\3\2\2\2\u01e1\u01e4\3\2\2\2\u01e2\u01e0\3\2\2\2\u01e2\u01e3"+ + "\3\2\2\2\u01e3\61\3\2\2\2\u01e4\u01e2\3\2\2\2\u01e5\u01e7\5<\37\2\u01e6"+ + "\u01e8\5\64\33\2\u01e7\u01e6\3\2\2\2\u01e7\u01e8\3\2\2\2\u01e8\63\3\2"+ + "\2\2\u01e9\u01eb\7A\2\2\u01ea\u01e9\3\2\2\2\u01ea\u01eb\3\2\2\2\u01eb"+ + "\u01ec\3\2\2\2\u01ec\u01ed\7\16\2\2\u01ed\u01ee\5<\37\2\u01ee\u01ef\7"+ + "\n\2\2\u01ef\u01f0\5<\37\2\u01f0\u0218\3\2\2\2\u01f1\u01f3\7A\2\2\u01f2"+ + "\u01f1\3\2\2\2\u01f2\u01f3\3\2\2\2\u01f3\u01f4\3\2\2\2\u01f4\u01f5\7\61"+ + "\2\2\u01f5\u01f6\7\3\2\2\u01f6\u01fb\5<\37\2\u01f7\u01f8\7\5\2\2\u01f8"+ + "\u01fa\5<\37\2\u01f9\u01f7\3\2\2\2\u01fa\u01fd\3\2\2\2\u01fb\u01f9\3\2"+ + "\2\2\u01fb\u01fc\3\2\2\2\u01fc\u01fe\3\2\2\2\u01fd\u01fb\3\2\2\2\u01fe"+ + "\u01ff\7\4\2\2\u01ff\u0218\3\2\2\2\u0200\u0202\7A\2\2\u0201\u0200\3\2"+ + "\2\2\u0201\u0202\3\2\2\2\u0202\u0203\3\2\2\2\u0203\u0204\7\61\2\2\u0204"+ + "\u0205\7\3\2\2\u0205\u0206\5\b\5\2\u0206\u0207\7\4\2\2\u0207\u0218\3\2"+ + "\2\2\u0208\u020a\7A\2\2\u0209\u0208\3\2\2\2\u0209\u020a\3\2\2\2\u020a"+ + "\u020b\3\2\2\2\u020b\u020c\78\2\2\u020c\u0218\58\35\2\u020d\u020f\7A\2"+ + "\2\u020e\u020d\3\2\2\2\u020e\u020f\3\2\2\2\u020f\u0210\3\2\2\2\u0210\u0211"+ + "\7M\2\2\u0211\u0218\5j\66\2\u0212\u0214\7\64\2\2\u0213\u0215\7A\2\2\u0214"+ + "\u0213\3\2\2\2\u0214\u0215\3\2\2\2\u0215\u0216\3\2\2\2\u0216\u0218\7B"+ + "\2\2\u0217\u01ea\3\2\2\2\u0217\u01f2\3\2\2\2\u0217\u0201\3\2\2\2\u0217"+ + "\u0209\3\2\2\2\u0217\u020e\3\2\2\2\u0217\u0212\3\2\2\2\u0218\65\3\2\2"+ + "\2\u0219\u021a\78\2\2\u021a\u021b\58\35\2\u021b\67\3\2\2\2\u021c\u021e"+ + "\5j\66\2\u021d\u021f\5:\36\2\u021e\u021d\3\2\2\2\u021e\u021f\3\2\2\2\u021f"+ + "9\3\2\2\2\u0220\u0221\7!\2\2\u0221\u0227\5j\66\2\u0222\u0223\7d\2\2\u0223"+ + "\u0224\5j\66\2\u0224\u0225\7k\2\2\u0225\u0227\3\2\2\2\u0226\u0220\3\2"+ + "\2\2\u0226\u0222\3\2\2\2\u0227;\3\2\2\2\u0228\u0229\b\37\1\2\u0229\u022d"+ + "\5> \2\u022a\u022b\t\7\2\2\u022b\u022d\5<\37\6\u022c\u0228\3\2\2\2\u022c"+ + "\u022a\3\2\2\2\u022d\u023a\3\2\2\2\u022e\u022f\f\5\2\2\u022f\u0230\t\f"+ + "\2\2\u0230\u0239\5<\37\6\u0231\u0232\f\4\2\2\u0232\u0233\t\7\2\2\u0233"+ + "\u0239\5<\37\5\u0234\u0235\f\3\2\2\u0235\u0236\5T+\2\u0236\u0237\5<\37"+ + "\4\u0237\u0239\3\2\2\2\u0238\u022e\3\2\2\2\u0238\u0231\3\2\2\2\u0238\u0234"+ + "\3\2\2\2\u0239\u023c\3\2\2\2\u023a\u0238\3\2\2\2\u023a\u023b\3\2\2\2\u023b"+ + "=\3\2\2\2\u023c\u023a\3\2\2\2\u023d\u023e\b \1\2\u023e\u0262\5B\"\2\u023f"+ + "\u0262\5H%\2\u0240\u0262\5@!\2\u0241\u0262\5R*\2\u0242\u0243\5^\60\2\u0243"+ + "\u0244\7z\2\2\u0244\u0246\3\2\2\2\u0245\u0242\3\2\2\2\u0245\u0246\3\2"+ + "\2\2\u0246\u0247\3\2\2\2\u0247\u0262\7u\2\2\u0248\u0262\5L\'\2\u0249\u024a"+ + "\7\3\2\2\u024a\u024b\5\b\5\2\u024b\u024c\7\4\2\2\u024c\u0262\3\2\2\2\u024d"+ + "\u0262\5^\60\2\u024e\u024f\7\3\2\2\u024f\u0250\5,\27\2\u0250\u0251\7\4"+ + "\2\2\u0251\u0262\3\2\2\2\u0252\u0254\7\20\2\2\u0253\u0255\5.\30\2\u0254"+ + "\u0253\3\2\2\2\u0254\u0255\3\2\2\2\u0255\u0257\3\2\2\2\u0256\u0258\5l"+ + "\67\2\u0257\u0256\3\2\2\2\u0258\u0259\3\2\2\2\u0259\u0257\3\2\2\2\u0259"+ + "\u025a\3\2\2\2\u025a\u025d\3\2\2\2\u025b\u025c\7\37\2\2\u025c\u025e\5"+ + ".\30\2\u025d\u025b\3\2\2\2\u025d\u025e\3\2\2\2\u025e\u025f\3\2\2\2\u025f"+ + "\u0260\7 \2\2\u0260\u0262\3\2\2\2\u0261\u023d\3\2\2\2\u0261\u023f\3\2"+ + "\2\2\u0261\u0240\3\2\2\2\u0261\u0241\3\2\2\2\u0261\u0245\3\2\2\2\u0261"+ + "\u0248\3\2\2\2\u0261\u0249\3\2\2\2\u0261\u024d\3\2\2\2\u0261\u024e\3\2"+ + "\2\2\u0261\u0252\3\2\2\2\u0262\u0268\3\2\2\2\u0263\u0264\f\f\2\2\u0264"+ + "\u0265\7x\2\2\u0265\u0267\5\\/\2\u0266\u0263\3\2\2\2\u0267\u026a\3\2\2"+ + "\2\u0268\u0266\3\2\2\2\u0268\u0269\3\2\2\2\u0269?\3\2\2\2\u026a\u0268"+ + "\3\2\2\2\u026b\u026f\7\30\2\2\u026c\u026f\7\26\2\2\u026d\u026f\7\27\2"+ + "\2\u026e\u026b\3\2\2\2\u026e\u026c\3\2\2\2\u026e\u026d\3\2\2\2\u026fA"+ + "\3\2\2\2\u0270\u027b\5D#\2\u0271\u0272\7e\2\2\u0272\u0273\5D#\2\u0273"+ + "\u0274\7k\2\2\u0274\u027b\3\2\2\2\u0275\u027b\5F$\2\u0276\u0277\7e\2\2"+ + "\u0277\u0278\5F$\2\u0278\u0279\7k\2\2\u0279\u027b\3\2\2\2\u027a\u0270"+ + "\3\2\2\2\u027a\u0271\3\2\2\2\u027a\u0275\3\2\2\2\u027a\u0276\3\2\2\2\u027b"+ + "C\3\2\2\2\u027c\u027d\7\21\2\2\u027d\u027e\7\3\2\2\u027e\u027f\5,\27\2"+ + "\u027f\u0280\7\f\2\2\u0280\u0281\5\\/\2\u0281\u0282\7\4\2\2\u0282E\3\2"+ + "\2\2\u0283\u0284\7\25\2\2\u0284\u0285\7\3\2\2\u0285\u0286\5,\27\2\u0286"+ + "\u0287\7\5\2\2\u0287\u0288\5\\/\2\u0288\u0289\7\4\2\2\u0289G\3\2\2\2\u028a"+ + "\u0290\5J&\2\u028b\u028c\7e\2\2\u028c\u028d\5J&\2\u028d\u028e\7k\2\2\u028e"+ + "\u0290\3\2\2\2\u028f\u028a\3\2\2\2\u028f\u028b\3\2\2\2\u0290I\3\2\2\2"+ + "\u0291\u0292\7%\2\2\u0292\u0293\7\3\2\2\u0293\u0294\5`\61\2\u0294\u0295"+ + "\7)\2\2\u0295\u0296\5<\37\2\u0296\u0297\7\4\2\2\u0297K\3\2\2\2\u0298\u029e"+ + "\5N(\2\u0299\u029a\7e\2\2\u029a\u029b\5N(\2\u029b\u029c\7k\2\2\u029c\u029e"+ + "\3\2\2\2\u029d\u0298\3\2\2\2\u029d\u0299\3\2\2\2\u029eM\3\2\2\2\u029f"+ + "\u02a0\5P)\2\u02a0\u02ac\7\3\2\2\u02a1\u02a3\5\36\20\2\u02a2\u02a1\3\2"+ + "\2\2\u02a2\u02a3\3\2\2\2\u02a3\u02a4\3\2\2\2\u02a4\u02a9\5,\27\2\u02a5"+ + "\u02a6\7\5\2\2\u02a6\u02a8\5,\27\2\u02a7\u02a5\3\2\2\2\u02a8\u02ab\3\2"+ + "\2\2\u02a9\u02a7\3\2\2\2\u02a9\u02aa\3\2\2\2\u02aa\u02ad\3\2\2\2\u02ab"+ + "\u02a9\3\2\2\2\u02ac\u02a2\3\2\2\2\u02ac\u02ad\3\2\2\2\u02ad\u02ae\3\2"+ + "\2\2\u02ae\u02af\7\4\2\2\u02afO\3\2\2\2\u02b0\u02b4\7\67\2\2\u02b1\u02b4"+ + "\7L\2\2\u02b2\u02b4\5`\61\2\u02b3\u02b0\3\2\2\2\u02b3\u02b1\3\2\2\2\u02b3"+ + "\u02b2\3\2\2\2\u02b4Q\3\2\2\2\u02b5\u02d0\7B\2\2\u02b6\u02d0\5X-\2\u02b7"+ + "\u02d0\5h\65\2\u02b8\u02d0\5V,\2\u02b9\u02bb\7|\2\2\u02ba\u02b9\3\2\2"+ + "\2\u02bb\u02bc\3\2\2\2\u02bc\u02ba\3\2\2\2\u02bc\u02bd\3\2\2\2\u02bd\u02d0"+ + "\3\2\2\2\u02be\u02d0\7{\2\2\u02bf\u02c0\7g\2\2\u02c0\u02c1\5j\66\2\u02c1"+ + "\u02c2\7k\2\2\u02c2\u02d0\3\2\2\2\u02c3\u02c4\7h\2\2\u02c4\u02c5\5j\66"+ + "\2\u02c5\u02c6\7k\2\2\u02c6\u02d0\3\2\2\2\u02c7\u02c8\7i\2\2\u02c8\u02c9"+ + "\5j\66\2\u02c9\u02ca\7k\2\2\u02ca\u02d0\3\2\2\2\u02cb\u02cc\7j\2\2\u02cc"+ + "\u02cd\5j\66\2\u02cd\u02ce\7k\2\2\u02ce\u02d0\3\2\2\2\u02cf\u02b5\3\2"+ + "\2\2\u02cf\u02b6\3\2\2\2\u02cf\u02b7\3\2\2\2\u02cf\u02b8\3\2\2\2\u02cf"+ + "\u02ba\3\2\2\2\u02cf\u02be\3\2\2\2\u02cf\u02bf\3\2\2\2\u02cf\u02c3\3\2"+ + "\2\2\u02cf\u02c7\3\2\2\2\u02cf\u02cb\3\2\2\2\u02d0S\3\2\2\2\u02d1\u02d2"+ + "\t\r\2\2\u02d2U\3\2\2\2\u02d3\u02d4\t\16\2\2\u02d4W\3\2\2\2\u02d5\u02d7"+ + "\7\63\2\2\u02d6\u02d8\t\7\2\2\u02d7\u02d6\3\2\2\2\u02d7\u02d8\3\2\2\2"+ + "\u02d8\u02db\3\2\2\2\u02d9\u02dc\5h\65\2\u02da\u02dc\5j\66\2\u02db\u02d9"+ + "\3\2\2\2\u02db\u02da\3\2\2\2\u02dc\u02dd\3\2\2\2\u02dd\u02e0\5Z.\2\u02de"+ + "\u02df\7Z\2\2\u02df\u02e1\5Z.\2\u02e0\u02de\3\2\2\2\u02e0\u02e1\3\2\2"+ + "\2\u02e1Y\3\2\2\2\u02e2\u02e3\t\17\2\2\u02e3[\3\2\2\2\u02e4\u02e5\5`\61"+ + "\2\u02e5]\3\2\2\2\u02e6\u02e7\5`\61\2\u02e7\u02e8\7z\2\2\u02e8\u02ea\3"+ + "\2\2\2\u02e9\u02e6\3\2\2\2\u02ea\u02ed\3\2\2\2\u02eb\u02e9\3\2\2\2\u02eb"+ + "\u02ec\3\2\2\2\u02ec\u02ee\3\2\2\2\u02ed\u02eb\3\2\2\2\u02ee\u02ef\5`"+ + "\61\2\u02ef_\3\2\2\2\u02f0\u02f3\5d\63\2\u02f1\u02f3\5f\64\2\u02f2\u02f0"+ + "\3\2\2\2\u02f2\u02f1\3\2\2\2\u02f3a\3\2\2\2\u02f4\u02f5\5`\61\2\u02f5"+ + "\u02f6\7\6\2\2\u02f6\u02f8\3\2\2\2\u02f7\u02f4\3\2\2\2\u02f7\u02f8\3\2"+ + "\2\2\u02f8\u02f9\3\2\2\2\u02f9\u0301\7\u0081\2\2\u02fa\u02fb\5`\61\2\u02fb"+ + "\u02fc\7\6\2\2\u02fc\u02fe\3\2\2\2\u02fd\u02fa\3\2\2\2\u02fd\u02fe\3\2"+ + "\2\2\u02fe\u02ff\3\2\2\2\u02ff\u0301\5`\61\2\u0300\u02f7\3\2\2\2\u0300"+ + "\u02fd\3\2\2\2\u0301c\3\2\2\2\u0302\u0305\7\u0082\2\2\u0303\u0305\7\u0083"+ + "\2\2\u0304\u0302\3\2\2\2\u0304\u0303\3\2\2\2\u0305e\3\2\2\2\u0306\u030a"+ + "\7\177\2\2\u0307\u030a\5n8\2\u0308\u030a\7\u0080\2\2\u0309\u0306\3\2\2"+ + "\2\u0309\u0307\3\2\2\2\u0309\u0308\3\2\2\2\u030ag\3\2\2\2\u030b\u030e"+ + "\7~\2\2\u030c\u030e\7}\2\2\u030d\u030b\3\2\2\2\u030d\u030c\3\2\2\2\u030e"+ + "i\3\2\2\2\u030f\u0310\t\20\2\2\u0310k\3\2\2\2\u0311\u0312\7_\2\2\u0312"+ + "\u0313\5,\27\2\u0313\u0314\7X\2\2\u0314\u0315\5,\27\2\u0315m\3\2\2\2\u0316"+ + "\u0317\t\21\2\2\u0317o\3\2\2\2k\177\u0081\u0085\u008e\u0090\u0094\u009b"+ + "\u00a2\u00a7\u00ac\u00b4\u00b8\u00c0\u00c3\u00c9\u00ce\u00d1\u00d6\u00d9"+ + "\u00db\u00e3\u00e6\u00f2\u00f5\u00f8\u00ff\u0106\u010a\u010e\u0112\u0119"+ + "\u011d\u0121\u0126\u012a\u0132\u0136\u013d\u0148\u014b\u014f\u015b\u015e"+ + "\u0164\u016b\u0172\u0175\u0179\u017d\u0181\u0183\u018e\u0193\u0197\u019a"+ + "\u01a0\u01a3\u01a9\u01ac\u01ae\u01d1\u01d9\u01db\u01e2\u01e7\u01ea\u01f2"+ + "\u01fb\u0201\u0209\u020e\u0214\u0217\u021e\u0226\u022c\u0238\u023a\u0245"+ + "\u0254\u0259\u025d\u0261\u0268\u026e\u027a\u028f\u029d\u02a2\u02a9\u02ac"+ + "\u02b3\u02bc\u02cf\u02d7\u02db\u02e0\u02eb\u02f2\u02f7\u02fd\u0300\u0304"+ + "\u0309\u030d"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseVisitor.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseVisitor.java index 56310aa66eb7c..7f44a1593c2ef 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseVisitor.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseVisitor.java @@ -414,6 +414,13 @@ interface SqlBaseVisitor extends ParseTreeVisitor { * @return the visitor result */ T visitSubqueryExpression(SqlBaseParser.SubqueryExpressionContext ctx); + /** + * Visit a parse tree produced by the {@code case} + * labeled alternative in {@link SqlBaseParser#primaryExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitCase(SqlBaseParser.CaseContext ctx); /** * Visit a parse tree produced by {@link SqlBaseParser#builtinDateTimeFunction}. * @param ctx the parse tree @@ -635,6 +642,12 @@ interface SqlBaseVisitor extends ParseTreeVisitor { * @return the visitor result */ T visitString(SqlBaseParser.StringContext ctx); + /** + * Visit a parse tree produced by {@link SqlBaseParser#whenClause}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitWhenClause(SqlBaseParser.WhenClauseContext ctx); /** * Visit a parse tree produced by {@link SqlBaseParser#nonReserved}. * @param ctx the parse tree diff --git a/x-pack/plugin/sql/src/main/resources/org/elasticsearch/xpack/sql/plugin/sql_whitelist.txt b/x-pack/plugin/sql/src/main/resources/org/elasticsearch/xpack/sql/plugin/sql_whitelist.txt index 56b911389447a..4ac4632572ca0 100644 --- a/x-pack/plugin/sql/src/main/resources/org/elasticsearch/xpack/sql/plugin/sql_whitelist.txt +++ b/x-pack/plugin/sql/src/main/resources/org/elasticsearch/xpack/sql/plugin/sql_whitelist.txt @@ -47,8 +47,9 @@ class org.elasticsearch.xpack.sql.expression.function.scalar.whitelist.InternalS Boolean isNotNull(Object) # -# Null +# Conditional # + def caseFunction(java.util.List) def coalesce(java.util.List) def greatest(java.util.List) def least(java.util.List) diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java index c2310aa331e74..24e993a2d8337 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java @@ -598,6 +598,26 @@ public void testConditionalWithDifferentDataTypes() { error("SELECT 1 = 1 OR " + arbirtraryArgsFunction + "(null, 3, '4') > 1")); } + public void testCaseWithNonBooleanConditionExpression() { + assertEquals("1:8: condition of [WHEN abs(int) THEN 'foo'] must be [boolean], found value [abs(int)] type [integer]", + error("SELECT CASE WHEN int = 1 THEN 'one' WHEN abs(int) THEN 'foo' END FROM test")); + } + + public void testCaseWithDifferentResultDataTypes() { + assertEquals("1:8: result of [WHEN int > 10 THEN 10] must be [keyword], found value [10] type [integer]", + error("SELECT CASE WHEN int > 20 THEN 'foo' WHEN int > 10 THEN 10 ELSE 'bar' END FROM test")); + } + + public void testCaseWithDifferentResultAndDefaultValueDataTypes() { + assertEquals("1:8: ELSE clause of [date] must be [keyword], found value [date] type [datetime]", + error("SELECT CASE WHEN int > 20 THEN 'foo' ELSE date END FROM test")); + } + + public void testCaseWithDifferentResultAndDefaultValueDataTypesAndNullTypesSkipped() { + assertEquals("1:8: ELSE clause of [date] must be [keyword], found value [date] type [datetime]", + error("SELECT CASE WHEN int > 20 THEN null WHEN int > 10 THEN null WHEN int > 5 THEN 'foo' ELSE date END FROM test")); + } + public void testAggsInWhere() { assertEquals("1:33: Cannot use WHERE filtering on aggregate function [MAX(int)], use HAVING instead", error("SELECT MAX(int) FROM test WHERE MAX(int) > 10 GROUP BY bool")); 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 new file mode 100644 index 0000000000000..efc86bb47d24d --- /dev/null +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/CaseTests.java @@ -0,0 +1,100 @@ +/* + * 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.sql.expression.predicate.conditional; + +import org.elasticsearch.xpack.sql.expression.Expression; +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 java.util.ArrayList; +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.SourceTests.randomSource; + +/** + * Needed to override tests in {@link NodeSubclassTests} as Case is special since its children are not usual + * expressions but {@link IfConditional}s. + */ +public class CaseTests extends AbstractNodeTestCase { + + public static Case randomCase() { + int noConditionals = randomIntBetween(1, 5); + List expressions = new ArrayList<>(noConditionals + 1); + for (int i = 0; i < noConditionals; i++) { + expressions.add(new IfConditional( + randomSource(), new Equals(randomSource(), randomStringLiteral(), randomStringLiteral()), randomIntLiteral())); + + } + // default else + expressions.add(randomIntLiteral()); + return new Case(randomSource(), expressions); + } + + @Override + protected Case randomInstance() { + return randomCase(); + } + + @Override + protected Case mutate(Case instance) { + Case c = randomCase(); + return new Case(c.source(), mutateChildren(c)); + } + + @Override + protected Case copy(Case instance) { + return new Case(instance.source(), instance.children()); + } + + @Override + public void testTransform() { + Case c = randomCase(); + + Source newSource = randomValueOtherThan(c.source(), SourceTests::randomSource); + assertEquals(new Case(c.source(), c.children()), + c.transformPropertiesOnly(p -> Objects.equals(p, c.source()) ? newSource: p, Object.class)); + + String newName = randomValueOtherThan(c.name(), () -> randomAlphaOfLength(5)); + assertEquals(new Case(c.source(), c.children()), + c.transformPropertiesOnly(p -> Objects.equals(p, c.name()) ? newName : p, Object.class)); + } + + @Override + public void testReplaceChildren() { + Case c = randomCase(); + + List newChildren = mutateChildren(c); + assertEquals(new Case(c.source(), newChildren), c.replaceChildren(newChildren)); + } + + private List mutateChildren(Case c) { + boolean removeConditional = randomBoolean(); + List expressions = new ArrayList<>(c.children().size()); + if (removeConditional) { + expressions.addAll(c.children().subList(0, c.children().size() - 2)); + } else { + int rndIdx = randomInt(c.conditions().size()); + for (int i = 0; i < c.conditions().size(); i++) { + if (i == rndIdx) { + expressions.add(new IfConditional(randomValueOtherThan(c.conditions().get(i).source(), SourceTests::randomSource), + new Equals(randomSource(), randomStringLiteral(), randomStringLiteral()), + randomValueOtherThan(c.conditions().get(i).condition(), FunctionTestUtils::randomStringLiteral))); + } else { + expressions.add(c.conditions().get(i)); + } + } + } + expressions.add(c.defaultElse()); + return expressions; + } +} 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 eb8ac2b4d1595..f77088c3fdfc1 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 @@ -44,9 +44,11 @@ import org.elasticsearch.xpack.sql.expression.predicate.BinaryOperator; import org.elasticsearch.xpack.sql.expression.predicate.Range; import org.elasticsearch.xpack.sql.expression.predicate.conditional.ArbitraryConditionalFunction; +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.IfConditional; import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfNull; import org.elasticsearch.xpack.sql.expression.predicate.conditional.Least; import org.elasticsearch.xpack.sql.expression.predicate.conditional.NullIf; @@ -112,9 +114,12 @@ import static org.elasticsearch.xpack.sql.expression.Literal.NULL; import static org.elasticsearch.xpack.sql.expression.Literal.TRUE; import static org.elasticsearch.xpack.sql.expression.Literal.of; +import static org.elasticsearch.xpack.sql.optimizer.Optimizer.SimplifyCase; +import static org.elasticsearch.xpack.sql.optimizer.Optimizer.SortAggregateOnOrderBy; import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; import static org.elasticsearch.xpack.sql.util.DateUtils.UTC; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.startsWith; public class OptimizerTests extends ESTestCase { @@ -595,6 +600,91 @@ public void testConcatFoldingIsNotNull() { assertEquals(StringUtils.EMPTY, foldNull.rule(new Concat(EMPTY, NULL, NULL)).fold()); } + public void testSimplifyCaseConditionsFoldWhenFalse() { + // CASE WHEN a = 1 THEN 'foo1' + // WHEN 1 = 2 THEN 'bar1' + // WHEN 2 = 1 THEN 'bar2' + // WHEN a > 1 THEN 'foo2' + // ELSE 'default' + // END + // + // ==> + // + // CASE WHEN a = 1 THEN 'foo1' + // WHEN a > 1 THEN 'foo2' + // ELSE 'default' + // END + + Case c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, getFieldAttribute(), ONE), Literal.of(EMPTY, "foo1")), + new IfConditional(EMPTY, new Equals(EMPTY, ONE, TWO), Literal.of(EMPTY, "bar1")), + new IfConditional(EMPTY, new Equals(EMPTY, TWO, ONE), Literal.of(EMPTY, "bar2")), + new IfConditional(EMPTY, new GreaterThan(EMPTY, getFieldAttribute(), ONE), Literal.of(EMPTY, "foo2")), + Literal.of(EMPTY, "default"))); + Expression e = new SimplifyCase().rule(c); + assertEquals(Case.class, e.getClass()); + c = (Case) e; + assertEquals(2, c.conditions().size()); + 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()); + } + + public void testSimplifyCaseConditionsFoldWhenTrue() { + // CASE WHEN a = 1 THEN 'foo1' + // WHEN 1 = 1 THEN 'bar1' + // WHEN 2 = 1 THEN 'bar2' + // WHEN a > 1 THEN 'foo2' + // ELSE 'default' + // END + // + // ==> + // + // CASE WHEN a = 1 THEN 'foo1' + // WHEN 1 = 1 THEN 'bar1' + // ELSE 'default' + // END + + SimplifyCase rule = new SimplifyCase(); + Case c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, getFieldAttribute(), ONE), Literal.of(EMPTY, "foo1")), + new IfConditional(EMPTY, new Equals(EMPTY, ONE, ONE), Literal.of(EMPTY, "bar1")), + new IfConditional(EMPTY, new Equals(EMPTY, TWO, ONE), Literal.of(EMPTY, "bar2")), + new IfConditional(EMPTY, new GreaterThan(EMPTY, getFieldAttribute(), ONE), Literal.of(EMPTY, "foo2")), + Literal.of(EMPTY, "default"))); + Expression e = rule.rule(c); + assertEquals(Case.class, e.getClass()); + c = (Case) e; + assertEquals(2, c.conditions().size()); + 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()); + } + + public void testSimplifyCaseConditionsFoldCompletely() { + // CASE WHEN 1 = 2 THEN 'foo1' + // WHEN 1 = 1 THEN 'foo2' + // ELSE 'default' + // END + // + // ==> + // + // 'foo2' + + SimplifyCase rule = new SimplifyCase(); + Case c = new Case(EMPTY, Arrays.asList( + new IfConditional(EMPTY, new Equals(EMPTY, ONE, TWO), Literal.of(EMPTY, "foo1")), + new IfConditional(EMPTY, new Equals(EMPTY, ONE, ONE), Literal.of(EMPTY, "foo2")), + Literal.of(EMPTY, "default"))); + Expression e = rule.rule(c); + assertEquals(Case.class, e.getClass()); + c = (Case) e; + assertEquals(1, c.conditions().size()); + assertThat(c.conditions().get(0).condition().toString(), startsWith("Equals[=1,=1]#")); + assertTrue(c.foldable()); + assertEquals("foo2", c.fold()); + } + // // Logical simplifications // @@ -1318,7 +1408,7 @@ public void testSortAggregateOnOrderByWithTwoFields() { OrderBy orderByPlan = new OrderBy(EMPTY, new Aggregate(EMPTY, FROM(), Arrays.asList(secondField, firstField), Arrays.asList(secondAlias, firstAlias)), Arrays.asList(firstOrderBy, secondOrderBy)); - LogicalPlan result = new Optimizer.SortAggregateOnOrderBy().apply(orderByPlan); + LogicalPlan result = new SortAggregateOnOrderBy().apply(orderByPlan); assertTrue(result instanceof OrderBy); List order = ((OrderBy) result).order(); @@ -1350,7 +1440,7 @@ public void testSortAggregateOnOrderByOnlyAliases() { OrderBy orderByPlan = new OrderBy(EMPTY, new Aggregate(EMPTY, FROM(), Arrays.asList(secondAlias, firstAlias), Arrays.asList(secondAlias, firstAlias)), Arrays.asList(firstOrderBy, secondOrderBy)); - LogicalPlan result = new Optimizer.SortAggregateOnOrderBy().apply(orderByPlan); + LogicalPlan result = new SortAggregateOnOrderBy().apply(orderByPlan); assertTrue(result instanceof OrderBy); List order = ((OrderBy) result).order(); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java index 98d7922d21839..a8e5df8e07caa 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java @@ -11,6 +11,8 @@ import org.elasticsearch.xpack.sql.expression.function.UnresolvedFunction; import org.elasticsearch.xpack.sql.expression.function.scalar.Cast; import org.elasticsearch.xpack.sql.expression.literal.Interval; +import org.elasticsearch.xpack.sql.expression.predicate.conditional.Case; +import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfConditional; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Add; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Mul; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Neg; @@ -26,6 +28,7 @@ import java.util.Locale; import static java.lang.String.format; +import static org.hamcrest.Matchers.startsWith; public class ExpressionTests extends ESTestCase { @@ -447,4 +450,63 @@ public void testSourceFunction() { Expression expr = parser.createExpression(s); assertEquals(s, expr.sourceText()); } + + public void testCaseWithoutOperand() { + Expression expr = parser.createExpression( + "CASE WHEN a = 1 THEN 'one'" + + " WHEN a > 2 THEN 'a few'" + + " WHEN a > 10 THEN 'many' " + + "END"); + + assertEquals(Case.class, expr.getClass()); + Case c = (Case) expr; + assertEquals(3, c.conditions().size()); + IfConditional ifc = c.conditions().get(0); + assertEquals("WHEN a = 1 THEN 'one'", ifc.sourceText()); + assertThat(ifc.condition().toString(), startsWith("Equals[?a,1]#")); + assertEquals("'one'=one", ifc.result().toString()); + assertEquals(Literal.NULL, c.defaultElse()); + + expr = parser.createExpression( + "CASE WHEN a = 1 THEN 'one'" + + " WHEN a <= 2 THEN 'a few'" + + "ELSE 'many' " + + "END"); + + assertEquals(Case.class, expr.getClass()); + c = (Case) expr; + assertEquals(2, c.conditions().size()); + ifc = c.conditions().get(0); + assertEquals("WHEN a = 1 THEN 'one'", ifc.sourceText()); + assertEquals("'many'=many", c.defaultElse().toString()); + } + + public void testCaseWithOperand() { + Expression expr = parser.createExpression( + "CASE a WHEN 1 THEN 'one'" + + " WHEN 2 THEN 'two'" + + " WHEN 3 THEN 'three' " + + "END"); + + assertEquals(Case.class, expr.getClass()); + Case c = (Case) expr; + assertEquals(3, c.conditions().size()); + IfConditional ifc = c.conditions().get(0); + assertEquals("WHEN 1 THEN 'one'", ifc.sourceText()); + assertThat(ifc.condition().toString(), startsWith("Equals[?a,1]#")); + assertEquals("'one'=one", ifc.result().toString()); + assertEquals(Literal.NULL, c.defaultElse()); + + expr = parser.createExpression( + "CASE a WHEN 1 THEN 'one'" + + " WHEN 2 THEN 'two'" + + "ELSE 'many' " + + "END"); + assertEquals(Case.class, expr.getClass()); + c = (Case) expr; + assertEquals(2, c.conditions().size()); + ifc = c.conditions().get(0); + assertEquals("WHEN 1 THEN 'one'", ifc.sourceText()); + assertEquals("'many'=many", c.defaultElse().toString()); + } } 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 85bc20596e9e3..ca3c3a92ce6d6 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 @@ -587,6 +587,21 @@ public void testTranslateNullIf_GroupBy_Painless() { assertEquals("[{v=int}, {v=10}]", scriptTemplate.params().toString()); } + public void testTranslateCase_GroupBy_Painless() { + LogicalPlan p = plan("SELECT CASE WHEN int > 10 THEN 'foo' WHEN int > 20 THEN 'bar' ELSE 'default' END FROM test GROUP BY 1"); + assertTrue(p instanceof Aggregate); + Expression condition = ((Aggregate) p).groupings().get(0); + assertFalse(condition.foldable()); + QueryTranslator.GroupingContext groupingContext = QueryTranslator.groupBy(((Aggregate) p).groupings()); + assertNotNull(groupingContext); + ScriptTemplate scriptTemplate = groupingContext.tail.script(); + assertEquals("InternalSqlScriptUtils.caseFunction([InternalSqlScriptUtils.gt(InternalSqlScriptUtils.docValue(" + "" + + "doc,params.v0),params.v1),params.v2,InternalSqlScriptUtils.gt(InternalSqlScriptUtils.docValue(doc,params.v3)," + + "params.v4),params.v5,params.v6])", + scriptTemplate.toString()); + assertEquals("[{v=int}, {v=10}, {v=foo}, {v=int}, {v=20}, {v=bar}, {v=default}]", scriptTemplate.params().toString()); + } + public void testGroupByDateHistogram() { LogicalPlan p = plan("SELECT MAX(int) FROM test GROUP BY HISTOGRAM(int, 1000)"); assertTrue(p instanceof Aggregate); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java index f11519f0f973f..cad06da0a824e 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.sql.tree; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; - import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.test.ESTestCase; @@ -29,6 +28,7 @@ import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe; import org.elasticsearch.xpack.sql.expression.gen.processor.ConstantProcessor; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; +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.fulltext.FullTextPredicate; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.In; @@ -92,8 +92,8 @@ */ public class NodeSubclassTests> extends ESTestCase { - private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = Arrays.asList(IfNull.class, In.class, InPipe.class, - Percentile.class, Percentiles.class, PercentileRanks.class); + private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = Arrays.asList(IfConditional.class, IfNull.class, + In.class, InPipe.class, Percentile.class, Percentiles.class, PercentileRanks.class); private final Class subclass; From f99946943a3350206b6bca774b2f060f41a787b3 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Mon, 22 Apr 2019 22:41:45 +0300 Subject: [PATCH 143/260] SQL: Use field caps inside DESCRIBE TABLE as well (#41377) Thanks to #34071, there is enough information in field caps to infer the table structure and thus use the same API consistently across the IndexResolver. --- .../xpack/sql/qa/security/JdbcSecurityIT.java | 14 +- .../single-node-only/command-sys.csv-spec | 1 + .../sql/analysis/index/IndexResolver.java | 352 +++++++++--------- .../plan/logical/command/sys/SysColumns.java | 2 +- .../analysis/index/IndexResolverTests.java | 64 ++-- 5 files changed, 235 insertions(+), 198 deletions(-) diff --git a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/JdbcSecurityIT.java b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/JdbcSecurityIT.java index a911e7d4854ae..1f04d6fb99c8b 100644 --- a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/JdbcSecurityIT.java +++ b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/JdbcSecurityIT.java @@ -234,12 +234,14 @@ public void checkNoMonitorMain(String user) throws Exception { expectUnauthorized("cluster:monitor/main", user, () -> es(userProperties(user))); expectUnauthorized("cluster:monitor/main", user, () -> es(userProperties(user)).getMetaData().getDatabaseMajorVersion()); expectUnauthorized("cluster:monitor/main", user, () -> es(userProperties(user)).getMetaData().getDatabaseMinorVersion()); - expectUnauthorized("cluster:monitor/main", user, - () -> es(userProperties(user)).createStatement().executeQuery("SELECT * FROM test")); - expectUnauthorized("cluster:monitor/main", user, - () -> es(userProperties(user)).createStatement().executeQuery("SHOW TABLES LIKE 'test'")); - expectUnauthorized("cluster:monitor/main", user, - () -> es(userProperties(user)).createStatement().executeQuery("DESCRIBE test")); + + // by moving to field caps these calls do not require the monitor permission + // expectUnauthorized("cluster:monitor/main", user, + // () -> es(userProperties(user)).createStatement().executeQuery("SELECT * FROM test")); + // expectUnauthorized("cluster:monitor/main", user, + // () -> es(userProperties(user)).createStatement().executeQuery("SHOW TABLES LIKE 'test'")); + // expectUnauthorized("cluster:monitor/main", user, + // () -> es(userProperties(user)).createStatement().executeQuery("DESCRIBE test")); } private void expectUnauthorized(String action, String user, ThrowingRunnable r) { diff --git a/x-pack/plugin/sql/qa/src/main/resources/single-node-only/command-sys.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/single-node-only/command-sys.csv-spec index f6b02ba4bea43..1b13841c47273 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/single-node-only/command-sys.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/single-node-only/command-sys.csv-spec @@ -107,6 +107,7 @@ x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp |salary |4 |INTEGER |11 |4 |null |10 |1 |null |null |4 |0 |null |11 |YES |null |null |null |null |NO |NO x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp_copy |birth_date |93 |DATETIME |29 |8 |null |null |1 |null |null |9 |3 |null |1 |YES |null |null |null |null |NO |NO x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp_copy |emp_no |4 |INTEGER |11 |4 |null |10 |1 |null |null |4 |0 |null |3 |YES |null |null |null |null |NO |NO +x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp_copy |extra.info.gender |12 |KEYWORD |32766 |2147483647 |null |null |1 |null |null |12 |0 |2147483647 |6 |YES |null |null |null |null |NO |NO x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp_copy |extra_gender |12 |KEYWORD |32766 |2147483647 |null |null |1 |null |null |12 |0 |2147483647 |7 |YES |null |null |null |null |NO |NO x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp_copy |extra_no |4 |INTEGER |11 |4 |null |10 |1 |null |null |4 |0 |null |8 |YES |null |null |null |null |NO |NO x-pack_plugin_sql_qa_single-node_integTestCluster |null |test_emp_copy |first_name |12 |TEXT |2147483647 |2147483647 |null |null |1 |null |null |12 |0 |2147483647 |9 |YES |null |null |null |null |NO |NO diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java index 367c9ea3a149f..8f51fa65b7463 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.sql.analysis.index; import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.action.ActionListener; @@ -22,17 +21,15 @@ import org.elasticsearch.action.support.IndicesOptions.WildcardStates; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.AliasMetaData; -import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.type.DateEsField; import org.elasticsearch.xpack.sql.type.EsField; import org.elasticsearch.xpack.sql.type.InvalidMappedField; import org.elasticsearch.xpack.sql.type.KeywordEsField; import org.elasticsearch.xpack.sql.type.TextEsField; -import org.elasticsearch.xpack.sql.type.Types; import org.elasticsearch.xpack.sql.type.UnsupportedEsField; import org.elasticsearch.xpack.sql.util.CollectionUtils; @@ -41,20 +38,25 @@ import java.util.Collections; import java.util.Comparator; import java.util.EnumSet; +import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Map.Entry; -import java.util.NavigableSet; import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.regex.Pattern; +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; public class IndexResolver { @@ -136,6 +138,7 @@ public boolean equals(Object obj) { private static final IndicesOptions INDICES_ONLY_OPTIONS = new IndicesOptions( EnumSet.of(Option.ALLOW_NO_INDICES, Option.IGNORE_UNAVAILABLE, Option.IGNORE_ALIASES), EnumSet.of(WildcardStates.OPEN)); private static final List FIELD_NAMES_BLACKLIST = Arrays.asList("_size"); + private static final String UNMAPPED = "unmapped"; private final Client client; private final String clusterName; @@ -242,103 +245,82 @@ private void filterResults(String javaRegex, GetAliasesResponse aliases, GetInde public void resolveAsMergedMapping(String indexWildcard, String javaRegex, ActionListener listener) { FieldCapabilitiesRequest fieldRequest = createFieldCapsRequest(indexWildcard); client.fieldCaps(fieldRequest, - ActionListener.wrap(response -> listener.onResponse(mergedMapping(indexWildcard, response.get())), listener::onFailure)); + ActionListener.wrap( + response -> listener.onResponse(mergedMappings(indexWildcard, response.getIndices(), response.get())), + listener::onFailure)); } - static IndexResolution mergedMapping(String indexPattern, Map> fieldCaps) { + static IndexResolution mergedMappings(String indexPattern, String[] indexNames, Map> fieldCaps) { if (fieldCaps == null || fieldCaps.isEmpty()) { return IndexResolution.notFound(indexPattern); } - StringBuilder errorMessage = new StringBuilder(); + // merge all indices onto the same one + List indices = buildIndices(indexNames, null, fieldCaps, i -> indexPattern, (n, types) -> { + StringBuilder errorMessage = new StringBuilder(); - NavigableSet>> sortedFields = new TreeSet<>( - // for some reason .reversed doesn't work (prolly due to inference) - Collections.reverseOrder(Comparator.comparing(Entry::getKey))); - sortedFields.addAll(fieldCaps.entrySet()); - - Map hierarchicalMapping = new TreeMap<>(); - Map flattedMapping = new LinkedHashMap<>(); - - // sort keys descending in order to easily detect multi-fields (a.b.c multi-field of a.b) - // without sorting, they can still be detected however without the emptyMap optimization - // (fields without multi-fields have no children) - for (Entry> entry : sortedFields) { + boolean hasUnmapped = types.containsKey(UNMAPPED); - InvalidMappedField invalidField = null; - FieldCapabilities fieldCap = null; - errorMessage.setLength(0); + if (types.size() > (hasUnmapped ? 2 : 1)) { + // build the error message + // and create a MultiTypeField - String name = entry.getKey(); + for (Entry type : types.entrySet()) { + // skip unmapped + if (UNMAPPED.equals(type.getKey())) { + continue; + } - // Skip any of the blacklisted field names. - if (!FIELD_NAMES_BLACKLIST.contains(name)) { - Map types = entry.getValue(); - // field is mapped differently across indices - if (types.size() > 1) { - // build the error message - // and create a MultiTypeField - - for (Entry type : types.entrySet()) { - if (errorMessage.length() > 0) { - errorMessage.append(", "); - } - errorMessage.append("["); - errorMessage.append(type.getKey()); - errorMessage.append("] in "); - errorMessage.append(Arrays.toString(type.getValue().indices())); + if (errorMessage.length() > 0) { + errorMessage.append(", "); } + errorMessage.append("["); + errorMessage.append(type.getKey()); + errorMessage.append("] in "); + errorMessage.append(Arrays.toString(type.getValue().indices())); + } + + errorMessage.insert(0, "mapped as [" + (types.size() - (hasUnmapped ? 1 : 0)) + "] incompatible types: "); - errorMessage.insert(0, "mapped as [" + types.size() + "] incompatible types: "); + return new InvalidMappedField(n, errorMessage.toString()); + } + // type is okay, check aggregation + else { + FieldCapabilities fieldCap = types.values().iterator().next(); - invalidField = new InvalidMappedField(name, errorMessage.toString()); + // validate search/agg-able + if (fieldCap.isAggregatable() && fieldCap.nonAggregatableIndices() != null) { + errorMessage.append("mapped as aggregatable except in "); + errorMessage.append(Arrays.toString(fieldCap.nonAggregatableIndices())); } - // type is okay, check aggregation - else { - fieldCap = types.values().iterator().next(); - - // Skip internal fields (name starting with underscore and its type reported by field_caps starts with underscore - // as well). A meta field named "_version", for example, has the type named "_version". - if (name.startsWith("_") && fieldCap.getType().startsWith("_")) { - continue; - } - // validate search/agg-able - if (fieldCap.isAggregatable() && fieldCap.nonAggregatableIndices() != null) { - errorMessage.append("mapped as aggregatable except in "); - errorMessage.append(Arrays.toString(fieldCap.nonAggregatableIndices())); - } - if (fieldCap.isSearchable() && fieldCap.nonSearchableIndices() != null) { - if (errorMessage.length() > 0) { - errorMessage.append(","); - } - errorMessage.append("mapped as searchable except in "); - errorMessage.append(Arrays.toString(fieldCap.nonSearchableIndices())); - } - + if (fieldCap.isSearchable() && fieldCap.nonSearchableIndices() != null) { if (errorMessage.length() > 0) { - invalidField = new InvalidMappedField(name, errorMessage.toString()); + errorMessage.append(","); } + errorMessage.append("mapped as searchable except in "); + errorMessage.append(Arrays.toString(fieldCap.nonSearchableIndices())); } - - // validation passes - create the field - // if the name wasn't added before - final InvalidMappedField invalidF = invalidField; - final FieldCapabilities fieldCapab = fieldCap; - - EsField esField = flattedMapping.get(name); - if (esField == null || (invalidF != null && (esField instanceof InvalidMappedField) == false)) { - createField(name, fieldCaps, hierarchicalMapping, flattedMapping, s -> { - return invalidF != null ? invalidF : createField(s, fieldCapab.getType(), emptyMap(), fieldCapab.isAggregatable()); - }); + + if (errorMessage.length() > 0) { + return new InvalidMappedField(n, errorMessage.toString()); } } + + // everything checks + return null; + }); + + if (indices.size() != 1) { + throw new SqlIllegalArgumentException("Incorrect merging of mappings (likely due to a bug) - expect 1 but found [{}]", + indices.size()); } - - return IndexResolution.valid(new EsIndex(indexPattern, hierarchicalMapping)); + + return IndexResolution.valid(indices.get(0)); } private static EsField createField(String fieldName, Map> globalCaps, - Map hierarchicalMapping, Map flattedMapping, + Map hierarchicalMapping, + Map flattedMapping, Function field) { Map parentProps = hierarchicalMapping; @@ -359,17 +341,22 @@ private static EsField createField(String fieldName, Map createField(s, DataType.OBJECT.name(), new TreeMap<>(), false); } else { - FieldCapabilities parentCap = map.values().iterator().next(); - fieldFunction = s -> createField(s, parentCap.getType(), new TreeMap<>(), parentCap.isAggregatable()); + Iterator iterator = map.values().iterator(); + FieldCapabilities parentCap = iterator.next(); + if (iterator.hasNext() && UNMAPPED.equals(parentCap.getType())) { + parentCap = iterator.next(); + } + final FieldCapabilities parentC = parentCap; + fieldFunction = s -> createField(s, parentC.getType(), new TreeMap<>(), parentC.isAggregatable()); } - + parent = createField(parentName, globalCaps, hierarchicalMapping, flattedMapping, fieldFunction); } parentProps = parent.getProperties(); } EsField esField = field.apply(fieldName); - + parentProps.put(fieldName, esField); flattedMapping.put(fullFieldName, esField); @@ -394,108 +381,133 @@ private static EsField createField(String fieldName, String typeName, Map> listener) { - GetIndexRequest getIndexRequest = createGetIndexRequest(indexWildcard); - client.admin().indices().getIndex(getIndexRequest, ActionListener.wrap(getIndexResponse -> { - ImmutableOpenMap> mappings = getIndexResponse.getMappings(); - ImmutableOpenMap> aliases = getIndexResponse.getAliases(); - - List results = new ArrayList<>(mappings.size()); - Pattern pattern = javaRegex != null ? Pattern.compile(javaRegex) : null; - for (ObjectObjectCursor> indexMappings : mappings) { - /* - * We support wildcard expressions here, and it's only for commands that only perform the get index call. - * We can and simply have to use the concrete index name and show that to users. - * Get index against an alias with security enabled, where the user has only access to get mappings for the alias - * and not the concrete index: there is a well known information leak of the concrete index name in the response. - */ - String concreteIndex = indexMappings.key; - - // take into account aliases - List aliasMetadata = aliases.get(concreteIndex); - boolean matchesAlias = false; - if (pattern != null && aliasMetadata != null) { - for (AliasMetaData aliasMeta : aliasMetadata) { - if (pattern.matcher(aliasMeta.alias()).matches()) { - matchesAlias = true; - break; - } - } - } - - if (pattern == null || matchesAlias || pattern.matcher(concreteIndex).matches()) { - IndexResolution getIndexResult = buildGetIndexResult(concreteIndex, concreteIndex, indexMappings.value); - if (getIndexResult.isValid()) { - results.add(getIndexResult.get()); - } - } - } - results.sort(Comparator.comparing(EsIndex::name)); - listener.onResponse(results); - }, listener::onFailure)); + FieldCapabilitiesRequest fieldRequest = createFieldCapsRequest(indexWildcard); + client.fieldCaps(fieldRequest, + ActionListener.wrap( + response -> listener.onResponse(separateMappings(indexWildcard, javaRegex, response.getIndices(), response.get())), + listener::onFailure)); + } - private static GetIndexRequest createGetIndexRequest(String index) { - return new GetIndexRequest() - .local(true) - .indices(Strings.commaDelimitedListToStringArray(index)) - //lenient because we throw our own errors looking at the response e.g. if something was not resolved - //also because this way security doesn't throw authorization exceptions but rather honours ignore_unavailable - .indicesOptions(IndicesOptions.lenientExpandOpen()); + static List separateMappings(String indexPattern, String javaRegex, String[] indexNames, + Map> fieldCaps) { + return buildIndices(indexNames, javaRegex, fieldCaps, Function.identity(), (s, cap) -> null); + } + + private static class Fields { + final Map hierarchicalMapping = new TreeMap<>(); + final Map flattedMapping = new LinkedHashMap<>(); } - private static IndexResolution buildGetIndexResult(String concreteIndex, String indexOrAlias, - ImmutableOpenMap mappings) { + /** + * Assemble an index-based mapping from the field caps (which is field based) by looking at the indices associated with + * each field. + */ + private static List buildIndices(String[] indexNames, String javaRegex, Map> fieldCaps, + Function indexNameProcessor, + BiFunction, InvalidMappedField> validityVerifier) { + + if (indexNames == null || indexNames.length == 0) { + return emptyList(); + } - // Make sure that the index contains only a single type - MappingMetaData singleType = null; - List typeNames = null; - for (ObjectObjectCursor type : mappings) { - //Default mappings are ignored as they are applied to each type. Each type alone holds all of its fields. - if ("_default_".equals(type.key)) { + final List resolvedIndices = asList(indexNames); + Map indices = new LinkedHashMap<>(resolvedIndices.size()); + Pattern pattern = javaRegex != null ? Pattern.compile(javaRegex) : null; + + // sort fields in reverse order to build the field hierarchy + Set>> sortedFields = new TreeSet<>( + Collections.reverseOrder(Comparator.comparing(Entry::getKey))); + + sortedFields.addAll(fieldCaps.entrySet()); + + for (Entry> entry : sortedFields) { + String fieldName = entry.getKey(); + Map types = entry.getValue(); + + // ignore size added by the mapper plugin + if (FIELD_NAMES_BLACKLIST.contains(fieldName)) { continue; } - if (singleType != null) { - // There are more than one types - if (typeNames == null) { - typeNames = new ArrayList<>(); - typeNames.add(singleType.type()); + + // apply verification + final InvalidMappedField invalidField = validityVerifier.apply(fieldName, types); + + // filter meta fields and unmapped + FieldCapabilities unmapped = types.get(UNMAPPED); + Set unmappedIndices = unmapped != null ? new HashSet<>(asList(unmapped.indices())) : emptySet(); + + // check each type + for (Entry typeEntry : types.entrySet()) { + FieldCapabilities typeCap = typeEntry.getValue(); + String[] capIndices = typeCap.indices(); + + // Skip internal fields (name starting with underscore and its type reported by field_caps starts + // with underscore as well). A meta field named "_version", for example, has the type named "_version". + if (typeEntry.getKey().startsWith("_") && typeCap.getType().startsWith("_")) { + continue; + } + + // compute the actual indices - if any are specified, take into account the unmapped indices + List concreteIndices = null; + if (capIndices != null) { + if (unmappedIndices.isEmpty() == true) { + concreteIndices = asList(capIndices); + } else { + concreteIndices = new ArrayList<>(capIndices.length - unmappedIndices.size() + 1); + for (String capIndex : capIndices) { + // add only indices that have a mapping + if (unmappedIndices.contains(capIndex) == false) { + concreteIndices.add(capIndex); + } + } + } + } else { + concreteIndices = resolvedIndices; + } + + // put the field in their respective mappings + for (String index : concreteIndices) { + if (pattern == null || pattern.matcher(index).matches()) { + String indexName = indexNameProcessor.apply(index); + Fields indexFields = indices.get(indexName); + if (indexFields == null) { + indexFields = new Fields(); + indices.put(indexName, indexFields); + } + EsField field = indexFields.flattedMapping.get(fieldName); + if (field == null || (invalidField != null && (field instanceof InvalidMappedField) == false)) { + createField(fieldName, fieldCaps, indexFields.hierarchicalMapping, indexFields.flattedMapping, s -> + invalidField != null ? invalidField : + createField(s, typeCap.getType(), emptyMap(), typeCap.isAggregatable())); + } + } } - typeNames.add(type.key); } - singleType = type.value; } - if (singleType == null) { - return IndexResolution.invalid("[" + indexOrAlias + "] doesn't have any types so it is incompatible with sql"); - } else if (typeNames != null) { - Collections.sort(typeNames); - return IndexResolution.invalid( - "[" + indexOrAlias + "] contains more than one type " + typeNames + " so it is incompatible with sql"); - } else { - try { - Map mapping = Types.fromEs(singleType.sourceAsMap()); - return IndexResolution.valid(new EsIndex(indexOrAlias, mapping)); - } catch (MappingException ex) { - return IndexResolution.invalid(ex.getMessage()); - } + // return indices in ascending order + List foundIndices = new ArrayList<>(indices.size()); + for (Entry entry : indices.entrySet()) { + foundIndices.add(new EsIndex(entry.getKey(), entry.getValue().hierarchicalMapping)); } + foundIndices.sort(Comparator.comparing(EsIndex::name)); + return foundIndices; + } + + private static FieldCapabilitiesRequest createFieldCapsRequest(String index) { + return new FieldCapabilitiesRequest() + .indices(Strings.commaDelimitedListToStringArray(index)) + .fields("*") + .includeUnmapped(true) + //lenient because we throw our own errors looking at the response e.g. if something was not resolved + //also because this way security doesn't throw authorization exceptions but rather honors ignore_unavailable + .indicesOptions(IndicesOptions.lenientExpandOpen()); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumns.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumns.java index 68cfefe7fb572..674045ab692fa 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumns.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumns.java @@ -116,7 +116,7 @@ public void execute(SqlSession session, ActionListener listener) { Pattern columnMatcher = columnPattern != null ? Pattern.compile(columnPattern.asJavaRegex()) : null; - // special case fo '%' (translated to *) + // special case for '%' (translated to *) if ("*".equals(idx)) { session.indexResolver().resolveAsSeparateMappings(idx, regex, ActionListener.wrap(esIndices -> { List> rows = new ArrayList<>(); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java index 0f4f8f030506c..561347b89979c 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java @@ -17,6 +17,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.stream.Stream; import static org.elasticsearch.common.logging.LoggerMessageFormat.format; @@ -28,11 +29,7 @@ public void testMergeSameMapping() throws Exception { assertNotSame(oneMapping, sameMapping); assertEquals(oneMapping, sameMapping); - String wildcard = "*"; - - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, fromMappings( - new EsIndex("a", oneMapping), - new EsIndex("b", sameMapping))); + IndexResolution resolution = merge(new EsIndex("a", oneMapping), new EsIndex("b", sameMapping)); assertTrue(resolution.isValid()); assertEqualsMaps(oneMapping, resolution.get().mapping()); @@ -44,10 +41,7 @@ public void testMergeCompatibleMapping() throws Exception { assertNotEquals(basicMapping, numericMapping); - String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, fromMappings( - new EsIndex("basic", basicMapping), - new EsIndex("numeric", numericMapping))); + IndexResolution resolution = merge(new EsIndex("basic", basicMapping), new EsIndex("numeric", numericMapping)); assertTrue(resolution.isValid()); assertEquals(basicMapping.size() + numericMapping.size(), resolution.get().mapping().size()); @@ -60,8 +54,7 @@ public void testMergeIncompatibleTypes() throws Exception { assertNotEquals(basicMapping, incompatible); String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, - fromMappings(new EsIndex("basic", basicMapping), new EsIndex("incompatible", incompatible))); + IndexResolution resolution = merge(new EsIndex("basic", basicMapping), new EsIndex("incompatible", incompatible)); assertTrue(resolution.isValid()); @@ -82,8 +75,7 @@ public void testMergeIncompatibleCapabilities() throws Exception { assertNotEquals(basicMapping, incompatible); String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, - fromMappings(new EsIndex("basic", basicMapping), new EsIndex("incompatible", incompatible))); + IndexResolution resolution = merge(new EsIndex("basic", basicMapping), new EsIndex("incompatible", incompatible)); assertTrue(resolution.isValid()); @@ -97,8 +89,7 @@ public void testMergeIncompatibleCapabilities() throws Exception { public void testMultiLevelObjectMappings() throws Exception { Map dottedMapping = TypesTests.loadMapping("mapping-dotted-field.json", true); - String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, fromMappings(new EsIndex("a", dottedMapping))); + IndexResolution resolution = merge(new EsIndex("a", dottedMapping)); assertTrue(resolution.isValid()); assertEqualsMaps(dottedMapping, resolution.get().mapping()); @@ -107,8 +98,7 @@ public void testMultiLevelObjectMappings() throws Exception { public void testMultiLevelNestedMappings() throws Exception { Map nestedMapping = TypesTests.loadMapping("mapping-nested.json", true); - String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, fromMappings(new EsIndex("a", nestedMapping))); + IndexResolution resolution = merge(new EsIndex("a", nestedMapping)); assertTrue(resolution.isValid()); assertEqualsMaps(nestedMapping, resolution.get().mapping()); @@ -122,7 +112,7 @@ public void testMetaFieldsAreIgnored() throws Exception { addFieldCaps(fieldCaps, "text", "keyword", true, true); String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, fieldCaps); + IndexResolution resolution = IndexResolver.mergedMappings(wildcard, new String[] { "index" }, fieldCaps); assertTrue(resolution.isValid()); EsIndex esIndex = resolution.get(); @@ -157,7 +147,7 @@ public void testMergeIncompatibleCapabilitiesOfObjectFields() throws Exception { String wildcard = "*"; - IndexResolution resolution = IndexResolver.mergedMapping(wildcard, fieldCaps); + IndexResolution resolution = IndexResolver.mergedMappings(wildcard, new String[] { "one-index" }, fieldCaps); assertTrue(resolution.isValid()); @@ -175,8 +165,40 @@ public void testMergeIncompatibleCapabilitiesOfObjectFields() throws Exception { } + + public void testSeparateSameMappingDifferentIndices() throws Exception { + Map oneMapping = TypesTests.loadMapping("mapping-basic.json", true); + Map sameMapping = TypesTests.loadMapping("mapping-basic.json", true); + assertNotSame(oneMapping, sameMapping); + assertEquals(oneMapping, sameMapping); + + List indices = separate(new EsIndex("a", oneMapping), new EsIndex("b", sameMapping)); + + assertEquals(2, indices.size()); + assertEqualsMaps(oneMapping, indices.get(0).mapping()); + assertEqualsMaps(sameMapping, indices.get(1).mapping()); + } + + public void testSeparateIncompatibleTypes() throws Exception { + Map basicMapping = TypesTests.loadMapping("mapping-basic.json", true); + Map incompatible = TypesTests.loadMapping("mapping-basic-incompatible.json"); + + assertNotEquals(basicMapping, incompatible); + + List indices = separate(new EsIndex("basic", basicMapping), new EsIndex("incompatible", incompatible)); + + assertEquals(2, indices.size()); + assertEqualsMaps(basicMapping, indices.get(0).mapping()); + assertEqualsMaps(incompatible, indices.get(1).mapping()); + } + public static IndexResolution merge(EsIndex... indices) { - return IndexResolver.mergedMapping("*", fromMappings(indices)); + return IndexResolver.mergedMappings("*", Stream.of(indices).map(EsIndex::name).toArray(String[]::new), fromMappings(indices)); + } + + public static List separate(EsIndex... indices) { + return IndexResolver.separateMappings("*", null, Stream.of(indices).map(EsIndex::name).toArray(String[]::new), + fromMappings(indices)); } public static Map> fromMappings(EsIndex... indices) { @@ -215,7 +237,7 @@ private static void addFieldCaps(String parent, EsField field, String indexName, } FieldCapabilities caps = map.computeIfAbsent(field.getDataType().typeName, esType -> new UpdateableFieldCapabilities(fieldName, esType, - isSearchable(field.getDataType()), + isSearchable(field.getDataType()), isAggregatable(field.getDataType()))); if (!field.isAggregatable()) { From 553b9ff082acf99c0346b7b958615a6e76455d5c Mon Sep 17 00:00:00 2001 From: Melori Arellano Date: Mon, 22 Apr 2019 14:38:53 -0600 Subject: [PATCH 144/260] [DOCS] Add missing setting skip_unavailable to example The example to delete a remote cluster is missing the `skip_unavailable` setting which results in an error: ``` "type": "illegal_argument_exception", "reason": "missing required setting [cluster.remote.tiny-test.seeds] for setting [cluster.remote.tiny-test.skip_unavailable]" ``` --- .../modules/remote-clusters.asciidoc | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/docs/reference/modules/remote-clusters.asciidoc b/docs/reference/modules/remote-clusters.asciidoc index 1c51cda907cf1..e654a50e6122e 100644 --- a/docs/reference/modules/remote-clusters.asciidoc +++ b/docs/reference/modules/remote-clusters.asciidoc @@ -79,6 +79,7 @@ cluster: cluster_two: seeds: 127.0.0.1:9301 transport.compress: true <3> + skip_unavailable: true <4> -------------------------------- <1> `cluster_one` and `cluster_two` are arbitrary _cluster aliases_ representing @@ -86,10 +87,12 @@ the connection to each cluster. These names are subsequently used to distinguish between local and remote indices. <2> A keep-alive ping is configured for `cluster_one`. <3> Compression is explicitly enabled for requests to `cluster_two`. +<4> Disconnected remote clusters are optional for `cluster_two`. For more information about the optional transport settings, see <>. + If you use <>, the remote clusters are available on every node in the cluster. For example: @@ -110,7 +113,8 @@ PUT _cluster/settings "seeds": [ "127.0.0.1:9301" ], - "transport.compress": true + "transport.compress": true, + "skip_unavailable": true }, "cluster_three": { "seeds": [ @@ -160,8 +164,7 @@ NOTE: When the compression or ping schedule settings change, all the existing node connections must close and re-open, which can cause in-flight requests to fail. -A remote cluster can be deleted from the cluster settings by setting its seeds -to `null`: +A remote cluster can be deleted from the cluster settings by setting its seeds and optional settings to `null` : [source,js] -------------------------------- @@ -170,8 +173,12 @@ PUT _cluster/settings "persistent": { "cluster": { "remote": { - "cluster_three": { - "seeds": null <1> + "cluster_two": { <1> + "seeds": null, + "skip_unavailable": null, + "transport": { + "compress": null + } } } } @@ -180,8 +187,8 @@ PUT _cluster/settings -------------------------------- // CONSOLE // TEST[continued] -<1> `cluster_three` would be removed from the cluster settings, leaving -`cluster_one` and `cluster_two` intact. +<1> `cluster_two` would be removed from the cluster settings, leaving +`cluster_one` and `cluster_three` intact. [float] [[remote-cluster-settings]] From 73a7a13304e4a07c8dec0aeee6041555a3b836a1 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 22 Apr 2019 21:45:04 -0400 Subject: [PATCH 145/260] Strengthen typing in sorted differences (#41410) We have some convenience methods for computing modifiable and unmodifiable sorted differences. The unmodifiable sorted difference method was not typed to return a SortedSet. This commit addresses that by using the new method Collections#unmodifiableSortedSet so that we can return a SortedSet here too. --- .../cluster/coordination/Reconfigurator.java | 7 ++++--- .../org/elasticsearch/common/util/set/Sets.java | 16 ++++++++-------- .../admin/indices/RestGetMappingAction.java | 3 ++- .../elasticsearch/common/util/set/SetsTests.java | 5 +++-- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java index 81a12ea4d47b4..b189b7ec2cc2d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Reconfigurator.java @@ -32,6 +32,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Set; +import java.util.SortedSet; import java.util.TreeSet; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -123,8 +124,8 @@ public VotingConfiguration reconfigure(Set liveNodes, Set final Set liveInConfigIds = new TreeSet<>(currentConfig.getNodeIds()); liveInConfigIds.retainAll(liveNodeIds); - final Set inConfigNotLiveIds = Sets.unmodifiableSortedDifference(currentConfig.getNodeIds(), liveInConfigIds); - final Set nonRetiredInConfigNotLiveIds = new TreeSet<>(inConfigNotLiveIds); + final SortedSet inConfigNotLiveIds = Sets.unmodifiableSortedDifference(currentConfig.getNodeIds(), liveInConfigIds); + final SortedSet nonRetiredInConfigNotLiveIds = new TreeSet<>(inConfigNotLiveIds); nonRetiredInConfigNotLiveIds.removeAll(retiredNodeIds); final Set nonRetiredInConfigLiveIds = new TreeSet<>(liveInConfigIds); @@ -141,7 +142,7 @@ public VotingConfiguration reconfigure(Set liveNodes, Set nonRetiredInConfigLiveMasterIds = Collections.emptySet(); } - final Set nonRetiredLiveNotInConfigIds = Sets.sortedDifference(liveNodeIds, currentConfig.getNodeIds()); + final SortedSet nonRetiredLiveNotInConfigIds = Sets.sortedDifference(liveNodeIds, currentConfig.getNodeIds()); nonRetiredLiveNotInConfigIds.removeAll(retiredNodeIds); /* diff --git a/server/src/main/java/org/elasticsearch/common/util/set/Sets.java b/server/src/main/java/org/elasticsearch/common/util/set/Sets.java index 9ab4ee59b4189..c17c056c0a70c 100644 --- a/server/src/main/java/org/elasticsearch/common/util/set/Sets.java +++ b/server/src/main/java/org/elasticsearch/common/util/set/Sets.java @@ -112,7 +112,7 @@ public static SortedSet sortedDifference(final Set left, final Set * @param the type of the elements of the sets * @return the unmodifiable sorted relative complement of the left set with respect to the right set */ - public static Set unmodifiableSortedDifference(final Set left, final Set right) { + public static SortedSet unmodifiableSortedDifference(final Set left, final Set right) { Objects.requireNonNull(left); Objects.requireNonNull(right); return left.stream().filter(k -> right.contains(k) == false).collect(toUnmodifiableSortedSet()); @@ -135,11 +135,11 @@ public static Collector, SortedSet> toSortedSet() { * @param the type of the input elements * @return an unmodifiable set where the underlying set is sorted */ - public static Collector, Set> toUnmodifiableSortedSet() { + public static Collector, SortedSet> toUnmodifiableSortedSet() { return new UnmodifiableSortedSetCollector<>(); } - abstract static class AbstractSortedSetCollector> implements Collector, R> { + abstract static class AbstractSortedSetCollector implements Collector, SortedSet> { @Override public Supplier> supplier() { @@ -159,13 +159,13 @@ public BinaryOperator> combiner() { }; } - public abstract Function, R> finisher(); + public abstract Function, SortedSet> finisher(); public abstract Set characteristics(); } - private static class SortedSetCollector extends AbstractSortedSetCollector> { + private static class SortedSetCollector extends AbstractSortedSetCollector { @Override public Function, SortedSet> finisher() { @@ -182,11 +182,11 @@ public Set characteristics() { } - private static class UnmodifiableSortedSetCollector extends AbstractSortedSetCollector> { + private static class UnmodifiableSortedSetCollector extends AbstractSortedSetCollector { @Override - public Function, Set> finisher() { - return Collections::unmodifiableSet; + public Function, SortedSet> finisher() { + return Collections::unmodifiableSortedSet; } @Override diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java index a1854eaf27e6d..f87332438b123 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java @@ -50,6 +50,7 @@ import java.util.List; import java.util.Locale; import java.util.Set; +import java.util.SortedSet; import java.util.stream.Collectors; import static org.elasticsearch.rest.RestRequest.Method.GET; @@ -116,7 +117,7 @@ public RestResponse buildResponse(final GetMappingsResponse response, final XCon } } - final Set difference = Sets.sortedDifference(Arrays.stream(types).collect(Collectors.toSet()), typeNames); + final SortedSet difference = Sets.sortedDifference(Arrays.stream(types).collect(Collectors.toSet()), typeNames); // now remove requested aliases that contain wildcards that are simple matches final List matches = new ArrayList<>(); diff --git a/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java b/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java index df6f264bdd58d..6e52299d2255d 100644 --- a/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/set/SetsTests.java @@ -25,6 +25,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Set; +import java.util.SortedSet; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -54,11 +55,11 @@ public void testUnmodifiableSortedDifference() { } private void runSortedDifferenceTest( - final BiFunction, Set, Set> sortedDifference, + final BiFunction, Set, SortedSet> sortedDifference, final Consumer> asserter) { final int endExclusive = randomIntBetween(0, 256); final Tuple, Set> sets = randomSets(endExclusive); - final Set difference = sortedDifference.apply(sets.v1(), sets.v2()); + final SortedSet difference = sortedDifference.apply(sets.v1(), sets.v2()); assertDifference(endExclusive, sets, difference); final Iterator it = difference.iterator(); if (it.hasNext()) { From 01d4c541ae10bc337e8a3dc8cca0da8a7f82eabf Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 22 Apr 2019 21:46:32 -0400 Subject: [PATCH 146/260] Avoid concurrent modification in mock log appender (#41424) It can be the case that while we are setting up expectations that also a log message is appended. For example, if we are setting up these expectations after a cluster has formed and messages start being sent around the cluster. In this case, we would hit a concurrent modification exception while we are mutating the expectations, and also while the expectations are being iterated over as a message is appended. This commit avoids this by using a copy-on-write array list which is safe for concurrent modification and iteration. Note that another possible approach here is to use synchronized, but that seems unnecessary since we don't appear to rely on messages that are sent while we are setting up expectations. Rather, we are setting up some expectations and some situation that we think will cause those expectations to be met. Using copy-on-write array list here is nice since we avoid bottlenecking these tests on synchronizing these methods. --- .../java/org/elasticsearch/test/MockLogAppender.java | 9 +++++++-- .../transport/AbstractSimpleTransportTestCase.java | 1 - 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/MockLogAppender.java b/test/framework/src/main/java/org/elasticsearch/test/MockLogAppender.java index ccb010e2a915e..22a12e74c3516 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/MockLogAppender.java +++ b/test/framework/src/main/java/org/elasticsearch/test/MockLogAppender.java @@ -24,8 +24,8 @@ import org.apache.logging.log4j.core.filter.RegexFilter; import org.elasticsearch.common.regex.Regex; -import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.regex.Pattern; import static org.hamcrest.CoreMatchers.equalTo; @@ -42,7 +42,12 @@ public class MockLogAppender extends AbstractAppender { public MockLogAppender() throws IllegalAccessException { super("mock", RegexFilter.createFilter(".*(\n.*)*", new String[0], false, null, null), null); - expectations = new ArrayList<>(); + /* + * We use a copy-on-write array list since log messages could be appended while we are setting up expectations. When that occurs, + * we would run into a concurrent modification exception from the iteration over the expectations in #append, concurrent with a + * modification from #addExpectation. + */ + expectations = new CopyOnWriteArrayList<>(); } public void addExpectation(LoggingExpectation expectation) { diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index 3fddc27297b1d..960d25c960946 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -993,7 +993,6 @@ public void handleException(TransportException exp) { } @TestLogging(value = "org.elasticsearch.transport.TransportService.tracer:trace") - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/40586") public void testTracerLog() throws Exception { TransportRequestHandler handler = (request, channel, task) -> channel.sendResponse(new StringMessageResponse("")); TransportRequestHandler handlerWithError = (request, channel, task) -> { From c3b1f8ddbd00a8fc344b8e5e24a8d3da9951dc23 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Tue, 23 Apr 2019 10:37:29 +0100 Subject: [PATCH 147/260] [TEST] Mute BulkRejectionIT.testBulkRejectionAfterDynamicMappingUpdate Due to https://github.com/elastic/elasticsearch/issues/41363 --- .../test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java index c82d825b9424f..1d7a5245a225e 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java @@ -49,6 +49,7 @@ protected int numberOfShards() { return 5; } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41363") public void testBulkRejectionAfterDynamicMappingUpdate() throws Exception { final String index = "test"; assertAcked(prepareCreate(index)); From f9af2d8c9beac2c0f06c2dd521f88338d4676926 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Tue, 23 Apr 2019 13:56:21 +0300 Subject: [PATCH 148/260] Preserver java 8 class format for in build tools (#41345) Keep the minimum compatability classes at the same version for a bit longer. --- buildSrc/build.gradle | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index 32845566a16f7..c2f0351c2f174 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -84,10 +84,8 @@ sourceSets { } } compileMinimumRuntimeGroovy { - // We can't use BuildPlugin here, so read from file - String minimumRuntimeVersion = file('src/main/resources/minimumRuntimeVersion').text.trim() - targetCompatibility = minimumRuntimeVersion - sourceCompatibility = minimumRuntimeVersion + targetCompatibility = 8 + sourceCompatibility = 8 } dependencies { if (project.ext.has("isEclipse") == false || project.ext.isEclipse == false) { From 3b0dd4a4ed6d27833ac59ac2f26c8883fbbef3f7 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Tue, 23 Apr 2019 13:38:35 +0200 Subject: [PATCH 149/260] [DOCS] change source and dest to be nested and add a query to the example (#41293) update source and dest to reflect the changes done in #40396 and add a query to the example. --- .../data-frames/apis/put-transform.asciidoc | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/docs/reference/data-frames/apis/put-transform.asciidoc b/docs/reference/data-frames/apis/put-transform.asciidoc index 68d7b4b049213..1889121ae5d38 100644 --- a/docs/reference/data-frames/apis/put-transform.asciidoc +++ b/docs/reference/data-frames/apis/put-transform.asciidoc @@ -25,9 +25,10 @@ Instantiates a {dataframe-transform}. ==== Request Body -`source`:: The source index or index pattern. +`source`:: (object) The source configuration, consisting of `index` and optionally +a `query`. -`dest`:: The destination index. +`dest`:: (object) The destination configuration, consisting of `index`. `pivot`:: Defines the pivot function `group by` fields and the aggregation to reduce the data. @@ -44,8 +45,19 @@ sample data: -------------------------------------------------- PUT _data_frame/transforms/ecommerce_transform { - "source": "kibana_sample_data_ecommerce", - "dest": "kibana_sample_data_ecommerce_transform", + "source": { + "index": "kibana_sample_data_ecommerce", + "query": { + "term": { + "geoip.continent_name": { + "value": "Asia" + } + } + } + }, + "dest": { + "index": "kibana_sample_data_ecommerce_transform" + }, "pivot": { "group_by": { "customer_id": { @@ -74,4 +86,4 @@ When the transform is created, you receive the following results: "acknowledged" : true } ---- -// NOTCONSOLE \ No newline at end of file +// NOTCONSOLE From a730b4489b424ff2d58748cbdef52b0c3220634f Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 23 Apr 2019 07:18:27 -0500 Subject: [PATCH 150/260] [ML][Data Frame] Moving destination creation to _start (#41416) * [ML][Data Frame] Moving destination creation to _start * slight refactor of DataFrameAuditor constructor --- .../core/dataframe/DataFrameMessages.java | 1 - .../integration/DataFrameMetaDataIT.java | 3 +- .../integration/DataFramePivotRestIT.java | 10 +- .../integration/DataFrameRestTestCase.java | 2 +- .../xpack/dataframe/DataFrame.java | 12 +- .../TransportPutDataFrameTransformAction.java | 60 +++------- ...ransportStartDataFrameTransformAction.java | 103 +++++++++++++++--- .../notifications/DataFrameAuditor.java | 22 ++++ .../transforms/DataFrameIndexer.java | 7 +- ...FrameTransformPersistentTasksExecutor.java | 7 +- .../transforms/DataFrameTransformTask.java | 9 +- .../transforms/DataFrameIndexerTests.java | 8 +- .../test/data_frame/transforms_crud.yml | 21 ---- .../test/data_frame/transforms_start_stop.yml | 25 +++++ 14 files changed, 173 insertions(+), 117 deletions(-) create mode 100644 x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/notifications/DataFrameAuditor.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java index 86dce1b331420..e027191d8a554 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/DataFrameMessages.java @@ -22,7 +22,6 @@ public class DataFrameMessages { public static final String REST_PUT_DATA_FRAME_FAILED_PERSIST_TRANSFORM_CONFIGURATION = "Failed to persist data frame configuration"; public static final String REST_PUT_DATA_FRAME_FAILED_TO_DEDUCE_DEST_MAPPINGS = "Failed to deduce dest mappings"; public static final String REST_PUT_DATA_FRAME_FAILED_TO_CREATE_DEST_INDEX = "Failed to create dest index"; - public static final String REST_PUT_DATA_FRAME_DEST_INDEX_ALREADY_EXISTS = "dest index [{0}] already exists"; public static final String REST_PUT_DATA_FRAME_SOURCE_INDEX_MISSING = "Source index [{0}] does not exist"; public static final String REST_PUT_DATA_FRAME_INCONSISTENT_ID = "Inconsistent id; ''{0}'' specified in the body differs from ''{1}'' specified as a URL argument"; diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameMetaDataIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameMetaDataIT.java index d278c78842c39..26a957ea055c2 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameMetaDataIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameMetaDataIT.java @@ -37,9 +37,10 @@ public void createIndexes() throws IOException { indicesCreated = true; } - public void testMetaData() throws IOException { + public void testMetaData() throws Exception { long testStarted = System.currentTimeMillis(); createPivotReviewsTransform("test_meta", "pivot_reviews", null); + startAndWaitForTransform("test_meta", "pivot_reviews"); Response mappingResponse = client().performRequest(new Request("GET", "pivot_reviews/_mapping")); diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java index 6ff97e1ed9d26..fd89a32a81713 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java @@ -115,9 +115,9 @@ public void testHistogramPivot() throws Exception { createDataframeTransformRequest.setJsonEntity(config); Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); - assertTrue(indexExists(dataFrameIndex)); startAndWaitForTransform(transformId, dataFrameIndex); + assertTrue(indexExists(dataFrameIndex)); // we expect 3 documents as there shall be 5 unique star values and we are bucketing every 2 starting at 0 Map indexStats = getAsMap(dataFrameIndex + "/_stats"); @@ -174,9 +174,9 @@ public void testBiggerPivot() throws Exception { createDataframeTransformRequest.setJsonEntity(config); Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); - assertTrue(indexExists(dataFrameIndex)); startAndWaitForTransform(transformId, dataFrameIndex, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); + assertTrue(indexExists(dataFrameIndex)); // we expect 27 documents as there shall be 27 user_id's Map indexStats = getAsMap(dataFrameIndex + "/_stats"); @@ -228,9 +228,9 @@ public void testDateHistogramPivot() throws Exception { createDataframeTransformRequest.setJsonEntity(config); Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); - assertTrue(indexExists(dataFrameIndex)); startAndWaitForTransform(transformId, dataFrameIndex, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); + assertTrue(indexExists(dataFrameIndex)); // we expect 21 documents as there shall be 21 days worth of docs Map indexStats = getAsMap(dataFrameIndex + "/_stats"); @@ -301,9 +301,9 @@ public void testPivotWithMaxOnDateField() throws Exception { createDataframeTransformRequest.setJsonEntity(config); Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); - assertTrue(indexExists(dataFrameIndex)); startAndWaitForTransform(transformId, dataFrameIndex, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); + assertTrue(indexExists(dataFrameIndex)); // we expect 21 documents as there shall be 21 days worth of docs Map indexStats = getAsMap(dataFrameIndex + "/_stats"); @@ -351,9 +351,9 @@ public void testPivotWithScriptedMetricAgg() throws Exception { createDataframeTransformRequest.setJsonEntity(config); Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); - assertTrue(indexExists(dataFrameIndex)); startAndWaitForTransform(transformId, dataFrameIndex, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); + assertTrue(indexExists(dataFrameIndex)); // we expect 27 documents as there shall be 27 user_id's Map indexStats = getAsMap(dataFrameIndex + "/_stats"); diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java index 89bdafbde2b28..c0e6c97fd6915 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java @@ -164,7 +164,6 @@ protected void createPivotReviewsTransform(String transformId, String dataFrameI Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); - assertTrue(indexExists(dataFrameIndex)); } protected void startDataframeTransform(String transformId, boolean force) throws IOException { @@ -195,6 +194,7 @@ protected void startAndWaitForTransform(String transformId, String dataFrameInde protected void startAndWaitForTransform(String transformId, String dataFrameIndex, String authHeader) throws Exception { // start the transform startDataframeTransform(transformId, false, authHeader); + assertTrue(indexExists(dataFrameIndex)); // wait until the dataframe has been created and all data is available waitForDataFrameCheckpoint(transformId); refreshIndex(dataFrameIndex); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrame.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrame.java index 4f689c6a5afcd..b7e6c235f8e6c 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrame.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/DataFrame.java @@ -40,7 +40,6 @@ import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; -import org.elasticsearch.xpack.core.common.notifications.Auditor; import org.elasticsearch.xpack.core.dataframe.action.DeleteDataFrameTransformAction; import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsAction; import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction; @@ -49,7 +48,6 @@ import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformAction; import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction; import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction; -import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; import org.elasticsearch.xpack.dataframe.action.TransportDeleteDataFrameTransformAction; import org.elasticsearch.xpack.dataframe.action.TransportGetDataFrameTransformsAction; @@ -60,6 +58,7 @@ import org.elasticsearch.xpack.dataframe.action.TransportStartDataFrameTransformTaskAction; import org.elasticsearch.xpack.dataframe.action.TransportStopDataFrameTransformAction; import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService; +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.rest.action.RestDeleteDataFrameTransformAction; @@ -85,7 +84,6 @@ import java.util.function.UnaryOperator; import static java.util.Collections.emptyList; -import static org.elasticsearch.xpack.core.ClientHelper.DATA_FRAME_ORIGIN; public class DataFrame extends Plugin implements ActionPlugin, PersistentTaskPlugin { @@ -102,7 +100,7 @@ public class DataFrame extends Plugin implements ActionPlugin, PersistentTaskPlu private final Settings settings; private final boolean transportClientMode; private final SetOnce dataFrameTransformsConfigManager = new SetOnce<>(); - private final SetOnce> dataFrameAuditor = new SetOnce<>(); + private final SetOnce dataFrameAuditor = new SetOnce<>(); private final SetOnce dataFrameTransformsCheckpointService = new SetOnce<>(); private final SetOnce schedulerEngine = new SetOnce<>(); @@ -184,11 +182,7 @@ public Collection createComponents(Client client, ClusterService cluster if (enabled == false || transportClientMode) { return emptyList(); } - dataFrameAuditor.set(new Auditor<>(client, - clusterService.getNodeName(), - DataFrameInternalIndex.AUDIT_INDEX, - DATA_FRAME_ORIGIN, - DataFrameAuditMessage.builder())); + dataFrameAuditor.set(new DataFrameAuditor(client, clusterService.getNodeName())); dataFrameTransformsConfigManager.set(new DataFrameTransformsConfigManager(client, xContentRegistry)); dataFrameTransformsCheckpointService.set(new DataFrameTransformsCheckpointService(client, dataFrameTransformsConfigManager.get())); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java index 6f364d91e44fb..c7f750ecdb210 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java @@ -11,8 +11,6 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; -import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.TransportMasterNodeAction; @@ -50,7 +48,6 @@ import org.elasticsearch.xpack.core.security.authz.permission.ResourcePrivileges; import org.elasticsearch.xpack.core.security.support.Exceptions; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; -import org.elasticsearch.xpack.dataframe.persistence.DataframeIndex; import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; import java.io.IOException; @@ -117,18 +114,6 @@ protected void masterOperation(Request request, ClusterState clusterState, Actio return; } - final String[] dest = indexNameExpressionResolver.concreteIndexNames(clusterState, - IndicesOptions.lenientExpandOpen(), - config.getDestination().getIndex()); - - if (dest.length > 0) { - listener.onFailure(new ElasticsearchStatusException( - DataFrameMessages.getMessage(DataFrameMessages.REST_PUT_DATA_FRAME_DEST_INDEX_ALREADY_EXISTS, - config.getDestination().getIndex()), - RestStatus.BAD_REQUEST)); - return; - } - for(String src : config.getSource().getIndex()) { if (indexNameExpressionResolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), src).length == 0) { listener.onFailure(new ElasticsearchStatusException( @@ -145,9 +130,19 @@ protected void masterOperation(Request request, ClusterState clusterState, Actio .indices(config.getSource().getIndex()) .privileges("read") .build(); + String[] destPrivileges = new String[3]; + destPrivileges[0] = "read"; + destPrivileges[1] = "index"; + // If the destination index does not exist, we can assume that we may have to create it on start. + // We should check that the creating user has the privileges to create the index. + if (indexNameExpressionResolver.concreteIndexNames(clusterState, + IndicesOptions.lenientExpandOpen(), + config.getDestination().getIndex()).length == 0) { + destPrivileges[2] = "create_index"; + } RoleDescriptor.IndicesPrivileges destIndexPrivileges = RoleDescriptor.IndicesPrivileges.builder() .indices(config.getDestination().getIndex()) - .privileges("read", "index", "create_index") + .privileges(destPrivileges) .build(); HasPrivilegesRequest privRequest = new HasPrivilegesRequest(); @@ -202,41 +197,12 @@ private void putDataFrame(DataFrameTransformConfig config, ActionListener Return the listener, or clean up destination index on failure. ActionListener putTransformConfigurationListener = ActionListener.wrap( putTransformConfigurationResult -> listener.onResponse(new Response(true)), - putTransformConfigurationException -> - ClientHelper.executeAsyncWithOrigin(client, - ClientHelper.DATA_FRAME_ORIGIN, - DeleteIndexAction.INSTANCE, - new DeleteIndexRequest(config.getDestination().getIndex()), ActionListener.wrap( - deleteIndexResponse -> listener.onFailure(putTransformConfigurationException), - deleteIndexException -> { - String msg = "Failed to delete destination index after creating transform [" + config.getId() + "] failed"; - listener.onFailure( - new ElasticsearchStatusException(msg, - RestStatus.INTERNAL_SERVER_ERROR, - putTransformConfigurationException)); - }) - ) + listener::onFailure ); // <4> Put our transform - ActionListener createDestinationIndexListener = ActionListener.wrap( - createIndexResult -> dataFrameTransformsConfigManager.putTransformConfiguration(config, putTransformConfigurationListener), - createDestinationIndexException -> listener.onFailure( - new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_CREATE_DEST_INDEX, - createDestinationIndexException)) - ); - - // <3> Create the destination index - ActionListener> deduceMappingsListener = ActionListener.wrap( - mappings -> DataframeIndex.createDestinationIndex(client, config, mappings, createDestinationIndexListener), - deduceTargetMappingsException -> listener.onFailure( - new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_DEDUCE_DEST_MAPPINGS, - deduceTargetMappingsException)) - ); - - // <2> Deduce our mappings for the destination index ActionListener pivotValidationListener = ActionListener.wrap( - validationResult -> pivot.deduceMappings(client, deduceMappingsListener), + validationResult -> dataFrameTransformsConfigManager.putTransformConfiguration(config, putTransformConfigurationListener), validationException -> listener.onFailure( new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_VALIDATE_DATA_FRAME_CONFIGURATION, validationException)) diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java index f68e246ed860b..0aeb757e5625c 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java @@ -6,10 +6,14 @@ package org.elasticsearch.xpack.dataframe.action; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; @@ -35,32 +39,40 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; +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.transforms.pivot.Pivot; import java.util.Collection; +import java.util.Map; import java.util.function.Consumer; import java.util.function.Predicate; public class TransportStartDataFrameTransformAction extends TransportMasterNodeAction { + private static final Logger logger = LogManager.getLogger(TransportStartDataFrameTransformAction.class); private final XPackLicenseState licenseState; private final DataFrameTransformsConfigManager dataFrameTransformsConfigManager; private final PersistentTasksService persistentTasksService; private final Client client; + private final DataFrameAuditor auditor; @Inject public TransportStartDataFrameTransformAction(TransportService transportService, ActionFilters actionFilters, ClusterService clusterService, XPackLicenseState licenseState, ThreadPool threadPool, IndexNameExpressionResolver indexNameExpressionResolver, DataFrameTransformsConfigManager dataFrameTransformsConfigManager, - PersistentTasksService persistentTasksService, Client client) { + PersistentTasksService persistentTasksService, Client client, + DataFrameAuditor auditor) { super(StartDataFrameTransformAction.NAME, transportService, clusterService, threadPool, actionFilters, StartDataFrameTransformAction.Request::new, indexNameExpressionResolver); this.licenseState = licenseState; this.dataFrameTransformsConfigManager = dataFrameTransformsConfigManager; this.persistentTasksService = persistentTasksService; this.client = client; + this.auditor = auditor; } @Override @@ -83,7 +95,7 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, } final DataFrameTransform transformTask = createDataFrameTransform(request.getId(), threadPool); - // <3> Set the allocated task's state to STARTED + // <4> Set the allocated task's state to STARTED ActionListener> persistentTaskActionListener = ActionListener.wrap( task -> { waitForDataFrameTaskAllocated(task.getId(), @@ -102,16 +114,9 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, listener::onFailure ); - // <2> Create the task in cluster state so that it will start executing on the node - ActionListener getTransformListener = ActionListener.wrap( - config -> { - if (config.isValid() == false) { - listener.onFailure(new ElasticsearchStatusException( - DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_CONFIG_INVALID, request.getId()), - RestStatus.BAD_REQUEST - )); - return; - } + // <3> Create the task in cluster state so that it will start executing on the node + ActionListener createOrGetIndexListener = ActionListener.wrap( + unused -> { PersistentTasksCustomMetaData.PersistentTask existingTask = getExistingTask(transformTask.getId(), state); if (existingTask == null) { @@ -123,14 +128,14 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, DataFrameTransformState transformState = (DataFrameTransformState)existingTask.getState(); if(transformState.getTaskState() == DataFrameTransformTaskState.FAILED && request.isForce() == false) { listener.onFailure(new ElasticsearchStatusException( - "Unable to start data frame transform [" + config.getId() + + "Unable to start data frame transform [" + request.getId() + "] as it is in a failed state with failure: [" + transformState.getReason() + "]. Use force start to restart data frame transform once error is resolved.", RestStatus.CONFLICT)); } else if (transformState.getTaskState() != DataFrameTransformTaskState.STOPPED && transformState.getTaskState() != DataFrameTransformTaskState.FAILED) { listener.onFailure(new ElasticsearchStatusException( - "Unable to start data frame transform [" + config.getId() + + "Unable to start data frame transform [" + request.getId() + "] as it is in state [" + transformState.getTaskState() + "]", RestStatus.CONFLICT)); } else { persistentTaskActionListener.onResponse(existingTask); @@ -140,10 +145,80 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, listener::onFailure ); + // <2> If the destination index exists, start the task, otherwise deduce our mappings for the destination index and create it + ActionListener getTransformListener = ActionListener.wrap( + config -> { + if (config.isValid() == false) { + listener.onFailure(new ElasticsearchStatusException( + DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_CONFIG_INVALID, request.getId()), + RestStatus.BAD_REQUEST + )); + return; + } + final String destinationIndex = config.getDestination().getIndex(); + String[] dest = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.lenientExpandOpen(), + destinationIndex); + + if(dest.length == 0) { + auditor.info(request.getId(), + "Could not find destination index [" + destinationIndex + "]." + + " Creating index with deduced mappings."); + createDestinationIndex(config, createOrGetIndexListener); + } else { + auditor.info(request.getId(), "Destination index [" + destinationIndex + "] already exists."); + ClientHelper.executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ClientHelper.DATA_FRAME_ORIGIN, + client.admin() + .indices() + .prepareStats(dest) + .clear() + .setDocs(true) + .request(), + ActionListener.wrap( + r -> { + long docTotal = r.getTotal().docs.getCount(); + if (docTotal > 0L) { + auditor.warning(request.getId(), "Non-empty destination index [" + destinationIndex + "]. " + + "Contains [" + docTotal + "] total documents."); + } + createOrGetIndexListener.onResponse(null); + }, + e -> { + String msg = "Unable to determine destination index stats, error: " + e.getMessage(); + logger.error(msg, e); + auditor.warning(request.getId(), msg); + createOrGetIndexListener.onResponse(null); + }), + client.admin().indices()::stats); + } + }, + listener::onFailure + ); + // <1> Get the config to verify it exists and is valid dataFrameTransformsConfigManager.getTransformConfiguration(request.getId(), getTransformListener); } + private void createDestinationIndex(final DataFrameTransformConfig config, final ActionListener listener) { + + final Pivot pivot = new Pivot(config.getSource().getIndex(), + config.getSource().getQueryConfig().getQuery(), + config.getPivotConfig()); + + ActionListener> deduceMappingsListener = ActionListener.wrap( + mappings -> DataframeIndex.createDestinationIndex(client, + config, + mappings, + ActionListener.wrap(r -> listener.onResponse(null), listener::onFailure)), + deduceTargetMappingsException -> listener.onFailure( + new RuntimeException(DataFrameMessages.REST_PUT_DATA_FRAME_FAILED_TO_DEDUCE_DEST_MAPPINGS, + deduceTargetMappingsException)) + ); + + pivot.deduceMappings(client, deduceMappingsListener); + } + @Override protected ClusterBlockException checkBlock(StartDataFrameTransformAction.Request request, ClusterState state) { return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/notifications/DataFrameAuditor.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/notifications/DataFrameAuditor.java new file mode 100644 index 0000000000000..e02954a280b25 --- /dev/null +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/notifications/DataFrameAuditor.java @@ -0,0 +1,22 @@ +/* + * 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.notifications; + +import org.elasticsearch.client.Client; +import org.elasticsearch.xpack.core.common.notifications.Auditor; +import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; +import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex; + +import static org.elasticsearch.xpack.core.ClientHelper.DATA_FRAME_ORIGIN; + +/** + * DataFrameAuditor class that abstracts away generic templating for easier injection + */ +public class DataFrameAuditor extends Auditor { + public DataFrameAuditor(Client client, String nodeName) { + super(client, nodeName, DataFrameInternalIndex.AUDIT_INDEX, DATA_FRAME_ORIGIN, DataFrameAuditMessage.builder()); + } +} 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 c670f32740c28..823ccaff71b1f 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 @@ -19,15 +19,14 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; -import org.elasticsearch.xpack.core.common.notifications.Auditor; import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; -import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer; import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.indexing.IterationResult; +import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; import java.io.IOException; @@ -47,13 +46,13 @@ public abstract class DataFrameIndexer extends AsyncTwoPhaseIndexer auditor; + protected final DataFrameAuditor auditor; private Pivot pivot; private int pageSize = 0; public DataFrameIndexer(Executor executor, - Auditor auditor, + DataFrameAuditor auditor, AtomicReference initialState, Map initialPosition, DataFrameIndexerTransformStats jobStats) { 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 e3c27fd21fe03..b6f38a5dd2355 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 @@ -18,15 +18,14 @@ import org.elasticsearch.persistent.PersistentTasksExecutor; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.common.notifications.Auditor; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; import org.elasticsearch.xpack.dataframe.DataFrame; 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; @@ -40,13 +39,13 @@ public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksEx private final DataFrameTransformsCheckpointService dataFrameTransformsCheckpointService; private final SchedulerEngine schedulerEngine; private final ThreadPool threadPool; - private final Auditor auditor; + private final DataFrameAuditor auditor; public DataFrameTransformPersistentTasksExecutor(Client client, DataFrameTransformsConfigManager transformsConfigManager, DataFrameTransformsCheckpointService dataFrameTransformsCheckpointService, SchedulerEngine schedulerEngine, - Auditor auditor, + DataFrameAuditor auditor, ThreadPool threadPool) { super(DataFrameField.TASK_NAME, DataFrame.TASK_THREAD_POOL_NAME); this.client = client; 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 b8ceb2e7bd497..f142fc36179f7 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 @@ -24,13 +24,11 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; -import org.elasticsearch.xpack.core.common.notifications.Auditor; import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction; import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction.Response; import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction; -import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; @@ -40,6 +38,7 @@ import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine.Event; 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.transforms.pivot.SchemaUtil; @@ -64,7 +63,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S private final SchedulerEngine schedulerEngine; private final ThreadPool threadPool; private final DataFrameIndexer indexer; - private final Auditor auditor; + private final DataFrameAuditor auditor; private final DataFrameIndexerTransformStats previousStats; private final AtomicReference taskState; @@ -77,7 +76,7 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S public DataFrameTransformTask(long id, String type, String action, TaskId parentTask, DataFrameTransform transform, DataFrameTransformState state, Client client, DataFrameTransformsConfigManager transformsConfigManager, DataFrameTransformsCheckpointService transformsCheckpointService, - SchedulerEngine schedulerEngine, Auditor auditor, + SchedulerEngine schedulerEngine, DataFrameAuditor auditor, ThreadPool threadPool, Map headers) { super(id, type, action, DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + transform.getId(), parentTask, headers); this.transform = transform; @@ -309,7 +308,7 @@ protected class ClientDataFrameIndexer extends DataFrameIndexer { public ClientDataFrameIndexer(String transformId, DataFrameTransformsConfigManager transformsConfigManager, DataFrameTransformsCheckpointService transformsCheckpointService, AtomicReference initialState, Map initialPosition, Client client, - Auditor auditor) { + DataFrameAuditor auditor) { super(threadPool.executor(ThreadPool.Names.GENERIC), auditor, initialState, initialPosition, new DataFrameIndexerTransformStats(transformId)); this.transformId = transformId; diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java index b121e8091c138..89388d82e08a2 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java @@ -21,12 +21,11 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.common.notifications.Auditor; -import org.elasticsearch.xpack.core.dataframe.notifications.DataFrameAuditMessage; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfigTests; import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; import org.junit.Before; @@ -66,7 +65,7 @@ class MockedDataFrameIndexer extends DataFrameIndexer { Executor executor, DataFrameTransformConfig transformConfig, Map fieldMappings, - Auditor auditor, + DataFrameAuditor auditor, AtomicReference initialState, Map initialPosition, DataFrameIndexerTransformStats jobStats, @@ -202,8 +201,7 @@ public void testPageSizeAdapt() throws InterruptedException { final ExecutorService executor = Executors.newFixedThreadPool(1); try { - Auditor auditor = new Auditor<>(client, "node_1", TEST_INDEX, TEST_ORIGIN, - DataFrameAuditMessage.builder()); + DataFrameAuditor auditor = new DataFrameAuditor(client, "node_1"); MockedDataFrameIndexer indexer = new MockedDataFrameIndexer(executor, config, Collections.emptyMap(), auditor, state, null, new DataFrameIndexerTransformStats(config.getId()), searchFunction, bulkFunction, failureConsumer); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml index 00b091330027f..353fc0aa9326a 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml @@ -188,24 +188,3 @@ setup: from: 0 size: 10000 ---- -"Verify put transform creates destination index with appropriate mapping": - - do: - data_frame.put_data_frame_transform: - transform_id: "airline-transform" - body: > - { - "source": { "index": "airline-data" }, - "dest": { "index": "airline-data-by-airline" }, - "pivot": { - "group_by": { "airline": {"terms": {"field": "airline"}}}, - "aggs": {"avg_response": {"avg": {"field": "responsetime"}}, "time": {"max": {"field": "time"}}} - } - } - - match: { acknowledged: true } - - do: - indices.get_mapping: - index: airline-data-by-airline - - match: { airline-data-by-airline.mappings.properties.airline.type: keyword } - - match: { airline-data-by-airline.mappings.properties.avg_response.type: double } - - match: { airline-data-by-airline.mappings.properties.time.type: date } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml index 96f6b6d0a4150..f1ac07b72340c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_start_stop.yml @@ -64,6 +64,31 @@ teardown: transform_id: "airline-transform-start-stop" --- +"Verify start transform creates destination index with appropriate mapping": + - do: + data_frame.start_data_frame_transform: + transform_id: "airline-transform-start-stop" + - match: { started: true } + - do: + indices.get_mapping: + index: airline-data-by-airline-start-stop + - match: { airline-data-by-airline-start-stop.mappings.properties.airline.type: keyword } + - match: { airline-data-by-airline-start-stop.mappings.properties.avg_response.type: double } + +--- +"Verify start transform reuses destination index": + - do: + indices.create: + index: airline-data-by-airline-start-stop + - do: + data_frame.start_data_frame_transform: + transform_id: "airline-transform-start-stop" + - match: { started: true } + - do: + indices.get_mapping: + index: airline-data-by-airline-start-stop + - match: { airline-data-by-airline-start-stop.mappings: {} } +--- "Test start/stop/start transform": - do: data_frame.start_data_frame_transform: From 6a696f4638955143206bd2d7c9f5671f117b9fd9 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Tue, 23 Apr 2019 14:54:57 +0300 Subject: [PATCH 151/260] Dix the packer cache script --- distribution/docker/build.gradle | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/distribution/docker/build.gradle b/distribution/docker/build.gradle index 00fceb9df6a14..56ad936bdfe5d 100644 --- a/distribution/docker/build.gradle +++ b/distribution/docker/build.gradle @@ -142,4 +142,6 @@ assemble.dependsOn "buildDockerImage" // We build the images used in compose locally, but the pull command insists on using a repository // thus we must disable it to prevent it from doing so. // Everything will still be pulled since we will build the local images on a pull -tasks.matching { name == "composePull" }.all { enabled = false } +if (tasks.findByName("composePull")) { + tasks.composePull.enabled = false +} From add02f4f553ad472026dcc1eaa84245a0558a4b0 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 23 Apr 2019 16:30:37 +0300 Subject: [PATCH 152/260] SQL: Implement IIF(, , ) (#41420) Implement a more trivial case of the CASE expression which is expressed as a traditional function with 2 or 3 arguments. e.g.: IIF(a = 1, 'one', 'many') IIF(a > 0, 'positive') Closes: #40917 --- .../sql/functions/conditional.asciidoc | 46 ++++++ docs/reference/sql/functions/index.asciidoc | 1 + .../qa/src/main/resources/command.csv-spec | 1 + .../src/main/resources/conditionals.csv-spec | 139 ++++++++++++++++++ .../qa/src/main/resources/docs/docs.csv-spec | 24 +++ .../expression/function/FunctionRegistry.java | 8 +- .../predicate/conditional/Case.java | 28 ++-- .../expression/predicate/conditional/Iif.java | 59 ++++++++ .../xpack/sql/optimizer/Optimizer.java | 2 +- .../analyzer/VerifierErrorMessagesTests.java | 10 ++ .../predicate/conditional/CaseTests.java | 2 +- .../predicate/conditional/IifTests.java | 87 +++++++++++ .../xpack/sql/optimizer/OptimizerTests.java | 23 +++ .../xpack/sql/parser/ExpressionTests.java | 8 +- .../sql/planner/QueryTranslatorTests.java | 14 ++ .../xpack/sql/tree/NodeSubclassTests.java | 5 +- 16 files changed, 432 insertions(+), 25 deletions(-) create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Iif.java create mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IifTests.java diff --git a/docs/reference/sql/functions/conditional.asciidoc b/docs/reference/sql/functions/conditional.asciidoc index cf15504bbe379..cfce98d803cb3 100644 --- a/docs/reference/sql/functions/conditional.asciidoc +++ b/docs/reference/sql/functions/conditional.asciidoc @@ -223,6 +223,52 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[ifNullReturnFirst] include-tagged::{sql-specs}/docs/docs.csv-spec[ifNullReturnSecond] ---- +[[sql-functions-conditional-iif]] +==== `IFF` + +.Synopsis: +[source, sql] +---- +IIF(expression<1>, expression<2>, [expression<3>]) +---- + +*Input*: + +<1> boolean condition to check + +<2> return value if the boolean condition evaluates to `true` + +<3> return value if the boolean condition evaluates `false`; optional + +*Output*: 2nd expression if 1st expression (condition) evaluates to `true`. If it evaluates to `false` +return 3rd expression. If 3rd expression is not provided return `null`. + +.Description + +Conditional function that implements the standard _IF THEN ELSE _ +logic of programming languages. If the 3rd expression is not provided and the condition evaluates to `false`, +`null` is returned. + + +["source","sql",subs="attributes,callouts,macros"] +---- +include-tagged::{sql-specs}/docs/docs.csv-spec[iifWithDefaultValue] +---- + +["source","sql",subs="attributes,callouts,macros"] +---- +include-tagged::{sql-specs}/docs/docs.csv-spec[iifWithoutDefaultValue] +---- + +[TIP] +================= +*IIF* functions can be combined to implement more complex logic simulating the <> +expression. E.g.: + +[source, sql] +IIF(a = 1, 'one', IIF(a = 2, 'two', IIF(a = 3, 'three', 'many'))) +================= + [[sql-functions-conditional-isnull]] ==== `ISNULL` diff --git a/docs/reference/sql/functions/index.asciidoc b/docs/reference/sql/functions/index.asciidoc index 831a9d0a4e75b..382adeecea4ed 100644 --- a/docs/reference/sql/functions/index.asciidoc +++ b/docs/reference/sql/functions/index.asciidoc @@ -131,6 +131,7 @@ ** <> ** <> ** <> +** <> ** <> ** <> ** <> diff --git a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec index 73ef70a96a4ed..f60f686cb8a3d 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec @@ -29,6 +29,7 @@ CASE |CONDITIONAL COALESCE |CONDITIONAL GREATEST |CONDITIONAL IFNULL |CONDITIONAL +IIF |CONDITIONAL ISNULL |CONDITIONAL LEAST |CONDITIONAL NULLIF |CONDITIONAL 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 2a4eaa501a191..a5d7411caf6ea 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 @@ -179,3 +179,142 @@ count | gender | languages 11 | M | 3 11 | M | 4 ; + + +iifField +SELECT emp_no, IIF(emp_no - 10000 < 10, 'First 10', 'Second 10') as "iif_result" FROM test_emp WHERE emp_no >= 10005 +ORDER BY emp_no LIMIT 10; + + emp_no | iif_result +--------+----------- +10005 | First 10 +10006 | First 10 +10007 | First 10 +10008 | First 10 +10009 | First 10 +10010 | Second 10 +10011 | Second 10 +10012 | Second 10 +10013 | Second 10 +10014 | Second 10 +; + +iifFieldWithoutAlias +SELECT emp_no, IIF(emp_no - 10000 < 10, emp_no, emp_no % 10) FROM test_emp WHERE emp_no >= 10005 +ORDER BY emp_no LIMIT 10; + + emp_no | IIF(emp_no - 10000 < 10, emp_no, emp_no % 10) +--------+---------------------------------------------- +10005 | 10005 +10006 | 10006 +10007 | 10007 +10008 | 10008 +10009 | 10009 +10010 | 0 +10011 | 1 +10012 | 2 +10013 | 3 +10014 | 4 +; + +iifFieldNoElse +SELECT emp_no, IIF(emp_no - 10000 < 10, 'First 10') as "iif_result" FROM test_emp WHERE emp_no >= 10005 +ORDER BY emp_no LIMIT 10; + + emp_no | iif_result +--------+---------- +10005 | First 10 +10006 | First 10 +10007 | First 10 +10008 | First 10 +10009 | First 10 +10010 | null +10011 | null +10012 | null +10013 | null +10014 | null +; + +iifWhere +SELECT last_name FROM test_emp WHERE IIF(LENGTH(last_name) < 7, 'ShortName') IS NOT NULL ORDER BY emp_no LIMIT 10; + + last_name +----------- +Simmel +Peac +Sluis +Terkki +Genin +Peha +Erde +Famili +Pettey +Heyers +; + +iifOrderBy +SELECT last_name FROM test_emp ORDER BY IIF(languages >= 3, 'first', 'second'), emp_no LIMIT 10; + + last_name +----------- +Simmel +Bamford +Koblick +Preusig +Zielinski +Piveteau +Sluis +Bridgland +Genin +Nooteboom +; + +iifGroupBy +schema::count:l|lang_skills:s +SELECT count(*) AS count, IIF(NVL(languages, 0) <= 1 , 'zero-to-one', 'multilingual') as lang_skills FROM test_emp +GROUP BY lang_skills ORDER BY 2; + + count | lang_skills +---------------+--------------- +75 |multilingual +25 |zero-to-one +; + +iifGroupByComplexNested +schema::count:l|lang_skills:s +SELECT count(*) AS count, +IIF(NVL(languages, 0) = 0, 'zero', + IIF(languages = 1, 'one', + IIF(languages = 2, 'bilingual', + IIF(languages = 3, 'trilingual', 'multilingual')))) as lang_skills FROM test_emp GROUP BY lang_skills ORDER BY 2; + + count | lang_skills +---------------+--------------- +19 |bilingual +39 |multilingual +15 |one +17 |trilingual +10 |zero +; + +iifGroupByAndHaving +schema::count:l|gender:s|languages:byte +SELECT count(*) AS count, gender, languages FROM test_emp +GROUP BY 2, 3 HAVING IIF(count(*) > 10, 'many', 'a few') = 'many' +ORDER BY 2, 3; + + count | gender | languages +---------------+---------------+--------------- +11 |M |2 +11 |M |3 +11 |M |4 +; + +iifWithConvertAndGroupBy +SELECT CONVERT(IIF(languages > 1, IIF(languages = 3, '3')), SQL_BIGINT) AS cond FROM test_emp GROUP BY cond ORDER BY cond DESC; + + cond:l +------- +3 +null +; diff --git a/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec index bc7180de4d619..2a9804dfd5d56 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec @@ -206,6 +206,7 @@ CASE |CONDITIONAL COALESCE |CONDITIONAL GREATEST |CONDITIONAL IFNULL |CONDITIONAL +IIF |CONDITIONAL ISNULL |CONDITIONAL LEAST |CONDITIONAL NULLIF |CONDITIONAL @@ -2095,6 +2096,29 @@ elastic ; +iifWithDefaultValue +schema::result1:s|result2:s +// tag::iifWithDefaultValue +SELECT IIF(1 < 2, 'TRUE', 'FALSE') AS result1, IIF(1 > 2, 'TRUE', 'FALSE') AS result2; + + result1 | result2 +---------------+--------------- +TRUE |FALSE +// end::iifWithDefaultValue +; + +iifWithoutDefaultValue +schema::result1:s|result2:s +// tag::iifWithoutDefaultValue +SELECT IIF(1 < 2, 'TRUE') AS result1, IIF(1 > 2 , 'TRUE') AS result2; + + result1 | result2 +---------------+--------------- +TRUE |null +// end::iifWithoutDefaultValue +; + + ifNullReturnSecond // tag::ifNullReturnSecond SELECT IFNULL(null, 'search') AS "ifnull"; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java index 883dd0e076a05..0e9f07ef2132c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/FunctionRegistry.java @@ -97,6 +97,7 @@ 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.Greatest; +import org.elasticsearch.xpack.sql.expression.predicate.conditional.Iif; import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfNull; import org.elasticsearch.xpack.sql.expression.predicate.conditional.Least; import org.elasticsearch.xpack.sql.expression.predicate.conditional.NullIf; @@ -172,6 +173,7 @@ private void defineDefaultFunctions() { // Conditional addToMap(def(Case.class, Case::new, "CASE"), def(Coalesce.class, Coalesce::new, "COALESCE"), + def(Iif.class, Iif::new, "IIF"), def(IfNull.class, IfNull::new, "IFNULL", "ISNULL", "NVL"), def(NullIf.class, NullIf::new, "NULLIF"), def(Greatest.class, Greatest::new, "GREATEST"), @@ -544,10 +546,10 @@ private interface FunctionBuilder { static FunctionDefinition def(Class function, ThreeParametersFunctionBuilder ctorRef, String... names) { FunctionBuilder builder = (source, children, distinct, cfg) -> { - boolean isLocateFunction = function.isAssignableFrom(Locate.class); - if (isLocateFunction && (children.size() > 3 || children.size() < 2)) { + boolean hasMinimumTwo = function.isAssignableFrom(Locate.class) || function.isAssignableFrom(Iif.class); + if (hasMinimumTwo && (children.size() > 3 || children.size() < 2)) { throw new SqlIllegalArgumentException("expects two or three arguments"); - } else if (!isLocateFunction && children.size() != 3) { + } else if (!hasMinimumTwo && children.size() != 3) { throw new SqlIllegalArgumentException("expects exactly three arguments"); } if (distinct) { 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 772e75f414a27..59ec2c38d00b9 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 @@ -30,28 +30,28 @@ public class Case extends ConditionalFunction { private final List conditions; - private final Expression defaultElse; + private final Expression elseResult; @SuppressWarnings("unchecked") public Case(Source source, List expressions) { super(source, expressions); this.conditions = (List) (List) expressions.subList(0, expressions.size() - 1); - this.defaultElse = expressions.get(expressions.size() - 1); + this.elseResult = expressions.get(expressions.size() - 1); } public List conditions() { return conditions; } - public Expression defaultElse() { - return defaultElse; + public Expression elseResult() { + return elseResult; } @Override public DataType dataType() { if (dataType == null) { if (conditions.isEmpty()) { - dataType = defaultElse().dataType(); + dataType = elseResult().dataType(); } else { dataType = DataType.NULL; @@ -83,7 +83,7 @@ protected TypeResolution resolveType() { } } if (expectedResultDataType == null) { - expectedResultDataType = defaultElse().dataType(); + expectedResultDataType = elseResult().dataType(); } for (IfConditional conditional : conditions) { @@ -102,12 +102,12 @@ protected TypeResolution resolveType() { } } - if (DataTypes.areTypesCompatible(expectedResultDataType, defaultElse.dataType()) == false) { + if (DataTypes.areTypesCompatible(expectedResultDataType, elseResult.dataType()) == false) { return new TypeResolution(format(null, "ELSE clause of [{}] must be [{}], found value [{}] type [{}]", - defaultElse.sourceText(), + elseResult.sourceText(), expectedResultDataType.typeName, - Expressions.name(defaultElse), - defaultElse.dataType().typeName)); + Expressions.name(elseResult), + elseResult.dataType().typeName)); } return TypeResolution.TYPE_RESOLVED; @@ -119,7 +119,7 @@ protected TypeResolution resolveType() { */ @Override public boolean foldable() { - return (conditions.isEmpty() && defaultElse.foldable()) || + return (conditions.isEmpty() && elseResult.foldable()) || (conditions.size() == 1 && conditions.get(0).condition().foldable() && conditions.get(0).result().foldable()); } @@ -128,7 +128,7 @@ public Object fold() { if (conditions.isEmpty() == false && conditions.get(0).condition().fold() == Boolean.TRUE) { return conditions.get(0).result().fold(); } - return defaultElse.fold(); + return elseResult.fold(); } @Override @@ -138,7 +138,7 @@ protected Pipe makePipe() { pipes.add(Expressions.pipe(ifConditional.condition())); pipes.add(Expressions.pipe(ifConditional.result())); } - pipes.add(Expressions.pipe(defaultElse)); + pipes.add(Expressions.pipe(elseResult)); return new CasePipe(source(), this, pipes); } @@ -149,7 +149,7 @@ public ScriptTemplate asScript() { templates.add(asScript(ifConditional.condition())); templates.add(asScript(ifConditional.result())); } - templates.add(asScript(defaultElse)); + templates.add(asScript(elseResult)); StringJoiner template = new StringJoiner(",", "{sql}.caseFunction([", "])"); ParamsBuilder params = paramsBuilder(); 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 new file mode 100644 index 0000000000000..49faeb1233a73 --- /dev/null +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/Iif.java @@ -0,0 +1,59 @@ +/* + * 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.sql.expression.predicate.conditional; + +import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Expressions; +import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; +import org.elasticsearch.xpack.sql.type.DataType; +import org.elasticsearch.xpack.sql.type.DataTypes; + +import java.util.Arrays; +import java.util.List; + +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.sql.expression.TypeResolutions.isBoolean; + +public class Iif extends Case { + + public Iif(Source source, Expression condition, Expression thenResult, Expression elseResult) { + super(source, Arrays.asList(new IfConditional(source, condition, thenResult), elseResult != null ? elseResult : Literal.NULL)); + } + + private 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()); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Iif(source(), newChildren); + } + + @Override + protected TypeResolution resolveType() { + TypeResolution conditionTypeResolution = isBoolean(conditions().get(0).condition(), sourceText(), Expressions.ParamOrdinal.FIRST); + if (conditionTypeResolution.unresolved()) { + return conditionTypeResolution; + } + + DataType resultDataType = conditions().get(0).dataType(); + if (DataTypes.areTypesCompatible(resultDataType, elseResult().dataType()) == false) { + return new TypeResolution(format(null, "third argument of [{}] must be [{}], found value [{}] type [{}]", + sourceText(), + resultDataType.typeName, + Expressions.name(elseResult()), + elseResult().dataType().typeName)); + } + return TypeResolution.TYPE_RESOLVED; + } +} diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java index 78cc16470ca56..72371ab9617d5 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java @@ -1257,7 +1257,7 @@ protected Expression rule(Expression e) { } if (newConditions.size() < c.children().size()) { - return c.replaceChildren(combine(newConditions, c.defaultElse())); + return c.replaceChildren(combine(newConditions, c.elseResult())); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java index 24e993a2d8337..dcf8dad5ecb79 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java @@ -618,6 +618,16 @@ public void testCaseWithDifferentResultAndDefaultValueDataTypesAndNullTypesSkipp error("SELECT CASE WHEN int > 20 THEN null WHEN int > 10 THEN null WHEN int > 5 THEN 'foo' ELSE date END FROM test")); } + public void testIifWithNonBooleanConditionExpression() { + assertEquals("1:8: first argument of [IIF(int, 'one', 'zero')] must be [boolean], found value [int] type [integer]", + error("SELECT IIF(int, 'one', 'zero') FROM test")); + } + + public void testIifWithDifferentResultAndDefaultValueDataTypes() { + assertEquals("1:8: third argument of [IIF(int > 20, 'foo', date)] must be [keyword], found value [date] type [datetime]", + error("SELECT IIF(int > 20, 'foo', date) FROM test")); + } + public void testAggsInWhere() { assertEquals("1:33: Cannot use WHERE filtering on aggregate function [MAX(int)], use HAVING instead", error("SELECT MAX(int) FROM test WHERE MAX(int) > 10 GROUP BY bool")); 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 efc86bb47d24d..807be397f91b6 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 @@ -94,7 +94,7 @@ private List mutateChildren(Case c) { } } } - expressions.add(c.defaultElse()); + expressions.add(c.elseResult()); return expressions; } } 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 new file mode 100644 index 0000000000000..054bf2c879f7c --- /dev/null +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/predicate/conditional/IifTests.java @@ -0,0 +1,87 @@ +/* + * 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.sql.expression.predicate.conditional; + +import org.elasticsearch.xpack.sql.expression.Expression; +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 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.SourceTests.randomSource; + +/** + * Needed to override tests in {@link NodeSubclassTests} as If is special since its children are not usual + * expressions but {@link IfConditional}s. + */ +public class IifTests extends AbstractNodeTestCase { + + public static Iif randomIif() { + return new Iif(randomSource(), new Equals(randomSource(), randomStringLiteral(), randomStringLiteral()), + randomIntLiteral(), randomIntLiteral()); + } + + @Override + protected Iif randomInstance() { + return randomIif(); + } + + @Override + protected Iif mutate(Iif instance) { + Iif iif = randomIif(); + List mutatedChildren = mutateChildren(iif); + return new Iif(iif.source(), mutatedChildren.get(0), mutatedChildren.get(1), mutatedChildren.get(2)); + } + + @Override + protected Iif copy(Iif instance) { + return new Iif(instance.source(), instance.conditions().get(0).condition(), instance.conditions().get(0).result(), + instance.elseResult()); + } + + @Override + public void testTransform() { + Iif iif = randomIif(); + + Source newSource = randomValueOtherThan(iif.source(), SourceTests::randomSource); + assertEquals(new Iif(iif.source(), iif.conditions().get(0).condition(), iif.conditions().get(0).result(), iif.elseResult()), + iif.transformPropertiesOnly(p -> Objects.equals(p, iif.source()) ? newSource: p, Object.class)); + + String newName = randomValueOtherThan(iif.name(), () -> randomAlphaOfLength(5)); + assertEquals(new Iif(iif.source(), iif.conditions().get(0).condition(), iif.conditions().get(0).result(), iif.elseResult()), + iif.transformPropertiesOnly(p -> Objects.equals(p, iif.name()) ? newName : p, Object.class)); + } + + @Override + public void testReplaceChildren() { + Iif iif = randomIif(); + + List newChildren = mutateChildren(iif); + assertEquals(new Iif(iif.source(), newChildren.get(0), newChildren.get(1), newChildren.get(2)), + iif.replaceChildren(Arrays.asList(new IfConditional(iif.source(), newChildren.get(0), newChildren.get(1)), + newChildren.get(2)))); + } + + private List mutateChildren(Iif iif) { + List expressions = new ArrayList<>(3); + Equals eq = (Equals) iif.conditions().get(0).condition(); + expressions.add(new Equals(randomSource(), + randomValueOtherThan(eq.left(), FunctionTestUtils::randomStringLiteral), + randomValueOtherThan(eq.right(), FunctionTestUtils::randomStringLiteral))); + expressions.add(randomValueOtherThan(iif.conditions().get(0).result(), FunctionTestUtils::randomIntLiteral)); + expressions.add(randomValueOtherThan(iif.elseResult(), FunctionTestUtils::randomIntLiteral)); + return expressions; + } +} 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 f77088c3fdfc1..b2e5eebe5ea5f 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 @@ -48,6 +48,7 @@ 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.Least; @@ -685,6 +686,28 @@ public void testSimplifyCaseConditionsFoldCompletely() { assertEquals("foo2", c.fold()); } + public void testSimplifyIif_ConditionTrue() { + SimplifyCase rule = new SimplifyCase(); + Iif iif = new Iif(EMPTY, new Equals(EMPTY, ONE, ONE), Literal.of(EMPTY, "foo"), Literal.of(EMPTY, "bar")); + Expression e = rule.rule(iif); + assertEquals(Iif.class, e.getClass()); + iif = (Iif) e; + assertEquals(1, iif.conditions().size()); + assertTrue(iif.foldable()); + assertEquals("foo", iif.fold()); + } + + public void testSimplifyIif_ConditionFalse() { + SimplifyCase rule = new SimplifyCase(); + Iif iif = new Iif(EMPTY, new Equals(EMPTY, ONE, TWO), Literal.of(EMPTY, "foo"), Literal.of(EMPTY, "bar")); + Expression e = rule.rule(iif); + assertEquals(Iif.class, e.getClass()); + iif = (Iif) e; + assertEquals(0, iif.conditions().size()); + assertTrue(iif.foldable()); + assertEquals("bar", iif.fold()); + } + // // Logical simplifications // diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java index a8e5df8e07caa..aff8d7b8fd774 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java @@ -465,7 +465,7 @@ public void testCaseWithoutOperand() { assertEquals("WHEN a = 1 THEN 'one'", ifc.sourceText()); assertThat(ifc.condition().toString(), startsWith("Equals[?a,1]#")); assertEquals("'one'=one", ifc.result().toString()); - assertEquals(Literal.NULL, c.defaultElse()); + assertEquals(Literal.NULL, c.elseResult()); expr = parser.createExpression( "CASE WHEN a = 1 THEN 'one'" + @@ -478,7 +478,7 @@ public void testCaseWithoutOperand() { assertEquals(2, c.conditions().size()); ifc = c.conditions().get(0); assertEquals("WHEN a = 1 THEN 'one'", ifc.sourceText()); - assertEquals("'many'=many", c.defaultElse().toString()); + assertEquals("'many'=many", c.elseResult().toString()); } public void testCaseWithOperand() { @@ -495,7 +495,7 @@ public void testCaseWithOperand() { assertEquals("WHEN 1 THEN 'one'", ifc.sourceText()); assertThat(ifc.condition().toString(), startsWith("Equals[?a,1]#")); assertEquals("'one'=one", ifc.result().toString()); - assertEquals(Literal.NULL, c.defaultElse()); + assertEquals(Literal.NULL, c.elseResult()); expr = parser.createExpression( "CASE a WHEN 1 THEN 'one'" + @@ -507,6 +507,6 @@ public void testCaseWithOperand() { assertEquals(2, c.conditions().size()); ifc = c.conditions().get(0); assertEquals("WHEN 1 THEN 'one'", ifc.sourceText()); - assertEquals("'many'=many", c.defaultElse().toString()); + assertEquals("'many'=many", c.elseResult().toString()); } } 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 ca3c3a92ce6d6..0543e65d4ae46 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 @@ -602,6 +602,20 @@ public void testTranslateCase_GroupBy_Painless() { assertEquals("[{v=int}, {v=10}, {v=foo}, {v=int}, {v=20}, {v=bar}, {v=default}]", scriptTemplate.params().toString()); } + public void testTranslateIif_GroupBy_Painless() { + LogicalPlan p = plan("SELECT IIF(int > 20, 'foo', 'bar') FROM test GROUP BY 1"); + assertTrue(p instanceof Aggregate); + Expression condition = ((Aggregate) p).groupings().get(0); + assertFalse(condition.foldable()); + QueryTranslator.GroupingContext groupingContext = QueryTranslator.groupBy(((Aggregate) p).groupings()); + assertNotNull(groupingContext); + ScriptTemplate scriptTemplate = groupingContext.tail.script(); + assertEquals("InternalSqlScriptUtils.caseFunction([InternalSqlScriptUtils.gt(" + + "InternalSqlScriptUtils.docValue(doc,params.v0),params.v1),params.v2,params.v3])", + scriptTemplate.toString()); + assertEquals("[{v=int}, {v=20}, {v=foo}, {v=bar}]", scriptTemplate.params().toString()); + } + public void testGroupByDateHistogram() { LogicalPlan p = plan("SELECT MAX(int) FROM test GROUP BY HISTOGRAM(int, 1000)"); assertTrue(p instanceof Aggregate); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java index cad06da0a824e..ddcbe27906e6c 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/tree/NodeSubclassTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe; import org.elasticsearch.xpack.sql.expression.gen.processor.ConstantProcessor; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; +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.fulltext.FullTextPredicate; @@ -92,8 +93,8 @@ */ public class NodeSubclassTests> extends ESTestCase { - private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = Arrays.asList(IfConditional.class, IfNull.class, - In.class, InPipe.class, Percentile.class, Percentiles.class, PercentileRanks.class); + private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = Arrays.asList(Iif.class, IfConditional.class, + IfNull.class, In.class, InPipe.class, Percentile.class, Percentiles.class, PercentileRanks.class); private final Class subclass; From 7e06023e7603b7584bfd9ee4e8a1ccd82c208ce7 Mon Sep 17 00:00:00 2001 From: Martijn Laarman Date: Tue, 23 Apr 2019 16:23:20 +0200 Subject: [PATCH 153/260] fix #35262 define deprecations of API's as a whole and urls (#39063) * fix #35262 define deprecations of API's as a whole and urls * document hot threads deprecated paths * deprecate scroll_id as part of the URL, documented only as part of the body which is a safer behaviour as well * use version numbers up to patch version * rest spec parser picks up deprecated paths as paths too --- .../rest-api-spec/api/clear_scroll.json | 11 ++++- .../resources/rest-api-spec/api/count.json | 9 +++- .../resources/rest-api-spec/api/create.json | 9 +++- .../resources/rest-api-spec/api/delete.json | 9 +++- .../rest-api-spec/api/delete_by_query.json | 9 +++- .../resources/rest-api-spec/api/exists.json | 9 +++- .../rest-api-spec/api/exists_source.json | 9 +++- .../resources/rest-api-spec/api/explain.json | 9 +++- .../main/resources/rest-api-spec/api/get.json | 9 +++- .../rest-api-spec/api/get_source.json | 9 +++- .../resources/rest-api-spec/api/index.json | 14 ++++++- .../api/indices.exists_type.json | 4 ++ .../api/indices.get_field_mapping.json | 14 ++++++- .../api/indices.get_mapping.json | 14 ++++++- .../api/indices.put_mapping.json | 41 ++++++++++++++++++- .../api/indices.validate_query.json | 9 +++- .../resources/rest-api-spec/api/mget.json | 9 +++- .../resources/rest-api-spec/api/msearch.json | 9 +++- .../rest-api-spec/api/msearch_template.json | 9 +++- .../rest-api-spec/api/mtermvectors.json | 9 +++- .../rest-api-spec/api/nodes.hot_threads.json | 34 ++++++++++++++- .../resources/rest-api-spec/api/scroll.json | 9 +++- .../resources/rest-api-spec/api/search.json | 9 +++- .../rest-api-spec/api/search_template.json | 9 +++- .../rest-api-spec/api/termvectors.json | 15 ++++++- .../resources/rest-api-spec/api/update.json | 9 +++- .../rest-api-spec/api/update_by_query.json | 9 +++- .../ClientYamlSuiteRestApiParser.java | 20 +++++++-- .../rest-api-spec/api/graph.explore.json | 9 +++- .../rest-api-spec/api/monitoring.bulk.json | 9 +++- .../api/rollup.rollup_search.json | 9 +++- 31 files changed, 329 insertions(+), 36 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json b/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json index 6b82d1dc4742a..b2b0343230c66 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/clear_scroll.json @@ -3,8 +3,15 @@ "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/search-request-scroll.html", "methods": ["DELETE"], "url": { - "path": "/_search/scroll/{scroll_id}", - "paths": ["/_search/scroll/{scroll_id}", "/_search/scroll"], + "path": "/_search/scroll", + "paths": [ "/_search/scroll"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_search/scroll/{scroll_id}", + "description" : "A scroll id can be quite large and should be specified as part of the body" + } + ], "parts": { "scroll_id": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json index d14f4ab784a57..b933091b9a416 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json @@ -4,7 +4,14 @@ "methods": ["POST", "GET"], "url": { "path": "/_count", - "paths": ["/_count", "/{index}/_count", "/{index}/{type}/_count"], + "paths": ["/_count", "/{index}/_count"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_count", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/create.json b/rest-api-spec/src/main/resources/rest-api-spec/api/create.json index 6a21620423d6b..f21d2606364d1 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/create.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/create.json @@ -4,7 +4,14 @@ "methods": ["PUT","POST"], "url": { "path": "/{index}/_create/{id}", - "paths": ["/{index}/_create/{id}", "/{index}/{type}/{id}/_create"], + "paths": ["/{index}/_create/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}/_create", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json index 4b698b371a16d..792f9d89609bf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json @@ -4,7 +4,14 @@ "methods": ["DELETE"], "url": { "path": "/{index}/_doc/{id}", - "paths": ["/{index}/{type}/{id}", "/{index}/_doc/{id}"], + "paths": ["/{index}/_doc/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json index dfdc00680828f..37b88e0f24d93 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json @@ -4,7 +4,14 @@ "methods": ["POST"], "url": { "path": "/{index}/_delete_by_query", - "paths": ["/{index}/_delete_by_query", "/{index}/{type}/_delete_by_query"], + "paths": ["/{index}/_delete_by_query"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_delete_by_query", + "description" : "Specifying types in urls has been deprecated" + } + ], "comment": "most things below this are just copied from search.json", "parts": { "index": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json b/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json index 1d3749a5d8c4f..3debd3edce585 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/exists.json @@ -4,7 +4,14 @@ "methods": ["HEAD"], "url": { "path": "/{index}/_doc/{id}", - "paths": ["/{index}/_doc/{id}", "/{index}/{type}/{id}"], + "paths": ["/{index}/_doc/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json b/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json index a3edff1d111a5..89f9c33e5fb44 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/exists_source.json @@ -4,7 +4,14 @@ "methods": ["HEAD"], "url": { "path": "/{index}/_source/{id}", - "paths": ["/{index}/_source/{id}", "/{index}/{type}/{id}/_source"], + "paths": ["/{index}/_source/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}/_source", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json b/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json index 005cbadccb0b7..12aa7a8dca942 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/explain.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/{index}/_explain/{id}", - "paths": ["/{index}/_explain/{id}", "/{index}/{type}/{id}/_explain"], + "paths": ["/{index}/_explain/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}/_explain", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json index ac3024c9c245d..cc20ceecc4e4c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json @@ -4,7 +4,14 @@ "methods": ["GET"], "url": { "path": "/{index}/_doc/{id}", - "paths": ["/{index}/_doc/{id}", "/{index}/{type}/{id}"], + "paths": ["/{index}/_doc/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json index cd737e32d6ca8..a26691edc41fc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get_source.json @@ -4,7 +4,14 @@ "methods": ["GET"], "url": { "path": "/{index}/_source/{id}", - "paths": ["/{index}/_source/{id}", "/{index}/{type}/{id}/_source"], + "paths": ["/{index}/_source/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}/_source", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index dcba05f6a3859..2a2053d2250a0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -4,7 +4,19 @@ "methods": ["POST", "PUT"], "url": { "path": "/{index}/_doc", - "paths": ["/{index}/{type}", "/{index}/{type}/{id}", "/{index}/_doc/{id}", "/{index}/_doc"], + "paths": ["/{index}/_doc/{id}", "/{index}/_doc"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json index d793199bc2776..a788202c93ae2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json @@ -2,6 +2,10 @@ "indices.exists_type": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-types-exists.html", "methods": ["HEAD"], + "deprecated" : { + "version" : "7.0.0", + "description" : "Types are being removed from elasticsearch and therefor this API is on the way out. Read more here: https://www.elastic.co/guide/en/elasticsearch/reference/master/removal-of-types.html" + }, "url": { "path": "/{index}/_mapping/{type}", "paths": ["/{index}/_mapping/{type}"], diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json index 3ce610153b5e6..305cb700ceae9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json @@ -4,7 +4,19 @@ "methods": ["GET"], "url": { "path": "/_mapping/field/{fields}", - "paths": ["/_mapping/field/{fields}", "/{index}/_mapping/field/{fields}", "/_mapping/{type}/field/{fields}", "/{index}/_mapping/{type}/field/{fields}"], + "paths": ["/_mapping/field/{fields}", "/{index}/_mapping/field/{fields}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_mapping/{type}/field/{fields}", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/_mapping/{type}/field/{fields}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json index d9016ec402498..abfc3d3867e9a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json @@ -4,7 +4,19 @@ "methods": ["GET"], "url": { "path": "/_mapping", - "paths": ["/_mapping", "/{index}/_mapping", "/_mapping/{type}", "/{index}/_mapping/{type}"], + "paths": ["/_mapping", "/{index}/_mapping"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_mapping/{type}", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/_mapping/{type}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json index cc55ffccdd1ef..c46e6b15549cc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json @@ -3,8 +3,45 @@ "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-put-mapping.html", "methods": ["PUT", "POST"], "url": { - "path": "/{index}/{type}/_mapping", - "paths": ["/{index}/{type}/_mapping", "/{index}/_mapping/{type}", "/_mapping/{type}", "/{index}/{type}/_mappings", "/{index}/_mappings/{type}", "/_mappings/{type}", "{index}/_mappings", "{index}/_mapping"], + "path": "{index}/_mapping", + "paths": ["{index}/_mapping"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_mapping", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/_mapping/{type}", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_mappings", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/_mappings/{type}", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/_mappings/{type}", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "{index}/_mappings", + "description" : "The plural mappings is accepted but only /_mapping is documented" + }, + { + "version" : "7.0.0", + "path" : "/_mapping/{type}", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json index 2f9aac054382c..6f1395b1582a0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_validate/query", - "paths": ["/_validate/query", "/{index}/_validate/query", "/{index}/{type}/_validate/query"], + "paths": ["/_validate/query", "/{index}/_validate/query"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_validate/query", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json b/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json index 62fbb59a4e451..c04fb34beac18 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/mget.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_mget", - "paths": ["/_mget", "/{index}/_mget", "/{index}/{type}/_mget"], + "paths": ["/_mget", "/{index}/_mget"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_mget", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json index 398dcbd29515d..9c416e841362a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_msearch", - "paths": ["/_msearch", "/{index}/_msearch", "/{index}/{type}/_msearch"], + "paths": ["/_msearch", "/{index}/_msearch"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_msearch", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json index e89f96e06960f..87aa6b22b8d45 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/msearch_template.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_msearch/template", - "paths": ["/_msearch/template", "/{index}/_msearch/template", "/{index}/{type}/_msearch/template"], + "paths": ["/_msearch/template", "/{index}/_msearch/template"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_msearch/template", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json b/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json index 58978b7d190fa..8cf4b22e90da9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/mtermvectors.json @@ -4,7 +4,14 @@ "methods" : ["GET", "POST"], "url" : { "path" : "/_mtermvectors", - "paths" : ["/_mtermvectors", "/{index}/_mtermvectors", "/{index}/{type}/_mtermvectors"], + "paths" : ["/_mtermvectors", "/{index}/_mtermvectors"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_mtermvectors", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts" : { "index" : { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json index 854cde1a9e731..95df4f53aa579 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/nodes.hot_threads.json @@ -4,7 +4,39 @@ "methods": ["GET"], "url": { "path": "/_nodes/hot_threads", - "paths": ["/_cluster/nodes/hotthreads", "/_cluster/nodes/hot_threads", "/_cluster/nodes/{node_id}/hotthreads", "/_cluster/nodes/{node_id}/hot_threads", "/_nodes/hotthreads", "/_nodes/hot_threads", "/_nodes/{node_id}/hotthreads", "/_nodes/{node_id}/hot_threads"], + "paths": ["/_nodes/hot_threads", "/_nodes/{node_id}/hot_threads"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_cluster/nodes/hotthreads", + "description" : "The hot threads API accepts `hotthreads` but only `hot_threads` is documented" + }, + { + "version" : "7.0.0", + "path" : "/_cluster/nodes/{node_id}/hotthreads", + "description" : "The hot threads API accepts `hotthreads` but only `hot_threads` is documented" + }, + { + "version" : "7.0.0", + "path" : "/_nodes/hotthreads", + "description" : "The hot threads API accepts `hotthreads` but only `hot_threads` is documented" + }, + { + "version" : "7.0.0", + "path" : "/_nodes/{node_id}/hotthreads", + "description" : "The hot threads API accepts `hotthreads` but only `hot_threads` is documented" + }, + { + "version" : "7.0.0", + "path" : "/_cluster/nodes/hot_threads", + "description" : "The hot accepts /_cluster/nodes as prefix for backwards compatibility reasons" + }, + { + "version" : "7.0.0", + "path" :"/_cluster/nodes/{node_id}/hot_threads", + "description" : "The hot accepts /_cluster/nodes as prefix for backwards compatibility reasons" + } + ], "parts": { "node_id": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json b/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json index fc04caeb808c5..31e02a172fd0e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/scroll.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_search/scroll", - "paths": ["/_search/scroll", "/_search/scroll/{scroll_id}"], + "paths": ["/_search/scroll"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_search/scroll/{scroll_id}", + "description" : "A scroll id can be quite large and should be specified as part of the body" + } + ], "parts": { "scroll_id": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index f44c0f74b2c3d..00ed8d113a00c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_search", - "paths": ["/_search", "/{index}/_search", "/{index}/{type}/_search"], + "paths": ["/_search", "/{index}/_search"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_search", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json index 24b7fa135b331..5395e4f59c10e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search_template.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/_search/template", - "paths": ["/_search/template", "/{index}/_search/template", "/{index}/{type}/_search/template"], + "paths": ["/_search/template", "/{index}/_search/template"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_search/template", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type" : "list", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json b/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json index 3485c7a6cc103..44b972b355f68 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/termvectors.json @@ -4,8 +4,19 @@ "methods" : ["GET", "POST"], "url" : { "path" : "/{index}/_termvectors/{id}", - "paths" : ["/{index}/_termvectors/{id}", "/{index}/_termvectors/", - "/{index}/{type}/{id}/_termvectors", "/{index}/{type}/_termvectors"], + "paths" : ["/{index}/_termvectors/{id}", "/{index}/_termvectors"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}/_termvectors", + "description" : "Specifying types in urls has been deprecated" + }, + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_termvectors", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts" : { "index" : { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update.json index 106b29b252ad3..b85c70be57d9e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update.json @@ -4,7 +4,14 @@ "methods": ["POST"], "url": { "path": "/{index}/_update/{id}", - "paths": ["/{index}/_update/{id}", "/{index}/{type}/{id}/_update"], + "paths": ["/{index}/_update/{id}"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/{id}/_update", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "id": { "type": "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json index 427a7e04ad8fb..bdb4c565bd965 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json @@ -4,7 +4,14 @@ "methods": ["POST"], "url": { "path": "/{index}/_update_by_query", - "paths": ["/{index}/_update_by_query", "/{index}/{type}/_update_by_query"], + "paths": ["/{index}/_update_by_query"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_update_by_query", + "description" : "Specifying types in urls has been deprecated" + } + ], "comment": "most things below this are just copied from search.json", "parts": { "index": { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/restspec/ClientYamlSuiteRestApiParser.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/restspec/ClientYamlSuiteRestApiParser.java index fefd6e6a276c6..3c6216eed76ed 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/restspec/ClientYamlSuiteRestApiParser.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/restspec/ClientYamlSuiteRestApiParser.java @@ -70,11 +70,16 @@ public ClientYamlSuiteRestApi parse(String location, XContentParser parser) thro } if (parser.currentToken() == XContentParser.Token.START_ARRAY && "paths".equals(currentFieldName)) { while (parser.nextToken() == XContentParser.Token.VALUE_STRING) { - String path = parser.text(); - if (restApi.getPaths().contains(path)) { - throw new IllegalArgumentException("Found duplicate path [" + path + "]"); + addPathToApi(parser.text(), restApi); + } + } + if (parser.currentToken() == XContentParser.Token.START_ARRAY && "deprecated_paths".equals(currentFieldName)) { + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + if (parser.currentToken() == XContentParser.Token.FIELD_NAME && "path".equals(parser.currentName())) + { + parser.nextToken(); + addPathToApi(parser.text(), restApi); } - restApi.addPath(path); } } @@ -142,6 +147,13 @@ public ClientYamlSuiteRestApi parse(String location, XContentParser parser) thro return restApi; } + private void addPathToApi(String path, ClientYamlSuiteRestApi restApi) { + if (restApi.getPaths().contains(path)) { + throw new IllegalArgumentException("Found duplicate path [" + path + "]"); + } + restApi.addPath(path); + } + private static class Parameter { private boolean required; public boolean isRequired() { diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/graph.explore.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/graph.explore.json index 293694d0ae8a0..b01c51bc6c381 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/graph.explore.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/graph.explore.json @@ -4,7 +4,14 @@ "methods": ["GET", "POST"], "url": { "path": "/{index}/_graph/explore", - "paths": ["/{index}/_graph/explore", "/{index}/{type}/_graph/explore"], + "paths": ["/{index}/_graph/explore"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/{index}/{type}/_graph/explore", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts" : { "index": { "type" : "list", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json index a299e2adc4bc7..78a6017cba2e6 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/monitoring.bulk.json @@ -4,7 +4,14 @@ "methods": ["POST", "PUT"], "url": { "path": "/_monitoring/bulk", - "paths": ["/_monitoring/bulk", "/_monitoring/{type}/bulk"], + "paths": ["/_monitoring/bulk"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "/_monitoring/{type}/bulk", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "type": { "type" : "string", diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/rollup.rollup_search.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/rollup.rollup_search.json index 5a63dfa3b42aa..0eeb7b29256cf 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/rollup.rollup_search.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/rollup.rollup_search.json @@ -4,7 +4,14 @@ "methods": [ "GET", "POST" ], "url": { "path": "/{index}/_rollup_search", - "paths": [ "{index}/_rollup_search", "{index}/{type}/_rollup_search" ], + "paths": [ "{index}/_rollup_search"], + "deprecated_paths" : [ + { + "version" : "7.0.0", + "path" : "{index}/{type}/_rollup_search", + "description" : "Specifying types in urls has been deprecated" + } + ], "parts": { "index": { "type": "list", From b47d65c917e6c5ba5aba8c7244f93644aca66262 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 23 Apr 2019 15:26:14 +0100 Subject: [PATCH 154/260] Mention the cost of tracking live docs in scrolls (#41375) Relates #41337, in which a heap dump shows hundreds of MBs allocated on the heap for tracking the live docs for each scroll. --- docs/reference/search/request/scroll.asciidoc | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/docs/reference/search/request/scroll.asciidoc b/docs/reference/search/request/scroll.asciidoc index ebc2f0aca6cb0..3503ad2c9c826 100644 --- a/docs/reference/search/request/scroll.asciidoc +++ b/docs/reference/search/request/scroll.asciidoc @@ -103,6 +103,12 @@ GET /_search?scroll=1m [[scroll-search-context]] ==== Keeping the search context alive +A scroll returns all the documents which matched the search at the time of the +initial search request. It ignores any subsequent changes to these documents. +The `scroll_id` identifies a _search context_ which keeps track of everything +that {es} needs to return the correct documents. The search context is created +by the initial request and kept alive by subsequent requests. + The `scroll` parameter (passed to the `search` request and to every `scroll` request) tells Elasticsearch how long it should keep the search context alive. Its value (e.g. `1m`, see <>) does not need to be long enough to @@ -112,17 +118,21 @@ new expiry time. If a `scroll` request doesn't pass in the `scroll` parameter, then the search context will be freed as part of _that_ `scroll` request. -Normally, the background merge process optimizes the -index by merging together smaller segments to create new bigger segments, at -which time the smaller segments are deleted. This process continues during -scrolling, but an open search context prevents the old segments from being -deleted while they are still in use. This is how Elasticsearch is able to -return the results of the initial search request, regardless of subsequent -changes to documents. - -TIP: Keeping older segments alive means that more file handles are needed. -Ensure that you have configured your nodes to have ample free file handles. -See <>. +Normally, the background merge process optimizes the index by merging together +smaller segments to create new, bigger segments. Once the smaller segments are +no longer needed they are deleted. This process continues during scrolling, but +an open search context prevents the old segments from being deleted since they +are still in use. + +TIP: Keeping older segments alive means that more disk space and file handles +are needed. Ensure that you have configured your nodes to have ample free file +handles. See <>. + +Additionally, if a segment contains deleted or updated documents then the +search context must keep track of whether each document in the segment was live +at the time of the initial search request. Ensure that your nodes have +sufficient heap space if you have many open scrolls on an index that is subject +to ongoing deletes or updates. NOTE: To prevent against issues caused by having too many scrolls open, the user is not allowed to open scrolls past a certain limit. By default, the From 125f5890ce5787c480c8fd60148d1a0d6e691033 Mon Sep 17 00:00:00 2001 From: Martijn Laarman Date: Tue, 23 Apr 2019 16:30:47 +0200 Subject: [PATCH 155/260] get.json documents deprecated _source_include (#38613) This brings it inline with other REST spec files which only document the plural --- .../src/main/resources/rest-api-spec/api/get.json | 8 -------- 1 file changed, 8 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json index cc20ceecc4e4c..5b2203c94deb9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json @@ -65,14 +65,6 @@ "type" : "list", "description" : "A list of fields to extract and return from the _source field" }, - "_source_exclude": { - "type" : "list", - "description" : "A list of fields to exclude from the returned _source field" - }, - "_source_include": { - "type" : "list", - "description" : "A list of fields to extract and return from the _source field" - }, "version" : { "type" : "number", "description" : "Explicit version number for concurrency control" From d880960584404fc12e7d42f4c6b0d9569691275a Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 23 Apr 2019 10:57:17 -0400 Subject: [PATCH 156/260] Docs: Rework close ml section (#41435) Reworks the close ml section to work properly in Asciidoctor. It renders a little funny in AsciiDoc but AsciiDoc is on its way out anyway.... --- docs/reference/upgrade/close-ml.asciidoc | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/reference/upgrade/close-ml.asciidoc b/docs/reference/upgrade/close-ml.asciidoc index 179d4f9adff39..affcd27926677 100644 --- a/docs/reference/upgrade/close-ml.asciidoc +++ b/docs/reference/upgrade/close-ml.asciidoc @@ -25,18 +25,16 @@ it puts increased load on the cluster. prevent new jobs from opening by using the <>: + --- [source,js] -------------------------------------------------- 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 active jobs during the upgrade and is faster than explicitly stopping {dfeeds} and closing jobs. --- * {stack-ov}/stopping-ml.html[Stop all {dfeeds} and close all jobs]. This option saves the model state at the time of closure. When you reopen the jobs after the From ce5e0762d8184794e0b8de012223559d6047684a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 23 Apr 2019 11:55:27 -0400 Subject: [PATCH 157/260] Introduce aliases version (#41397) This commit introduces aliases versions to index metadata. This will be useful in CCR when we replicate aliases. --- .../elasticsearch/cluster/ClusterState.java | 1 + .../cluster/metadata/IndexMetaData.java | 121 +++++- .../metadata/MetaDataIndexAliasesService.java | 16 +- .../snapshots/RestoreService.java | 19 +- .../elasticsearch/aliases/IndexAliasesIT.java | 349 +++++++++++++----- .../MetaDataIndexAliasesServiceTests.java | 132 ++++++- 6 files changed, 508 insertions(+), 130 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index 93ee8af779472..a5996cc5da9c6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -313,6 +313,7 @@ public String toString() { sb.append(": v[").append(indexMetaData.getVersion()) .append("], mv[").append(indexMetaData.getMappingVersion()) .append("], sv[").append(indexMetaData.getSettingsVersion()) + .append("], av[").append(indexMetaData.getAliasesVersion()) .append("]\n"); for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) { sb.append(TAB).append(TAB).append(shard).append(": "); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 6fc73ef5f7fa8..f3c0af84a06c9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -260,6 +260,7 @@ public Iterator> settings() { static final String KEY_VERSION = "version"; static final String KEY_MAPPING_VERSION = "mapping_version"; static final String KEY_SETTINGS_VERSION = "settings_version"; + static final String KEY_ALIASES_VERSION = "aliases_version"; static final String KEY_ROUTING_NUM_SHARDS = "routing_num_shards"; static final String KEY_SETTINGS = "settings"; static final String KEY_STATE = "state"; @@ -282,6 +283,8 @@ public Iterator> settings() { private final long mappingVersion; private final long settingsVersion; + + private final long aliasesVersion; private final long[] primaryTerms; @@ -310,15 +313,31 @@ public Iterator> settings() { private final ActiveShardCount waitForActiveShards; private final ImmutableOpenMap rolloverInfos; - private IndexMetaData(Index index, long version, long mappingVersion, long settingsVersion, long[] primaryTerms, State state, - int numberOfShards, int numberOfReplicas, Settings settings, - ImmutableOpenMap mappings, ImmutableOpenMap aliases, - ImmutableOpenMap customData, ImmutableOpenIntMap> inSyncAllocationIds, - DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters initialRecoveryFilters, - DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters, - Version indexCreatedVersion, Version indexUpgradedVersion, - int routingNumShards, int routingPartitionSize, ActiveShardCount waitForActiveShards, - ImmutableOpenMap rolloverInfos) { + private IndexMetaData( + final Index index, + final long version, + final long mappingVersion, + final long settingsVersion, + final long aliasesVersion, + final long[] primaryTerms, + final State state, + final int numberOfShards, + final int numberOfReplicas, + final Settings settings, + final ImmutableOpenMap mappings, + final ImmutableOpenMap aliases, + final ImmutableOpenMap customData, + final ImmutableOpenIntMap> inSyncAllocationIds, + final DiscoveryNodeFilters requireFilters, + final DiscoveryNodeFilters initialRecoveryFilters, + final DiscoveryNodeFilters includeFilters, + final DiscoveryNodeFilters excludeFilters, + final Version indexCreatedVersion, + final Version indexUpgradedVersion, + final int routingNumShards, + final int routingPartitionSize, + final ActiveShardCount waitForActiveShards, + final ImmutableOpenMap rolloverInfos) { this.index = index; this.version = version; @@ -326,6 +345,8 @@ private IndexMetaData(Index index, long version, long mappingVersion, long setti this.mappingVersion = mappingVersion; assert settingsVersion >= 0 : settingsVersion; this.settingsVersion = settingsVersion; + assert aliasesVersion >= 0 : aliasesVersion; + this.aliasesVersion = aliasesVersion; this.primaryTerms = primaryTerms; assert primaryTerms.length == numberOfShards; this.state = state; @@ -383,6 +404,10 @@ public long getSettingsVersion() { return settingsVersion; } + public long getAliasesVersion() { + return aliasesVersion; + } + /** * The term of the current selected primary. This is a non-negative number incremented when * a primary shard is assigned after a full cluster restart or a replica shard is promoted to a primary. @@ -652,6 +677,7 @@ private static class IndexMetaDataDiff implements Diff { private final long version; private final long mappingVersion; private final long settingsVersion; + private final long aliasesVersion; private final long[] primaryTerms; private final State state; private final Settings settings; @@ -666,6 +692,7 @@ private static class IndexMetaDataDiff implements Diff { version = after.version; mappingVersion = after.mappingVersion; settingsVersion = after.settingsVersion; + aliasesVersion = after.aliasesVersion; routingNumShards = after.routingNumShards; state = after.state; settings = after.settings; @@ -684,6 +711,11 @@ private static class IndexMetaDataDiff implements Diff { version = in.readLong(); mappingVersion = in.readVLong(); settingsVersion = in.readVLong(); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + aliasesVersion = in.readVLong(); + } else { + aliasesVersion = 0; + } state = State.fromId(in.readByte()); settings = Settings.readSettingsFromStream(in); primaryTerms = in.readVLongArray(); @@ -706,6 +738,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeVLong(mappingVersion); out.writeVLong(settingsVersion); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeVLong(aliasesVersion); + } out.writeByte(state.id); Settings.writeSettingsToStream(settings, out); out.writeVLongArray(primaryTerms); @@ -722,6 +757,7 @@ public IndexMetaData apply(IndexMetaData part) { builder.version(version); builder.mappingVersion(mappingVersion); builder.settingsVersion(settingsVersion); + builder.aliasesVersion(aliasesVersion); builder.setRoutingNumShards(routingNumShards); builder.state(state); builder.settings(settings); @@ -740,6 +776,9 @@ public static IndexMetaData readFrom(StreamInput in) throws IOException { builder.version(in.readLong()); builder.mappingVersion(in.readVLong()); builder.settingsVersion(in.readVLong()); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + builder.aliasesVersion(in.readVLong()); + } builder.setRoutingNumShards(in.readInt()); builder.state(State.fromId(in.readByte())); builder.settings(readSettingsFromStream(in)); @@ -779,6 +818,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeVLong(mappingVersion); out.writeVLong(settingsVersion); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeVLong(aliasesVersion); + } out.writeInt(routingNumShards); out.writeByte(state.id()); writeSettingsToStream(settings, out); @@ -822,6 +864,7 @@ public static class Builder { private long version = 1; private long mappingVersion = 1; private long settingsVersion = 1; + private long aliasesVersion = 1; private long[] primaryTerms = null; private Settings settings = Settings.Builder.EMPTY_SETTINGS; private final ImmutableOpenMap.Builder mappings; @@ -846,6 +889,7 @@ public Builder(IndexMetaData indexMetaData) { this.version = indexMetaData.version; this.mappingVersion = indexMetaData.mappingVersion; this.settingsVersion = indexMetaData.settingsVersion; + this.aliasesVersion = indexMetaData.aliasesVersion; this.settings = indexMetaData.getSettings(); this.primaryTerms = indexMetaData.primaryTerms.clone(); this.mappings = ImmutableOpenMap.builder(indexMetaData.mappings); @@ -994,20 +1038,29 @@ public long mappingVersion() { return mappingVersion; } - public long settingsVersion() { - return settingsVersion; - } - public Builder mappingVersion(final long mappingVersion) { this.mappingVersion = mappingVersion; return this; } - + + public long settingsVersion() { + return settingsVersion; + } + public Builder settingsVersion(final long settingsVersion) { this.settingsVersion = settingsVersion; return this; } - + + public long aliasesVersion() { + return aliasesVersion; + } + + public Builder aliasesVersion(final long aliasesVersion) { + this.aliasesVersion = aliasesVersion; + return this; + } + /** * returns the primary term for the given shard. * See {@link IndexMetaData#primaryTerm(int)} for more information. @@ -1136,11 +1189,31 @@ public IndexMetaData build() { final String uuid = settings.get(SETTING_INDEX_UUID, INDEX_UUID_NA_VALUE); - return new IndexMetaData(new Index(index, uuid), version, mappingVersion, settingsVersion, primaryTerms, state, - numberOfShards, numberOfReplicas, tmpSettings, mappings.build(), tmpAliases.build(), customMetaData.build(), - filledInSyncAllocationIds.build(), requireFilters, initialRecoveryFilters, includeFilters, excludeFilters, - indexCreatedVersion, indexUpgradedVersion, getRoutingNumShards(), routingPartitionSize, waitForActiveShards, - rolloverInfos.build()); + return new IndexMetaData( + new Index(index, uuid), + version, + mappingVersion, + settingsVersion, + aliasesVersion, + primaryTerms, + state, + numberOfShards, + numberOfReplicas, + tmpSettings, + mappings.build(), + tmpAliases.build(), + customMetaData.build(), + filledInSyncAllocationIds.build(), + requireFilters, + initialRecoveryFilters, + includeFilters, + excludeFilters, + indexCreatedVersion, + indexUpgradedVersion, + getRoutingNumShards(), + routingPartitionSize, + waitForActiveShards, + rolloverInfos.build()); } public static void toXContent(IndexMetaData indexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { @@ -1149,6 +1222,7 @@ public static void toXContent(IndexMetaData indexMetaData, XContentBuilder build builder.field(KEY_VERSION, indexMetaData.getVersion()); builder.field(KEY_MAPPING_VERSION, indexMetaData.getMappingVersion()); builder.field(KEY_SETTINGS_VERSION, indexMetaData.getSettingsVersion()); + builder.field(KEY_ALIASES_VERSION, indexMetaData.getAliasesVersion()); builder.field(KEY_ROUTING_NUM_SHARDS, indexMetaData.getRoutingNumShards()); builder.field(KEY_STATE, indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH)); @@ -1223,6 +1297,7 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti } boolean mappingVersion = false; boolean settingsVersion = false; + boolean aliasesVersion = false; while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); @@ -1321,6 +1396,9 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti } else if (KEY_SETTINGS_VERSION.equals(currentFieldName)) { settingsVersion = true; builder.settingsVersion(parser.longValue()); + } else if (KEY_ALIASES_VERSION.equals(currentFieldName)) { + aliasesVersion = true; + builder.aliasesVersion(parser.longValue()); } else if (KEY_ROUTING_NUM_SHARDS.equals(currentFieldName)) { builder.setRoutingNumShards(parser.intValue()); } else { @@ -1336,6 +1414,9 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_6_5_0)) { assert settingsVersion : "settings version should be present for indices created on or after 6.5.0"; } + if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_8_0_0)) { + assert aliasesVersion : "aliases version should be present for indices created on or after 8.0.0"; + } return builder.build(); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java index e6d0fc0832445..84e2f512e569f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesService.java @@ -115,6 +115,7 @@ ClusterState innerExecute(ClusterState currentState, Iterable actio } MetaData.Builder metadata = MetaData.builder(currentState.metaData()); // Run the remaining alias actions + final Set maybeModifiedIndices = new HashSet<>(); for (AliasAction action : actions) { if (action.removeIndex()) { // Handled above @@ -151,7 +152,20 @@ ClusterState innerExecute(ClusterState currentState, Iterable actio xContentRegistry); } }; - changed |= action.apply(newAliasValidator, metadata, index); + if (action.apply(newAliasValidator, metadata, index)) { + changed = true; + maybeModifiedIndices.add(index.getIndex().getName()); + } + } + + for (final String maybeModifiedIndex : maybeModifiedIndices) { + final IndexMetaData currentIndexMetaData = currentState.metaData().index(maybeModifiedIndex); + final IndexMetaData newIndexMetaData = metadata.get(maybeModifiedIndex); + // only increment the aliases version if the aliases actually changed for this index + if (currentIndexMetaData.getAliases().equals(newIndexMetaData.getAliases()) == false) { + assert currentIndexMetaData.getAliasesVersion() == newIndexMetaData.getAliasesVersion(); + metadata.put(new IndexMetaData.Builder(newIndexMetaData).aliasesVersion(1 + currentIndexMetaData.getAliasesVersion())); + } } if (changed) { diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 3a81a9956870a..cb9e7fee04249 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -301,13 +301,18 @@ public ClusterState execute(ClusterState currentState) { } else { validateExistingIndex(currentIndexMetaData, snapshotIndexMetaData, renamedIndexName, partial); // Index exists and it's closed - open it in metadata and start recovery - IndexMetaData.Builder indexMdBuilder = IndexMetaData.builder(snapshotIndexMetaData) - .state(IndexMetaData.State.OPEN); - indexMdBuilder.version(Math.max(snapshotIndexMetaData.getVersion(), currentIndexMetaData.getVersion() + 1)); - indexMdBuilder.mappingVersion(Math.max(snapshotIndexMetaData.getMappingVersion(), - currentIndexMetaData.getMappingVersion() + 1)); - indexMdBuilder.settingsVersion(Math.max(snapshotIndexMetaData.getSettingsVersion(), - currentIndexMetaData.getSettingsVersion() + 1)); + IndexMetaData.Builder indexMdBuilder = + IndexMetaData.builder(snapshotIndexMetaData).state(IndexMetaData.State.OPEN); + indexMdBuilder.version( + Math.max(snapshotIndexMetaData.getVersion(), 1 + currentIndexMetaData.getVersion())); + indexMdBuilder.mappingVersion( + Math.max(snapshotIndexMetaData.getMappingVersion(), 1 + currentIndexMetaData.getMappingVersion())); + indexMdBuilder.settingsVersion( + Math.max( + snapshotIndexMetaData.getSettingsVersion(), + 1 + currentIndexMetaData.getSettingsVersion())); + indexMdBuilder.aliasesVersion( + Math.max(snapshotIndexMetaData.getAliasesVersion(), 1 + currentIndexMetaData.getAliasesVersion())); for (int shard = 0; shard < snapshotIndexMetaData.getNumberOfShards(); shard++) { indexMdBuilder.primaryTerm(shard, diff --git a/server/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java b/server/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java index a1058b03dacb0..ce7ea7fbb6bd0 100644 --- a/server/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java +++ b/server/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.rest.action.admin.indices.AliasesNotFoundException; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; @@ -49,6 +50,7 @@ import org.elasticsearch.test.ESIntegTestCase; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -81,14 +83,18 @@ import static org.hamcrest.Matchers.nullValue; public class IndexAliasesIT extends ESIntegTestCase { + public void testAliases() throws Exception { logger.info("--> creating index [test]"); createIndex("test"); ensureGreen(); - logger.info("--> aliasing index [test] with [alias1]"); - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1", false)); + assertAliasesVersionIncreases( + "test", () -> { + logger.info("--> aliasing index [test] with [alias1]"); + assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1", false)); + }); logger.info("--> indexing against [alias1], should fail now"); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, @@ -98,8 +104,13 @@ public void testAliases() throws Exception { " The write index may be explicitly disabled using is_write_index=false or the alias points to multiple" + " indices without one being designated as a write index")); - logger.info("--> aliasing index [test] with [alias1]"); - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1")); + assertAliasesVersionIncreases( + "test", + () -> { + logger.info("--> aliasing index [test] with [alias1]"); + assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1")); + } + ); logger.info("--> indexing against [alias1], should work now"); IndexResponse indexResponse = client().index(indexRequest("alias1").type("type1").id("1") @@ -111,8 +122,10 @@ public void testAliases() throws Exception { ensureGreen(); - logger.info("--> add index [test_x] with [alias1]"); - assertAcked(admin().indices().prepareAliases().addAlias("test_x", "alias1")); + assertAliasesVersionIncreases("test_x", () -> { + logger.info("--> add index [test_x] with [alias1]"); + assertAcked(admin().indices().prepareAliases().addAlias("test_x", "alias1")); + }); logger.info("--> indexing against [alias1], should fail now"); exception = expectThrows(IllegalArgumentException.class, @@ -129,16 +142,20 @@ public void testAliases() throws Exception { " The write index may be explicitly disabled using is_write_index=false or the alias points to multiple" + " indices without one being designated as a write index")); - logger.info("--> remove aliasing index [test_x] with [alias1]"); - assertAcked(admin().indices().prepareAliases().removeAlias("test_x", "alias1")); + assertAliasesVersionIncreases("test_x", () -> { + logger.info("--> remove aliasing index [test_x] with [alias1]"); + assertAcked(admin().indices().prepareAliases().removeAlias("test_x", "alias1")); + }); logger.info("--> indexing against [alias1], should work now"); indexResponse = client().index(indexRequest("alias1").type("type1").id("1") .source(source("1", "test"), XContentType.JSON)).actionGet(); assertThat(indexResponse.getIndex(), equalTo("test")); - logger.info("--> add index [test_x] with [alias1] as write-index"); - assertAcked(admin().indices().prepareAliases().addAlias("test_x", "alias1", true)); + assertAliasesVersionIncreases("test_x", () -> { + logger.info("--> add index [test_x] with [alias1] as write-index"); + assertAcked(admin().indices().prepareAliases().addAlias("test_x", "alias1", true)); + }); logger.info("--> indexing against [alias1], should work now"); indexResponse = client().index(indexRequest("alias1").type("type1").id("1") @@ -149,8 +166,10 @@ public void testAliases() throws Exception { DeleteResponse deleteResponse = client().delete(deleteRequest("alias1").type("type1").id("1")).actionGet(); assertThat(deleteResponse.getIndex(), equalTo("test_x")); - logger.info("--> remove [alias1], Aliasing index [test_x] with [alias1]"); - assertAcked(admin().indices().prepareAliases().removeAlias("test", "alias1").addAlias("test_x", "alias1")); + assertAliasesVersionIncreases("test_x", () -> { + logger.info("--> remove [alias1], Aliasing index [test_x] with [alias1]"); + assertAcked(admin().indices().prepareAliases().removeAlias("test", "alias1").addAlias("test_x", "alias1")); + }); logger.info("--> indexing against [alias1], should work against [test_x]"); indexResponse = client().index(indexRequest("alias1").type("type1").id("1") @@ -181,7 +200,7 @@ public void testFilteringAliases() throws Exception { logger.info("--> aliasing index [test] with [alias1] and filter [user:kimchy]"); QueryBuilder filter = termQuery("user", "kimchy"); - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1", filter)); + assertAliasesVersionIncreases("test", () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1", filter))); // For now just making sure that filter was stored with the alias logger.info("--> making sure that filter was stored with alias [alias1] and filter [user:kimchy]"); @@ -210,11 +229,18 @@ public void testSearchingFilteringAliasesSingleIndex() throws Exception { ensureGreen(); logger.info("--> adding filtering aliases to index [test]"); - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1")); - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias2")); - assertAcked(admin().indices().prepareAliases().addAlias("test", "foos", termQuery("name", "foo"))); - assertAcked(admin().indices().prepareAliases().addAlias("test", "bars", termQuery("name", "bar"))); - assertAcked(admin().indices().prepareAliases().addAlias("test", "tests", termQuery("name", "test"))); + + assertAliasesVersionIncreases("test", () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1"))); + assertAliasesVersionIncreases("test", () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "alias2"))); + assertAliasesVersionIncreases( + "test", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "foos", termQuery("name", "foo")))); + assertAliasesVersionIncreases( + "test", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "bars", termQuery("name", "bar")))); + assertAliasesVersionIncreases( + "test", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "tests", termQuery("name", "test")))); logger.info("--> indexing against [test]"); client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test"), XContentType.JSON) @@ -295,15 +321,21 @@ public void testSearchingFilteringAliasesTwoIndices() throws Exception { ensureGreen(); logger.info("--> adding filtering aliases to index [test1]"); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTest1")); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTests")); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "foos", termQuery("name", "foo"))); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "bars", termQuery("name", "bar"))); + assertAliasesVersionIncreases("test1", () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTest1"))); + assertAliasesVersionIncreases("test1", () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTests"))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "foos", termQuery("name", "foo")))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "bars", termQuery("name", "bar")))); logger.info("--> adding filtering aliases to index [test2]"); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTest2")); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTests")); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "foos", termQuery("name", "foo"))); + assertAliasesVersionIncreases("test2", () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTest2"))); + assertAliasesVersionIncreases("test2", () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTests"))); + assertAliasesVersionIncreases( + "test2", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "foos", termQuery("name", "foo")))); logger.info("--> indexing against [test1]"); client().index(indexRequest("test1").type("type1").id("1").source(source("1", "foo test"), XContentType.JSON)).get(); @@ -367,17 +399,27 @@ public void testSearchingFilteringAliasesMultipleIndices() throws Exception { ensureGreen(); logger.info("--> adding aliases to indices"); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "alias12")); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "alias12")); + assertAliasesVersionIncreases("test1", () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "alias12"))); + assertAliasesVersionIncreases("test2", () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "alias12"))); logger.info("--> adding filtering aliases to indices"); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "filter1", termQuery("name", "test1"))); - - assertAcked(admin().indices().prepareAliases().addAlias("test2", "filter23", termQuery("name", "foo"))); - assertAcked(admin().indices().prepareAliases().addAlias("test3", "filter23", termQuery("name", "foo"))); - - assertAcked(admin().indices().prepareAliases().addAlias("test1", "filter13", termQuery("name", "baz"))); - assertAcked(admin().indices().prepareAliases().addAlias("test3", "filter13", termQuery("name", "baz"))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "filter1", termQuery("name", "test1")))); + + assertAliasesVersionIncreases( + "test2", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "filter23", termQuery("name", "foo")))); + assertAliasesVersionIncreases( + "test3", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test3", "filter23", termQuery("name", "foo")))); + + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "filter13", termQuery("name", "baz")))); + assertAliasesVersionIncreases( + "test3", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test3", "filter13", termQuery("name", "baz")))); logger.info("--> indexing against [test1]"); client().index(indexRequest("test1").type("type1").id("11").source(source("11", "foo test1"), XContentType.JSON)).get(); @@ -433,17 +475,27 @@ public void testDeletingByQueryFilteringAliases() throws Exception { ensureGreen(); logger.info("--> adding filtering aliases to index [test1]"); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTest1")); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTests")); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "foos", termQuery("name", "foo"))); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "bars", termQuery("name", "bar"))); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "tests", termQuery("name", "test"))); + assertAliasesVersionIncreases("test1", () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTest1"))); + assertAliasesVersionIncreases("test1", () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTests"))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "foos", termQuery("name", "foo")))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "bars", termQuery("name", "bar")))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test1", "tests", termQuery("name", "test")))); logger.info("--> adding filtering aliases to index [test2]"); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTest2")); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTests")); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "foos", termQuery("name", "foo"))); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "tests", termQuery("name", "test"))); + assertAliasesVersionIncreases("test2", () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTest2"))); + assertAliasesVersionIncreases("test2", () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTests"))); + assertAliasesVersionIncreases( + "test2", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "foos", termQuery("name", "foo")))); + assertAliasesVersionIncreases( + "test2", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test2", "tests", termQuery("name", "test")))); logger.info("--> indexing against [test1]"); client().index(indexRequest("test1").type("type1").id("1").source(source("1", "foo test"), XContentType.JSON)).get(); @@ -471,22 +523,30 @@ public void testDeleteAliases() throws Exception { ensureGreen(); logger.info("--> adding filtering aliases to index [test1]"); - assertAcked(admin().indices().prepareAliases().addAlias("test1", "aliasToTest1") - .addAlias("test1", "aliasToTests") - .addAlias("test1", "foos", termQuery("name", "foo")) - .addAlias("test1", "bars", termQuery("name", "bar")) - .addAlias("test1", "tests", termQuery("name", "test"))); + assertAliasesVersionIncreases( + "test1", + () -> assertAcked(admin().indices() + .prepareAliases() + .addAlias("test1", "aliasToTest1") + .addAlias("test1", "aliasToTests") + .addAlias("test1", "foos", termQuery("name", "foo")) + .addAlias("test1", "bars", termQuery("name", "bar")) + .addAlias("test1", "tests", termQuery("name", "test")))); logger.info("--> adding filtering aliases to index [test2]"); - assertAcked(admin().indices().prepareAliases().addAlias("test2", "aliasToTest2") - .addAlias("test2", "aliasToTests") - .addAlias("test2", "foos", termQuery("name", "foo")) - .addAlias("test2", "tests", termQuery("name", "test"))); + assertAliasesVersionIncreases( + "test2", + () -> assertAcked(admin().indices() + .prepareAliases() + .addAlias("test2", "aliasToTest2") + .addAlias("test2", "aliasToTests") + .addAlias("test2", "foos", termQuery("name", "foo")) + .addAlias("test2", "tests", termQuery("name", "test")))); String[] indices = {"test1", "test2"}; String[] aliases = {"aliasToTest1", "foos", "bars", "tests", "aliasToTest2", "aliasToTests"}; - admin().indices().prepareAliases().removeAlias(indices, aliases).get(); + assertAliasesVersionIncreases(indices, () -> admin().indices().prepareAliases().removeAlias(indices, aliases).get()); AliasesExistResponse response = admin().indices().prepareAliasesExist(aliases).get(); assertThat(response.exists(), equalTo(false)); @@ -497,10 +557,12 @@ public void testDeleteAliases() throws Exception { ensureGreen(); logger.info("--> adding [foo] alias to [foo_foo] and [bar_bar]"); - assertAcked(admin().indices().prepareAliases().addAlias("foo_foo", "foo")); - assertAcked(admin().indices().prepareAliases().addAlias("bar_bar", "foo")); + assertAliasesVersionIncreases("foo_foo", () -> assertAcked(admin().indices().prepareAliases().addAlias("foo_foo", "foo"))); + assertAliasesVersionIncreases("bar_bar", () -> assertAcked(admin().indices().prepareAliases().addAlias("bar_bar", "foo"))); - assertAcked(admin().indices().prepareAliases().addAliasAction(AliasActions.remove().index("foo*").alias("foo")).execute().get()); + assertAliasesVersionIncreases( + "foo_foo", + () -> assertAcked(admin().indices().prepareAliases().addAliasAction(AliasActions.remove().index("foo*").alias("foo")))); assertTrue(admin().indices().prepareAliasesExist("foo").get().exists()); assertFalse(admin().indices().prepareAliasesExist("foo").setIndices("foo_foo").get().exists()); @@ -518,8 +580,9 @@ public void testWaitForAliasCreationMultipleShards() throws Exception { ensureGreen(); for (int i = 0; i < 10; i++) { - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias" + i)); - client().index(indexRequest("alias" + i).type("type1").id("1").source(source("1", "test"), XContentType.JSON)).get(); + final String aliasName = "alias" + i; + assertAliasesVersionIncreases("test", () -> assertAcked(admin().indices().prepareAliases().addAlias("test", aliasName))); + client().index(indexRequest(aliasName).type("type1").id("1").source(source("1", "test"), XContentType.JSON)).get(); } } @@ -533,8 +596,9 @@ public void testWaitForAliasCreationSingleShard() throws Exception { ensureGreen(); for (int i = 0; i < 10; i++) { - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias" + i)); - client().index(indexRequest("alias" + i).type("type1").id("1").source(source("1", "test"), + final String aliasName = "alias" + i; + assertAliasesVersionIncreases("test", () -> assertAcked(admin().indices().prepareAliases().addAlias("test", aliasName))); + client().index(indexRequest(aliasName).type("type1").id("1").source(source("1", "test"), XContentType.JSON)).get(); } } @@ -553,7 +617,9 @@ public void testWaitForAliasSimultaneousUpdate() throws Exception { executor.submit(new Runnable() { @Override public void run() { - assertAcked(admin().indices().prepareAliases().addAlias("test", aliasName)); + assertAliasesVersionIncreases( + "test", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test", aliasName))); client().index(indexRequest(aliasName).type("type1").id("1").source(source("1", "test"), XContentType.JSON)) .actionGet(); } @@ -573,27 +639,37 @@ public void testSameAlias() throws Exception { ensureGreen(); logger.info("--> creating alias1 "); - assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1"))); + assertAliasesVersionIncreases("test", () -> assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1")))); TimeValue timeout = TimeValue.timeValueSeconds(2); logger.info("--> recreating alias1 "); StopWatch stopWatch = new StopWatch(); stopWatch.start(); - assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1").setTimeout(timeout))); + assertAliasesVersionUnchanged( + "test", + () -> assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1").setTimeout(timeout)))); assertThat(stopWatch.stop().lastTaskTime().millis(), lessThan(timeout.millis())); logger.info("--> modifying alias1 to have a filter"); stopWatch.start(); - assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1", termQuery("name", "foo")).setTimeout(timeout))); + final TermQueryBuilder fooFilter = termQuery("name", "foo"); + assertAliasesVersionIncreases( + "test", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1", fooFilter).setTimeout(timeout))); assertThat(stopWatch.stop().lastTaskTime().millis(), lessThan(timeout.millis())); logger.info("--> recreating alias1 with the same filter"); stopWatch.start(); - assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1", termQuery("name", "foo")).setTimeout(timeout))); + assertAliasesVersionUnchanged( + "test", + () -> assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1", fooFilter).setTimeout(timeout)))); assertThat(stopWatch.stop().lastTaskTime().millis(), lessThan(timeout.millis())); logger.info("--> recreating alias1 with a different filter"); stopWatch.start(); - assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1", termQuery("name", "bar")).setTimeout(timeout))); + final TermQueryBuilder barFilter = termQuery("name", "bar"); + assertAliasesVersionIncreases( + "test", + () -> assertAcked((admin().indices().prepareAliases().addAlias("test", "alias1", barFilter).setTimeout(timeout)))); assertThat(stopWatch.stop().lastTaskTime().millis(), lessThan(timeout.millis())); logger.info("--> verify that filter was updated"); @@ -603,10 +679,10 @@ public void testSameAlias() throws Exception { logger.info("--> deleting alias1"); stopWatch.start(); - assertAcked((admin().indices().prepareAliases().removeAlias("test", "alias1").setTimeout(timeout))); + assertAliasesVersionIncreases( + "test", + () -> assertAcked((admin().indices().prepareAliases().removeAlias("test", "alias1").setTimeout(timeout)))); assertThat(stopWatch.stop().lastTaskTime().millis(), lessThan(timeout.millis())); - - } public void testIndicesRemoveNonExistingAliasResponds404() throws Exception { @@ -635,7 +711,9 @@ public void testIndicesGetAliases() throws Exception { ensureGreen(); logger.info("--> creating aliases [alias1, alias2]"); - assertAcked(admin().indices().prepareAliases().addAlias("foobar", "alias1").addAlias("foobar", "alias2")); + assertAliasesVersionIncreases( + "foobar", + () -> assertAcked(admin().indices().prepareAliases().addAlias("foobar", "alias1").addAlias("foobar", "alias2"))); logger.info("--> getting alias1"); GetAliasesResponse getResponse = admin().indices().prepareGetAliases("alias1").get(); @@ -670,13 +748,19 @@ public void testIndicesGetAliases() throws Exception { logger.info("--> creating aliases [bar, baz, foo]"); - assertAcked(admin().indices().prepareAliases() - .addAlias("bazbar", "bar") - .addAlias("bazbar", "bac", termQuery("field", "value")) - .addAlias("foobar", "foo")); - - assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("foobar").alias("bac").routing("bla"))); + assertAliasesVersionIncreases( + new String[]{"bazbar", "foobar"}, + () -> assertAcked(admin().indices() + .prepareAliases() + .addAlias("bazbar", "bar") + .addAlias("bazbar", "bac", termQuery("field", "value")) + .addAlias("foobar", "foo"))); + + assertAliasesVersionIncreases( + "foobar", + () -> assertAcked(admin().indices() + .prepareAliases() + .addAliasAction(AliasActions.add().index("foobar").alias("bac").routing("bla")))); logger.info("--> getting bar and baz for index bazbar"); getResponse = admin().indices().prepareGetAliases("bar", "bac").addIndices("bazbar").get(); @@ -826,7 +910,9 @@ public void testGetAllAliasesWorks() { createIndex("index1"); createIndex("index2"); - assertAcked(admin().indices().prepareAliases().addAlias("index1", "alias1").addAlias("index2", "alias2")); + assertAliasesVersionIncreases( + new String[]{"index1", "index2"}, + () -> assertAcked(admin().indices().prepareAliases().addAlias("index1", "alias1").addAlias("index2", "alias2"))); GetAliasesResponse response = admin().indices().prepareGetAliases().get(); assertThat(response.getAliases(), hasKey("index1")); @@ -911,23 +997,41 @@ public void testAliasesCanBeAddedToIndicesOnly() throws Exception { // fields mentioned in filters don't need to exist in the mapping. public void testAddAliasWithFilterNoMapping() throws Exception { assertAcked(prepareCreate("test")); - client().admin().indices().prepareAliases() - .addAlias("test", "a", QueryBuilders.termQuery("field1", "term")) - .get(); - client().admin().indices().prepareAliases() - .addAlias("test", "a", QueryBuilders.rangeQuery("field2").from(0).to(1)) - .get(); - client().admin().indices().prepareAliases() - .addAlias("test", "a", QueryBuilders.matchAllQuery()) - .get(); + assertAliasesVersionIncreases( + "test", + () -> client().admin() + .indices() + .prepareAliases() + .addAlias("test", "a", QueryBuilders.termQuery("field1", "term")) + .get()); + assertAliasesVersionIncreases( + "test", + () -> client().admin() + .indices() + .prepareAliases() + .addAlias("test", "a", QueryBuilders.rangeQuery("field2").from(0).to(1)) + .get()); + assertAliasesVersionIncreases( + "test", + () -> client().admin() + .indices() + .prepareAliases() + .addAlias("test", "a", QueryBuilders.matchAllQuery()) + .get()); } public void testAliasFilterWithNowInRangeFilterAndQuery() throws Exception { assertAcked(prepareCreate("my-index").addMapping("my-type", "timestamp", "type=date")); - assertAcked(admin().indices().prepareAliases() - .addAlias("my-index", "filter1", rangeQuery("timestamp").from("2016-12-01").to("2016-12-31"))); - assertAcked(admin().indices().prepareAliases() - .addAlias("my-index", "filter2", rangeQuery("timestamp").from("2016-01-01").to("2016-12-31"))); + assertAliasesVersionIncreases( + "my-index", + () -> assertAcked(admin().indices() + .prepareAliases() + .addAlias("my-index", "filter1", rangeQuery("timestamp").from("2016-12-01").to("2016-12-31")))); + assertAliasesVersionIncreases( + "my-index", + () -> assertAcked(admin().indices() + .prepareAliases() + .addAlias("my-index", "filter2", rangeQuery("timestamp").from("2016-01-01").to("2016-12-31")))); final int numDocs = scaledRandomIntBetween(5, 52); for (int i = 1; i <= numDocs; i++) { @@ -951,8 +1055,10 @@ public void testAliasesWithBlocks() { try { enableIndexBlock("test", block); - assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1").addAlias("test", "alias2")); - assertAcked(admin().indices().prepareAliases().removeAlias("test", "alias1")); + assertAliasesVersionIncreases( + "test", + () -> assertAcked(admin().indices().prepareAliases().addAlias("test", "alias1").addAlias("test", "alias2"))); + assertAliasesVersionIncreases("test", () -> assertAcked(admin().indices().prepareAliases().removeAlias("test", "alias1"))); assertThat(admin().indices().prepareGetAliases("alias2").execute().actionGet().getAliases().get("test").size(), equalTo(1)); assertThat(admin().indices().prepareAliasesExist("alias2").get().exists(), equalTo(true)); } finally { @@ -963,8 +1069,12 @@ public void testAliasesWithBlocks() { try { enableIndexBlock("test", SETTING_READ_ONLY); - assertBlocked(admin().indices().prepareAliases().addAlias("test", "alias3"), INDEX_READ_ONLY_BLOCK); - assertBlocked(admin().indices().prepareAliases().removeAlias("test", "alias2"), INDEX_READ_ONLY_BLOCK); + assertAliasesVersionUnchanged( + "test", + () -> assertBlocked(admin().indices().prepareAliases().addAlias("test", "alias3"), INDEX_READ_ONLY_BLOCK)); + assertAliasesVersionUnchanged( + "test", + () -> assertBlocked(admin().indices().prepareAliases().removeAlias("test", "alias2"), INDEX_READ_ONLY_BLOCK)); assertThat(admin().indices().prepareGetAliases("alias2").execute().actionGet().getAliases().get("test").size(), equalTo(1)); assertThat(admin().indices().prepareAliasesExist("alias2").get().exists(), equalTo(true)); @@ -975,8 +1085,12 @@ public void testAliasesWithBlocks() { try { enableIndexBlock("test", SETTING_BLOCKS_METADATA); - assertBlocked(admin().indices().prepareAliases().addAlias("test", "alias3"), INDEX_METADATA_BLOCK); - assertBlocked(admin().indices().prepareAliases().removeAlias("test", "alias2"), INDEX_METADATA_BLOCK); + assertAliasesVersionUnchanged( + "test", + () -> assertBlocked(admin().indices().prepareAliases().addAlias("test", "alias3"), INDEX_METADATA_BLOCK)); + assertAliasesVersionUnchanged( + "test", + () -> assertBlocked(admin().indices().prepareAliases().removeAlias("test", "alias2"), INDEX_METADATA_BLOCK)); assertBlocked(admin().indices().prepareGetAliases("alias2"), INDEX_METADATA_BLOCK); assertBlocked(admin().indices().prepareAliasesExist("alias2"), INDEX_METADATA_BLOCK); @@ -988,15 +1102,19 @@ public void testAliasesWithBlocks() { public void testAliasActionRemoveIndex() throws InterruptedException, ExecutionException { assertAcked(prepareCreate("foo_foo")); assertAcked(prepareCreate("bar_bar")); - assertAcked(admin().indices().prepareAliases().addAlias("foo_foo", "foo")); - assertAcked(admin().indices().prepareAliases().addAlias("bar_bar", "foo")); + assertAliasesVersionIncreases( + new String[]{"foo_foo", "bar_bar"}, + () -> { + assertAcked(admin().indices().prepareAliases().addAlias("foo_foo", "foo")); + assertAcked(admin().indices().prepareAliases().addAlias("bar_bar", "foo")); + }); IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> client().admin().indices().prepareAliases().removeIndex("foo").execute().actionGet()); assertEquals("The provided expression [foo] matches an alias, specify the corresponding concrete indices instead.", iae.getMessage()); - assertAcked(client().admin().indices().prepareAliases().removeIndex("foo*").execute().get()); + assertAcked(client().admin().indices().prepareAliases().removeIndex("foo*")); assertFalse(client().admin().indices().prepareExists("foo_foo").execute().actionGet().isExists()); assertTrue(admin().indices().prepareAliasesExist("foo").get().exists()); assertTrue(client().admin().indices().prepareExists("bar_bar").execute().actionGet().isExists()); @@ -1010,7 +1128,9 @@ public void testAliasActionRemoveIndex() throws InterruptedException, ExecutionE public void testRemoveIndexAndReplaceWithAlias() throws InterruptedException, ExecutionException { assertAcked(client().admin().indices().prepareCreate("test")); indexRandom(true, client().prepareIndex("test_2", "test", "test").setSource("test", "test")); - assertAcked(client().admin().indices().prepareAliases().addAlias("test_2", "test").removeIndex("test")); + assertAliasesVersionIncreases( + "test_2", + () -> assertAcked(client().admin().indices().prepareAliases().addAlias("test_2", "test").removeIndex("test"))); assertHitCount(client().prepareSearch("test").get(), 1); } @@ -1052,4 +1172,31 @@ private void assertHits(SearchHits hits, String... ids) { private String source(String id, String nameValue) { return "{ \"id\" : \"" + id + "\", \"name\" : \"" + nameValue + "\" }"; } + + private void assertAliasesVersionIncreases(final String index, final Runnable runnable) { + assertAliasesVersionIncreases(new String[]{index}, runnable); + } + + private void assertAliasesVersionIncreases(final String[] indices, final Runnable runnable) { + final var beforeAliasesVersions = new HashMap(indices.length); + final var beforeMetaData = admin().cluster().prepareState().get().getState().metaData(); + for (final var index : indices) { + beforeAliasesVersions.put(index, beforeMetaData.index(index).getAliasesVersion()); + } + runnable.run(); + final var afterMetaData = admin().cluster().prepareState().get().getState().metaData(); + for (final String index : indices) { + assertThat(afterMetaData.index(index).getAliasesVersion(), equalTo(1 + beforeAliasesVersions.get(index))); + } + } + + private void assertAliasesVersionUnchanged(final String index, final Runnable runnable) { + final long beforeAliasesVersion = + admin().cluster().prepareState().get().getState().metaData().index(index).getAliasesVersion(); + runnable.run(); + final long afterAliasesVersion = + admin().cluster().prepareState().get().getState().metaData().index(index).getAliasesVersion(); + assertThat(afterAliasesVersion, equalTo(beforeAliasesVersion)); + } + } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java index 9b2d58ac28758..e07865f308ef5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java @@ -23,14 +23,17 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import static java.util.Collections.singletonList; @@ -74,6 +77,7 @@ public void testAddAndRemove() { assertNotNull(alias); assertTrue(alias.isAlias()); assertThat(alias.getIndices(), contains(after.metaData().index(index))); + assertAliasesVersionIncreased(index, before, after); // Remove the alias from it while adding another one before = after; @@ -85,12 +89,102 @@ public void testAddAndRemove() { assertNotNull(alias); assertTrue(alias.isAlias()); assertThat(alias.getIndices(), contains(after.metaData().index(index))); + assertAliasesVersionIncreased(index, before, after); // Now just remove on its own before = after; after = service.innerExecute(before, singletonList(new AliasAction.Remove(index, "test_2"))); assertNull(after.metaData().getAliasAndIndexLookup().get("test")); assertNull(after.metaData().getAliasAndIndexLookup().get("test_2")); + assertAliasesVersionIncreased(index, before, after); + } + + public void testMultipleIndices() { + final var length = randomIntBetween(2, 8); + final var indices = new HashSet(length); + var before = ClusterState.builder(ClusterName.DEFAULT).build(); + final var addActions = new ArrayList(length); + for (int i = 0; i < length; i++) { + final String index = randomValueOtherThanMany(v -> indices.add(v) == false, () -> randomAlphaOfLength(8)); + before = createIndex(before, index); + addActions.add(new AliasAction.Add(index, "alias-" + index, null, null, null, null)); + } + final var afterAddingAliasesToAll = service.innerExecute(before, addActions); + assertAliasesVersionIncreased(indices.toArray(new String[0]), before, afterAddingAliasesToAll); + + // now add some aliases randomly + final var randomIndices = new HashSet(length); + final var randomAddActions = new ArrayList(length); + for (var index : indices) { + if (randomBoolean()) { + randomAddActions.add(new AliasAction.Add(index, "random-alias-" + index, null, null, null, null)); + randomIndices.add(index); + } + } + final var afterAddingRandomAliases = service.innerExecute(afterAddingAliasesToAll, randomAddActions); + assertAliasesVersionIncreased(randomIndices.toArray(new String[0]), afterAddingAliasesToAll, afterAddingRandomAliases); + assertAliasesVersionUnchanged( + Sets.difference(indices, randomIndices).toArray(new String[0]), + afterAddingAliasesToAll, + afterAddingRandomAliases); + } + + public void testChangingWriteAliasStateIncreasesAliasesVersion() { + final String index = randomAlphaOfLength(8); + final ClusterState before = createIndex(ClusterState.builder(ClusterName.DEFAULT).build(), index); + + final ClusterState afterAddWriteAlias = + service.innerExecute(before, singletonList(new AliasAction.Add(index, "test", null, null, null, true))); + assertAliasesVersionIncreased(index, before, afterAddWriteAlias); + + final ClusterState afterChangeWriteAliasToNonWriteAlias = + service.innerExecute(afterAddWriteAlias, singletonList(new AliasAction.Add(index, "test", null, null, null, false))); + assertAliasesVersionIncreased(index, afterAddWriteAlias, afterChangeWriteAliasToNonWriteAlias); + + final ClusterState afterChangeNonWriteAliasToWriteAlias = + service.innerExecute( + afterChangeWriteAliasToNonWriteAlias, + singletonList(new AliasAction.Add(index, "test", null, null, null, true))); + assertAliasesVersionIncreased(index, afterChangeWriteAliasToNonWriteAlias, afterChangeNonWriteAliasToWriteAlias); + } + + public void testAddingAliasMoreThanOnceShouldOnlyIncreaseAliasesVersionByOne() { + final String index = randomAlphaOfLength(8); + final ClusterState before = createIndex(ClusterState.builder(ClusterName.DEFAULT).build(), index); + + // add an alias to the index multiple times + final int length = randomIntBetween(2, 8); + final var addActions = new ArrayList(length); + for (int i = 0; i < length; i++) { + addActions.add(new AliasAction.Add(index, "test", null, null, null, null)); + } + final ClusterState afterAddingAliases = service.innerExecute(before, addActions); + + assertAliasesVersionIncreased(index, before, afterAddingAliases); + } + + public void testAliasesVersionUnchangedWhenActionsAreIdempotent() { + final String index = randomAlphaOfLength(8); + final ClusterState before = createIndex(ClusterState.builder(ClusterName.DEFAULT).build(), index); + + // add some aliases to the index + final int length = randomIntBetween(1, 8); + final var aliasNames = new HashSet(); + final var addActions = new ArrayList(length); + for (int i = 0; i < length; i++) { + final String aliasName = randomValueOtherThanMany(v -> aliasNames.add(v) == false, () -> randomAlphaOfLength(8)); + addActions.add(new AliasAction.Add(index, aliasName, null, null, null, null)); + } + final ClusterState afterAddingAlias = service.innerExecute(before, addActions); + + // now perform a remove and add for each alias which is idempotent, the resulting aliases are unchanged + final var removeAndAddActions = new ArrayList(2 * length); + for (final var aliasName : aliasNames) { + removeAndAddActions.add(new AliasAction.Remove(index, aliasName)); + removeAndAddActions.add(new AliasAction.Add(index, aliasName, null, null, null, null)); + } + final ClusterState afterRemoveAndAddAlias = service.innerExecute(afterAddingAlias, removeAndAddActions); + assertAliasesVersionUnchanged(index, afterAddingAlias, afterRemoveAndAddAlias); } public void testSwapIndexWithAlias() { @@ -106,6 +200,7 @@ public void testSwapIndexWithAlias() { assertNotNull(alias); assertTrue(alias.isAlias()); assertThat(alias.getIndices(), contains(after.metaData().index("test_2"))); + assertAliasesVersionIncreased("test_2", before, after); } public void testAddAliasToRemovedIndex() { @@ -137,18 +232,21 @@ public void testAddWriteOnlyWithNoExistingAliases() { new AliasAction.Add("test", "alias", null, null, null, false))); assertFalse(after.metaData().index("test").getAliases().get("alias").writeIndex()); assertNull(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex()); + assertAliasesVersionIncreased("test", before, after); after = service.innerExecute(before, Arrays.asList( new AliasAction.Add("test", "alias", null, null, null, null))); assertNull(after.metaData().index("test").getAliases().get("alias").writeIndex()); assertThat(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex(), equalTo(after.metaData().index("test"))); + assertAliasesVersionIncreased("test", before, after); after = service.innerExecute(before, Arrays.asList( new AliasAction.Add("test", "alias", null, null, null, true))); assertTrue(after.metaData().index("test").getAliases().get("alias").writeIndex()); assertThat(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex(), equalTo(after.metaData().index("test"))); + assertAliasesVersionIncreased("test", before, after); } public void testAddWriteOnlyWithExistingWriteIndex() { @@ -165,6 +263,8 @@ public void testAddWriteOnlyWithExistingWriteIndex() { assertNull(after.metaData().index("test").getAliases().get("alias").writeIndex()); assertThat(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex(), equalTo(after.metaData().index("test2"))); + assertAliasesVersionIncreased("test", before, after); + assertAliasesVersionUnchanged("test2", before, after); Exception exception = expectThrows(IllegalStateException.class, () -> service.innerExecute(before, Arrays.asList( new AliasAction.Add("test", "alias", null, null, null, true)))); @@ -191,6 +291,8 @@ public void testSwapWriteOnlyIndex() { assertTrue(after.metaData().index("test2").getAliases().get("alias").writeIndex()); assertThat(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex(), equalTo(after.metaData().index("test2"))); + assertAliasesVersionIncreased("test", before, after); + assertAliasesVersionIncreased("test2", before, after); } public void testAddWriteOnlyWithExistingNonWriteIndices() { @@ -212,7 +314,9 @@ public void testAddWriteOnlyWithExistingNonWriteIndices() { assertTrue(after.metaData().index("test3").getAliases().get("alias").writeIndex()); assertThat(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex(), equalTo(after.metaData().index("test3"))); - + assertAliasesVersionUnchanged("test", before, after); + assertAliasesVersionUnchanged("test2", before, after); + assertAliasesVersionIncreased("test3", before, after); } public void testAddWriteOnlyWithIndexRemoved() { @@ -233,6 +337,7 @@ public void testAddWriteOnlyWithIndexRemoved() { assertNull(after.metaData().index("test2").getAliases().get("alias").writeIndex()); assertThat(((AliasOrIndex.Alias) after.metaData().getAliasAndIndexLookup().get("alias")).getWriteIndex(), equalTo(after.metaData().index("test2"))); + assertAliasesVersionUnchanged("test2", before, after); } public void testAddWriteOnlyValidatesAgainstMetaDataBuilder() { @@ -260,4 +365,29 @@ private ClusterState createIndex(ClusterState state, String index) { .metaData(MetaData.builder(state.metaData()).put(indexMetaData, false)) .build(); } + + private void assertAliasesVersionUnchanged(final String index, final ClusterState before, final ClusterState after) { + assertAliasesVersionUnchanged(new String[]{index}, before, after); + } + + private void assertAliasesVersionUnchanged(final String[] indices, final ClusterState before, final ClusterState after) { + for (final var index : indices) { + final long expected = before.metaData().index(index).getAliasesVersion(); + final long actual = after.metaData().index(index).getAliasesVersion(); + assertThat("index metadata aliases version mismatch", actual, equalTo(expected)); + } + } + + private void assertAliasesVersionIncreased(final String index, final ClusterState before, final ClusterState after) { + assertAliasesVersionIncreased(new String[]{index}, before, after); + } + + private void assertAliasesVersionIncreased(final String[] indices, final ClusterState before, final ClusterState after) { + for (final var index : indices) { + final long expected = 1 + before.metaData().index(index).getAliasesVersion(); + final long actual = after.metaData().index(index).getAliasesVersion(); + assertThat("index metadata aliases version mismatch", actual, equalTo(expected)); + } + } + } From 9fc0e81cb327835327b6c363895f395ac8fb9782 Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Tue, 23 Apr 2019 09:10:47 -0700 Subject: [PATCH 158/260] Task to generate Painless API's per context (#41233) This adds a gradle task called generateContextDoc in the Painless module. The task will start a cluster, issue commands against the context rest api for Painless, and generate documentation for each API per context. Each context has a first page of classes sorted by package first and class name second, along with a page per package with each classes' constructors, methods, and fields. A link is generated for each constructor, method, and field to a JavaDoc page when possible. --- docs/Versions.asciidoc | 3 +- docs/painless/painless-api-reference.asciidoc | 14 +- .../AbstractChronology.asciidoc | 7 - .../AbstractCollection.asciidoc | 7 - .../AbstractList.asciidoc | 7 - .../AbstractMap.SimpleEntry.asciidoc | 9 - .../AbstractMap.SimpleImmutableEntry.asciidoc | 9 - .../AbstractMap.asciidoc | 7 - .../AbstractQueue.asciidoc | 7 - .../AbstractSequentialList.asciidoc | 7 - .../AbstractSet.asciidoc | 7 - .../Annotation.asciidoc | 9 - .../Appendable.asciidoc | 8 - .../ArithmeticException.asciidoc | 9 - .../ArrayDeque.asciidoc | 10 - .../ArrayIndexOutOfBoundsException.asciidoc | 9 - .../painless-api-reference/ArrayList.asciidoc | 11 - .../ArrayStoreException.asciidoc | 9 - .../painless-api-reference/Arrays.asciidoc | 11 - ...ibutedCharacterIterator.Attribute.asciidoc | 10 - .../AttributedCharacterIterator.asciidoc | 14 - .../AttributedString.asciidoc | 15 - .../Base64.Decoder.asciidoc | 9 - .../Base64.Encoder.asciidoc | 10 - .../painless-api-reference/Base64.asciidoc | 14 - .../BaseStream.asciidoc | 13 - .../BiConsumer.asciidoc | 9 - .../BiFunction.asciidoc | 9 - .../BiPredicate.asciidoc | 11 - .../painless-api-reference/Bidi.asciidoc | 28 - .../BigDecimal.asciidoc | 57 - .../BigInteger.asciidoc | 50 - .../BinaryOperator.asciidoc | 9 - .../painless-api-reference/BitSet.asciidoc | 34 - .../painless-api-reference/Boolean.asciidoc | 19 - .../BooleanSupplier.asciidoc | 8 - .../BreakIterator.asciidoc | 29 - .../painless-api-reference/Byte.asciidoc | 22 - .../painless-api-reference/BytesRef.asciidoc | 12 - .../Calendar.Builder.asciidoc | 21 - .../painless-api-reference/Calendar.asciidoc | 102 - .../CharSequence.asciidoc | 15 - .../Character.Subset.asciidoc | 7 - .../Character.UnicodeBlock.asciidoc | 229 - .../Character.UnicodeScript.asciidoc | 114 - .../painless-api-reference/Character.asciidoc | 125 - .../CharacterIterator.asciidoc | 18 - .../ChoiceFormat.asciidoc | 17 - .../ChronoField.asciidoc | 41 - .../ChronoLocalDate.asciidoc | 31 - .../ChronoLocalDateTime.asciidoc | 29 - .../ChronoPeriod.asciidoc | 20 - .../ChronoUnit.asciidoc | 25 - .../ChronoZonedDateTime.asciidoc | 35 - .../Chronology.asciidoc | 33 - .../ClassCastException.asciidoc | 9 - .../ClassNotFoundException.asciidoc | 9 - .../painless-api-reference/Clock.asciidoc | 13 - .../CloneNotSupportedException.asciidoc | 9 - .../CollationElementIterator.asciidoc | 18 - .../CollationKey.asciidoc | 10 - .../painless-api-reference/Collator.asciidoc | 24 - .../Collection.asciidoc | 28 - .../Collections.asciidoc | 59 - .../Collector.Characteristics.asciidoc | 12 - .../painless-api-reference/Collector.asciidoc | 14 - .../Collectors.asciidoc | 38 - .../Comparable.asciidoc | 8 - .../Comparator.asciidoc | 24 - .../ConcurrentModificationException.asciidoc | 9 - .../painless-api-reference/Consumer.asciidoc | 9 - .../painless-api-reference/Currency.asciidoc | 16 - .../painless-api-reference/Date.asciidoc | 16 - .../DateFormat.Field.asciidoc | 27 - .../DateFormat.asciidoc | 51 - .../DateFormatSymbols.asciidoc | 30 - .../DateTimeException.asciidoc | 8 - .../DateTimeFormatter.asciidoc | 50 - .../DateTimeFormatterBuilder.asciidoc | 44 - .../DateTimeParseException.asciidoc | 10 - .../painless-api-reference/DayOfWeek.asciidoc | 22 - .../painless-api-reference/Debug.asciidoc | 8 - .../DecimalFormat.asciidoc | 32 - .../DecimalFormatSymbols.asciidoc | 43 - .../DecimalStyle.asciidoc | 19 - .../painless-api-reference/Deque.asciidoc | 25 - .../Dictionary.asciidoc | 14 - .../painless-api-reference/Double.asciidoc | 35 - .../DoubleBinaryOperator.asciidoc | 8 - .../DoubleConsumer.asciidoc | 9 - .../DoubleFunction.asciidoc | 8 - .../DoublePredicate.asciidoc | 11 - .../DoubleStream.Builder.asciidoc | 9 - .../DoubleStream.asciidoc | 43 - .../DoubleSummaryStatistics.asciidoc | 14 - .../DoubleSupplier.asciidoc | 8 - .../DoubleToIntFunction.asciidoc | 8 - .../DoubleToLongFunction.asciidoc | 8 - .../DoubleUnaryOperator.asciidoc | 11 - .../DuplicateFormatFlagsException.asciidoc | 9 - .../painless-api-reference/Duration.asciidoc | 51 - .../EmptyStackException.asciidoc | 8 - .../painless-api-reference/Enum.asciidoc | 10 - .../EnumConstantNotPresentException.asciidoc | 8 - .../Enumeration.asciidoc | 9 - .../painless-api-reference/Era.asciidoc | 9 - .../EventListener.asciidoc | 7 - .../EventListenerProxy.asciidoc | 8 - .../EventObject.asciidoc | 9 - .../painless-api-reference/Exception.asciidoc | 12 - .../FieldPosition.asciidoc | 15 - .../painless-api-reference/Float.asciidoc | 35 - .../Format.Field.asciidoc | 7 - .../painless-api-reference/Format.asciidoc | 13 - ...tFlagsConversionMismatchException.asciidoc | 10 - .../FormatStyle.asciidoc | 13 - .../Formattable.asciidoc | 8 - .../FormattableFlags.asciidoc | 10 - .../Formatter.BigDecimalLayoutForm.asciidoc | 9 - .../painless-api-reference/Formatter.asciidoc | 14 - .../FormatterClosedException.asciidoc | 8 - .../painless-api-reference/Function.asciidoc | 11 - .../GregorianCalendar.asciidoc | 20 - .../painless-api-reference/HashMap.asciidoc | 10 - .../painless-api-reference/HashSet.asciidoc | 10 - .../painless-api-reference/Hashtable.asciidoc | 10 - .../HijrahChronology.asciidoc | 16 - .../HijrahDate.asciidoc | 18 - .../painless-api-reference/HijrahEra.asciidoc | 12 - .../IdentityHashMap.asciidoc | 10 - .../IllegalAccessException.asciidoc | 9 - .../IllegalArgumentException.asciidoc | 9 - .../IllegalFormatCodePointException.asciidoc | 9 - .../IllegalFormatConversionException.asciidoc | 8 - .../IllegalFormatException.asciidoc | 7 - .../IllegalFormatFlagsException.asciidoc | 9 - .../IllegalFormatPrecisionException.asciidoc | 9 - .../IllegalFormatWidthException.asciidoc | 9 - .../IllegalMonitorStateException.asciidoc | 9 - .../IllegalStateException.asciidoc | 9 - .../IllegalThreadStateException.asciidoc | 9 - .../IllformedLocaleException.asciidoc | 11 - .../IndexOutOfBoundsException.asciidoc | 9 - .../InputMismatchException.asciidoc | 9 - .../painless-api-reference/Instant.asciidoc | 36 - .../InstantiationException.asciidoc | 9 - .../IntBinaryOperator.asciidoc | 8 - .../IntConsumer.asciidoc | 9 - .../IntFunction.asciidoc | 8 - .../IntPredicate.asciidoc | 11 - .../IntStream.Builder.asciidoc | 9 - .../painless-api-reference/IntStream.asciidoc | 47 - .../IntSummaryStatistics.asciidoc | 14 - .../IntSupplier.asciidoc | 8 - .../IntToDoubleFunction.asciidoc | 8 - .../IntToLongFunction.asciidoc | 8 - .../IntUnaryOperator.asciidoc | 11 - .../painless-api-reference/Integer.asciidoc | 44 - .../InterruptedException.asciidoc | 9 - .../IsoChronology.asciidoc | 20 - .../painless-api-reference/IsoEra.asciidoc | 13 - .../painless-api-reference/IsoFields.asciidoc | 13 - .../painless-api-reference/Iterable.asciidoc | 21 - .../painless-api-reference/Iterator.asciidoc | 11 - .../JapaneseChronology.asciidoc | 16 - .../JapaneseDate.asciidoc | 17 - .../JapaneseEra.asciidoc | 15 - .../JulianFields.asciidoc | 10 - .../LinkedHashMap.asciidoc | 9 - .../LinkedHashSet.asciidoc | 9 - .../LinkedList.asciidoc | 10 - .../painless-api-reference/List.asciidoc | 22 - .../ListIterator.asciidoc | 12 - .../painless-api-reference/LocalDate.asciidoc | 47 - .../LocalDateTime.asciidoc | 61 - .../painless-api-reference/LocalTime.asciidoc | 50 - .../Locale.Builder.asciidoc | 21 - .../Locale.Category.asciidoc | 11 - .../Locale.FilteringMode.asciidoc | 14 - .../Locale.LanguageRange.asciidoc | 16 - .../painless-api-reference/Locale.asciidoc | 69 - .../painless-api-reference/Long.asciidoc | 44 - .../LongBinaryOperator.asciidoc | 8 - .../LongConsumer.asciidoc | 9 - .../LongFunction.asciidoc | 8 - .../LongPredicate.asciidoc | 11 - .../LongStream.Builder.asciidoc | 9 - .../LongStream.asciidoc | 46 - .../LongSummaryStatistics.asciidoc | 14 - .../LongSupplier.asciidoc | 8 - .../LongToDoubleFunction.asciidoc | 8 - .../LongToIntFunction.asciidoc | 8 - .../LongUnaryOperator.asciidoc | 11 - .../painless-api-reference/Map.Entry.asciidoc | 16 - .../painless-api-reference/Map.asciidoc | 42 - .../painless-api-reference/Matcher.asciidoc | 34 - .../painless-api-reference/Math.asciidoc | 46 - .../MathContext.asciidoc | 15 - .../MessageFormat.Field.asciidoc | 8 - .../MessageFormat.asciidoc | 20 - .../MinguoChronology.asciidoc | 16 - .../MinguoDate.asciidoc | 17 - .../painless-api-reference/MinguoEra.asciidoc | 13 - .../MissingFormatArgumentException.asciidoc | 9 - .../MissingFormatWidthException.asciidoc | 9 - .../MissingResourceException.asciidoc | 10 - .../painless-api-reference/Month.asciidoc | 32 - .../painless-api-reference/MonthDay.asciidoc | 23 - .../NavigableMap.asciidoc | 24 - .../NavigableSet.asciidoc | 18 - .../NegativeArraySizeException.asciidoc | 9 - .../NoSuchElementException.asciidoc | 9 - .../NoSuchFieldException.asciidoc | 9 - .../NoSuchMethodException.asciidoc | 9 - .../Normalizer.Form.asciidoc | 13 - .../Normalizer.asciidoc | 9 - .../NullPointerException.asciidoc | 9 - .../painless-api-reference/Number.asciidoc | 13 - .../NumberFormat.Field.asciidoc | 18 - .../NumberFormat.asciidoc | 38 - .../NumberFormatException.asciidoc | 9 - .../ObjDoubleConsumer.asciidoc | 8 - .../ObjIntConsumer.asciidoc | 8 - .../ObjLongConsumer.asciidoc | 8 - .../painless-api-reference/Object.asciidoc | 9 - .../painless-api-reference/Objects.asciidoc | 18 - .../Observable.asciidoc | 15 - .../painless-api-reference/Observer.asciidoc | 8 - .../OffsetDateTime.asciidoc | 75 - .../OffsetTime.asciidoc | 47 - .../painless-api-reference/Optional.asciidoc | 19 - .../OptionalDouble.asciidoc | 15 - .../OptionalInt.asciidoc | 15 - .../OptionalLong.asciidoc | 15 - .../ParseException.asciidoc | 9 - .../ParsePosition.asciidoc | 12 - .../painless-api-reference/Pattern.asciidoc | 15 - .../painless-api-reference/Period.asciidoc | 35 - .../painless-api-reference/Predicate.asciidoc | 12 - .../PrimitiveIterator.OfDouble.asciidoc | 9 - .../PrimitiveIterator.OfInt.asciidoc | 9 - .../PrimitiveIterator.OfLong.asciidoc | 9 - .../PrimitiveIterator.asciidoc | 8 - .../PriorityQueue.asciidoc | 9 - .../painless-api-reference/Queue.asciidoc | 12 - .../painless-api-reference/Random.asciidoc | 24 - .../RandomAccess.asciidoc | 7 - .../ReflectiveOperationException.asciidoc | 9 - .../ResolverStyle.asciidoc | 12 - .../RoundingMode.asciidoc | 17 - .../RuleBasedCollator.asciidoc | 10 - .../RuntimeException.asciidoc | 9 - .../SecurityException.asciidoc | 9 - .../painless-api-reference/Set.asciidoc | 10 - .../painless-api-reference/Short.asciidoc | 23 - .../painless-api-reference/SignStyle.asciidoc | 14 - .../SimpleDateFormat.asciidoc | 18 - .../SimpleTimeZone.asciidoc | 23 - .../painless-api-reference/SortedMap.asciidoc | 13 - .../painless-api-reference/SortedSet.asciidoc | 13 - .../Spliterator.OfDouble.asciidoc | 8 - .../Spliterator.OfInt.asciidoc | 8 - .../Spliterator.OfLong.asciidoc | 8 - .../Spliterator.OfPrimitive.asciidoc | 10 - .../Spliterator.asciidoc | 23 - .../Spliterators.asciidoc | 15 - .../painless-api-reference/Stack.asciidoc | 13 - .../StackTraceElement.asciidoc | 13 - .../Stream.Builder.asciidoc | 9 - .../painless-api-reference/Stream.asciidoc | 43 - .../StrictMath.asciidoc | 46 - .../painless-api-reference/String.asciidoc | 46 - .../StringBuffer.asciidoc | 31 - .../StringBuilder.asciidoc | 31 - .../StringCharacterIterator.asciidoc | 11 - .../StringIndexOutOfBoundsException.asciidoc | 9 - .../StringJoiner.asciidoc | 13 - .../StringTokenizer.asciidoc | 14 - .../painless-api-reference/Supplier.asciidoc | 8 - .../painless-api-reference/System.asciidoc | 10 - .../painless-api-reference/Temporal.asciidoc | 14 - .../TemporalAccessor.asciidoc | 12 - .../TemporalAdjuster.asciidoc | 8 - .../TemporalAdjusters.asciidoc | 21 - .../TemporalAmount.asciidoc | 11 - .../TemporalField.asciidoc | 19 - .../TemporalQueries.asciidoc | 14 - .../TemporalQuery.asciidoc | 8 - .../TemporalUnit.asciidoc | 15 - .../painless-api-reference/TextStyle.asciidoc | 18 - .../ThaiBuddhistChronology.asciidoc | 16 - .../ThaiBuddhistDate.asciidoc | 17 - .../ThaiBuddhistEra.asciidoc | 13 - .../painless-api-reference/TimeZone.asciidoc | 29 - .../ToDoubleBiFunction.asciidoc | 8 - .../ToDoubleFunction.asciidoc | 8 - .../ToIntBiFunction.asciidoc | 8 - .../ToIntFunction.asciidoc | 8 - .../ToLongBiFunction.asciidoc | 8 - .../ToLongFunction.asciidoc | 8 - .../TooManyListenersException.asciidoc | 9 - .../painless-api-reference/TreeMap.asciidoc | 10 - .../painless-api-reference/TreeSet.asciidoc | 10 - .../TypeNotPresentException.asciidoc | 8 - .../painless-api-reference/UUID.asciidoc | 18 - .../UnaryOperator.asciidoc | 8 - .../UnknownFormatConversionException.asciidoc | 9 - .../UnknownFormatFlagsException.asciidoc | 9 - .../UnsupportedOperationException.asciidoc | 9 - .../UnsupportedTemporalTypeException.asciidoc | 8 - .../ValueRange.asciidoc | 20 - .../painless-api-reference/Vector.asciidoc | 22 - .../WeekFields.asciidoc | 19 - .../painless-api-reference/Year.asciidoc | 32 - .../painless-api-reference/YearMonth.asciidoc | 36 - .../painless-api-reference/ZoneId.asciidoc | 18 - .../ZoneOffset.asciidoc | 17 - .../ZoneOffsetTransition.asciidoc | 19 - ...fsetTransitionRule.TimeDefinition.asciidoc | 13 - .../ZoneOffsetTransitionRule.asciidoc | 18 - .../painless-api-reference/ZoneRules.asciidoc | 21 - .../ZoneRulesException.asciidoc | 8 - .../ZoneRulesProvider.asciidoc | 10 - .../ZonedDateTime.asciidoc | 66 - .../painless-api-reference/index.asciidoc | 380 +- ...elasticsearch.common.geo.GeoPoint.asciidoc | 9 - ...ielddata.ScriptDocValues.Booleans.asciidoc | 10 - ...elddata.ScriptDocValues.BytesRefs.asciidoc | 10 - ...x.fielddata.ScriptDocValues.Dates.asciidoc | 12 - ...fielddata.ScriptDocValues.Doubles.asciidoc | 10 - ...elddata.ScriptDocValues.GeoPoints.asciidoc | 20 - ...x.fielddata.ScriptDocValues.Longs.asciidoc | 12 - ...fielddata.ScriptDocValues.Strings.asciidoc | 10 - ...per.IpFieldType.IpScriptDocValues.asciidoc | 10 - ...similarity.ScriptedSimilarity.Doc.asciidoc | 9 - ...milarity.ScriptedSimilarity.Field.asciidoc | 10 - ...milarity.ScriptedSimilarity.Query.asciidoc | 8 - ...imilarity.ScriptedSimilarity.Term.asciidoc | 9 - ...lasticsearch.painless.FeatureTest.asciidoc | 19 - ...csearch.search.lookup.FieldLookup.asciidoc | 10 - .../org.joda.time.ReadableDateTime.asciidoc | 27 - .../org.joda.time.ReadableInstant.asciidoc | 14 - .../index.asciidoc | 16 + .../packages.asciidoc | 22 + .../index.asciidoc | 16 + .../packages.asciidoc | 20 + .../index.asciidoc | 16 + .../packages.asciidoc | 23 + .../index.asciidoc | 18 + .../packages.asciidoc | 169 + .../index.asciidoc | 432 + .../packages.asciidoc | 8608 +++++++++++++++++ docs/reference/redirects.asciidoc | 6 +- modules/lang-painless/build.gradle | 33 +- .../painless/ContextDocGenerator.java | 670 ++ .../PainlessContextClassBindingInfo.java | 31 + .../action/PainlessContextClassInfo.java | 41 + .../PainlessContextConstructorInfo.java | 16 + .../action/PainlessContextFieldInfo.java | 21 + .../painless/action/PainlessContextInfo.java | 31 + .../PainlessContextInstanceBindingInfo.java | 26 + .../action/PainlessContextMethodInfo.java | 26 + 362 files changed, 10277 insertions(+), 6427 deletions(-) delete mode 100644 docs/painless/painless-api-reference/AbstractChronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractCollection.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractList.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractMap.SimpleEntry.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractMap.SimpleImmutableEntry.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractQueue.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractSequentialList.asciidoc delete mode 100644 docs/painless/painless-api-reference/AbstractSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/Annotation.asciidoc delete mode 100644 docs/painless/painless-api-reference/Appendable.asciidoc delete mode 100644 docs/painless/painless-api-reference/ArithmeticException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ArrayDeque.asciidoc delete mode 100644 docs/painless/painless-api-reference/ArrayIndexOutOfBoundsException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ArrayList.asciidoc delete mode 100644 docs/painless/painless-api-reference/ArrayStoreException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Arrays.asciidoc delete mode 100644 docs/painless/painless-api-reference/AttributedCharacterIterator.Attribute.asciidoc delete mode 100644 docs/painless/painless-api-reference/AttributedCharacterIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/AttributedString.asciidoc delete mode 100644 docs/painless/painless-api-reference/Base64.Decoder.asciidoc delete mode 100644 docs/painless/painless-api-reference/Base64.Encoder.asciidoc delete mode 100644 docs/painless/painless-api-reference/Base64.asciidoc delete mode 100644 docs/painless/painless-api-reference/BaseStream.asciidoc delete mode 100644 docs/painless/painless-api-reference/BiConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/BiFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/BiPredicate.asciidoc delete mode 100644 docs/painless/painless-api-reference/Bidi.asciidoc delete mode 100644 docs/painless/painless-api-reference/BigDecimal.asciidoc delete mode 100644 docs/painless/painless-api-reference/BigInteger.asciidoc delete mode 100644 docs/painless/painless-api-reference/BinaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/BitSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/Boolean.asciidoc delete mode 100644 docs/painless/painless-api-reference/BooleanSupplier.asciidoc delete mode 100644 docs/painless/painless-api-reference/BreakIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/Byte.asciidoc delete mode 100644 docs/painless/painless-api-reference/BytesRef.asciidoc delete mode 100644 docs/painless/painless-api-reference/Calendar.Builder.asciidoc delete mode 100644 docs/painless/painless-api-reference/Calendar.asciidoc delete mode 100644 docs/painless/painless-api-reference/CharSequence.asciidoc delete mode 100644 docs/painless/painless-api-reference/Character.Subset.asciidoc delete mode 100644 docs/painless/painless-api-reference/Character.UnicodeBlock.asciidoc delete mode 100644 docs/painless/painless-api-reference/Character.UnicodeScript.asciidoc delete mode 100644 docs/painless/painless-api-reference/Character.asciidoc delete mode 100644 docs/painless/painless-api-reference/CharacterIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChoiceFormat.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChronoField.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChronoLocalDate.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChronoLocalDateTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChronoPeriod.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChronoUnit.asciidoc delete mode 100644 docs/painless/painless-api-reference/ChronoZonedDateTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/Chronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/ClassCastException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ClassNotFoundException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Clock.asciidoc delete mode 100644 docs/painless/painless-api-reference/CloneNotSupportedException.asciidoc delete mode 100644 docs/painless/painless-api-reference/CollationElementIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/CollationKey.asciidoc delete mode 100644 docs/painless/painless-api-reference/Collator.asciidoc delete mode 100644 docs/painless/painless-api-reference/Collection.asciidoc delete mode 100644 docs/painless/painless-api-reference/Collections.asciidoc delete mode 100644 docs/painless/painless-api-reference/Collector.Characteristics.asciidoc delete mode 100644 docs/painless/painless-api-reference/Collector.asciidoc delete mode 100644 docs/painless/painless-api-reference/Collectors.asciidoc delete mode 100644 docs/painless/painless-api-reference/Comparable.asciidoc delete mode 100644 docs/painless/painless-api-reference/Comparator.asciidoc delete mode 100644 docs/painless/painless-api-reference/ConcurrentModificationException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Consumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/Currency.asciidoc delete mode 100644 docs/painless/painless-api-reference/Date.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateFormat.Field.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateFormat.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateFormatSymbols.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateTimeException.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateTimeFormatter.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateTimeFormatterBuilder.asciidoc delete mode 100644 docs/painless/painless-api-reference/DateTimeParseException.asciidoc delete mode 100644 docs/painless/painless-api-reference/DayOfWeek.asciidoc delete mode 100644 docs/painless/painless-api-reference/Debug.asciidoc delete mode 100644 docs/painless/painless-api-reference/DecimalFormat.asciidoc delete mode 100644 docs/painless/painless-api-reference/DecimalFormatSymbols.asciidoc delete mode 100644 docs/painless/painless-api-reference/DecimalStyle.asciidoc delete mode 100644 docs/painless/painless-api-reference/Deque.asciidoc delete mode 100644 docs/painless/painless-api-reference/Dictionary.asciidoc delete mode 100644 docs/painless/painless-api-reference/Double.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleBinaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoublePredicate.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleStream.Builder.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleStream.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleSummaryStatistics.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleSupplier.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleToIntFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleToLongFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/DoubleUnaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/DuplicateFormatFlagsException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Duration.asciidoc delete mode 100644 docs/painless/painless-api-reference/EmptyStackException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Enum.asciidoc delete mode 100644 docs/painless/painless-api-reference/EnumConstantNotPresentException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Enumeration.asciidoc delete mode 100644 docs/painless/painless-api-reference/Era.asciidoc delete mode 100644 docs/painless/painless-api-reference/EventListener.asciidoc delete mode 100644 docs/painless/painless-api-reference/EventListenerProxy.asciidoc delete mode 100644 docs/painless/painless-api-reference/EventObject.asciidoc delete mode 100644 docs/painless/painless-api-reference/Exception.asciidoc delete mode 100644 docs/painless/painless-api-reference/FieldPosition.asciidoc delete mode 100644 docs/painless/painless-api-reference/Float.asciidoc delete mode 100644 docs/painless/painless-api-reference/Format.Field.asciidoc delete mode 100644 docs/painless/painless-api-reference/Format.asciidoc delete mode 100644 docs/painless/painless-api-reference/FormatFlagsConversionMismatchException.asciidoc delete mode 100644 docs/painless/painless-api-reference/FormatStyle.asciidoc delete mode 100644 docs/painless/painless-api-reference/Formattable.asciidoc delete mode 100644 docs/painless/painless-api-reference/FormattableFlags.asciidoc delete mode 100644 docs/painless/painless-api-reference/Formatter.BigDecimalLayoutForm.asciidoc delete mode 100644 docs/painless/painless-api-reference/Formatter.asciidoc delete mode 100644 docs/painless/painless-api-reference/FormatterClosedException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Function.asciidoc delete mode 100644 docs/painless/painless-api-reference/GregorianCalendar.asciidoc delete mode 100644 docs/painless/painless-api-reference/HashMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/HashSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/Hashtable.asciidoc delete mode 100644 docs/painless/painless-api-reference/HijrahChronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/HijrahDate.asciidoc delete mode 100644 docs/painless/painless-api-reference/HijrahEra.asciidoc delete mode 100644 docs/painless/painless-api-reference/IdentityHashMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalAccessException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalArgumentException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalFormatCodePointException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalFormatConversionException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalFormatException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalFormatFlagsException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalFormatPrecisionException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalFormatWidthException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalMonitorStateException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalStateException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllegalThreadStateException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IllformedLocaleException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IndexOutOfBoundsException.asciidoc delete mode 100644 docs/painless/painless-api-reference/InputMismatchException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Instant.asciidoc delete mode 100644 docs/painless/painless-api-reference/InstantiationException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntBinaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntPredicate.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntStream.Builder.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntStream.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntSummaryStatistics.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntSupplier.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntToDoubleFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntToLongFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/IntUnaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/Integer.asciidoc delete mode 100644 docs/painless/painless-api-reference/InterruptedException.asciidoc delete mode 100644 docs/painless/painless-api-reference/IsoChronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/IsoEra.asciidoc delete mode 100644 docs/painless/painless-api-reference/IsoFields.asciidoc delete mode 100644 docs/painless/painless-api-reference/Iterable.asciidoc delete mode 100644 docs/painless/painless-api-reference/Iterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/JapaneseChronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/JapaneseDate.asciidoc delete mode 100644 docs/painless/painless-api-reference/JapaneseEra.asciidoc delete mode 100644 docs/painless/painless-api-reference/JulianFields.asciidoc delete mode 100644 docs/painless/painless-api-reference/LinkedHashMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/LinkedHashSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/LinkedList.asciidoc delete mode 100644 docs/painless/painless-api-reference/List.asciidoc delete mode 100644 docs/painless/painless-api-reference/ListIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/LocalDate.asciidoc delete mode 100644 docs/painless/painless-api-reference/LocalDateTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/LocalTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/Locale.Builder.asciidoc delete mode 100644 docs/painless/painless-api-reference/Locale.Category.asciidoc delete mode 100644 docs/painless/painless-api-reference/Locale.FilteringMode.asciidoc delete mode 100644 docs/painless/painless-api-reference/Locale.LanguageRange.asciidoc delete mode 100644 docs/painless/painless-api-reference/Locale.asciidoc delete mode 100644 docs/painless/painless-api-reference/Long.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongBinaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongPredicate.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongStream.Builder.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongStream.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongSummaryStatistics.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongSupplier.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongToDoubleFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongToIntFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/LongUnaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/Map.Entry.asciidoc delete mode 100644 docs/painless/painless-api-reference/Map.asciidoc delete mode 100644 docs/painless/painless-api-reference/Matcher.asciidoc delete mode 100644 docs/painless/painless-api-reference/Math.asciidoc delete mode 100644 docs/painless/painless-api-reference/MathContext.asciidoc delete mode 100644 docs/painless/painless-api-reference/MessageFormat.Field.asciidoc delete mode 100644 docs/painless/painless-api-reference/MessageFormat.asciidoc delete mode 100644 docs/painless/painless-api-reference/MinguoChronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/MinguoDate.asciidoc delete mode 100644 docs/painless/painless-api-reference/MinguoEra.asciidoc delete mode 100644 docs/painless/painless-api-reference/MissingFormatArgumentException.asciidoc delete mode 100644 docs/painless/painless-api-reference/MissingFormatWidthException.asciidoc delete mode 100644 docs/painless/painless-api-reference/MissingResourceException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Month.asciidoc delete mode 100644 docs/painless/painless-api-reference/MonthDay.asciidoc delete mode 100644 docs/painless/painless-api-reference/NavigableMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/NavigableSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/NegativeArraySizeException.asciidoc delete mode 100644 docs/painless/painless-api-reference/NoSuchElementException.asciidoc delete mode 100644 docs/painless/painless-api-reference/NoSuchFieldException.asciidoc delete mode 100644 docs/painless/painless-api-reference/NoSuchMethodException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Normalizer.Form.asciidoc delete mode 100644 docs/painless/painless-api-reference/Normalizer.asciidoc delete mode 100644 docs/painless/painless-api-reference/NullPointerException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Number.asciidoc delete mode 100644 docs/painless/painless-api-reference/NumberFormat.Field.asciidoc delete mode 100644 docs/painless/painless-api-reference/NumberFormat.asciidoc delete mode 100644 docs/painless/painless-api-reference/NumberFormatException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ObjDoubleConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/ObjIntConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/ObjLongConsumer.asciidoc delete mode 100644 docs/painless/painless-api-reference/Object.asciidoc delete mode 100644 docs/painless/painless-api-reference/Objects.asciidoc delete mode 100644 docs/painless/painless-api-reference/Observable.asciidoc delete mode 100644 docs/painless/painless-api-reference/Observer.asciidoc delete mode 100644 docs/painless/painless-api-reference/OffsetDateTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/OffsetTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/Optional.asciidoc delete mode 100644 docs/painless/painless-api-reference/OptionalDouble.asciidoc delete mode 100644 docs/painless/painless-api-reference/OptionalInt.asciidoc delete mode 100644 docs/painless/painless-api-reference/OptionalLong.asciidoc delete mode 100644 docs/painless/painless-api-reference/ParseException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ParsePosition.asciidoc delete mode 100644 docs/painless/painless-api-reference/Pattern.asciidoc delete mode 100644 docs/painless/painless-api-reference/Period.asciidoc delete mode 100644 docs/painless/painless-api-reference/Predicate.asciidoc delete mode 100644 docs/painless/painless-api-reference/PrimitiveIterator.OfDouble.asciidoc delete mode 100644 docs/painless/painless-api-reference/PrimitiveIterator.OfInt.asciidoc delete mode 100644 docs/painless/painless-api-reference/PrimitiveIterator.OfLong.asciidoc delete mode 100644 docs/painless/painless-api-reference/PrimitiveIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/PriorityQueue.asciidoc delete mode 100644 docs/painless/painless-api-reference/Queue.asciidoc delete mode 100644 docs/painless/painless-api-reference/Random.asciidoc delete mode 100644 docs/painless/painless-api-reference/RandomAccess.asciidoc delete mode 100644 docs/painless/painless-api-reference/ReflectiveOperationException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ResolverStyle.asciidoc delete mode 100644 docs/painless/painless-api-reference/RoundingMode.asciidoc delete mode 100644 docs/painless/painless-api-reference/RuleBasedCollator.asciidoc delete mode 100644 docs/painless/painless-api-reference/RuntimeException.asciidoc delete mode 100644 docs/painless/painless-api-reference/SecurityException.asciidoc delete mode 100644 docs/painless/painless-api-reference/Set.asciidoc delete mode 100644 docs/painless/painless-api-reference/Short.asciidoc delete mode 100644 docs/painless/painless-api-reference/SignStyle.asciidoc delete mode 100644 docs/painless/painless-api-reference/SimpleDateFormat.asciidoc delete mode 100644 docs/painless/painless-api-reference/SimpleTimeZone.asciidoc delete mode 100644 docs/painless/painless-api-reference/SortedMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/SortedSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/Spliterator.OfDouble.asciidoc delete mode 100644 docs/painless/painless-api-reference/Spliterator.OfInt.asciidoc delete mode 100644 docs/painless/painless-api-reference/Spliterator.OfLong.asciidoc delete mode 100644 docs/painless/painless-api-reference/Spliterator.OfPrimitive.asciidoc delete mode 100644 docs/painless/painless-api-reference/Spliterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/Spliterators.asciidoc delete mode 100644 docs/painless/painless-api-reference/Stack.asciidoc delete mode 100644 docs/painless/painless-api-reference/StackTraceElement.asciidoc delete mode 100644 docs/painless/painless-api-reference/Stream.Builder.asciidoc delete mode 100644 docs/painless/painless-api-reference/Stream.asciidoc delete mode 100644 docs/painless/painless-api-reference/StrictMath.asciidoc delete mode 100644 docs/painless/painless-api-reference/String.asciidoc delete mode 100644 docs/painless/painless-api-reference/StringBuffer.asciidoc delete mode 100644 docs/painless/painless-api-reference/StringBuilder.asciidoc delete mode 100644 docs/painless/painless-api-reference/StringCharacterIterator.asciidoc delete mode 100644 docs/painless/painless-api-reference/StringIndexOutOfBoundsException.asciidoc delete mode 100644 docs/painless/painless-api-reference/StringJoiner.asciidoc delete mode 100644 docs/painless/painless-api-reference/StringTokenizer.asciidoc delete mode 100644 docs/painless/painless-api-reference/Supplier.asciidoc delete mode 100644 docs/painless/painless-api-reference/System.asciidoc delete mode 100644 docs/painless/painless-api-reference/Temporal.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalAccessor.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalAdjuster.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalAdjusters.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalAmount.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalField.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalQueries.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalQuery.asciidoc delete mode 100644 docs/painless/painless-api-reference/TemporalUnit.asciidoc delete mode 100644 docs/painless/painless-api-reference/TextStyle.asciidoc delete mode 100644 docs/painless/painless-api-reference/ThaiBuddhistChronology.asciidoc delete mode 100644 docs/painless/painless-api-reference/ThaiBuddhistDate.asciidoc delete mode 100644 docs/painless/painless-api-reference/ThaiBuddhistEra.asciidoc delete mode 100644 docs/painless/painless-api-reference/TimeZone.asciidoc delete mode 100644 docs/painless/painless-api-reference/ToDoubleBiFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/ToDoubleFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/ToIntBiFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/ToIntFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/ToLongBiFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/ToLongFunction.asciidoc delete mode 100644 docs/painless/painless-api-reference/TooManyListenersException.asciidoc delete mode 100644 docs/painless/painless-api-reference/TreeMap.asciidoc delete mode 100644 docs/painless/painless-api-reference/TreeSet.asciidoc delete mode 100644 docs/painless/painless-api-reference/TypeNotPresentException.asciidoc delete mode 100644 docs/painless/painless-api-reference/UUID.asciidoc delete mode 100644 docs/painless/painless-api-reference/UnaryOperator.asciidoc delete mode 100644 docs/painless/painless-api-reference/UnknownFormatConversionException.asciidoc delete mode 100644 docs/painless/painless-api-reference/UnknownFormatFlagsException.asciidoc delete mode 100644 docs/painless/painless-api-reference/UnsupportedOperationException.asciidoc delete mode 100644 docs/painless/painless-api-reference/UnsupportedTemporalTypeException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ValueRange.asciidoc delete mode 100644 docs/painless/painless-api-reference/Vector.asciidoc delete mode 100644 docs/painless/painless-api-reference/WeekFields.asciidoc delete mode 100644 docs/painless/painless-api-reference/Year.asciidoc delete mode 100644 docs/painless/painless-api-reference/YearMonth.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneId.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneOffset.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneOffsetTransition.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneOffsetTransitionRule.TimeDefinition.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneOffsetTransitionRule.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneRules.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneRulesException.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZoneRulesProvider.asciidoc delete mode 100644 docs/painless/painless-api-reference/ZonedDateTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.common.geo.GeoPoint.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Booleans.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.BytesRefs.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Dates.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Doubles.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Longs.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Strings.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.mapper.IpFieldMapper.IpFieldType.IpScriptDocValues.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Doc.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Field.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Query.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Term.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.painless.FeatureTest.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.elasticsearch.search.lookup.FieldLookup.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.joda.time.ReadableDateTime.asciidoc delete mode 100644 docs/painless/painless-api-reference/org.joda.time.ReadableInstant.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-analysis/index.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-analysis/packages.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-ingest/index.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-ingest/packages.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-moving-function/index.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-moving-function/packages.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-score/index.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-shared/index.asciidoc create mode 100644 docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc create mode 100644 modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java diff --git a/docs/Versions.asciidoc b/docs/Versions.asciidoc index 07d735b66547a..f728d76751dc3 100644 --- a/docs/Versions.asciidoc +++ b/docs/Versions.asciidoc @@ -30,8 +30,7 @@ release-state can be: released | prerelease | unreleased /////// Javadoc roots used to generate links from Painless's API reference /////// -:java8-javadoc: https://docs.oracle.com/javase/8/docs/api -:java9-javadoc: https://docs.oracle.com/javase/9/docs/api +:java11-javadoc: https://docs.oracle.com/en/java/javase/11/docs/api :joda-time-javadoc: http://www.joda.org/joda-time/apidocs :lucene-core-javadoc: http://lucene.apache.org/core/{lucene_version_path}/core diff --git a/docs/painless/painless-api-reference.asciidoc b/docs/painless/painless-api-reference.asciidoc index 814824b0db9a2..2e9db174f2c06 100644 --- a/docs/painless/painless-api-reference.asciidoc +++ b/docs/painless/painless-api-reference.asciidoc @@ -1,13 +1,11 @@ [[painless-api-reference]] == Painless API Reference -Painless has a strict whitelist for methods and classes to ensure all -painless scripts are secure. Most of these methods are exposed directly -from the Java Runtime Environment (JRE) while others are part of -Elasticsearch or Painless itself. Below is a list of all available -classes grouped with their respected methods. Clicking on the method -name takes you to the documentation for that specific method. Methods -defined in the JRE also have a `(java 9)` link which can be used to see -the method's documentation in Java 9. +Painless has a strict whitelist for methods and classes per context to +ensure all painless scripts are secure. Most of these methods are +exposed directly from the Java Runtime Environment (JRE) while others +are part of Elasticsearch or Painless itself. Below is a list of the available +APIs per context. The shared API is available to all contexts, while the +specialized API available differs between contexts. include::painless-api-reference/index.asciidoc[] diff --git a/docs/painless/painless-api-reference/AbstractChronology.asciidoc b/docs/painless/painless-api-reference/AbstractChronology.asciidoc deleted file mode 100644 index 3cc90ee65cde2..0000000000000 --- a/docs/painless/painless-api-reference/AbstractChronology.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractChronology]]++AbstractChronology++:: -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractCollection.asciidoc b/docs/painless/painless-api-reference/AbstractCollection.asciidoc deleted file mode 100644 index 313c327ac07cc..0000000000000 --- a/docs/painless/painless-api-reference/AbstractCollection.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractCollection]]++AbstractCollection++:: -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractList.asciidoc b/docs/painless/painless-api-reference/AbstractList.asciidoc deleted file mode 100644 index 9ce7c9a683001..0000000000000 --- a/docs/painless/painless-api-reference/AbstractList.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractList]]++AbstractList++:: -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractMap.SimpleEntry.asciidoc b/docs/painless/painless-api-reference/AbstractMap.SimpleEntry.asciidoc deleted file mode 100644 index bdd784070f4fb..0000000000000 --- a/docs/painless/painless-api-reference/AbstractMap.SimpleEntry.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractMap-SimpleEntry]]++AbstractMap.SimpleEntry++:: -* ++[[painless-api-reference-AbstractMap-SimpleEntry-AbstractMap.SimpleEntry-1]]link:{java8-javadoc}/java/util/AbstractMap.SimpleEntry.html#AbstractMap.SimpleEntry%2Djava.util.Map$Entry%2D[AbstractMap.SimpleEntry](<>)++ (link:{java9-javadoc}/java/util/AbstractMap.SimpleEntry.html#AbstractMap.SimpleEntry%2Djava.util.Map$Entry%2D[java 9]) -* ++[[painless-api-reference-AbstractMap-SimpleEntry-AbstractMap.SimpleEntry-2]]link:{java8-javadoc}/java/util/AbstractMap.SimpleEntry.html#AbstractMap.SimpleEntry%2Djava.lang.Object%2Djava.lang.Object%2D[AbstractMap.SimpleEntry](def, def)++ (link:{java9-javadoc}/java/util/AbstractMap.SimpleEntry.html#AbstractMap.SimpleEntry%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractMap.SimpleImmutableEntry.asciidoc b/docs/painless/painless-api-reference/AbstractMap.SimpleImmutableEntry.asciidoc deleted file mode 100644 index 5b5605a42ae81..0000000000000 --- a/docs/painless/painless-api-reference/AbstractMap.SimpleImmutableEntry.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractMap-SimpleImmutableEntry]]++AbstractMap.SimpleImmutableEntry++:: -* ++[[painless-api-reference-AbstractMap-SimpleImmutableEntry-AbstractMap.SimpleImmutableEntry-1]]link:{java8-javadoc}/java/util/AbstractMap.SimpleImmutableEntry.html#AbstractMap.SimpleImmutableEntry%2Djava.util.Map$Entry%2D[AbstractMap.SimpleImmutableEntry](<>)++ (link:{java9-javadoc}/java/util/AbstractMap.SimpleImmutableEntry.html#AbstractMap.SimpleImmutableEntry%2Djava.util.Map$Entry%2D[java 9]) -* ++[[painless-api-reference-AbstractMap-SimpleImmutableEntry-AbstractMap.SimpleImmutableEntry-2]]link:{java8-javadoc}/java/util/AbstractMap.SimpleImmutableEntry.html#AbstractMap.SimpleImmutableEntry%2Djava.lang.Object%2Djava.lang.Object%2D[AbstractMap.SimpleImmutableEntry](def, def)++ (link:{java9-javadoc}/java/util/AbstractMap.SimpleImmutableEntry.html#AbstractMap.SimpleImmutableEntry%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractMap.asciidoc b/docs/painless/painless-api-reference/AbstractMap.asciidoc deleted file mode 100644 index ea3be0d125a98..0000000000000 --- a/docs/painless/painless-api-reference/AbstractMap.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractMap]]++AbstractMap++:: -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractQueue.asciidoc b/docs/painless/painless-api-reference/AbstractQueue.asciidoc deleted file mode 100644 index 37fc53bd73d65..0000000000000 --- a/docs/painless/painless-api-reference/AbstractQueue.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractQueue]]++AbstractQueue++:: -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractSequentialList.asciidoc b/docs/painless/painless-api-reference/AbstractSequentialList.asciidoc deleted file mode 100644 index 822b203851648..0000000000000 --- a/docs/painless/painless-api-reference/AbstractSequentialList.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractSequentialList]]++AbstractSequentialList++:: -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AbstractSet.asciidoc b/docs/painless/painless-api-reference/AbstractSet.asciidoc deleted file mode 100644 index 7baa3a58f7995..0000000000000 --- a/docs/painless/painless-api-reference/AbstractSet.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AbstractSet]]++AbstractSet++:: -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Annotation.asciidoc b/docs/painless/painless-api-reference/Annotation.asciidoc deleted file mode 100644 index 33252543951b8..0000000000000 --- a/docs/painless/painless-api-reference/Annotation.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Annotation]]++Annotation++:: -* ++[[painless-api-reference-Annotation-Annotation-1]]link:{java8-javadoc}/java/text/Annotation.html#Annotation%2Djava.lang.Object%2D[Annotation](<>)++ (link:{java9-javadoc}/java/text/Annotation.html#Annotation%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Annotation-getValue-0]]def link:{java8-javadoc}/java/text/Annotation.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/text/Annotation.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Appendable.asciidoc b/docs/painless/painless-api-reference/Appendable.asciidoc deleted file mode 100644 index 61fcbaf5a6052..0000000000000 --- a/docs/painless/painless-api-reference/Appendable.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Appendable]]++Appendable++:: -* ++[[painless-api-reference-Appendable-append-3]]<> link:{java8-javadoc}/java/lang/Appendable.html#append%2Djava.lang.CharSequence%2Dint%2Dint%2D[append](<>, int, int)++ (link:{java9-javadoc}/java/lang/Appendable.html#append%2Djava.lang.CharSequence%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ArithmeticException.asciidoc b/docs/painless/painless-api-reference/ArithmeticException.asciidoc deleted file mode 100644 index f486289320570..0000000000000 --- a/docs/painless/painless-api-reference/ArithmeticException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ArithmeticException]]++ArithmeticException++:: -* ++[[painless-api-reference-ArithmeticException-ArithmeticException-0]]link:{java8-javadoc}/java/lang/ArithmeticException.html#ArithmeticException%2D%2D[ArithmeticException]()++ (link:{java9-javadoc}/java/lang/ArithmeticException.html#ArithmeticException%2D%2D[java 9]) -* ++[[painless-api-reference-ArithmeticException-ArithmeticException-1]]link:{java8-javadoc}/java/lang/ArithmeticException.html#ArithmeticException%2Djava.lang.String%2D[ArithmeticException](<>)++ (link:{java9-javadoc}/java/lang/ArithmeticException.html#ArithmeticException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ArrayDeque.asciidoc b/docs/painless/painless-api-reference/ArrayDeque.asciidoc deleted file mode 100644 index 668a86185e690..0000000000000 --- a/docs/painless/painless-api-reference/ArrayDeque.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ArrayDeque]]++ArrayDeque++:: -* ++[[painless-api-reference-ArrayDeque-ArrayDeque-0]]link:{java8-javadoc}/java/util/ArrayDeque.html#ArrayDeque%2D%2D[ArrayDeque]()++ (link:{java9-javadoc}/java/util/ArrayDeque.html#ArrayDeque%2D%2D[java 9]) -* ++[[painless-api-reference-ArrayDeque-ArrayDeque-1]]link:{java8-javadoc}/java/util/ArrayDeque.html#ArrayDeque%2Djava.util.Collection%2D[ArrayDeque](<>)++ (link:{java9-javadoc}/java/util/ArrayDeque.html#ArrayDeque%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-ArrayDeque-clone-0]]<> link:{java8-javadoc}/java/util/ArrayDeque.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/ArrayDeque.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ArrayIndexOutOfBoundsException.asciidoc b/docs/painless/painless-api-reference/ArrayIndexOutOfBoundsException.asciidoc deleted file mode 100644 index 49b13361163f4..0000000000000 --- a/docs/painless/painless-api-reference/ArrayIndexOutOfBoundsException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ArrayIndexOutOfBoundsException]]++ArrayIndexOutOfBoundsException++:: -* ++[[painless-api-reference-ArrayIndexOutOfBoundsException-ArrayIndexOutOfBoundsException-0]]link:{java8-javadoc}/java/lang/ArrayIndexOutOfBoundsException.html#ArrayIndexOutOfBoundsException%2D%2D[ArrayIndexOutOfBoundsException]()++ (link:{java9-javadoc}/java/lang/ArrayIndexOutOfBoundsException.html#ArrayIndexOutOfBoundsException%2D%2D[java 9]) -* ++[[painless-api-reference-ArrayIndexOutOfBoundsException-ArrayIndexOutOfBoundsException-1]]link:{java8-javadoc}/java/lang/ArrayIndexOutOfBoundsException.html#ArrayIndexOutOfBoundsException%2Djava.lang.String%2D[ArrayIndexOutOfBoundsException](<>)++ (link:{java9-javadoc}/java/lang/ArrayIndexOutOfBoundsException.html#ArrayIndexOutOfBoundsException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ArrayList.asciidoc b/docs/painless/painless-api-reference/ArrayList.asciidoc deleted file mode 100644 index 90386a3742cf7..0000000000000 --- a/docs/painless/painless-api-reference/ArrayList.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ArrayList]]++ArrayList++:: -* ++[[painless-api-reference-ArrayList-ArrayList-0]]link:{java8-javadoc}/java/util/ArrayList.html#ArrayList%2D%2D[ArrayList]()++ (link:{java9-javadoc}/java/util/ArrayList.html#ArrayList%2D%2D[java 9]) -* ++[[painless-api-reference-ArrayList-ArrayList-1]]link:{java8-javadoc}/java/util/ArrayList.html#ArrayList%2Djava.util.Collection%2D[ArrayList](<>)++ (link:{java9-javadoc}/java/util/ArrayList.html#ArrayList%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-ArrayList-clone-0]]def link:{java8-javadoc}/java/util/ArrayList.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/ArrayList.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-ArrayList-trimToSize-0]]void link:{java8-javadoc}/java/util/ArrayList.html#trimToSize%2D%2D[trimToSize]()++ (link:{java9-javadoc}/java/util/ArrayList.html#trimToSize%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ArrayStoreException.asciidoc b/docs/painless/painless-api-reference/ArrayStoreException.asciidoc deleted file mode 100644 index 29f7a536eadf9..0000000000000 --- a/docs/painless/painless-api-reference/ArrayStoreException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ArrayStoreException]]++ArrayStoreException++:: -* ++[[painless-api-reference-ArrayStoreException-ArrayStoreException-0]]link:{java8-javadoc}/java/lang/ArrayStoreException.html#ArrayStoreException%2D%2D[ArrayStoreException]()++ (link:{java9-javadoc}/java/lang/ArrayStoreException.html#ArrayStoreException%2D%2D[java 9]) -* ++[[painless-api-reference-ArrayStoreException-ArrayStoreException-1]]link:{java8-javadoc}/java/lang/ArrayStoreException.html#ArrayStoreException%2Djava.lang.String%2D[ArrayStoreException](<>)++ (link:{java9-javadoc}/java/lang/ArrayStoreException.html#ArrayStoreException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Arrays.asciidoc b/docs/painless/painless-api-reference/Arrays.asciidoc deleted file mode 100644 index 97598d64d5790..0000000000000 --- a/docs/painless/painless-api-reference/Arrays.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Arrays]]++Arrays++:: -* ++[[painless-api-reference-Arrays-asList-1]]static <> link:{java8-javadoc}/java/util/Arrays.html#asList%2Djava.lang.Object:A%2D[asList](<>[])++ (link:{java9-javadoc}/java/util/Arrays.html#asList%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Arrays-deepEquals-2]]static boolean link:{java8-javadoc}/java/util/Arrays.html#deepEquals%2Djava.lang.Object:A%2Djava.lang.Object:A%2D[deepEquals](<>[], <>[])++ (link:{java9-javadoc}/java/util/Arrays.html#deepEquals%2Djava.lang.Object:A%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Arrays-deepHashCode-1]]static int link:{java8-javadoc}/java/util/Arrays.html#deepHashCode%2Djava.lang.Object:A%2D[deepHashCode](<>[])++ (link:{java9-javadoc}/java/util/Arrays.html#deepHashCode%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Arrays-deepToString-1]]static <> link:{java8-javadoc}/java/util/Arrays.html#deepToString%2Djava.lang.Object:A%2D[deepToString](<>[])++ (link:{java9-javadoc}/java/util/Arrays.html#deepToString%2Djava.lang.Object:A%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/AttributedCharacterIterator.Attribute.asciidoc b/docs/painless/painless-api-reference/AttributedCharacterIterator.Attribute.asciidoc deleted file mode 100644 index dbcdd9446f3f9..0000000000000 --- a/docs/painless/painless-api-reference/AttributedCharacterIterator.Attribute.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AttributedCharacterIterator-Attribute]]++AttributedCharacterIterator.Attribute++:: -** [[painless-api-reference-AttributedCharacterIterator-Attribute-INPUT_METHOD_SEGMENT]]static <> link:{java8-javadoc}/java/text/AttributedCharacterIterator.Attribute.html#INPUT_METHOD_SEGMENT[INPUT_METHOD_SEGMENT] (link:{java9-javadoc}/java/text/AttributedCharacterIterator.Attribute.html#INPUT_METHOD_SEGMENT[java 9]) -** [[painless-api-reference-AttributedCharacterIterator-Attribute-LANGUAGE]]static <> link:{java8-javadoc}/java/text/AttributedCharacterIterator.Attribute.html#LANGUAGE[LANGUAGE] (link:{java9-javadoc}/java/text/AttributedCharacterIterator.Attribute.html#LANGUAGE[java 9]) -** [[painless-api-reference-AttributedCharacterIterator-Attribute-READING]]static <> link:{java8-javadoc}/java/text/AttributedCharacterIterator.Attribute.html#READING[READING] (link:{java9-javadoc}/java/text/AttributedCharacterIterator.Attribute.html#READING[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/AttributedCharacterIterator.asciidoc b/docs/painless/painless-api-reference/AttributedCharacterIterator.asciidoc deleted file mode 100644 index a71d3906fa167..0000000000000 --- a/docs/painless/painless-api-reference/AttributedCharacterIterator.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AttributedCharacterIterator]]++AttributedCharacterIterator++:: -* ++[[painless-api-reference-AttributedCharacterIterator-getAllAttributeKeys-0]]<> link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getAllAttributeKeys%2D%2D[getAllAttributeKeys]()++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getAllAttributeKeys%2D%2D[java 9]) -* ++[[painless-api-reference-AttributedCharacterIterator-getAttribute-1]]def link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getAttribute%2Djava.text.AttributedCharacterIterator$Attribute%2D[getAttribute](<>)++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getAttribute%2Djava.text.AttributedCharacterIterator$Attribute%2D[java 9]) -* ++[[painless-api-reference-AttributedCharacterIterator-getAttributes-0]]<> link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getAttributes%2D%2D[getAttributes]()++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getAttributes%2D%2D[java 9]) -* ++[[painless-api-reference-AttributedCharacterIterator-getRunLimit-0]]int link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getRunLimit%2D%2D[getRunLimit]()++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getRunLimit%2D%2D[java 9]) -* ++[[painless-api-reference-AttributedCharacterIterator-getRunLimit-1]]int link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getRunLimit%2Djava.util.Set%2D[getRunLimit](<>)++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getRunLimit%2Djava.util.Set%2D[java 9]) -* ++[[painless-api-reference-AttributedCharacterIterator-getRunStart-0]]int link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getRunStart%2D%2D[getRunStart]()++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getRunStart%2D%2D[java 9]) -* ++[[painless-api-reference-AttributedCharacterIterator-getRunStart-1]]int link:{java8-javadoc}/java/text/AttributedCharacterIterator.html#getRunStart%2Djava.util.Set%2D[getRunStart](<>)++ (link:{java9-javadoc}/java/text/AttributedCharacterIterator.html#getRunStart%2Djava.util.Set%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/AttributedString.asciidoc b/docs/painless/painless-api-reference/AttributedString.asciidoc deleted file mode 100644 index 0393838a73717..0000000000000 --- a/docs/painless/painless-api-reference/AttributedString.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-AttributedString]]++AttributedString++:: -* ++[[painless-api-reference-AttributedString-AttributedString-1]]link:{java8-javadoc}/java/text/AttributedString.html#AttributedString%2Djava.lang.String%2D[AttributedString](<>)++ (link:{java9-javadoc}/java/text/AttributedString.html#AttributedString%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-AttributedString-AttributedString-2]]link:{java8-javadoc}/java/text/AttributedString.html#AttributedString%2Djava.lang.String%2Djava.util.Map%2D[AttributedString](<>, <>)++ (link:{java9-javadoc}/java/text/AttributedString.html#AttributedString%2Djava.lang.String%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-AttributedString-addAttribute-2]]void link:{java8-javadoc}/java/text/AttributedString.html#addAttribute%2Djava.text.AttributedCharacterIterator$Attribute%2Djava.lang.Object%2D[addAttribute](<>, <>)++ (link:{java9-javadoc}/java/text/AttributedString.html#addAttribute%2Djava.text.AttributedCharacterIterator$Attribute%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-AttributedString-addAttribute-4]]void link:{java8-javadoc}/java/text/AttributedString.html#addAttribute%2Djava.text.AttributedCharacterIterator$Attribute%2Djava.lang.Object%2Dint%2Dint%2D[addAttribute](<>, <>, int, int)++ (link:{java9-javadoc}/java/text/AttributedString.html#addAttribute%2Djava.text.AttributedCharacterIterator$Attribute%2Djava.lang.Object%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-AttributedString-addAttributes-3]]void link:{java8-javadoc}/java/text/AttributedString.html#addAttributes%2Djava.util.Map%2Dint%2Dint%2D[addAttributes](<>, int, int)++ (link:{java9-javadoc}/java/text/AttributedString.html#addAttributes%2Djava.util.Map%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-AttributedString-getIterator-0]]<> link:{java8-javadoc}/java/text/AttributedString.html#getIterator%2D%2D[getIterator]()++ (link:{java9-javadoc}/java/text/AttributedString.html#getIterator%2D%2D[java 9]) -* ++[[painless-api-reference-AttributedString-getIterator-1]]<> link:{java8-javadoc}/java/text/AttributedString.html#getIterator%2Djava.text.AttributedCharacterIterator$Attribute:A%2D[getIterator](<>[])++ (link:{java9-javadoc}/java/text/AttributedString.html#getIterator%2Djava.text.AttributedCharacterIterator$Attribute:A%2D[java 9]) -* ++[[painless-api-reference-AttributedString-getIterator-3]]<> link:{java8-javadoc}/java/text/AttributedString.html#getIterator%2Djava.text.AttributedCharacterIterator$Attribute:A%2Dint%2Dint%2D[getIterator](<>[], int, int)++ (link:{java9-javadoc}/java/text/AttributedString.html#getIterator%2Djava.text.AttributedCharacterIterator$Attribute:A%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Base64.Decoder.asciidoc b/docs/painless/painless-api-reference/Base64.Decoder.asciidoc deleted file mode 100644 index 3324f1a230308..0000000000000 --- a/docs/painless/painless-api-reference/Base64.Decoder.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Base64-Decoder]]++Base64.Decoder++:: -* ++[[painless-api-reference-Base64-Decoder-decode-1]]byte[] link:{java8-javadoc}/java/util/Base64.Decoder.html#decode%2Djava.lang.String%2D[decode](<>)++ (link:{java9-javadoc}/java/util/Base64.Decoder.html#decode%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Base64-Decoder-decode-2]]int link:{java8-javadoc}/java/util/Base64.Decoder.html#decode%2Dbyte:A%2Dbyte:A%2D[decode](byte[], byte[])++ (link:{java9-javadoc}/java/util/Base64.Decoder.html#decode%2Dbyte:A%2Dbyte:A%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Base64.Encoder.asciidoc b/docs/painless/painless-api-reference/Base64.Encoder.asciidoc deleted file mode 100644 index 6cdcd0536e5be..0000000000000 --- a/docs/painless/painless-api-reference/Base64.Encoder.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Base64-Encoder]]++Base64.Encoder++:: -* ++[[painless-api-reference-Base64-Encoder-encode-2]]int link:{java8-javadoc}/java/util/Base64.Encoder.html#encode%2Dbyte:A%2Dbyte:A%2D[encode](byte[], byte[])++ (link:{java9-javadoc}/java/util/Base64.Encoder.html#encode%2Dbyte:A%2Dbyte:A%2D[java 9]) -* ++[[painless-api-reference-Base64-Encoder-encodeToString-1]]<> link:{java8-javadoc}/java/util/Base64.Encoder.html#encodeToString%2Dbyte:A%2D[encodeToString](byte[])++ (link:{java9-javadoc}/java/util/Base64.Encoder.html#encodeToString%2Dbyte:A%2D[java 9]) -* ++[[painless-api-reference-Base64-Encoder-withoutPadding-0]]<> link:{java8-javadoc}/java/util/Base64.Encoder.html#withoutPadding%2D%2D[withoutPadding]()++ (link:{java9-javadoc}/java/util/Base64.Encoder.html#withoutPadding%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Base64.asciidoc b/docs/painless/painless-api-reference/Base64.asciidoc deleted file mode 100644 index 795f538d7d479..0000000000000 --- a/docs/painless/painless-api-reference/Base64.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Base64]]++Base64++:: -* ++[[painless-api-reference-Base64-getDecoder-0]]static <> link:{java8-javadoc}/java/util/Base64.html#getDecoder%2D%2D[getDecoder]()++ (link:{java9-javadoc}/java/util/Base64.html#getDecoder%2D%2D[java 9]) -* ++[[painless-api-reference-Base64-getEncoder-0]]static <> link:{java8-javadoc}/java/util/Base64.html#getEncoder%2D%2D[getEncoder]()++ (link:{java9-javadoc}/java/util/Base64.html#getEncoder%2D%2D[java 9]) -* ++[[painless-api-reference-Base64-getMimeDecoder-0]]static <> link:{java8-javadoc}/java/util/Base64.html#getMimeDecoder%2D%2D[getMimeDecoder]()++ (link:{java9-javadoc}/java/util/Base64.html#getMimeDecoder%2D%2D[java 9]) -* ++[[painless-api-reference-Base64-getMimeEncoder-0]]static <> link:{java8-javadoc}/java/util/Base64.html#getMimeEncoder%2D%2D[getMimeEncoder]()++ (link:{java9-javadoc}/java/util/Base64.html#getMimeEncoder%2D%2D[java 9]) -* ++[[painless-api-reference-Base64-getMimeEncoder-2]]static <> link:{java8-javadoc}/java/util/Base64.html#getMimeEncoder%2Dint%2Dbyte:A%2D[getMimeEncoder](int, byte[])++ (link:{java9-javadoc}/java/util/Base64.html#getMimeEncoder%2Dint%2Dbyte:A%2D[java 9]) -* ++[[painless-api-reference-Base64-getUrlDecoder-0]]static <> link:{java8-javadoc}/java/util/Base64.html#getUrlDecoder%2D%2D[getUrlDecoder]()++ (link:{java9-javadoc}/java/util/Base64.html#getUrlDecoder%2D%2D[java 9]) -* ++[[painless-api-reference-Base64-getUrlEncoder-0]]static <> link:{java8-javadoc}/java/util/Base64.html#getUrlEncoder%2D%2D[getUrlEncoder]()++ (link:{java9-javadoc}/java/util/Base64.html#getUrlEncoder%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BaseStream.asciidoc b/docs/painless/painless-api-reference/BaseStream.asciidoc deleted file mode 100644 index 680e9fff71d79..0000000000000 --- a/docs/painless/painless-api-reference/BaseStream.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BaseStream]]++BaseStream++:: -* ++[[painless-api-reference-BaseStream-close-0]]void link:{java8-javadoc}/java/util/stream/BaseStream.html#close%2D%2D[close]()++ (link:{java9-javadoc}/java/util/stream/BaseStream.html#close%2D%2D[java 9]) -* ++[[painless-api-reference-BaseStream-isParallel-0]]boolean link:{java8-javadoc}/java/util/stream/BaseStream.html#isParallel%2D%2D[isParallel]()++ (link:{java9-javadoc}/java/util/stream/BaseStream.html#isParallel%2D%2D[java 9]) -* ++[[painless-api-reference-BaseStream-iterator-0]]<> link:{java8-javadoc}/java/util/stream/BaseStream.html#iterator%2D%2D[iterator]()++ (link:{java9-javadoc}/java/util/stream/BaseStream.html#iterator%2D%2D[java 9]) -* ++[[painless-api-reference-BaseStream-sequential-0]]<> link:{java8-javadoc}/java/util/stream/BaseStream.html#sequential%2D%2D[sequential]()++ (link:{java9-javadoc}/java/util/stream/BaseStream.html#sequential%2D%2D[java 9]) -* ++[[painless-api-reference-BaseStream-spliterator-0]]<> link:{java8-javadoc}/java/util/stream/BaseStream.html#spliterator%2D%2D[spliterator]()++ (link:{java9-javadoc}/java/util/stream/BaseStream.html#spliterator%2D%2D[java 9]) -* ++[[painless-api-reference-BaseStream-unordered-0]]<> link:{java8-javadoc}/java/util/stream/BaseStream.html#unordered%2D%2D[unordered]()++ (link:{java9-javadoc}/java/util/stream/BaseStream.html#unordered%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BiConsumer.asciidoc b/docs/painless/painless-api-reference/BiConsumer.asciidoc deleted file mode 100644 index a13bbdbd300e2..0000000000000 --- a/docs/painless/painless-api-reference/BiConsumer.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BiConsumer]]++BiConsumer++:: -* ++[[painless-api-reference-BiConsumer-accept-2]]void link:{java8-javadoc}/java/util/function/BiConsumer.html#accept%2Djava.lang.Object%2Djava.lang.Object%2D[accept](def, def)++ (link:{java9-javadoc}/java/util/function/BiConsumer.html#accept%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-BiConsumer-andThen-1]]<> link:{java8-javadoc}/java/util/function/BiConsumer.html#andThen%2Djava.util.function.BiConsumer%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/BiConsumer.html#andThen%2Djava.util.function.BiConsumer%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BiFunction.asciidoc b/docs/painless/painless-api-reference/BiFunction.asciidoc deleted file mode 100644 index 88b316300321e..0000000000000 --- a/docs/painless/painless-api-reference/BiFunction.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BiFunction]]++BiFunction++:: -* ++[[painless-api-reference-BiFunction-andThen-1]]<> link:{java8-javadoc}/java/util/function/BiFunction.html#andThen%2Djava.util.function.Function%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/BiFunction.html#andThen%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-BiFunction-apply-2]]def link:{java8-javadoc}/java/util/function/BiFunction.html#apply%2Djava.lang.Object%2Djava.lang.Object%2D[apply](def, def)++ (link:{java9-javadoc}/java/util/function/BiFunction.html#apply%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BiPredicate.asciidoc b/docs/painless/painless-api-reference/BiPredicate.asciidoc deleted file mode 100644 index 7961800814cc6..0000000000000 --- a/docs/painless/painless-api-reference/BiPredicate.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BiPredicate]]++BiPredicate++:: -* ++[[painless-api-reference-BiPredicate-and-1]]<> link:{java8-javadoc}/java/util/function/BiPredicate.html#and%2Djava.util.function.BiPredicate%2D[and](<>)++ (link:{java9-javadoc}/java/util/function/BiPredicate.html#and%2Djava.util.function.BiPredicate%2D[java 9]) -* ++[[painless-api-reference-BiPredicate-negate-0]]<> link:{java8-javadoc}/java/util/function/BiPredicate.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/util/function/BiPredicate.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-BiPredicate-or-1]]<> link:{java8-javadoc}/java/util/function/BiPredicate.html#or%2Djava.util.function.BiPredicate%2D[or](<>)++ (link:{java9-javadoc}/java/util/function/BiPredicate.html#or%2Djava.util.function.BiPredicate%2D[java 9]) -* ++[[painless-api-reference-BiPredicate-test-2]]boolean link:{java8-javadoc}/java/util/function/BiPredicate.html#test%2Djava.lang.Object%2Djava.lang.Object%2D[test](def, def)++ (link:{java9-javadoc}/java/util/function/BiPredicate.html#test%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Bidi.asciidoc b/docs/painless/painless-api-reference/Bidi.asciidoc deleted file mode 100644 index cf6272dd49cf5..0000000000000 --- a/docs/painless/painless-api-reference/Bidi.asciidoc +++ /dev/null @@ -1,28 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Bidi]]++Bidi++:: -** [[painless-api-reference-Bidi-DIRECTION_DEFAULT_LEFT_TO_RIGHT]]static int link:{java8-javadoc}/java/text/Bidi.html#DIRECTION_DEFAULT_LEFT_TO_RIGHT[DIRECTION_DEFAULT_LEFT_TO_RIGHT] (link:{java9-javadoc}/java/text/Bidi.html#DIRECTION_DEFAULT_LEFT_TO_RIGHT[java 9]) -** [[painless-api-reference-Bidi-DIRECTION_DEFAULT_RIGHT_TO_LEFT]]static int link:{java8-javadoc}/java/text/Bidi.html#DIRECTION_DEFAULT_RIGHT_TO_LEFT[DIRECTION_DEFAULT_RIGHT_TO_LEFT] (link:{java9-javadoc}/java/text/Bidi.html#DIRECTION_DEFAULT_RIGHT_TO_LEFT[java 9]) -** [[painless-api-reference-Bidi-DIRECTION_LEFT_TO_RIGHT]]static int link:{java8-javadoc}/java/text/Bidi.html#DIRECTION_LEFT_TO_RIGHT[DIRECTION_LEFT_TO_RIGHT] (link:{java9-javadoc}/java/text/Bidi.html#DIRECTION_LEFT_TO_RIGHT[java 9]) -** [[painless-api-reference-Bidi-DIRECTION_RIGHT_TO_LEFT]]static int link:{java8-javadoc}/java/text/Bidi.html#DIRECTION_RIGHT_TO_LEFT[DIRECTION_RIGHT_TO_LEFT] (link:{java9-javadoc}/java/text/Bidi.html#DIRECTION_RIGHT_TO_LEFT[java 9]) -* ++[[painless-api-reference-Bidi-reorderVisually-5]]static void link:{java8-javadoc}/java/text/Bidi.html#reorderVisually%2Dbyte:A%2Dint%2Djava.lang.Object:A%2Dint%2Dint%2D[reorderVisually](byte[], int, <>[], int, int)++ (link:{java9-javadoc}/java/text/Bidi.html#reorderVisually%2Dbyte:A%2Dint%2Djava.lang.Object:A%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-requiresBidi-3]]static boolean link:{java8-javadoc}/java/text/Bidi.html#requiresBidi%2Dchar:A%2Dint%2Dint%2D[requiresBidi](char[], int, int)++ (link:{java9-javadoc}/java/text/Bidi.html#requiresBidi%2Dchar:A%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-Bidi-1]]link:{java8-javadoc}/java/text/Bidi.html#Bidi%2Djava.text.AttributedCharacterIterator%2D[Bidi](<>)++ (link:{java9-javadoc}/java/text/Bidi.html#Bidi%2Djava.text.AttributedCharacterIterator%2D[java 9]) -* ++[[painless-api-reference-Bidi-Bidi-2]]link:{java8-javadoc}/java/text/Bidi.html#Bidi%2Djava.lang.String%2Dint%2D[Bidi](<>, int)++ (link:{java9-javadoc}/java/text/Bidi.html#Bidi%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-Bidi-6]]link:{java8-javadoc}/java/text/Bidi.html#Bidi%2Dchar:A%2Dint%2Dbyte:A%2Dint%2Dint%2Dint%2D[Bidi](char[], int, byte[], int, int, int)++ (link:{java9-javadoc}/java/text/Bidi.html#Bidi%2Dchar:A%2Dint%2Dbyte:A%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-baseIsLeftToRight-0]]boolean link:{java8-javadoc}/java/text/Bidi.html#baseIsLeftToRight%2D%2D[baseIsLeftToRight]()++ (link:{java9-javadoc}/java/text/Bidi.html#baseIsLeftToRight%2D%2D[java 9]) -* ++[[painless-api-reference-Bidi-createLineBidi-2]]<> link:{java8-javadoc}/java/text/Bidi.html#createLineBidi%2Dint%2Dint%2D[createLineBidi](int, int)++ (link:{java9-javadoc}/java/text/Bidi.html#createLineBidi%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-getBaseLevel-0]]int link:{java8-javadoc}/java/text/Bidi.html#getBaseLevel%2D%2D[getBaseLevel]()++ (link:{java9-javadoc}/java/text/Bidi.html#getBaseLevel%2D%2D[java 9]) -* ++[[painless-api-reference-Bidi-getLength-0]]int link:{java8-javadoc}/java/text/Bidi.html#getLength%2D%2D[getLength]()++ (link:{java9-javadoc}/java/text/Bidi.html#getLength%2D%2D[java 9]) -* ++[[painless-api-reference-Bidi-getLevelAt-1]]int link:{java8-javadoc}/java/text/Bidi.html#getLevelAt%2Dint%2D[getLevelAt](int)++ (link:{java9-javadoc}/java/text/Bidi.html#getLevelAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-getRunCount-0]]int link:{java8-javadoc}/java/text/Bidi.html#getRunCount%2D%2D[getRunCount]()++ (link:{java9-javadoc}/java/text/Bidi.html#getRunCount%2D%2D[java 9]) -* ++[[painless-api-reference-Bidi-getRunLevel-1]]int link:{java8-javadoc}/java/text/Bidi.html#getRunLevel%2Dint%2D[getRunLevel](int)++ (link:{java9-javadoc}/java/text/Bidi.html#getRunLevel%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-getRunLimit-1]]int link:{java8-javadoc}/java/text/Bidi.html#getRunLimit%2Dint%2D[getRunLimit](int)++ (link:{java9-javadoc}/java/text/Bidi.html#getRunLimit%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-getRunStart-1]]int link:{java8-javadoc}/java/text/Bidi.html#getRunStart%2Dint%2D[getRunStart](int)++ (link:{java9-javadoc}/java/text/Bidi.html#getRunStart%2Dint%2D[java 9]) -* ++[[painless-api-reference-Bidi-isLeftToRight-0]]boolean link:{java8-javadoc}/java/text/Bidi.html#isLeftToRight%2D%2D[isLeftToRight]()++ (link:{java9-javadoc}/java/text/Bidi.html#isLeftToRight%2D%2D[java 9]) -* ++[[painless-api-reference-Bidi-isMixed-0]]boolean link:{java8-javadoc}/java/text/Bidi.html#isMixed%2D%2D[isMixed]()++ (link:{java9-javadoc}/java/text/Bidi.html#isMixed%2D%2D[java 9]) -* ++[[painless-api-reference-Bidi-isRightToLeft-0]]boolean link:{java8-javadoc}/java/text/Bidi.html#isRightToLeft%2D%2D[isRightToLeft]()++ (link:{java9-javadoc}/java/text/Bidi.html#isRightToLeft%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BigDecimal.asciidoc b/docs/painless/painless-api-reference/BigDecimal.asciidoc deleted file mode 100644 index ab8a78f35c15b..0000000000000 --- a/docs/painless/painless-api-reference/BigDecimal.asciidoc +++ /dev/null @@ -1,57 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BigDecimal]]++BigDecimal++:: -** [[painless-api-reference-BigDecimal-ONE]]static <> link:{java8-javadoc}/java/math/BigDecimal.html#ONE[ONE] (link:{java9-javadoc}/java/math/BigDecimal.html#ONE[java 9]) -** [[painless-api-reference-BigDecimal-TEN]]static <> link:{java8-javadoc}/java/math/BigDecimal.html#TEN[TEN] (link:{java9-javadoc}/java/math/BigDecimal.html#TEN[java 9]) -** [[painless-api-reference-BigDecimal-ZERO]]static <> link:{java8-javadoc}/java/math/BigDecimal.html#ZERO[ZERO] (link:{java9-javadoc}/java/math/BigDecimal.html#ZERO[java 9]) -* ++[[painless-api-reference-BigDecimal-valueOf-1]]static <> link:{java8-javadoc}/java/math/BigDecimal.html#valueOf%2Ddouble%2D[valueOf](double)++ (link:{java9-javadoc}/java/math/BigDecimal.html#valueOf%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-BigDecimal-1]]link:{java8-javadoc}/java/math/BigDecimal.html#BigDecimal%2Djava.lang.String%2D[BigDecimal](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#BigDecimal%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-BigDecimal-2]]link:{java8-javadoc}/java/math/BigDecimal.html#BigDecimal%2Djava.lang.String%2Djava.math.MathContext%2D[BigDecimal](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#BigDecimal%2Djava.lang.String%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-abs-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#abs%2D%2D[abs]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#abs%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-abs-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#abs%2Djava.math.MathContext%2D[abs](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#abs%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-add-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#add%2Djava.math.BigDecimal%2D[add](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#add%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-add-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#add%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[add](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#add%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-byteValueExact-0]]byte link:{java8-javadoc}/java/math/BigDecimal.html#byteValueExact%2D%2D[byteValueExact]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#byteValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-compareTo-1]]int link:{java8-javadoc}/java/math/BigDecimal.html#compareTo%2Djava.math.BigDecimal%2D[compareTo](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#compareTo%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-divide-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#divide%2Djava.math.BigDecimal%2D[divide](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#divide%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-divide-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#divide%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[divide](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#divide%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-divideAndRemainder-1]]<>[] link:{java8-javadoc}/java/math/BigDecimal.html#divideAndRemainder%2Djava.math.BigDecimal%2D[divideAndRemainder](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#divideAndRemainder%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-divideAndRemainder-2]]<>[] link:{java8-javadoc}/java/math/BigDecimal.html#divideAndRemainder%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[divideAndRemainder](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#divideAndRemainder%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-divideToIntegralValue-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#divideToIntegralValue%2Djava.math.BigDecimal%2D[divideToIntegralValue](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#divideToIntegralValue%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-divideToIntegralValue-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#divideToIntegralValue%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[divideToIntegralValue](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#divideToIntegralValue%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-intValueExact-0]]int link:{java8-javadoc}/java/math/BigDecimal.html#intValueExact%2D%2D[intValueExact]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#intValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-longValueExact-0]]long link:{java8-javadoc}/java/math/BigDecimal.html#longValueExact%2D%2D[longValueExact]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#longValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-max-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#max%2Djava.math.BigDecimal%2D[max](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#max%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-min-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#min%2Djava.math.BigDecimal%2D[min](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#min%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-movePointLeft-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#movePointLeft%2Dint%2D[movePointLeft](int)++ (link:{java9-javadoc}/java/math/BigDecimal.html#movePointLeft%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-movePointRight-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#movePointRight%2Dint%2D[movePointRight](int)++ (link:{java9-javadoc}/java/math/BigDecimal.html#movePointRight%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-multiply-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#multiply%2Djava.math.BigDecimal%2D[multiply](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#multiply%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-multiply-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#multiply%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[multiply](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#multiply%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-negate-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-negate-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#negate%2Djava.math.MathContext%2D[negate](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#negate%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-plus-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#plus%2D%2D[plus]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#plus%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-plus-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#plus%2Djava.math.MathContext%2D[plus](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#plus%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-pow-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#pow%2Dint%2D[pow](int)++ (link:{java9-javadoc}/java/math/BigDecimal.html#pow%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-pow-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#pow%2Dint%2Djava.math.MathContext%2D[pow](int, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#pow%2Dint%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-precision-0]]int link:{java8-javadoc}/java/math/BigDecimal.html#precision%2D%2D[precision]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#precision%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-remainder-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#remainder%2Djava.math.BigDecimal%2D[remainder](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#remainder%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-remainder-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#remainder%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[remainder](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#remainder%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-round-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#round%2Djava.math.MathContext%2D[round](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#round%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-scale-0]]int link:{java8-javadoc}/java/math/BigDecimal.html#scale%2D%2D[scale]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#scale%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-scaleByPowerOfTen-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#scaleByPowerOfTen%2Dint%2D[scaleByPowerOfTen](int)++ (link:{java9-javadoc}/java/math/BigDecimal.html#scaleByPowerOfTen%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-setScale-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#setScale%2Dint%2D[setScale](int)++ (link:{java9-javadoc}/java/math/BigDecimal.html#setScale%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-setScale-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#setScale%2Dint%2Djava.math.RoundingMode%2D[setScale](int, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#setScale%2Dint%2Djava.math.RoundingMode%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-shortValueExact-0]]short link:{java8-javadoc}/java/math/BigDecimal.html#shortValueExact%2D%2D[shortValueExact]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#shortValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-signum-0]]int link:{java8-javadoc}/java/math/BigDecimal.html#signum%2D%2D[signum]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#signum%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-stripTrailingZeros-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#stripTrailingZeros%2D%2D[stripTrailingZeros]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#stripTrailingZeros%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-subtract-1]]<> link:{java8-javadoc}/java/math/BigDecimal.html#subtract%2Djava.math.BigDecimal%2D[subtract](<>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#subtract%2Djava.math.BigDecimal%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-subtract-2]]<> link:{java8-javadoc}/java/math/BigDecimal.html#subtract%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[subtract](<>, <>)++ (link:{java9-javadoc}/java/math/BigDecimal.html#subtract%2Djava.math.BigDecimal%2Djava.math.MathContext%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-toBigInteger-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#toBigInteger%2D%2D[toBigInteger]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#toBigInteger%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-toBigIntegerExact-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#toBigIntegerExact%2D%2D[toBigIntegerExact]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#toBigIntegerExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-toEngineeringString-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#toEngineeringString%2D%2D[toEngineeringString]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#toEngineeringString%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-toPlainString-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#toPlainString%2D%2D[toPlainString]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#toPlainString%2D%2D[java 9]) -* ++[[painless-api-reference-BigDecimal-ulp-0]]<> link:{java8-javadoc}/java/math/BigDecimal.html#ulp%2D%2D[ulp]()++ (link:{java9-javadoc}/java/math/BigDecimal.html#ulp%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/BigInteger.asciidoc b/docs/painless/painless-api-reference/BigInteger.asciidoc deleted file mode 100644 index 796355cadd3b7..0000000000000 --- a/docs/painless/painless-api-reference/BigInteger.asciidoc +++ /dev/null @@ -1,50 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BigInteger]]++BigInteger++:: -** [[painless-api-reference-BigInteger-ONE]]static <> link:{java8-javadoc}/java/math/BigInteger.html#ONE[ONE] (link:{java9-javadoc}/java/math/BigInteger.html#ONE[java 9]) -** [[painless-api-reference-BigInteger-TEN]]static <> link:{java8-javadoc}/java/math/BigInteger.html#TEN[TEN] (link:{java9-javadoc}/java/math/BigInteger.html#TEN[java 9]) -** [[painless-api-reference-BigInteger-ZERO]]static <> link:{java8-javadoc}/java/math/BigInteger.html#ZERO[ZERO] (link:{java9-javadoc}/java/math/BigInteger.html#ZERO[java 9]) -* ++[[painless-api-reference-BigInteger-valueOf-1]]static <> link:{java8-javadoc}/java/math/BigInteger.html#valueOf%2Dlong%2D[valueOf](long)++ (link:{java9-javadoc}/java/math/BigInteger.html#valueOf%2Dlong%2D[java 9]) -* ++[[painless-api-reference-BigInteger-BigInteger-1]]link:{java8-javadoc}/java/math/BigInteger.html#BigInteger%2Djava.lang.String%2D[BigInteger](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#BigInteger%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-BigInteger-BigInteger-2]]link:{java8-javadoc}/java/math/BigInteger.html#BigInteger%2Djava.lang.String%2Dint%2D[BigInteger](<>, int)++ (link:{java9-javadoc}/java/math/BigInteger.html#BigInteger%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-abs-0]]<> link:{java8-javadoc}/java/math/BigInteger.html#abs%2D%2D[abs]()++ (link:{java9-javadoc}/java/math/BigInteger.html#abs%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-add-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#add%2Djava.math.BigInteger%2D[add](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#add%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-and-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#and%2Djava.math.BigInteger%2D[and](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#and%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-andNot-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#andNot%2Djava.math.BigInteger%2D[andNot](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#andNot%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-bitCount-0]]int link:{java8-javadoc}/java/math/BigInteger.html#bitCount%2D%2D[bitCount]()++ (link:{java9-javadoc}/java/math/BigInteger.html#bitCount%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-bitLength-0]]int link:{java8-javadoc}/java/math/BigInteger.html#bitLength%2D%2D[bitLength]()++ (link:{java9-javadoc}/java/math/BigInteger.html#bitLength%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-byteValueExact-0]]byte link:{java8-javadoc}/java/math/BigInteger.html#byteValueExact%2D%2D[byteValueExact]()++ (link:{java9-javadoc}/java/math/BigInteger.html#byteValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-clearBit-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#clearBit%2Dint%2D[clearBit](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#clearBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-compareTo-1]]int link:{java8-javadoc}/java/math/BigInteger.html#compareTo%2Djava.math.BigInteger%2D[compareTo](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#compareTo%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-divide-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#divide%2Djava.math.BigInteger%2D[divide](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#divide%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-divideAndRemainder-1]]<>[] link:{java8-javadoc}/java/math/BigInteger.html#divideAndRemainder%2Djava.math.BigInteger%2D[divideAndRemainder](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#divideAndRemainder%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-flipBit-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#flipBit%2Dint%2D[flipBit](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#flipBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-gcd-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#gcd%2Djava.math.BigInteger%2D[gcd](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#gcd%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-getLowestSetBit-0]]int link:{java8-javadoc}/java/math/BigInteger.html#getLowestSetBit%2D%2D[getLowestSetBit]()++ (link:{java9-javadoc}/java/math/BigInteger.html#getLowestSetBit%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-intValueExact-0]]int link:{java8-javadoc}/java/math/BigInteger.html#intValueExact%2D%2D[intValueExact]()++ (link:{java9-javadoc}/java/math/BigInteger.html#intValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-longValueExact-0]]long link:{java8-javadoc}/java/math/BigInteger.html#longValueExact%2D%2D[longValueExact]()++ (link:{java9-javadoc}/java/math/BigInteger.html#longValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-max-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#max%2Djava.math.BigInteger%2D[max](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#max%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-min-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#min%2Djava.math.BigInteger%2D[min](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#min%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-mod-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#mod%2Djava.math.BigInteger%2D[mod](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#mod%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-modInverse-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#modInverse%2Djava.math.BigInteger%2D[modInverse](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#modInverse%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-modPow-2]]<> link:{java8-javadoc}/java/math/BigInteger.html#modPow%2Djava.math.BigInteger%2Djava.math.BigInteger%2D[modPow](<>, <>)++ (link:{java9-javadoc}/java/math/BigInteger.html#modPow%2Djava.math.BigInteger%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-multiply-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#multiply%2Djava.math.BigInteger%2D[multiply](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#multiply%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-negate-0]]<> link:{java8-javadoc}/java/math/BigInteger.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/math/BigInteger.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-not-0]]<> link:{java8-javadoc}/java/math/BigInteger.html#not%2D%2D[not]()++ (link:{java9-javadoc}/java/math/BigInteger.html#not%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-or-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#or%2Djava.math.BigInteger%2D[or](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#or%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-pow-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#pow%2Dint%2D[pow](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#pow%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-remainder-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#remainder%2Djava.math.BigInteger%2D[remainder](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#remainder%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-setBit-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#setBit%2Dint%2D[setBit](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#setBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-shiftLeft-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#shiftLeft%2Dint%2D[shiftLeft](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#shiftLeft%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-shiftRight-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#shiftRight%2Dint%2D[shiftRight](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#shiftRight%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-shortValueExact-0]]short link:{java8-javadoc}/java/math/BigInteger.html#shortValueExact%2D%2D[shortValueExact]()++ (link:{java9-javadoc}/java/math/BigInteger.html#shortValueExact%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-signum-0]]int link:{java8-javadoc}/java/math/BigInteger.html#signum%2D%2D[signum]()++ (link:{java9-javadoc}/java/math/BigInteger.html#signum%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-subtract-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#subtract%2Djava.math.BigInteger%2D[subtract](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#subtract%2Djava.math.BigInteger%2D[java 9]) -* ++[[painless-api-reference-BigInteger-testBit-1]]boolean link:{java8-javadoc}/java/math/BigInteger.html#testBit%2Dint%2D[testBit](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#testBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-toByteArray-0]]byte[] link:{java8-javadoc}/java/math/BigInteger.html#toByteArray%2D%2D[toByteArray]()++ (link:{java9-javadoc}/java/math/BigInteger.html#toByteArray%2D%2D[java 9]) -* ++[[painless-api-reference-BigInteger-toString-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#toString%2Dint%2D[toString](int)++ (link:{java9-javadoc}/java/math/BigInteger.html#toString%2Dint%2D[java 9]) -* ++[[painless-api-reference-BigInteger-xor-1]]<> link:{java8-javadoc}/java/math/BigInteger.html#xor%2Djava.math.BigInteger%2D[xor](<>)++ (link:{java9-javadoc}/java/math/BigInteger.html#xor%2Djava.math.BigInteger%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/BinaryOperator.asciidoc b/docs/painless/painless-api-reference/BinaryOperator.asciidoc deleted file mode 100644 index c7e53be47f720..0000000000000 --- a/docs/painless/painless-api-reference/BinaryOperator.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BinaryOperator]]++BinaryOperator++:: -* ++[[painless-api-reference-BinaryOperator-maxBy-1]]static <> link:{java8-javadoc}/java/util/function/BinaryOperator.html#maxBy%2Djava.util.Comparator%2D[maxBy](<>)++ (link:{java9-javadoc}/java/util/function/BinaryOperator.html#maxBy%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-BinaryOperator-minBy-1]]static <> link:{java8-javadoc}/java/util/function/BinaryOperator.html#minBy%2Djava.util.Comparator%2D[minBy](<>)++ (link:{java9-javadoc}/java/util/function/BinaryOperator.html#minBy%2Djava.util.Comparator%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/BitSet.asciidoc b/docs/painless/painless-api-reference/BitSet.asciidoc deleted file mode 100644 index 158f64675d93e..0000000000000 --- a/docs/painless/painless-api-reference/BitSet.asciidoc +++ /dev/null @@ -1,34 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BitSet]]++BitSet++:: -* ++[[painless-api-reference-BitSet-valueOf-1]]static <> link:{java8-javadoc}/java/util/BitSet.html#valueOf%2Dlong:A%2D[valueOf](long[])++ (link:{java9-javadoc}/java/util/BitSet.html#valueOf%2Dlong:A%2D[java 9]) -* ++[[painless-api-reference-BitSet-BitSet-0]]link:{java8-javadoc}/java/util/BitSet.html#BitSet%2D%2D[BitSet]()++ (link:{java9-javadoc}/java/util/BitSet.html#BitSet%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-BitSet-1]]link:{java8-javadoc}/java/util/BitSet.html#BitSet%2Dint%2D[BitSet](int)++ (link:{java9-javadoc}/java/util/BitSet.html#BitSet%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-and-1]]void link:{java8-javadoc}/java/util/BitSet.html#and%2Djava.util.BitSet%2D[and](<>)++ (link:{java9-javadoc}/java/util/BitSet.html#and%2Djava.util.BitSet%2D[java 9]) -* ++[[painless-api-reference-BitSet-andNot-1]]void link:{java8-javadoc}/java/util/BitSet.html#andNot%2Djava.util.BitSet%2D[andNot](<>)++ (link:{java9-javadoc}/java/util/BitSet.html#andNot%2Djava.util.BitSet%2D[java 9]) -* ++[[painless-api-reference-BitSet-cardinality-0]]int link:{java8-javadoc}/java/util/BitSet.html#cardinality%2D%2D[cardinality]()++ (link:{java9-javadoc}/java/util/BitSet.html#cardinality%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-clear-0]]void link:{java8-javadoc}/java/util/BitSet.html#clear%2D%2D[clear]()++ (link:{java9-javadoc}/java/util/BitSet.html#clear%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-clear-1]]void link:{java8-javadoc}/java/util/BitSet.html#clear%2Dint%2D[clear](int)++ (link:{java9-javadoc}/java/util/BitSet.html#clear%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-clear-2]]void link:{java8-javadoc}/java/util/BitSet.html#clear%2Dint%2Dint%2D[clear](int, int)++ (link:{java9-javadoc}/java/util/BitSet.html#clear%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-clone-0]]def link:{java8-javadoc}/java/util/BitSet.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/BitSet.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-flip-1]]void link:{java8-javadoc}/java/util/BitSet.html#flip%2Dint%2D[flip](int)++ (link:{java9-javadoc}/java/util/BitSet.html#flip%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-flip-2]]void link:{java8-javadoc}/java/util/BitSet.html#flip%2Dint%2Dint%2D[flip](int, int)++ (link:{java9-javadoc}/java/util/BitSet.html#flip%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-intersects-1]]boolean link:{java8-javadoc}/java/util/BitSet.html#intersects%2Djava.util.BitSet%2D[intersects](<>)++ (link:{java9-javadoc}/java/util/BitSet.html#intersects%2Djava.util.BitSet%2D[java 9]) -* ++[[painless-api-reference-BitSet-isEmpty-0]]boolean link:{java8-javadoc}/java/util/BitSet.html#isEmpty%2D%2D[isEmpty]()++ (link:{java9-javadoc}/java/util/BitSet.html#isEmpty%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-length-0]]int link:{java8-javadoc}/java/util/BitSet.html#length%2D%2D[length]()++ (link:{java9-javadoc}/java/util/BitSet.html#length%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-nextClearBit-1]]int link:{java8-javadoc}/java/util/BitSet.html#nextClearBit%2Dint%2D[nextClearBit](int)++ (link:{java9-javadoc}/java/util/BitSet.html#nextClearBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-nextSetBit-1]]int link:{java8-javadoc}/java/util/BitSet.html#nextSetBit%2Dint%2D[nextSetBit](int)++ (link:{java9-javadoc}/java/util/BitSet.html#nextSetBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-or-1]]void link:{java8-javadoc}/java/util/BitSet.html#or%2Djava.util.BitSet%2D[or](<>)++ (link:{java9-javadoc}/java/util/BitSet.html#or%2Djava.util.BitSet%2D[java 9]) -* ++[[painless-api-reference-BitSet-previousClearBit-1]]int link:{java8-javadoc}/java/util/BitSet.html#previousClearBit%2Dint%2D[previousClearBit](int)++ (link:{java9-javadoc}/java/util/BitSet.html#previousClearBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-previousSetBit-1]]int link:{java8-javadoc}/java/util/BitSet.html#previousSetBit%2Dint%2D[previousSetBit](int)++ (link:{java9-javadoc}/java/util/BitSet.html#previousSetBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-set-1]]void link:{java8-javadoc}/java/util/BitSet.html#set%2Dint%2D[set](int)++ (link:{java9-javadoc}/java/util/BitSet.html#set%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-set-2]]void link:{java8-javadoc}/java/util/BitSet.html#set%2Dint%2Dint%2D[set](int, int)++ (link:{java9-javadoc}/java/util/BitSet.html#set%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-BitSet-set-3]]void link:{java8-javadoc}/java/util/BitSet.html#set%2Dint%2Dint%2Dboolean%2D[set](int, int, boolean)++ (link:{java9-javadoc}/java/util/BitSet.html#set%2Dint%2Dint%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-BitSet-size-0]]int link:{java8-javadoc}/java/util/BitSet.html#size%2D%2D[size]()++ (link:{java9-javadoc}/java/util/BitSet.html#size%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-toByteArray-0]]byte[] link:{java8-javadoc}/java/util/BitSet.html#toByteArray%2D%2D[toByteArray]()++ (link:{java9-javadoc}/java/util/BitSet.html#toByteArray%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-toLongArray-0]]long[] link:{java8-javadoc}/java/util/BitSet.html#toLongArray%2D%2D[toLongArray]()++ (link:{java9-javadoc}/java/util/BitSet.html#toLongArray%2D%2D[java 9]) -* ++[[painless-api-reference-BitSet-xor-1]]void link:{java8-javadoc}/java/util/BitSet.html#xor%2Djava.util.BitSet%2D[xor](<>)++ (link:{java9-javadoc}/java/util/BitSet.html#xor%2Djava.util.BitSet%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Boolean.asciidoc b/docs/painless/painless-api-reference/Boolean.asciidoc deleted file mode 100644 index 47000e58fa828..0000000000000 --- a/docs/painless/painless-api-reference/Boolean.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Boolean]]++Boolean++:: -** [[painless-api-reference-Boolean-FALSE]]static <> link:{java8-javadoc}/java/lang/Boolean.html#FALSE[FALSE] (link:{java9-javadoc}/java/lang/Boolean.html#FALSE[java 9]) -** [[painless-api-reference-Boolean-TRUE]]static <> link:{java8-javadoc}/java/lang/Boolean.html#TRUE[TRUE] (link:{java9-javadoc}/java/lang/Boolean.html#TRUE[java 9]) -* ++[[painless-api-reference-Boolean-compare-2]]static int link:{java8-javadoc}/java/lang/Boolean.html#compare%2Dboolean%2Dboolean%2D[compare](boolean, boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#compare%2Dboolean%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-hashCode-1]]static int link:{java8-javadoc}/java/lang/Boolean.html#hashCode%2Dboolean%2D[hashCode](boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#hashCode%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-logicalAnd-2]]static boolean link:{java8-javadoc}/java/lang/Boolean.html#logicalAnd%2Dboolean%2Dboolean%2D[logicalAnd](boolean, boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#logicalAnd%2Dboolean%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-logicalOr-2]]static boolean link:{java8-javadoc}/java/lang/Boolean.html#logicalOr%2Dboolean%2Dboolean%2D[logicalOr](boolean, boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#logicalOr%2Dboolean%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-logicalXor-2]]static boolean link:{java8-javadoc}/java/lang/Boolean.html#logicalXor%2Dboolean%2Dboolean%2D[logicalXor](boolean, boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#logicalXor%2Dboolean%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-parseBoolean-1]]static boolean link:{java8-javadoc}/java/lang/Boolean.html#parseBoolean%2Djava.lang.String%2D[parseBoolean](<>)++ (link:{java9-javadoc}/java/lang/Boolean.html#parseBoolean%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Boolean-toString-1]]static <> link:{java8-javadoc}/java/lang/Boolean.html#toString%2Dboolean%2D[toString](boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#toString%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Boolean.html#valueOf%2Dboolean%2D[valueOf](boolean)++ (link:{java9-javadoc}/java/lang/Boolean.html#valueOf%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Boolean-booleanValue-0]]boolean link:{java8-javadoc}/java/lang/Boolean.html#booleanValue%2D%2D[booleanValue]()++ (link:{java9-javadoc}/java/lang/Boolean.html#booleanValue%2D%2D[java 9]) -* ++[[painless-api-reference-Boolean-compareTo-1]]int link:{java8-javadoc}/java/lang/Boolean.html#compareTo%2Djava.lang.Boolean%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Boolean.html#compareTo%2Djava.lang.Boolean%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BooleanSupplier.asciidoc b/docs/painless/painless-api-reference/BooleanSupplier.asciidoc deleted file mode 100644 index 9a34d1caf2d99..0000000000000 --- a/docs/painless/painless-api-reference/BooleanSupplier.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BooleanSupplier]]++BooleanSupplier++:: -* ++[[painless-api-reference-BooleanSupplier-getAsBoolean-0]]boolean link:{java8-javadoc}/java/util/function/BooleanSupplier.html#getAsBoolean%2D%2D[getAsBoolean]()++ (link:{java9-javadoc}/java/util/function/BooleanSupplier.html#getAsBoolean%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/BreakIterator.asciidoc b/docs/painless/painless-api-reference/BreakIterator.asciidoc deleted file mode 100644 index 0e830859c457f..0000000000000 --- a/docs/painless/painless-api-reference/BreakIterator.asciidoc +++ /dev/null @@ -1,29 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BreakIterator]]++BreakIterator++:: -** [[painless-api-reference-BreakIterator-DONE]]static int link:{java8-javadoc}/java/text/BreakIterator.html#DONE[DONE] (link:{java9-javadoc}/java/text/BreakIterator.html#DONE[java 9]) -* ++[[painless-api-reference-BreakIterator-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/text/BreakIterator.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getCharacterInstance-0]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getCharacterInstance%2D%2D[getCharacterInstance]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#getCharacterInstance%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getCharacterInstance-1]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getCharacterInstance%2Djava.util.Locale%2D[getCharacterInstance](<>)++ (link:{java9-javadoc}/java/text/BreakIterator.html#getCharacterInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getLineInstance-0]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getLineInstance%2D%2D[getLineInstance]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#getLineInstance%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getLineInstance-1]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getLineInstance%2Djava.util.Locale%2D[getLineInstance](<>)++ (link:{java9-javadoc}/java/text/BreakIterator.html#getLineInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getSentenceInstance-0]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getSentenceInstance%2D%2D[getSentenceInstance]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#getSentenceInstance%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getSentenceInstance-1]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getSentenceInstance%2Djava.util.Locale%2D[getSentenceInstance](<>)++ (link:{java9-javadoc}/java/text/BreakIterator.html#getSentenceInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getWordInstance-0]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getWordInstance%2D%2D[getWordInstance]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#getWordInstance%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getWordInstance-1]]static <> link:{java8-javadoc}/java/text/BreakIterator.html#getWordInstance%2Djava.util.Locale%2D[getWordInstance](<>)++ (link:{java9-javadoc}/java/text/BreakIterator.html#getWordInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-clone-0]]def link:{java8-javadoc}/java/text/BreakIterator.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-current-0]]int link:{java8-javadoc}/java/text/BreakIterator.html#current%2D%2D[current]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#current%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-first-0]]int link:{java8-javadoc}/java/text/BreakIterator.html#first%2D%2D[first]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#first%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-following-1]]int link:{java8-javadoc}/java/text/BreakIterator.html#following%2Dint%2D[following](int)++ (link:{java9-javadoc}/java/text/BreakIterator.html#following%2Dint%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-getText-0]]<> link:{java8-javadoc}/java/text/BreakIterator.html#getText%2D%2D[getText]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#getText%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-isBoundary-1]]boolean link:{java8-javadoc}/java/text/BreakIterator.html#isBoundary%2Dint%2D[isBoundary](int)++ (link:{java9-javadoc}/java/text/BreakIterator.html#isBoundary%2Dint%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-last-0]]int link:{java8-javadoc}/java/text/BreakIterator.html#last%2D%2D[last]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#last%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-next-0]]int link:{java8-javadoc}/java/text/BreakIterator.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-next-1]]int link:{java8-javadoc}/java/text/BreakIterator.html#next%2Dint%2D[next](int)++ (link:{java9-javadoc}/java/text/BreakIterator.html#next%2Dint%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-preceding-1]]int link:{java8-javadoc}/java/text/BreakIterator.html#preceding%2Dint%2D[preceding](int)++ (link:{java9-javadoc}/java/text/BreakIterator.html#preceding%2Dint%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-previous-0]]int link:{java8-javadoc}/java/text/BreakIterator.html#previous%2D%2D[previous]()++ (link:{java9-javadoc}/java/text/BreakIterator.html#previous%2D%2D[java 9]) -* ++[[painless-api-reference-BreakIterator-setText-1]]void link:{java8-javadoc}/java/text/BreakIterator.html#setText%2Djava.lang.String%2D[setText](<>)++ (link:{java9-javadoc}/java/text/BreakIterator.html#setText%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Byte.asciidoc b/docs/painless/painless-api-reference/Byte.asciidoc deleted file mode 100644 index 2fc42b6362540..0000000000000 --- a/docs/painless/painless-api-reference/Byte.asciidoc +++ /dev/null @@ -1,22 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Byte]]++Byte++:: -** [[painless-api-reference-Byte-BYTES]]static int link:{java8-javadoc}/java/lang/Byte.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Byte.html#BYTES[java 9]) -** [[painless-api-reference-Byte-MAX_VALUE]]static byte link:{java8-javadoc}/java/lang/Byte.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Byte.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Byte-MIN_VALUE]]static byte link:{java8-javadoc}/java/lang/Byte.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Byte.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Byte-SIZE]]static int link:{java8-javadoc}/java/lang/Byte.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Byte.html#SIZE[java 9]) -* ++[[painless-api-reference-Byte-compare-2]]static int link:{java8-javadoc}/java/lang/Byte.html#compare%2Dbyte%2Dbyte%2D[compare](byte, byte)++ (link:{java9-javadoc}/java/lang/Byte.html#compare%2Dbyte%2Dbyte%2D[java 9]) -* ++[[painless-api-reference-Byte-decode-1]]static <> link:{java8-javadoc}/java/lang/Byte.html#decode%2Djava.lang.String%2D[decode](<>)++ (link:{java9-javadoc}/java/lang/Byte.html#decode%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Byte-hashCode-1]]static int link:{java8-javadoc}/java/lang/Byte.html#hashCode%2Dbyte%2D[hashCode](byte)++ (link:{java9-javadoc}/java/lang/Byte.html#hashCode%2Dbyte%2D[java 9]) -* ++[[painless-api-reference-Byte-parseByte-1]]static byte link:{java8-javadoc}/java/lang/Byte.html#parseByte%2Djava.lang.String%2D[parseByte](<>)++ (link:{java9-javadoc}/java/lang/Byte.html#parseByte%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Byte-parseByte-2]]static byte link:{java8-javadoc}/java/lang/Byte.html#parseByte%2Djava.lang.String%2Dint%2D[parseByte](<>, int)++ (link:{java9-javadoc}/java/lang/Byte.html#parseByte%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Byte-toString-1]]static <> link:{java8-javadoc}/java/lang/Byte.html#toString%2Dbyte%2D[toString](byte)++ (link:{java9-javadoc}/java/lang/Byte.html#toString%2Dbyte%2D[java 9]) -* ++[[painless-api-reference-Byte-toUnsignedInt-1]]static int link:{java8-javadoc}/java/lang/Byte.html#toUnsignedInt%2Dbyte%2D[toUnsignedInt](byte)++ (link:{java9-javadoc}/java/lang/Byte.html#toUnsignedInt%2Dbyte%2D[java 9]) -* ++[[painless-api-reference-Byte-toUnsignedLong-1]]static long link:{java8-javadoc}/java/lang/Byte.html#toUnsignedLong%2Dbyte%2D[toUnsignedLong](byte)++ (link:{java9-javadoc}/java/lang/Byte.html#toUnsignedLong%2Dbyte%2D[java 9]) -* ++[[painless-api-reference-Byte-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Byte.html#valueOf%2Dbyte%2D[valueOf](byte)++ (link:{java9-javadoc}/java/lang/Byte.html#valueOf%2Dbyte%2D[java 9]) -* ++[[painless-api-reference-Byte-valueOf-2]]static <> link:{java8-javadoc}/java/lang/Byte.html#valueOf%2Djava.lang.String%2Dint%2D[valueOf](<>, int)++ (link:{java9-javadoc}/java/lang/Byte.html#valueOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Byte-compareTo-1]]int link:{java8-javadoc}/java/lang/Byte.html#compareTo%2Djava.lang.Byte%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Byte.html#compareTo%2Djava.lang.Byte%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/BytesRef.asciidoc b/docs/painless/painless-api-reference/BytesRef.asciidoc deleted file mode 100644 index 6114a5625fcae..0000000000000 --- a/docs/painless/painless-api-reference/BytesRef.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-BytesRef]]++BytesRef++:: -** [[painless-api-reference-BytesRef-bytes]]byte[] link:{lucene-core-javadoc}/org/apache/lucene/util/BytesRef.html#bytes[bytes] -** [[painless-api-reference-BytesRef-length]]int link:{lucene-core-javadoc}/org/apache/lucene/util/BytesRef.html#length[length] -** [[painless-api-reference-BytesRef-offset]]int link:{lucene-core-javadoc}/org/apache/lucene/util/BytesRef.html#offset[offset] -* ++[[painless-api-reference-BytesRef-bytesEquals-1]]boolean link:{lucene-core-javadoc}/org/apache/lucene/util/BytesRef.html#bytesEquals%2Dorg.apache.lucene.util.BytesRef%2D[bytesEquals](<>)++ -* ++[[painless-api-reference-BytesRef-utf8ToString-0]]<> link:{lucene-core-javadoc}/org/apache/lucene/util/BytesRef.html#utf8ToString%2D%2D[utf8ToString]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Calendar.Builder.asciidoc b/docs/painless/painless-api-reference/Calendar.Builder.asciidoc deleted file mode 100644 index 2ab66c5ecac9e..0000000000000 --- a/docs/painless/painless-api-reference/Calendar.Builder.asciidoc +++ /dev/null @@ -1,21 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Calendar-Builder]]++Calendar.Builder++:: -* ++[[painless-api-reference-Calendar-Builder-Calendar.Builder-0]]link:{java8-javadoc}/java/util/Calendar.Builder.html#Calendar.Builder%2D%2D[Calendar.Builder]()++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#Calendar.Builder%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-build-0]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#build%2D%2D[build]()++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#build%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-set-2]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#set%2Dint%2Dint%2D[set](int, int)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#set%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setCalendarType-1]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setCalendarType%2Djava.lang.String%2D[setCalendarType](<>)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setCalendarType%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setDate-3]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setDate%2Dint%2Dint%2Dint%2D[setDate](int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setDate%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setFields-1]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setFields%2Dint:A%2D[setFields](int[])++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setFields%2Dint:A%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setInstant-1]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setInstant%2Dlong%2D[setInstant](long)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setInstant%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setLenient-1]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setLenient%2Dboolean%2D[setLenient](boolean)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setLenient%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setLocale-1]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setLocale%2Djava.util.Locale%2D[setLocale](<>)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setLocale%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setTimeOfDay-3]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setTimeOfDay%2Dint%2Dint%2Dint%2D[setTimeOfDay](int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setTimeOfDay%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setTimeOfDay-4]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setTimeOfDay%2Dint%2Dint%2Dint%2Dint%2D[setTimeOfDay](int, int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setTimeOfDay%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setTimeZone-1]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setTimeZone%2Djava.util.TimeZone%2D[setTimeZone](<>)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setTimeZone%2Djava.util.TimeZone%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setWeekDate-3]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setWeekDate%2Dint%2Dint%2Dint%2D[setWeekDate](int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setWeekDate%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-Builder-setWeekDefinition-2]]<> link:{java8-javadoc}/java/util/Calendar.Builder.html#setWeekDefinition%2Dint%2Dint%2D[setWeekDefinition](int, int)++ (link:{java9-javadoc}/java/util/Calendar.Builder.html#setWeekDefinition%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Calendar.asciidoc b/docs/painless/painless-api-reference/Calendar.asciidoc deleted file mode 100644 index e1a489d8c87d1..0000000000000 --- a/docs/painless/painless-api-reference/Calendar.asciidoc +++ /dev/null @@ -1,102 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Calendar]]++Calendar++:: -** [[painless-api-reference-Calendar-ALL_STYLES]]static int link:{java8-javadoc}/java/util/Calendar.html#ALL_STYLES[ALL_STYLES] (link:{java9-javadoc}/java/util/Calendar.html#ALL_STYLES[java 9]) -** [[painless-api-reference-Calendar-AM]]static int link:{java8-javadoc}/java/util/Calendar.html#AM[AM] (link:{java9-javadoc}/java/util/Calendar.html#AM[java 9]) -** [[painless-api-reference-Calendar-AM_PM]]static int link:{java8-javadoc}/java/util/Calendar.html#AM_PM[AM_PM] (link:{java9-javadoc}/java/util/Calendar.html#AM_PM[java 9]) -** [[painless-api-reference-Calendar-APRIL]]static int link:{java8-javadoc}/java/util/Calendar.html#APRIL[APRIL] (link:{java9-javadoc}/java/util/Calendar.html#APRIL[java 9]) -** [[painless-api-reference-Calendar-AUGUST]]static int link:{java8-javadoc}/java/util/Calendar.html#AUGUST[AUGUST] (link:{java9-javadoc}/java/util/Calendar.html#AUGUST[java 9]) -** [[painless-api-reference-Calendar-DATE]]static int link:{java8-javadoc}/java/util/Calendar.html#DATE[DATE] (link:{java9-javadoc}/java/util/Calendar.html#DATE[java 9]) -** [[painless-api-reference-Calendar-DAY_OF_MONTH]]static int link:{java8-javadoc}/java/util/Calendar.html#DAY_OF_MONTH[DAY_OF_MONTH] (link:{java9-javadoc}/java/util/Calendar.html#DAY_OF_MONTH[java 9]) -** [[painless-api-reference-Calendar-DAY_OF_WEEK]]static int link:{java8-javadoc}/java/util/Calendar.html#DAY_OF_WEEK[DAY_OF_WEEK] (link:{java9-javadoc}/java/util/Calendar.html#DAY_OF_WEEK[java 9]) -** [[painless-api-reference-Calendar-DAY_OF_WEEK_IN_MONTH]]static int link:{java8-javadoc}/java/util/Calendar.html#DAY_OF_WEEK_IN_MONTH[DAY_OF_WEEK_IN_MONTH] (link:{java9-javadoc}/java/util/Calendar.html#DAY_OF_WEEK_IN_MONTH[java 9]) -** [[painless-api-reference-Calendar-DAY_OF_YEAR]]static int link:{java8-javadoc}/java/util/Calendar.html#DAY_OF_YEAR[DAY_OF_YEAR] (link:{java9-javadoc}/java/util/Calendar.html#DAY_OF_YEAR[java 9]) -** [[painless-api-reference-Calendar-DECEMBER]]static int link:{java8-javadoc}/java/util/Calendar.html#DECEMBER[DECEMBER] (link:{java9-javadoc}/java/util/Calendar.html#DECEMBER[java 9]) -** [[painless-api-reference-Calendar-DST_OFFSET]]static int link:{java8-javadoc}/java/util/Calendar.html#DST_OFFSET[DST_OFFSET] (link:{java9-javadoc}/java/util/Calendar.html#DST_OFFSET[java 9]) -** [[painless-api-reference-Calendar-ERA]]static int link:{java8-javadoc}/java/util/Calendar.html#ERA[ERA] (link:{java9-javadoc}/java/util/Calendar.html#ERA[java 9]) -** [[painless-api-reference-Calendar-FEBRUARY]]static int link:{java8-javadoc}/java/util/Calendar.html#FEBRUARY[FEBRUARY] (link:{java9-javadoc}/java/util/Calendar.html#FEBRUARY[java 9]) -** [[painless-api-reference-Calendar-FIELD_COUNT]]static int link:{java8-javadoc}/java/util/Calendar.html#FIELD_COUNT[FIELD_COUNT] (link:{java9-javadoc}/java/util/Calendar.html#FIELD_COUNT[java 9]) -** [[painless-api-reference-Calendar-FRIDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#FRIDAY[FRIDAY] (link:{java9-javadoc}/java/util/Calendar.html#FRIDAY[java 9]) -** [[painless-api-reference-Calendar-HOUR]]static int link:{java8-javadoc}/java/util/Calendar.html#HOUR[HOUR] (link:{java9-javadoc}/java/util/Calendar.html#HOUR[java 9]) -** [[painless-api-reference-Calendar-HOUR_OF_DAY]]static int link:{java8-javadoc}/java/util/Calendar.html#HOUR_OF_DAY[HOUR_OF_DAY] (link:{java9-javadoc}/java/util/Calendar.html#HOUR_OF_DAY[java 9]) -** [[painless-api-reference-Calendar-JANUARY]]static int link:{java8-javadoc}/java/util/Calendar.html#JANUARY[JANUARY] (link:{java9-javadoc}/java/util/Calendar.html#JANUARY[java 9]) -** [[painless-api-reference-Calendar-JULY]]static int link:{java8-javadoc}/java/util/Calendar.html#JULY[JULY] (link:{java9-javadoc}/java/util/Calendar.html#JULY[java 9]) -** [[painless-api-reference-Calendar-JUNE]]static int link:{java8-javadoc}/java/util/Calendar.html#JUNE[JUNE] (link:{java9-javadoc}/java/util/Calendar.html#JUNE[java 9]) -** [[painless-api-reference-Calendar-LONG]]static int link:{java8-javadoc}/java/util/Calendar.html#LONG[LONG] (link:{java9-javadoc}/java/util/Calendar.html#LONG[java 9]) -** [[painless-api-reference-Calendar-LONG_FORMAT]]static int link:{java8-javadoc}/java/util/Calendar.html#LONG_FORMAT[LONG_FORMAT] (link:{java9-javadoc}/java/util/Calendar.html#LONG_FORMAT[java 9]) -** [[painless-api-reference-Calendar-LONG_STANDALONE]]static int link:{java8-javadoc}/java/util/Calendar.html#LONG_STANDALONE[LONG_STANDALONE] (link:{java9-javadoc}/java/util/Calendar.html#LONG_STANDALONE[java 9]) -** [[painless-api-reference-Calendar-MARCH]]static int link:{java8-javadoc}/java/util/Calendar.html#MARCH[MARCH] (link:{java9-javadoc}/java/util/Calendar.html#MARCH[java 9]) -** [[painless-api-reference-Calendar-MAY]]static int link:{java8-javadoc}/java/util/Calendar.html#MAY[MAY] (link:{java9-javadoc}/java/util/Calendar.html#MAY[java 9]) -** [[painless-api-reference-Calendar-MILLISECOND]]static int link:{java8-javadoc}/java/util/Calendar.html#MILLISECOND[MILLISECOND] (link:{java9-javadoc}/java/util/Calendar.html#MILLISECOND[java 9]) -** [[painless-api-reference-Calendar-MINUTE]]static int link:{java8-javadoc}/java/util/Calendar.html#MINUTE[MINUTE] (link:{java9-javadoc}/java/util/Calendar.html#MINUTE[java 9]) -** [[painless-api-reference-Calendar-MONDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#MONDAY[MONDAY] (link:{java9-javadoc}/java/util/Calendar.html#MONDAY[java 9]) -** [[painless-api-reference-Calendar-MONTH]]static int link:{java8-javadoc}/java/util/Calendar.html#MONTH[MONTH] (link:{java9-javadoc}/java/util/Calendar.html#MONTH[java 9]) -** [[painless-api-reference-Calendar-NARROW_FORMAT]]static int link:{java8-javadoc}/java/util/Calendar.html#NARROW_FORMAT[NARROW_FORMAT] (link:{java9-javadoc}/java/util/Calendar.html#NARROW_FORMAT[java 9]) -** [[painless-api-reference-Calendar-NARROW_STANDALONE]]static int link:{java8-javadoc}/java/util/Calendar.html#NARROW_STANDALONE[NARROW_STANDALONE] (link:{java9-javadoc}/java/util/Calendar.html#NARROW_STANDALONE[java 9]) -** [[painless-api-reference-Calendar-NOVEMBER]]static int link:{java8-javadoc}/java/util/Calendar.html#NOVEMBER[NOVEMBER] (link:{java9-javadoc}/java/util/Calendar.html#NOVEMBER[java 9]) -** [[painless-api-reference-Calendar-OCTOBER]]static int link:{java8-javadoc}/java/util/Calendar.html#OCTOBER[OCTOBER] (link:{java9-javadoc}/java/util/Calendar.html#OCTOBER[java 9]) -** [[painless-api-reference-Calendar-PM]]static int link:{java8-javadoc}/java/util/Calendar.html#PM[PM] (link:{java9-javadoc}/java/util/Calendar.html#PM[java 9]) -** [[painless-api-reference-Calendar-SATURDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#SATURDAY[SATURDAY] (link:{java9-javadoc}/java/util/Calendar.html#SATURDAY[java 9]) -** [[painless-api-reference-Calendar-SECOND]]static int link:{java8-javadoc}/java/util/Calendar.html#SECOND[SECOND] (link:{java9-javadoc}/java/util/Calendar.html#SECOND[java 9]) -** [[painless-api-reference-Calendar-SEPTEMBER]]static int link:{java8-javadoc}/java/util/Calendar.html#SEPTEMBER[SEPTEMBER] (link:{java9-javadoc}/java/util/Calendar.html#SEPTEMBER[java 9]) -** [[painless-api-reference-Calendar-SHORT]]static int link:{java8-javadoc}/java/util/Calendar.html#SHORT[SHORT] (link:{java9-javadoc}/java/util/Calendar.html#SHORT[java 9]) -** [[painless-api-reference-Calendar-SHORT_FORMAT]]static int link:{java8-javadoc}/java/util/Calendar.html#SHORT_FORMAT[SHORT_FORMAT] (link:{java9-javadoc}/java/util/Calendar.html#SHORT_FORMAT[java 9]) -** [[painless-api-reference-Calendar-SHORT_STANDALONE]]static int link:{java8-javadoc}/java/util/Calendar.html#SHORT_STANDALONE[SHORT_STANDALONE] (link:{java9-javadoc}/java/util/Calendar.html#SHORT_STANDALONE[java 9]) -** [[painless-api-reference-Calendar-SUNDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#SUNDAY[SUNDAY] (link:{java9-javadoc}/java/util/Calendar.html#SUNDAY[java 9]) -** [[painless-api-reference-Calendar-THURSDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#THURSDAY[THURSDAY] (link:{java9-javadoc}/java/util/Calendar.html#THURSDAY[java 9]) -** [[painless-api-reference-Calendar-TUESDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#TUESDAY[TUESDAY] (link:{java9-javadoc}/java/util/Calendar.html#TUESDAY[java 9]) -** [[painless-api-reference-Calendar-UNDECIMBER]]static int link:{java8-javadoc}/java/util/Calendar.html#UNDECIMBER[UNDECIMBER] (link:{java9-javadoc}/java/util/Calendar.html#UNDECIMBER[java 9]) -** [[painless-api-reference-Calendar-WEDNESDAY]]static int link:{java8-javadoc}/java/util/Calendar.html#WEDNESDAY[WEDNESDAY] (link:{java9-javadoc}/java/util/Calendar.html#WEDNESDAY[java 9]) -** [[painless-api-reference-Calendar-WEEK_OF_MONTH]]static int link:{java8-javadoc}/java/util/Calendar.html#WEEK_OF_MONTH[WEEK_OF_MONTH] (link:{java9-javadoc}/java/util/Calendar.html#WEEK_OF_MONTH[java 9]) -** [[painless-api-reference-Calendar-WEEK_OF_YEAR]]static int link:{java8-javadoc}/java/util/Calendar.html#WEEK_OF_YEAR[WEEK_OF_YEAR] (link:{java9-javadoc}/java/util/Calendar.html#WEEK_OF_YEAR[java 9]) -** [[painless-api-reference-Calendar-YEAR]]static int link:{java8-javadoc}/java/util/Calendar.html#YEAR[YEAR] (link:{java9-javadoc}/java/util/Calendar.html#YEAR[java 9]) -** [[painless-api-reference-Calendar-ZONE_OFFSET]]static int link:{java8-javadoc}/java/util/Calendar.html#ZONE_OFFSET[ZONE_OFFSET] (link:{java9-javadoc}/java/util/Calendar.html#ZONE_OFFSET[java 9]) -* ++[[painless-api-reference-Calendar-getAvailableCalendarTypes-0]]static <> link:{java8-javadoc}/java/util/Calendar.html#getAvailableCalendarTypes%2D%2D[getAvailableCalendarTypes]()++ (link:{java9-javadoc}/java/util/Calendar.html#getAvailableCalendarTypes%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/util/Calendar.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/util/Calendar.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getInstance-0]]static <> link:{java8-javadoc}/java/util/Calendar.html#getInstance%2D%2D[getInstance]()++ (link:{java9-javadoc}/java/util/Calendar.html#getInstance%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getInstance-1]]static <> link:{java8-javadoc}/java/util/Calendar.html#getInstance%2Djava.util.TimeZone%2D[getInstance](<>)++ (link:{java9-javadoc}/java/util/Calendar.html#getInstance%2Djava.util.TimeZone%2D[java 9]) -* ++[[painless-api-reference-Calendar-getInstance-2]]static <> link:{java8-javadoc}/java/util/Calendar.html#getInstance%2Djava.util.TimeZone%2Djava.util.Locale%2D[getInstance](<>, <>)++ (link:{java9-javadoc}/java/util/Calendar.html#getInstance%2Djava.util.TimeZone%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Calendar-add-2]]void link:{java8-javadoc}/java/util/Calendar.html#add%2Dint%2Dint%2D[add](int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#add%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-after-1]]boolean link:{java8-javadoc}/java/util/Calendar.html#after%2Djava.lang.Object%2D[after](<>)++ (link:{java9-javadoc}/java/util/Calendar.html#after%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Calendar-before-1]]boolean link:{java8-javadoc}/java/util/Calendar.html#before%2Djava.lang.Object%2D[before](<>)++ (link:{java9-javadoc}/java/util/Calendar.html#before%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Calendar-clear-0]]void link:{java8-javadoc}/java/util/Calendar.html#clear%2D%2D[clear]()++ (link:{java9-javadoc}/java/util/Calendar.html#clear%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-clear-1]]void link:{java8-javadoc}/java/util/Calendar.html#clear%2Dint%2D[clear](int)++ (link:{java9-javadoc}/java/util/Calendar.html#clear%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-clone-0]]def link:{java8-javadoc}/java/util/Calendar.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/Calendar.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-compareTo-1]]int link:{java8-javadoc}/java/util/Calendar.html#compareTo%2Djava.util.Calendar%2D[compareTo](<>)++ (link:{java9-javadoc}/java/util/Calendar.html#compareTo%2Djava.util.Calendar%2D[java 9]) -* ++[[painless-api-reference-Calendar-get-1]]int link:{java8-javadoc}/java/util/Calendar.html#get%2Dint%2D[get](int)++ (link:{java9-javadoc}/java/util/Calendar.html#get%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getActualMaximum-1]]int link:{java8-javadoc}/java/util/Calendar.html#getActualMaximum%2Dint%2D[getActualMaximum](int)++ (link:{java9-javadoc}/java/util/Calendar.html#getActualMaximum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getActualMinimum-1]]int link:{java8-javadoc}/java/util/Calendar.html#getActualMinimum%2Dint%2D[getActualMinimum](int)++ (link:{java9-javadoc}/java/util/Calendar.html#getActualMinimum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getCalendarType-0]]<> link:{java8-javadoc}/java/util/Calendar.html#getCalendarType%2D%2D[getCalendarType]()++ (link:{java9-javadoc}/java/util/Calendar.html#getCalendarType%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getDisplayName-3]]<> link:{java8-javadoc}/java/util/Calendar.html#getDisplayName%2Dint%2Dint%2Djava.util.Locale%2D[getDisplayName](int, int, <>)++ (link:{java9-javadoc}/java/util/Calendar.html#getDisplayName%2Dint%2Dint%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Calendar-getDisplayNames-3]]<> link:{java8-javadoc}/java/util/Calendar.html#getDisplayNames%2Dint%2Dint%2Djava.util.Locale%2D[getDisplayNames](int, int, <>)++ (link:{java9-javadoc}/java/util/Calendar.html#getDisplayNames%2Dint%2Dint%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Calendar-getFirstDayOfWeek-0]]int link:{java8-javadoc}/java/util/Calendar.html#getFirstDayOfWeek%2D%2D[getFirstDayOfWeek]()++ (link:{java9-javadoc}/java/util/Calendar.html#getFirstDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getGreatestMinimum-1]]int link:{java8-javadoc}/java/util/Calendar.html#getGreatestMinimum%2Dint%2D[getGreatestMinimum](int)++ (link:{java9-javadoc}/java/util/Calendar.html#getGreatestMinimum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getLeastMaximum-1]]int link:{java8-javadoc}/java/util/Calendar.html#getLeastMaximum%2Dint%2D[getLeastMaximum](int)++ (link:{java9-javadoc}/java/util/Calendar.html#getLeastMaximum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getMaximum-1]]int link:{java8-javadoc}/java/util/Calendar.html#getMaximum%2Dint%2D[getMaximum](int)++ (link:{java9-javadoc}/java/util/Calendar.html#getMaximum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getMinimalDaysInFirstWeek-0]]int link:{java8-javadoc}/java/util/Calendar.html#getMinimalDaysInFirstWeek%2D%2D[getMinimalDaysInFirstWeek]()++ (link:{java9-javadoc}/java/util/Calendar.html#getMinimalDaysInFirstWeek%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getMinimum-1]]int link:{java8-javadoc}/java/util/Calendar.html#getMinimum%2Dint%2D[getMinimum](int)++ (link:{java9-javadoc}/java/util/Calendar.html#getMinimum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-getTime-0]]<> link:{java8-javadoc}/java/util/Calendar.html#getTime%2D%2D[getTime]()++ (link:{java9-javadoc}/java/util/Calendar.html#getTime%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getTimeInMillis-0]]long link:{java8-javadoc}/java/util/Calendar.html#getTimeInMillis%2D%2D[getTimeInMillis]()++ (link:{java9-javadoc}/java/util/Calendar.html#getTimeInMillis%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getTimeZone-0]]<> link:{java8-javadoc}/java/util/Calendar.html#getTimeZone%2D%2D[getTimeZone]()++ (link:{java9-javadoc}/java/util/Calendar.html#getTimeZone%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getWeekYear-0]]int link:{java8-javadoc}/java/util/Calendar.html#getWeekYear%2D%2D[getWeekYear]()++ (link:{java9-javadoc}/java/util/Calendar.html#getWeekYear%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-getWeeksInWeekYear-0]]int link:{java8-javadoc}/java/util/Calendar.html#getWeeksInWeekYear%2D%2D[getWeeksInWeekYear]()++ (link:{java9-javadoc}/java/util/Calendar.html#getWeeksInWeekYear%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-isLenient-0]]boolean link:{java8-javadoc}/java/util/Calendar.html#isLenient%2D%2D[isLenient]()++ (link:{java9-javadoc}/java/util/Calendar.html#isLenient%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-isSet-1]]boolean link:{java8-javadoc}/java/util/Calendar.html#isSet%2Dint%2D[isSet](int)++ (link:{java9-javadoc}/java/util/Calendar.html#isSet%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-isWeekDateSupported-0]]boolean link:{java8-javadoc}/java/util/Calendar.html#isWeekDateSupported%2D%2D[isWeekDateSupported]()++ (link:{java9-javadoc}/java/util/Calendar.html#isWeekDateSupported%2D%2D[java 9]) -* ++[[painless-api-reference-Calendar-roll-2]]void link:{java8-javadoc}/java/util/Calendar.html#roll%2Dint%2Dint%2D[roll](int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#roll%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-set-2]]void link:{java8-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2D[set](int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-set-3]]void link:{java8-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2Dint%2D[set](int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-set-5]]void link:{java8-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2Dint%2Dint%2Dint%2D[set](int, int, int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-set-6]]void link:{java8-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[set](int, int, int, int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#set%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-setFirstDayOfWeek-1]]void link:{java8-javadoc}/java/util/Calendar.html#setFirstDayOfWeek%2Dint%2D[setFirstDayOfWeek](int)++ (link:{java9-javadoc}/java/util/Calendar.html#setFirstDayOfWeek%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-setLenient-1]]void link:{java8-javadoc}/java/util/Calendar.html#setLenient%2Dboolean%2D[setLenient](boolean)++ (link:{java9-javadoc}/java/util/Calendar.html#setLenient%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Calendar-setMinimalDaysInFirstWeek-1]]void link:{java8-javadoc}/java/util/Calendar.html#setMinimalDaysInFirstWeek%2Dint%2D[setMinimalDaysInFirstWeek](int)++ (link:{java9-javadoc}/java/util/Calendar.html#setMinimalDaysInFirstWeek%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-setTime-1]]void link:{java8-javadoc}/java/util/Calendar.html#setTime%2Djava.util.Date%2D[setTime](<>)++ (link:{java9-javadoc}/java/util/Calendar.html#setTime%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-Calendar-setTimeInMillis-1]]void link:{java8-javadoc}/java/util/Calendar.html#setTimeInMillis%2Dlong%2D[setTimeInMillis](long)++ (link:{java9-javadoc}/java/util/Calendar.html#setTimeInMillis%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Calendar-setTimeZone-1]]void link:{java8-javadoc}/java/util/Calendar.html#setTimeZone%2Djava.util.TimeZone%2D[setTimeZone](<>)++ (link:{java9-javadoc}/java/util/Calendar.html#setTimeZone%2Djava.util.TimeZone%2D[java 9]) -* ++[[painless-api-reference-Calendar-setWeekDate-3]]void link:{java8-javadoc}/java/util/Calendar.html#setWeekDate%2Dint%2Dint%2Dint%2D[setWeekDate](int, int, int)++ (link:{java9-javadoc}/java/util/Calendar.html#setWeekDate%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Calendar-toInstant-0]]<> link:{java8-javadoc}/java/util/Calendar.html#toInstant%2D%2D[toInstant]()++ (link:{java9-javadoc}/java/util/Calendar.html#toInstant%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/CharSequence.asciidoc b/docs/painless/painless-api-reference/CharSequence.asciidoc deleted file mode 100644 index 2618a22881e02..0000000000000 --- a/docs/painless/painless-api-reference/CharSequence.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-CharSequence]]++CharSequence++:: -* ++[[painless-api-reference-CharSequence-charAt-1]]char link:{java8-javadoc}/java/lang/CharSequence.html#charAt%2Dint%2D[charAt](int)++ (link:{java9-javadoc}/java/lang/CharSequence.html#charAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-CharSequence-chars-0]]<> link:{java8-javadoc}/java/lang/CharSequence.html#chars%2D%2D[chars]()++ (link:{java9-javadoc}/java/lang/CharSequence.html#chars%2D%2D[java 9]) -* ++[[painless-api-reference-CharSequence-codePoints-0]]<> link:{java8-javadoc}/java/lang/CharSequence.html#codePoints%2D%2D[codePoints]()++ (link:{java9-javadoc}/java/lang/CharSequence.html#codePoints%2D%2D[java 9]) -* ++[[painless-api-reference-CharSequence-length-0]]int link:{java8-javadoc}/java/lang/CharSequence.html#length%2D%2D[length]()++ (link:{java9-javadoc}/java/lang/CharSequence.html#length%2D%2D[java 9]) -* ++[[painless-api-reference-CharSequence-replaceAll-2]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#replaceAll%2Djava.lang.CharSequence%2Djava.util.regex.Pattern%2Djava.util.function.Function%2D[replaceAll](<>, <>)++ -* ++[[painless-api-reference-CharSequence-replaceFirst-2]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#replaceFirst%2Djava.lang.CharSequence%2Djava.util.regex.Pattern%2Djava.util.function.Function%2D[replaceFirst](<>, <>)++ -* ++[[painless-api-reference-CharSequence-subSequence-2]]<> link:{java8-javadoc}/java/lang/CharSequence.html#subSequence%2Dint%2Dint%2D[subSequence](int, int)++ (link:{java9-javadoc}/java/lang/CharSequence.html#subSequence%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-CharSequence-toString-0]]<> link:{java8-javadoc}/java/lang/CharSequence.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/lang/CharSequence.html#toString%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Character.Subset.asciidoc b/docs/painless/painless-api-reference/Character.Subset.asciidoc deleted file mode 100644 index e8dff322e6e27..0000000000000 --- a/docs/painless/painless-api-reference/Character.Subset.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Character-Subset]]++Character.Subset++:: -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Character.UnicodeBlock.asciidoc b/docs/painless/painless-api-reference/Character.UnicodeBlock.asciidoc deleted file mode 100644 index 6fdf81adfc922..0000000000000 --- a/docs/painless/painless-api-reference/Character.UnicodeBlock.asciidoc +++ /dev/null @@ -1,229 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Character-UnicodeBlock]]++Character.UnicodeBlock++:: -** [[painless-api-reference-Character-UnicodeBlock-AEGEAN_NUMBERS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#AEGEAN_NUMBERS[AEGEAN_NUMBERS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#AEGEAN_NUMBERS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ALCHEMICAL_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ALCHEMICAL_SYMBOLS[ALCHEMICAL_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ALCHEMICAL_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ALPHABETIC_PRESENTATION_FORMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ALPHABETIC_PRESENTATION_FORMS[ALPHABETIC_PRESENTATION_FORMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ALPHABETIC_PRESENTATION_FORMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ANCIENT_GREEK_MUSICAL_NOTATION]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ANCIENT_GREEK_MUSICAL_NOTATION[ANCIENT_GREEK_MUSICAL_NOTATION] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ANCIENT_GREEK_MUSICAL_NOTATION[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ANCIENT_GREEK_NUMBERS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ANCIENT_GREEK_NUMBERS[ANCIENT_GREEK_NUMBERS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ANCIENT_GREEK_NUMBERS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ANCIENT_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ANCIENT_SYMBOLS[ANCIENT_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ANCIENT_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARABIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC[ARABIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARABIC_EXTENDED_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_EXTENDED_A[ARABIC_EXTENDED_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_EXTENDED_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARABIC_MATHEMATICAL_ALPHABETIC_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_MATHEMATICAL_ALPHABETIC_SYMBOLS[ARABIC_MATHEMATICAL_ALPHABETIC_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_MATHEMATICAL_ALPHABETIC_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARABIC_PRESENTATION_FORMS_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_PRESENTATION_FORMS_A[ARABIC_PRESENTATION_FORMS_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_PRESENTATION_FORMS_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARABIC_PRESENTATION_FORMS_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_PRESENTATION_FORMS_B[ARABIC_PRESENTATION_FORMS_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_PRESENTATION_FORMS_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARABIC_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_SUPPLEMENT[ARABIC_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARABIC_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARMENIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARMENIAN[ARMENIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARMENIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ARROWS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ARROWS[ARROWS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ARROWS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-AVESTAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#AVESTAN[AVESTAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#AVESTAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BALINESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BALINESE[BALINESE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BALINESE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BAMUM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BAMUM[BAMUM] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BAMUM[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BAMUM_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BAMUM_SUPPLEMENT[BAMUM_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BAMUM_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BASIC_LATIN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BASIC_LATIN[BASIC_LATIN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BASIC_LATIN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BATAK]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BATAK[BATAK] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BATAK[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BENGALI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BENGALI[BENGALI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BENGALI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BLOCK_ELEMENTS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BLOCK_ELEMENTS[BLOCK_ELEMENTS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BLOCK_ELEMENTS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BOPOMOFO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BOPOMOFO[BOPOMOFO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BOPOMOFO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BOPOMOFO_EXTENDED]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BOPOMOFO_EXTENDED[BOPOMOFO_EXTENDED] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BOPOMOFO_EXTENDED[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BOX_DRAWING]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BOX_DRAWING[BOX_DRAWING] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BOX_DRAWING[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BRAHMI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BRAHMI[BRAHMI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BRAHMI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BRAILLE_PATTERNS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BRAILLE_PATTERNS[BRAILLE_PATTERNS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BRAILLE_PATTERNS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BUGINESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BUGINESE[BUGINESE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BUGINESE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BUHID]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BUHID[BUHID] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BUHID[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-BYZANTINE_MUSICAL_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#BYZANTINE_MUSICAL_SYMBOLS[BYZANTINE_MUSICAL_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#BYZANTINE_MUSICAL_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CARIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CARIAN[CARIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CARIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CHAKMA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CHAKMA[CHAKMA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CHAKMA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CHAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CHAM[CHAM] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CHAM[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CHEROKEE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CHEROKEE[CHEROKEE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CHEROKEE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_COMPATIBILITY]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY[CJK_COMPATIBILITY] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_COMPATIBILITY_FORMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY_FORMS[CJK_COMPATIBILITY_FORMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY_FORMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_COMPATIBILITY_IDEOGRAPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY_IDEOGRAPHS[CJK_COMPATIBILITY_IDEOGRAPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY_IDEOGRAPHS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_COMPATIBILITY_IDEOGRAPHS_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY_IDEOGRAPHS_SUPPLEMENT[CJK_COMPATIBILITY_IDEOGRAPHS_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_COMPATIBILITY_IDEOGRAPHS_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_RADICALS_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_RADICALS_SUPPLEMENT[CJK_RADICALS_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_RADICALS_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_STROKES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_STROKES[CJK_STROKES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_STROKES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_SYMBOLS_AND_PUNCTUATION]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_SYMBOLS_AND_PUNCTUATION[CJK_SYMBOLS_AND_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_SYMBOLS_AND_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_UNIFIED_IDEOGRAPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS[CJK_UNIFIED_IDEOGRAPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_UNIFIED_IDEOGRAPHS_EXTENSION_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_A[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_UNIFIED_IDEOGRAPHS_EXTENSION_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_B[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_UNIFIED_IDEOGRAPHS_EXTENSION_C]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_C[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_C] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_C[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CJK_UNIFIED_IDEOGRAPHS_EXTENSION_D]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_D[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_D] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_D[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COMBINING_DIACRITICAL_MARKS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_DIACRITICAL_MARKS[COMBINING_DIACRITICAL_MARKS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_DIACRITICAL_MARKS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COMBINING_DIACRITICAL_MARKS_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_DIACRITICAL_MARKS_SUPPLEMENT[COMBINING_DIACRITICAL_MARKS_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_DIACRITICAL_MARKS_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COMBINING_HALF_MARKS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_HALF_MARKS[COMBINING_HALF_MARKS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_HALF_MARKS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COMBINING_MARKS_FOR_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_MARKS_FOR_SYMBOLS[COMBINING_MARKS_FOR_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COMBINING_MARKS_FOR_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COMMON_INDIC_NUMBER_FORMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COMMON_INDIC_NUMBER_FORMS[COMMON_INDIC_NUMBER_FORMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COMMON_INDIC_NUMBER_FORMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CONTROL_PICTURES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CONTROL_PICTURES[CONTROL_PICTURES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CONTROL_PICTURES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COPTIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COPTIC[COPTIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COPTIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-COUNTING_ROD_NUMERALS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#COUNTING_ROD_NUMERALS[COUNTING_ROD_NUMERALS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#COUNTING_ROD_NUMERALS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CUNEIFORM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CUNEIFORM[CUNEIFORM] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CUNEIFORM[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CUNEIFORM_NUMBERS_AND_PUNCTUATION]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CUNEIFORM_NUMBERS_AND_PUNCTUATION[CUNEIFORM_NUMBERS_AND_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CUNEIFORM_NUMBERS_AND_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CURRENCY_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CURRENCY_SYMBOLS[CURRENCY_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CURRENCY_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CYPRIOT_SYLLABARY]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CYPRIOT_SYLLABARY[CYPRIOT_SYLLABARY] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CYPRIOT_SYLLABARY[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CYRILLIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC[CYRILLIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CYRILLIC_EXTENDED_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC_EXTENDED_A[CYRILLIC_EXTENDED_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC_EXTENDED_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CYRILLIC_EXTENDED_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC_EXTENDED_B[CYRILLIC_EXTENDED_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC_EXTENDED_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-CYRILLIC_SUPPLEMENTARY]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC_SUPPLEMENTARY[CYRILLIC_SUPPLEMENTARY] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#CYRILLIC_SUPPLEMENTARY[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-DESERET]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#DESERET[DESERET] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#DESERET[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-DEVANAGARI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#DEVANAGARI[DEVANAGARI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#DEVANAGARI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-DEVANAGARI_EXTENDED]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#DEVANAGARI_EXTENDED[DEVANAGARI_EXTENDED] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#DEVANAGARI_EXTENDED[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-DINGBATS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#DINGBATS[DINGBATS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#DINGBATS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-DOMINO_TILES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#DOMINO_TILES[DOMINO_TILES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#DOMINO_TILES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-EGYPTIAN_HIEROGLYPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#EGYPTIAN_HIEROGLYPHS[EGYPTIAN_HIEROGLYPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#EGYPTIAN_HIEROGLYPHS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-EMOTICONS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#EMOTICONS[EMOTICONS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#EMOTICONS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ENCLOSED_ALPHANUMERICS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_ALPHANUMERICS[ENCLOSED_ALPHANUMERICS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_ALPHANUMERICS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ENCLOSED_ALPHANUMERIC_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_ALPHANUMERIC_SUPPLEMENT[ENCLOSED_ALPHANUMERIC_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_ALPHANUMERIC_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ENCLOSED_CJK_LETTERS_AND_MONTHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_CJK_LETTERS_AND_MONTHS[ENCLOSED_CJK_LETTERS_AND_MONTHS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_CJK_LETTERS_AND_MONTHS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ENCLOSED_IDEOGRAPHIC_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_IDEOGRAPHIC_SUPPLEMENT[ENCLOSED_IDEOGRAPHIC_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ENCLOSED_IDEOGRAPHIC_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ETHIOPIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC[ETHIOPIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ETHIOPIC_EXTENDED]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC_EXTENDED[ETHIOPIC_EXTENDED] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC_EXTENDED[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ETHIOPIC_EXTENDED_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC_EXTENDED_A[ETHIOPIC_EXTENDED_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC_EXTENDED_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ETHIOPIC_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC_SUPPLEMENT[ETHIOPIC_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ETHIOPIC_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GENERAL_PUNCTUATION]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GENERAL_PUNCTUATION[GENERAL_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GENERAL_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GEOMETRIC_SHAPES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GEOMETRIC_SHAPES[GEOMETRIC_SHAPES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GEOMETRIC_SHAPES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GEORGIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GEORGIAN[GEORGIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GEORGIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GEORGIAN_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GEORGIAN_SUPPLEMENT[GEORGIAN_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GEORGIAN_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GLAGOLITIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GLAGOLITIC[GLAGOLITIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GLAGOLITIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GOTHIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GOTHIC[GOTHIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GOTHIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GREEK]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GREEK[GREEK] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GREEK[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GREEK_EXTENDED]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GREEK_EXTENDED[GREEK_EXTENDED] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GREEK_EXTENDED[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GUJARATI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GUJARATI[GUJARATI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GUJARATI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-GURMUKHI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#GURMUKHI[GURMUKHI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#GURMUKHI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HALFWIDTH_AND_FULLWIDTH_FORMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HALFWIDTH_AND_FULLWIDTH_FORMS[HALFWIDTH_AND_FULLWIDTH_FORMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HALFWIDTH_AND_FULLWIDTH_FORMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HANGUL_COMPATIBILITY_JAMO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_COMPATIBILITY_JAMO[HANGUL_COMPATIBILITY_JAMO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_COMPATIBILITY_JAMO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HANGUL_JAMO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_JAMO[HANGUL_JAMO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_JAMO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HANGUL_JAMO_EXTENDED_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_JAMO_EXTENDED_A[HANGUL_JAMO_EXTENDED_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_JAMO_EXTENDED_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HANGUL_JAMO_EXTENDED_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_JAMO_EXTENDED_B[HANGUL_JAMO_EXTENDED_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_JAMO_EXTENDED_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HANGUL_SYLLABLES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_SYLLABLES[HANGUL_SYLLABLES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HANGUL_SYLLABLES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HANUNOO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HANUNOO[HANUNOO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HANUNOO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HEBREW]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HEBREW[HEBREW] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HEBREW[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HIGH_PRIVATE_USE_SURROGATES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HIGH_PRIVATE_USE_SURROGATES[HIGH_PRIVATE_USE_SURROGATES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HIGH_PRIVATE_USE_SURROGATES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HIGH_SURROGATES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HIGH_SURROGATES[HIGH_SURROGATES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HIGH_SURROGATES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-HIRAGANA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#HIRAGANA[HIRAGANA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#HIRAGANA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-IDEOGRAPHIC_DESCRIPTION_CHARACTERS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#IDEOGRAPHIC_DESCRIPTION_CHARACTERS[IDEOGRAPHIC_DESCRIPTION_CHARACTERS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#IDEOGRAPHIC_DESCRIPTION_CHARACTERS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-IMPERIAL_ARAMAIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#IMPERIAL_ARAMAIC[IMPERIAL_ARAMAIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#IMPERIAL_ARAMAIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-INSCRIPTIONAL_PAHLAVI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#INSCRIPTIONAL_PAHLAVI[INSCRIPTIONAL_PAHLAVI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#INSCRIPTIONAL_PAHLAVI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-INSCRIPTIONAL_PARTHIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#INSCRIPTIONAL_PARTHIAN[INSCRIPTIONAL_PARTHIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#INSCRIPTIONAL_PARTHIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-IPA_EXTENSIONS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#IPA_EXTENSIONS[IPA_EXTENSIONS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#IPA_EXTENSIONS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-JAVANESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#JAVANESE[JAVANESE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#JAVANESE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KAITHI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KAITHI[KAITHI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KAITHI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KANA_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KANA_SUPPLEMENT[KANA_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KANA_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KANBUN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KANBUN[KANBUN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KANBUN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KANGXI_RADICALS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KANGXI_RADICALS[KANGXI_RADICALS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KANGXI_RADICALS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KANNADA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KANNADA[KANNADA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KANNADA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KATAKANA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KATAKANA[KATAKANA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KATAKANA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KATAKANA_PHONETIC_EXTENSIONS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KATAKANA_PHONETIC_EXTENSIONS[KATAKANA_PHONETIC_EXTENSIONS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KATAKANA_PHONETIC_EXTENSIONS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KAYAH_LI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KAYAH_LI[KAYAH_LI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KAYAH_LI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KHAROSHTHI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KHAROSHTHI[KHAROSHTHI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KHAROSHTHI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KHMER]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KHMER[KHMER] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KHMER[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-KHMER_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#KHMER_SYMBOLS[KHMER_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#KHMER_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LAO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LAO[LAO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LAO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LATIN_1_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_1_SUPPLEMENT[LATIN_1_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_1_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LATIN_EXTENDED_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_A[LATIN_EXTENDED_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LATIN_EXTENDED_ADDITIONAL]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_ADDITIONAL[LATIN_EXTENDED_ADDITIONAL] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_ADDITIONAL[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LATIN_EXTENDED_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_B[LATIN_EXTENDED_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LATIN_EXTENDED_C]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_C[LATIN_EXTENDED_C] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_C[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LATIN_EXTENDED_D]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_D[LATIN_EXTENDED_D] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LATIN_EXTENDED_D[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LEPCHA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LEPCHA[LEPCHA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LEPCHA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LETTERLIKE_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LETTERLIKE_SYMBOLS[LETTERLIKE_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LETTERLIKE_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LIMBU]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LIMBU[LIMBU] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LIMBU[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LINEAR_B_IDEOGRAMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LINEAR_B_IDEOGRAMS[LINEAR_B_IDEOGRAMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LINEAR_B_IDEOGRAMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LINEAR_B_SYLLABARY]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LINEAR_B_SYLLABARY[LINEAR_B_SYLLABARY] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LINEAR_B_SYLLABARY[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LISU]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LISU[LISU] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LISU[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LOW_SURROGATES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LOW_SURROGATES[LOW_SURROGATES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LOW_SURROGATES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LYCIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LYCIAN[LYCIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LYCIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-LYDIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#LYDIAN[LYDIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#LYDIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MAHJONG_TILES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MAHJONG_TILES[MAHJONG_TILES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MAHJONG_TILES[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MALAYALAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MALAYALAM[MALAYALAM] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MALAYALAM[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MANDAIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MANDAIC[MANDAIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MANDAIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MATHEMATICAL_ALPHANUMERIC_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MATHEMATICAL_ALPHANUMERIC_SYMBOLS[MATHEMATICAL_ALPHANUMERIC_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MATHEMATICAL_ALPHANUMERIC_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MATHEMATICAL_OPERATORS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MATHEMATICAL_OPERATORS[MATHEMATICAL_OPERATORS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MATHEMATICAL_OPERATORS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MEETEI_MAYEK]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MEETEI_MAYEK[MEETEI_MAYEK] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MEETEI_MAYEK[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MEETEI_MAYEK_EXTENSIONS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MEETEI_MAYEK_EXTENSIONS[MEETEI_MAYEK_EXTENSIONS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MEETEI_MAYEK_EXTENSIONS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MEROITIC_CURSIVE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MEROITIC_CURSIVE[MEROITIC_CURSIVE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MEROITIC_CURSIVE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MEROITIC_HIEROGLYPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MEROITIC_HIEROGLYPHS[MEROITIC_HIEROGLYPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MEROITIC_HIEROGLYPHS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MIAO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MIAO[MIAO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MIAO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MISCELLANEOUS_MATHEMATICAL_SYMBOLS_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_MATHEMATICAL_SYMBOLS_A[MISCELLANEOUS_MATHEMATICAL_SYMBOLS_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_MATHEMATICAL_SYMBOLS_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MISCELLANEOUS_MATHEMATICAL_SYMBOLS_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_MATHEMATICAL_SYMBOLS_B[MISCELLANEOUS_MATHEMATICAL_SYMBOLS_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_MATHEMATICAL_SYMBOLS_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MISCELLANEOUS_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_SYMBOLS[MISCELLANEOUS_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MISCELLANEOUS_SYMBOLS_AND_ARROWS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_SYMBOLS_AND_ARROWS[MISCELLANEOUS_SYMBOLS_AND_ARROWS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_SYMBOLS_AND_ARROWS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MISCELLANEOUS_SYMBOLS_AND_PICTOGRAPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_SYMBOLS_AND_PICTOGRAPHS[MISCELLANEOUS_SYMBOLS_AND_PICTOGRAPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_SYMBOLS_AND_PICTOGRAPHS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MISCELLANEOUS_TECHNICAL]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_TECHNICAL[MISCELLANEOUS_TECHNICAL] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MISCELLANEOUS_TECHNICAL[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MODIFIER_TONE_LETTERS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MODIFIER_TONE_LETTERS[MODIFIER_TONE_LETTERS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MODIFIER_TONE_LETTERS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MONGOLIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MONGOLIAN[MONGOLIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MONGOLIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MUSICAL_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MUSICAL_SYMBOLS[MUSICAL_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MUSICAL_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MYANMAR]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MYANMAR[MYANMAR] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MYANMAR[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-MYANMAR_EXTENDED_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#MYANMAR_EXTENDED_A[MYANMAR_EXTENDED_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#MYANMAR_EXTENDED_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-NEW_TAI_LUE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#NEW_TAI_LUE[NEW_TAI_LUE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#NEW_TAI_LUE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-NKO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#NKO[NKO] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#NKO[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-NUMBER_FORMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#NUMBER_FORMS[NUMBER_FORMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#NUMBER_FORMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OGHAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OGHAM[OGHAM] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OGHAM[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OLD_ITALIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_ITALIC[OLD_ITALIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_ITALIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OLD_PERSIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_PERSIAN[OLD_PERSIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_PERSIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OLD_SOUTH_ARABIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_SOUTH_ARABIAN[OLD_SOUTH_ARABIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_SOUTH_ARABIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OLD_TURKIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_TURKIC[OLD_TURKIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OLD_TURKIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OL_CHIKI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OL_CHIKI[OL_CHIKI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OL_CHIKI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OPTICAL_CHARACTER_RECOGNITION]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OPTICAL_CHARACTER_RECOGNITION[OPTICAL_CHARACTER_RECOGNITION] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OPTICAL_CHARACTER_RECOGNITION[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-ORIYA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#ORIYA[ORIYA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#ORIYA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-OSMANYA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#OSMANYA[OSMANYA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#OSMANYA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PHAGS_PA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PHAGS_PA[PHAGS_PA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PHAGS_PA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PHAISTOS_DISC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PHAISTOS_DISC[PHAISTOS_DISC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PHAISTOS_DISC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PHOENICIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PHOENICIAN[PHOENICIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PHOENICIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PHONETIC_EXTENSIONS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PHONETIC_EXTENSIONS[PHONETIC_EXTENSIONS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PHONETIC_EXTENSIONS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PHONETIC_EXTENSIONS_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PHONETIC_EXTENSIONS_SUPPLEMENT[PHONETIC_EXTENSIONS_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PHONETIC_EXTENSIONS_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PLAYING_CARDS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PLAYING_CARDS[PLAYING_CARDS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PLAYING_CARDS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-PRIVATE_USE_AREA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#PRIVATE_USE_AREA[PRIVATE_USE_AREA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#PRIVATE_USE_AREA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-REJANG]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#REJANG[REJANG] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#REJANG[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-RUMI_NUMERAL_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#RUMI_NUMERAL_SYMBOLS[RUMI_NUMERAL_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#RUMI_NUMERAL_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-RUNIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#RUNIC[RUNIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#RUNIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SAMARITAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SAMARITAN[SAMARITAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SAMARITAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SAURASHTRA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SAURASHTRA[SAURASHTRA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SAURASHTRA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SHARADA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SHARADA[SHARADA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SHARADA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SHAVIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SHAVIAN[SHAVIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SHAVIAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SINHALA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SINHALA[SINHALA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SINHALA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SMALL_FORM_VARIANTS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SMALL_FORM_VARIANTS[SMALL_FORM_VARIANTS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SMALL_FORM_VARIANTS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SORA_SOMPENG]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SORA_SOMPENG[SORA_SOMPENG] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SORA_SOMPENG[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SPACING_MODIFIER_LETTERS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SPACING_MODIFIER_LETTERS[SPACING_MODIFIER_LETTERS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SPACING_MODIFIER_LETTERS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SPECIALS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SPECIALS[SPECIALS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SPECIALS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUNDANESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUNDANESE[SUNDANESE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUNDANESE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUNDANESE_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUNDANESE_SUPPLEMENT[SUNDANESE_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUNDANESE_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPERSCRIPTS_AND_SUBSCRIPTS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPERSCRIPTS_AND_SUBSCRIPTS[SUPERSCRIPTS_AND_SUBSCRIPTS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPERSCRIPTS_AND_SUBSCRIPTS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPPLEMENTAL_ARROWS_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_ARROWS_A[SUPPLEMENTAL_ARROWS_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_ARROWS_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPPLEMENTAL_ARROWS_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_ARROWS_B[SUPPLEMENTAL_ARROWS_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_ARROWS_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPPLEMENTAL_MATHEMATICAL_OPERATORS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_MATHEMATICAL_OPERATORS[SUPPLEMENTAL_MATHEMATICAL_OPERATORS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_MATHEMATICAL_OPERATORS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPPLEMENTAL_PUNCTUATION]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_PUNCTUATION[SUPPLEMENTAL_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTAL_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPPLEMENTARY_PRIVATE_USE_AREA_A]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTARY_PRIVATE_USE_AREA_A[SUPPLEMENTARY_PRIVATE_USE_AREA_A] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTARY_PRIVATE_USE_AREA_A[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SUPPLEMENTARY_PRIVATE_USE_AREA_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTARY_PRIVATE_USE_AREA_B[SUPPLEMENTARY_PRIVATE_USE_AREA_B] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SUPPLEMENTARY_PRIVATE_USE_AREA_B[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SYLOTI_NAGRI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SYLOTI_NAGRI[SYLOTI_NAGRI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SYLOTI_NAGRI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-SYRIAC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#SYRIAC[SYRIAC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#SYRIAC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAGALOG]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAGALOG[TAGALOG] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAGALOG[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAGBANWA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAGBANWA[TAGBANWA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAGBANWA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAGS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAGS[TAGS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAGS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAI_LE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_LE[TAI_LE] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_LE[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAI_THAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_THAM[TAI_THAM] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_THAM[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAI_VIET]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_VIET[TAI_VIET] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_VIET[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAI_XUAN_JING_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_XUAN_JING_SYMBOLS[TAI_XUAN_JING_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAI_XUAN_JING_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAKRI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAKRI[TAKRI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAKRI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TAMIL]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TAMIL[TAMIL] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TAMIL[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TELUGU]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TELUGU[TELUGU] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TELUGU[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-THAANA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#THAANA[THAANA] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#THAANA[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-THAI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#THAI[THAI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#THAI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TIBETAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TIBETAN[TIBETAN] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TIBETAN[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TIFINAGH]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TIFINAGH[TIFINAGH] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TIFINAGH[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-TRANSPORT_AND_MAP_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#TRANSPORT_AND_MAP_SYMBOLS[TRANSPORT_AND_MAP_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#TRANSPORT_AND_MAP_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-UGARITIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#UGARITIC[UGARITIC] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#UGARITIC[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS[UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS_EXTENDED]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS_EXTENDED[UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS_EXTENDED] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS_EXTENDED[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-VAI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#VAI[VAI] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#VAI[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-VARIATION_SELECTORS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#VARIATION_SELECTORS[VARIATION_SELECTORS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#VARIATION_SELECTORS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-VARIATION_SELECTORS_SUPPLEMENT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#VARIATION_SELECTORS_SUPPLEMENT[VARIATION_SELECTORS_SUPPLEMENT] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#VARIATION_SELECTORS_SUPPLEMENT[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-VEDIC_EXTENSIONS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#VEDIC_EXTENSIONS[VEDIC_EXTENSIONS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#VEDIC_EXTENSIONS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-VERTICAL_FORMS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#VERTICAL_FORMS[VERTICAL_FORMS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#VERTICAL_FORMS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-YIJING_HEXAGRAM_SYMBOLS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#YIJING_HEXAGRAM_SYMBOLS[YIJING_HEXAGRAM_SYMBOLS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#YIJING_HEXAGRAM_SYMBOLS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-YI_RADICALS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#YI_RADICALS[YI_RADICALS] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#YI_RADICALS[java 9]) -** [[painless-api-reference-Character-UnicodeBlock-YI_SYLLABLES]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#YI_SYLLABLES[YI_SYLLABLES] (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#YI_SYLLABLES[java 9]) -* ++[[painless-api-reference-Character-UnicodeBlock-forName-1]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#forName%2Djava.lang.String%2D[forName](<>)++ (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#forName%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Character-UnicodeBlock-of-1]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeBlock.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/lang/Character.UnicodeBlock.html#of%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Character.UnicodeScript.asciidoc b/docs/painless/painless-api-reference/Character.UnicodeScript.asciidoc deleted file mode 100644 index 0e9ff73e1e53d..0000000000000 --- a/docs/painless/painless-api-reference/Character.UnicodeScript.asciidoc +++ /dev/null @@ -1,114 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Character-UnicodeScript]]++Character.UnicodeScript++:: -** [[painless-api-reference-Character-UnicodeScript-ARABIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#ARABIC[ARABIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#ARABIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-ARMENIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#ARMENIAN[ARMENIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#ARMENIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-AVESTAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#AVESTAN[AVESTAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#AVESTAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BALINESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BALINESE[BALINESE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BALINESE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BAMUM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BAMUM[BAMUM] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BAMUM[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BATAK]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BATAK[BATAK] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BATAK[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BENGALI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BENGALI[BENGALI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BENGALI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BOPOMOFO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BOPOMOFO[BOPOMOFO] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BOPOMOFO[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BRAHMI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BRAHMI[BRAHMI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BRAHMI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BRAILLE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BRAILLE[BRAILLE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BRAILLE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BUGINESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BUGINESE[BUGINESE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BUGINESE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-BUHID]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#BUHID[BUHID] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#BUHID[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CANADIAN_ABORIGINAL]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CANADIAN_ABORIGINAL[CANADIAN_ABORIGINAL] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CANADIAN_ABORIGINAL[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CARIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CARIAN[CARIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CARIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CHAKMA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CHAKMA[CHAKMA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CHAKMA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CHAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CHAM[CHAM] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CHAM[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CHEROKEE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CHEROKEE[CHEROKEE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CHEROKEE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-COMMON]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#COMMON[COMMON] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#COMMON[java 9]) -** [[painless-api-reference-Character-UnicodeScript-COPTIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#COPTIC[COPTIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#COPTIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CUNEIFORM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CUNEIFORM[CUNEIFORM] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CUNEIFORM[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CYPRIOT]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CYPRIOT[CYPRIOT] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CYPRIOT[java 9]) -** [[painless-api-reference-Character-UnicodeScript-CYRILLIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#CYRILLIC[CYRILLIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#CYRILLIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-DESERET]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#DESERET[DESERET] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#DESERET[java 9]) -** [[painless-api-reference-Character-UnicodeScript-DEVANAGARI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#DEVANAGARI[DEVANAGARI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#DEVANAGARI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-EGYPTIAN_HIEROGLYPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#EGYPTIAN_HIEROGLYPHS[EGYPTIAN_HIEROGLYPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#EGYPTIAN_HIEROGLYPHS[java 9]) -** [[painless-api-reference-Character-UnicodeScript-ETHIOPIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#ETHIOPIC[ETHIOPIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#ETHIOPIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-GEORGIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#GEORGIAN[GEORGIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#GEORGIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-GLAGOLITIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#GLAGOLITIC[GLAGOLITIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#GLAGOLITIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-GOTHIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#GOTHIC[GOTHIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#GOTHIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-GREEK]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#GREEK[GREEK] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#GREEK[java 9]) -** [[painless-api-reference-Character-UnicodeScript-GUJARATI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#GUJARATI[GUJARATI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#GUJARATI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-GURMUKHI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#GURMUKHI[GURMUKHI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#GURMUKHI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-HAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#HAN[HAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#HAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-HANGUL]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#HANGUL[HANGUL] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#HANGUL[java 9]) -** [[painless-api-reference-Character-UnicodeScript-HANUNOO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#HANUNOO[HANUNOO] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#HANUNOO[java 9]) -** [[painless-api-reference-Character-UnicodeScript-HEBREW]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#HEBREW[HEBREW] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#HEBREW[java 9]) -** [[painless-api-reference-Character-UnicodeScript-HIRAGANA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#HIRAGANA[HIRAGANA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#HIRAGANA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-IMPERIAL_ARAMAIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#IMPERIAL_ARAMAIC[IMPERIAL_ARAMAIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#IMPERIAL_ARAMAIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-INHERITED]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#INHERITED[INHERITED] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#INHERITED[java 9]) -** [[painless-api-reference-Character-UnicodeScript-INSCRIPTIONAL_PAHLAVI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#INSCRIPTIONAL_PAHLAVI[INSCRIPTIONAL_PAHLAVI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#INSCRIPTIONAL_PAHLAVI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-INSCRIPTIONAL_PARTHIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#INSCRIPTIONAL_PARTHIAN[INSCRIPTIONAL_PARTHIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#INSCRIPTIONAL_PARTHIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-JAVANESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#JAVANESE[JAVANESE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#JAVANESE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-KAITHI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#KAITHI[KAITHI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#KAITHI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-KANNADA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#KANNADA[KANNADA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#KANNADA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-KATAKANA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#KATAKANA[KATAKANA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#KATAKANA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-KAYAH_LI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#KAYAH_LI[KAYAH_LI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#KAYAH_LI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-KHAROSHTHI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#KHAROSHTHI[KHAROSHTHI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#KHAROSHTHI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-KHMER]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#KHMER[KHMER] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#KHMER[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LAO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LAO[LAO] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LAO[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LATIN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LATIN[LATIN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LATIN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LEPCHA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LEPCHA[LEPCHA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LEPCHA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LIMBU]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LIMBU[LIMBU] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LIMBU[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LINEAR_B]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LINEAR_B[LINEAR_B] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LINEAR_B[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LISU]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LISU[LISU] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LISU[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LYCIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LYCIAN[LYCIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LYCIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-LYDIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#LYDIAN[LYDIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#LYDIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MALAYALAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MALAYALAM[MALAYALAM] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MALAYALAM[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MANDAIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MANDAIC[MANDAIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MANDAIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MEETEI_MAYEK]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MEETEI_MAYEK[MEETEI_MAYEK] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MEETEI_MAYEK[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MEROITIC_CURSIVE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MEROITIC_CURSIVE[MEROITIC_CURSIVE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MEROITIC_CURSIVE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MEROITIC_HIEROGLYPHS]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MEROITIC_HIEROGLYPHS[MEROITIC_HIEROGLYPHS] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MEROITIC_HIEROGLYPHS[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MIAO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MIAO[MIAO] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MIAO[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MONGOLIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MONGOLIAN[MONGOLIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MONGOLIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-MYANMAR]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#MYANMAR[MYANMAR] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#MYANMAR[java 9]) -** [[painless-api-reference-Character-UnicodeScript-NEW_TAI_LUE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#NEW_TAI_LUE[NEW_TAI_LUE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#NEW_TAI_LUE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-NKO]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#NKO[NKO] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#NKO[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OGHAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OGHAM[OGHAM] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OGHAM[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OLD_ITALIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OLD_ITALIC[OLD_ITALIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OLD_ITALIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OLD_PERSIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OLD_PERSIAN[OLD_PERSIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OLD_PERSIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OLD_SOUTH_ARABIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OLD_SOUTH_ARABIAN[OLD_SOUTH_ARABIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OLD_SOUTH_ARABIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OLD_TURKIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OLD_TURKIC[OLD_TURKIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OLD_TURKIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OL_CHIKI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OL_CHIKI[OL_CHIKI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OL_CHIKI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-ORIYA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#ORIYA[ORIYA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#ORIYA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-OSMANYA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#OSMANYA[OSMANYA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#OSMANYA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-PHAGS_PA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#PHAGS_PA[PHAGS_PA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#PHAGS_PA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-PHOENICIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#PHOENICIAN[PHOENICIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#PHOENICIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-REJANG]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#REJANG[REJANG] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#REJANG[java 9]) -** [[painless-api-reference-Character-UnicodeScript-RUNIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#RUNIC[RUNIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#RUNIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SAMARITAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SAMARITAN[SAMARITAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SAMARITAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SAURASHTRA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SAURASHTRA[SAURASHTRA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SAURASHTRA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SHARADA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SHARADA[SHARADA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SHARADA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SHAVIAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SHAVIAN[SHAVIAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SHAVIAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SINHALA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SINHALA[SINHALA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SINHALA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SORA_SOMPENG]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SORA_SOMPENG[SORA_SOMPENG] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SORA_SOMPENG[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SUNDANESE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SUNDANESE[SUNDANESE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SUNDANESE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SYLOTI_NAGRI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SYLOTI_NAGRI[SYLOTI_NAGRI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SYLOTI_NAGRI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-SYRIAC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#SYRIAC[SYRIAC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#SYRIAC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAGALOG]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAGALOG[TAGALOG] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAGALOG[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAGBANWA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAGBANWA[TAGBANWA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAGBANWA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAI_LE]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAI_LE[TAI_LE] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAI_LE[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAI_THAM]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAI_THAM[TAI_THAM] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAI_THAM[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAI_VIET]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAI_VIET[TAI_VIET] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAI_VIET[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAKRI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAKRI[TAKRI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAKRI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TAMIL]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TAMIL[TAMIL] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TAMIL[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TELUGU]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TELUGU[TELUGU] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TELUGU[java 9]) -** [[painless-api-reference-Character-UnicodeScript-THAANA]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#THAANA[THAANA] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#THAANA[java 9]) -** [[painless-api-reference-Character-UnicodeScript-THAI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#THAI[THAI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#THAI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TIBETAN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TIBETAN[TIBETAN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TIBETAN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-TIFINAGH]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#TIFINAGH[TIFINAGH] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#TIFINAGH[java 9]) -** [[painless-api-reference-Character-UnicodeScript-UGARITIC]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#UGARITIC[UGARITIC] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#UGARITIC[java 9]) -** [[painless-api-reference-Character-UnicodeScript-UNKNOWN]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#UNKNOWN[UNKNOWN] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#UNKNOWN[java 9]) -** [[painless-api-reference-Character-UnicodeScript-VAI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#VAI[VAI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#VAI[java 9]) -** [[painless-api-reference-Character-UnicodeScript-YI]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#YI[YI] (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#YI[java 9]) -* ++[[painless-api-reference-Character-UnicodeScript-forName-1]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#forName%2Djava.lang.String%2D[forName](<>)++ (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#forName%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Character-UnicodeScript-of-1]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-UnicodeScript-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Character-UnicodeScript-values-0]]static <>[] link:{java8-javadoc}/java/lang/Character.UnicodeScript.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/lang/Character.UnicodeScript.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Character.asciidoc b/docs/painless/painless-api-reference/Character.asciidoc deleted file mode 100644 index 929a4ea3c5ec1..0000000000000 --- a/docs/painless/painless-api-reference/Character.asciidoc +++ /dev/null @@ -1,125 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Character]]++Character++:: -** [[painless-api-reference-Character-BYTES]]static int link:{java8-javadoc}/java/lang/Character.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Character.html#BYTES[java 9]) -** [[painless-api-reference-Character-COMBINING_SPACING_MARK]]static byte link:{java8-javadoc}/java/lang/Character.html#COMBINING_SPACING_MARK[COMBINING_SPACING_MARK] (link:{java9-javadoc}/java/lang/Character.html#COMBINING_SPACING_MARK[java 9]) -** [[painless-api-reference-Character-CONNECTOR_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#CONNECTOR_PUNCTUATION[CONNECTOR_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#CONNECTOR_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-CONTROL]]static byte link:{java8-javadoc}/java/lang/Character.html#CONTROL[CONTROL] (link:{java9-javadoc}/java/lang/Character.html#CONTROL[java 9]) -** [[painless-api-reference-Character-CURRENCY_SYMBOL]]static byte link:{java8-javadoc}/java/lang/Character.html#CURRENCY_SYMBOL[CURRENCY_SYMBOL] (link:{java9-javadoc}/java/lang/Character.html#CURRENCY_SYMBOL[java 9]) -** [[painless-api-reference-Character-DASH_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#DASH_PUNCTUATION[DASH_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#DASH_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-DECIMAL_DIGIT_NUMBER]]static byte link:{java8-javadoc}/java/lang/Character.html#DECIMAL_DIGIT_NUMBER[DECIMAL_DIGIT_NUMBER] (link:{java9-javadoc}/java/lang/Character.html#DECIMAL_DIGIT_NUMBER[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_ARABIC_NUMBER]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_ARABIC_NUMBER[DIRECTIONALITY_ARABIC_NUMBER] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_ARABIC_NUMBER[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_BOUNDARY_NEUTRAL]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_BOUNDARY_NEUTRAL[DIRECTIONALITY_BOUNDARY_NEUTRAL] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_BOUNDARY_NEUTRAL[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_COMMON_NUMBER_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_COMMON_NUMBER_SEPARATOR[DIRECTIONALITY_COMMON_NUMBER_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_COMMON_NUMBER_SEPARATOR[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_EUROPEAN_NUMBER]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER[DIRECTIONALITY_EUROPEAN_NUMBER] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_EUROPEAN_NUMBER_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER_SEPARATOR[DIRECTIONALITY_EUROPEAN_NUMBER_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER_SEPARATOR[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_EUROPEAN_NUMBER_TERMINATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER_TERMINATOR[DIRECTIONALITY_EUROPEAN_NUMBER_TERMINATOR] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER_TERMINATOR[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_LEFT_TO_RIGHT]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT[DIRECTIONALITY_LEFT_TO_RIGHT] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_LEFT_TO_RIGHT_EMBEDDING]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT_EMBEDDING[DIRECTIONALITY_LEFT_TO_RIGHT_EMBEDDING] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT_EMBEDDING[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_LEFT_TO_RIGHT_OVERRIDE]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT_OVERRIDE[DIRECTIONALITY_LEFT_TO_RIGHT_OVERRIDE] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT_OVERRIDE[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_NONSPACING_MARK]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_NONSPACING_MARK[DIRECTIONALITY_NONSPACING_MARK] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_NONSPACING_MARK[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_OTHER_NEUTRALS]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_OTHER_NEUTRALS[DIRECTIONALITY_OTHER_NEUTRALS] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_OTHER_NEUTRALS[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_PARAGRAPH_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_PARAGRAPH_SEPARATOR[DIRECTIONALITY_PARAGRAPH_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_PARAGRAPH_SEPARATOR[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_POP_DIRECTIONAL_FORMAT]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_POP_DIRECTIONAL_FORMAT[DIRECTIONALITY_POP_DIRECTIONAL_FORMAT] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_POP_DIRECTIONAL_FORMAT[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_RIGHT_TO_LEFT]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT[DIRECTIONALITY_RIGHT_TO_LEFT] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_RIGHT_TO_LEFT_ARABIC]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_ARABIC[DIRECTIONALITY_RIGHT_TO_LEFT_ARABIC] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_ARABIC[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_RIGHT_TO_LEFT_EMBEDDING]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_EMBEDDING[DIRECTIONALITY_RIGHT_TO_LEFT_EMBEDDING] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_EMBEDDING[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_RIGHT_TO_LEFT_OVERRIDE]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_OVERRIDE[DIRECTIONALITY_RIGHT_TO_LEFT_OVERRIDE] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_OVERRIDE[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_SEGMENT_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_SEGMENT_SEPARATOR[DIRECTIONALITY_SEGMENT_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_SEGMENT_SEPARATOR[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_UNDEFINED]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_UNDEFINED[DIRECTIONALITY_UNDEFINED] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_UNDEFINED[java 9]) -** [[painless-api-reference-Character-DIRECTIONALITY_WHITESPACE]]static byte link:{java8-javadoc}/java/lang/Character.html#DIRECTIONALITY_WHITESPACE[DIRECTIONALITY_WHITESPACE] (link:{java9-javadoc}/java/lang/Character.html#DIRECTIONALITY_WHITESPACE[java 9]) -** [[painless-api-reference-Character-ENCLOSING_MARK]]static byte link:{java8-javadoc}/java/lang/Character.html#ENCLOSING_MARK[ENCLOSING_MARK] (link:{java9-javadoc}/java/lang/Character.html#ENCLOSING_MARK[java 9]) -** [[painless-api-reference-Character-END_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#END_PUNCTUATION[END_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#END_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-FINAL_QUOTE_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#FINAL_QUOTE_PUNCTUATION[FINAL_QUOTE_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#FINAL_QUOTE_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-FORMAT]]static byte link:{java8-javadoc}/java/lang/Character.html#FORMAT[FORMAT] (link:{java9-javadoc}/java/lang/Character.html#FORMAT[java 9]) -** [[painless-api-reference-Character-INITIAL_QUOTE_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#INITIAL_QUOTE_PUNCTUATION[INITIAL_QUOTE_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#INITIAL_QUOTE_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-LETTER_NUMBER]]static byte link:{java8-javadoc}/java/lang/Character.html#LETTER_NUMBER[LETTER_NUMBER] (link:{java9-javadoc}/java/lang/Character.html#LETTER_NUMBER[java 9]) -** [[painless-api-reference-Character-LINE_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#LINE_SEPARATOR[LINE_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#LINE_SEPARATOR[java 9]) -** [[painless-api-reference-Character-LOWERCASE_LETTER]]static byte link:{java8-javadoc}/java/lang/Character.html#LOWERCASE_LETTER[LOWERCASE_LETTER] (link:{java9-javadoc}/java/lang/Character.html#LOWERCASE_LETTER[java 9]) -** [[painless-api-reference-Character-MATH_SYMBOL]]static byte link:{java8-javadoc}/java/lang/Character.html#MATH_SYMBOL[MATH_SYMBOL] (link:{java9-javadoc}/java/lang/Character.html#MATH_SYMBOL[java 9]) -** [[painless-api-reference-Character-MAX_CODE_POINT]]static int link:{java8-javadoc}/java/lang/Character.html#MAX_CODE_POINT[MAX_CODE_POINT] (link:{java9-javadoc}/java/lang/Character.html#MAX_CODE_POINT[java 9]) -** [[painless-api-reference-Character-MAX_HIGH_SURROGATE]]static char link:{java8-javadoc}/java/lang/Character.html#MAX_HIGH_SURROGATE[MAX_HIGH_SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#MAX_HIGH_SURROGATE[java 9]) -** [[painless-api-reference-Character-MAX_LOW_SURROGATE]]static char link:{java8-javadoc}/java/lang/Character.html#MAX_LOW_SURROGATE[MAX_LOW_SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#MAX_LOW_SURROGATE[java 9]) -** [[painless-api-reference-Character-MAX_RADIX]]static int link:{java8-javadoc}/java/lang/Character.html#MAX_RADIX[MAX_RADIX] (link:{java9-javadoc}/java/lang/Character.html#MAX_RADIX[java 9]) -** [[painless-api-reference-Character-MAX_SURROGATE]]static char link:{java8-javadoc}/java/lang/Character.html#MAX_SURROGATE[MAX_SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#MAX_SURROGATE[java 9]) -** [[painless-api-reference-Character-MAX_VALUE]]static char link:{java8-javadoc}/java/lang/Character.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Character.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Character-MIN_CODE_POINT]]static char link:{java8-javadoc}/java/lang/Character.html#MIN_CODE_POINT[MIN_CODE_POINT] (link:{java9-javadoc}/java/lang/Character.html#MIN_CODE_POINT[java 9]) -** [[painless-api-reference-Character-MIN_HIGH_SURROGATE]]static char link:{java8-javadoc}/java/lang/Character.html#MIN_HIGH_SURROGATE[MIN_HIGH_SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#MIN_HIGH_SURROGATE[java 9]) -** [[painless-api-reference-Character-MIN_LOW_SURROGATE]]static char link:{java8-javadoc}/java/lang/Character.html#MIN_LOW_SURROGATE[MIN_LOW_SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#MIN_LOW_SURROGATE[java 9]) -** [[painless-api-reference-Character-MIN_RADIX]]static int link:{java8-javadoc}/java/lang/Character.html#MIN_RADIX[MIN_RADIX] (link:{java9-javadoc}/java/lang/Character.html#MIN_RADIX[java 9]) -** [[painless-api-reference-Character-MIN_SUPPLEMENTARY_CODE_POINT]]static int link:{java8-javadoc}/java/lang/Character.html#MIN_SUPPLEMENTARY_CODE_POINT[MIN_SUPPLEMENTARY_CODE_POINT] (link:{java9-javadoc}/java/lang/Character.html#MIN_SUPPLEMENTARY_CODE_POINT[java 9]) -** [[painless-api-reference-Character-MIN_SURROGATE]]static char link:{java8-javadoc}/java/lang/Character.html#MIN_SURROGATE[MIN_SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#MIN_SURROGATE[java 9]) -** [[painless-api-reference-Character-MIN_VALUE]]static char link:{java8-javadoc}/java/lang/Character.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Character.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Character-MODIFIER_LETTER]]static byte link:{java8-javadoc}/java/lang/Character.html#MODIFIER_LETTER[MODIFIER_LETTER] (link:{java9-javadoc}/java/lang/Character.html#MODIFIER_LETTER[java 9]) -** [[painless-api-reference-Character-MODIFIER_SYMBOL]]static byte link:{java8-javadoc}/java/lang/Character.html#MODIFIER_SYMBOL[MODIFIER_SYMBOL] (link:{java9-javadoc}/java/lang/Character.html#MODIFIER_SYMBOL[java 9]) -** [[painless-api-reference-Character-NON_SPACING_MARK]]static byte link:{java8-javadoc}/java/lang/Character.html#NON_SPACING_MARK[NON_SPACING_MARK] (link:{java9-javadoc}/java/lang/Character.html#NON_SPACING_MARK[java 9]) -** [[painless-api-reference-Character-OTHER_LETTER]]static byte link:{java8-javadoc}/java/lang/Character.html#OTHER_LETTER[OTHER_LETTER] (link:{java9-javadoc}/java/lang/Character.html#OTHER_LETTER[java 9]) -** [[painless-api-reference-Character-OTHER_NUMBER]]static byte link:{java8-javadoc}/java/lang/Character.html#OTHER_NUMBER[OTHER_NUMBER] (link:{java9-javadoc}/java/lang/Character.html#OTHER_NUMBER[java 9]) -** [[painless-api-reference-Character-OTHER_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#OTHER_PUNCTUATION[OTHER_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#OTHER_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-OTHER_SYMBOL]]static byte link:{java8-javadoc}/java/lang/Character.html#OTHER_SYMBOL[OTHER_SYMBOL] (link:{java9-javadoc}/java/lang/Character.html#OTHER_SYMBOL[java 9]) -** [[painless-api-reference-Character-PARAGRAPH_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#PARAGRAPH_SEPARATOR[PARAGRAPH_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#PARAGRAPH_SEPARATOR[java 9]) -** [[painless-api-reference-Character-PRIVATE_USE]]static byte link:{java8-javadoc}/java/lang/Character.html#PRIVATE_USE[PRIVATE_USE] (link:{java9-javadoc}/java/lang/Character.html#PRIVATE_USE[java 9]) -** [[painless-api-reference-Character-SIZE]]static int link:{java8-javadoc}/java/lang/Character.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Character.html#SIZE[java 9]) -** [[painless-api-reference-Character-SPACE_SEPARATOR]]static byte link:{java8-javadoc}/java/lang/Character.html#SPACE_SEPARATOR[SPACE_SEPARATOR] (link:{java9-javadoc}/java/lang/Character.html#SPACE_SEPARATOR[java 9]) -** [[painless-api-reference-Character-START_PUNCTUATION]]static byte link:{java8-javadoc}/java/lang/Character.html#START_PUNCTUATION[START_PUNCTUATION] (link:{java9-javadoc}/java/lang/Character.html#START_PUNCTUATION[java 9]) -** [[painless-api-reference-Character-SURROGATE]]static byte link:{java8-javadoc}/java/lang/Character.html#SURROGATE[SURROGATE] (link:{java9-javadoc}/java/lang/Character.html#SURROGATE[java 9]) -** [[painless-api-reference-Character-TITLECASE_LETTER]]static byte link:{java8-javadoc}/java/lang/Character.html#TITLECASE_LETTER[TITLECASE_LETTER] (link:{java9-javadoc}/java/lang/Character.html#TITLECASE_LETTER[java 9]) -** [[painless-api-reference-Character-UNASSIGNED]]static byte link:{java8-javadoc}/java/lang/Character.html#UNASSIGNED[UNASSIGNED] (link:{java9-javadoc}/java/lang/Character.html#UNASSIGNED[java 9]) -** [[painless-api-reference-Character-UPPERCASE_LETTER]]static byte link:{java8-javadoc}/java/lang/Character.html#UPPERCASE_LETTER[UPPERCASE_LETTER] (link:{java9-javadoc}/java/lang/Character.html#UPPERCASE_LETTER[java 9]) -* ++[[painless-api-reference-Character-charCount-1]]static int link:{java8-javadoc}/java/lang/Character.html#charCount%2Dint%2D[charCount](int)++ (link:{java9-javadoc}/java/lang/Character.html#charCount%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-codePointAt-2]]static int link:{java8-javadoc}/java/lang/Character.html#codePointAt%2Djava.lang.CharSequence%2Dint%2D[codePointAt](<>, int)++ (link:{java9-javadoc}/java/lang/Character.html#codePointAt%2Djava.lang.CharSequence%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-codePointAt-3]]static int link:{java8-javadoc}/java/lang/Character.html#codePointAt%2Dchar:A%2Dint%2Dint%2D[codePointAt](char[], int, int)++ (link:{java9-javadoc}/java/lang/Character.html#codePointAt%2Dchar:A%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-codePointBefore-2]]static int link:{java8-javadoc}/java/lang/Character.html#codePointBefore%2Djava.lang.CharSequence%2Dint%2D[codePointBefore](<>, int)++ (link:{java9-javadoc}/java/lang/Character.html#codePointBefore%2Djava.lang.CharSequence%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-codePointBefore-3]]static int link:{java8-javadoc}/java/lang/Character.html#codePointBefore%2Dchar:A%2Dint%2Dint%2D[codePointBefore](char[], int, int)++ (link:{java9-javadoc}/java/lang/Character.html#codePointBefore%2Dchar:A%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-codePointCount-3]]static int link:{java8-javadoc}/java/lang/Character.html#codePointCount%2Djava.lang.CharSequence%2Dint%2Dint%2D[codePointCount](<>, int, int)++ (link:{java9-javadoc}/java/lang/Character.html#codePointCount%2Djava.lang.CharSequence%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-compare-2]]static int link:{java8-javadoc}/java/lang/Character.html#compare%2Dchar%2Dchar%2D[compare](char, char)++ (link:{java9-javadoc}/java/lang/Character.html#compare%2Dchar%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-digit-2]]static int link:{java8-javadoc}/java/lang/Character.html#digit%2Dint%2Dint%2D[digit](int, int)++ (link:{java9-javadoc}/java/lang/Character.html#digit%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-forDigit-2]]static char link:{java8-javadoc}/java/lang/Character.html#forDigit%2Dint%2Dint%2D[forDigit](int, int)++ (link:{java9-javadoc}/java/lang/Character.html#forDigit%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-getDirectionality-1]]static byte link:{java8-javadoc}/java/lang/Character.html#getDirectionality%2Dint%2D[getDirectionality](int)++ (link:{java9-javadoc}/java/lang/Character.html#getDirectionality%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-getName-1]]static <> link:{java8-javadoc}/java/lang/Character.html#getName%2Dint%2D[getName](int)++ (link:{java9-javadoc}/java/lang/Character.html#getName%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-getNumericValue-1]]static int link:{java8-javadoc}/java/lang/Character.html#getNumericValue%2Dint%2D[getNumericValue](int)++ (link:{java9-javadoc}/java/lang/Character.html#getNumericValue%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-getType-1]]static int link:{java8-javadoc}/java/lang/Character.html#getType%2Dint%2D[getType](int)++ (link:{java9-javadoc}/java/lang/Character.html#getType%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-hashCode-1]]static int link:{java8-javadoc}/java/lang/Character.html#hashCode%2Dchar%2D[hashCode](char)++ (link:{java9-javadoc}/java/lang/Character.html#hashCode%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-highSurrogate-1]]static char link:{java8-javadoc}/java/lang/Character.html#highSurrogate%2Dint%2D[highSurrogate](int)++ (link:{java9-javadoc}/java/lang/Character.html#highSurrogate%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isAlphabetic-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isAlphabetic%2Dint%2D[isAlphabetic](int)++ (link:{java9-javadoc}/java/lang/Character.html#isAlphabetic%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isBmpCodePoint-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isBmpCodePoint%2Dint%2D[isBmpCodePoint](int)++ (link:{java9-javadoc}/java/lang/Character.html#isBmpCodePoint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isDefined-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isDefined%2Dint%2D[isDefined](int)++ (link:{java9-javadoc}/java/lang/Character.html#isDefined%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isDigit-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isDigit%2Dint%2D[isDigit](int)++ (link:{java9-javadoc}/java/lang/Character.html#isDigit%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isHighSurrogate-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isHighSurrogate%2Dchar%2D[isHighSurrogate](char)++ (link:{java9-javadoc}/java/lang/Character.html#isHighSurrogate%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-isISOControl-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isISOControl%2Dint%2D[isISOControl](int)++ (link:{java9-javadoc}/java/lang/Character.html#isISOControl%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isIdentifierIgnorable-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isIdentifierIgnorable%2Dint%2D[isIdentifierIgnorable](int)++ (link:{java9-javadoc}/java/lang/Character.html#isIdentifierIgnorable%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isIdeographic-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isIdeographic%2Dint%2D[isIdeographic](int)++ (link:{java9-javadoc}/java/lang/Character.html#isIdeographic%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isJavaIdentifierPart-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isJavaIdentifierPart%2Dint%2D[isJavaIdentifierPart](int)++ (link:{java9-javadoc}/java/lang/Character.html#isJavaIdentifierPart%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isJavaIdentifierStart-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isJavaIdentifierStart%2Dint%2D[isJavaIdentifierStart](int)++ (link:{java9-javadoc}/java/lang/Character.html#isJavaIdentifierStart%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isLetter-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isLetter%2Dint%2D[isLetter](int)++ (link:{java9-javadoc}/java/lang/Character.html#isLetter%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isLetterOrDigit-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isLetterOrDigit%2Dint%2D[isLetterOrDigit](int)++ (link:{java9-javadoc}/java/lang/Character.html#isLetterOrDigit%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isLowerCase-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isLowerCase%2Dint%2D[isLowerCase](int)++ (link:{java9-javadoc}/java/lang/Character.html#isLowerCase%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isMirrored-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isMirrored%2Dint%2D[isMirrored](int)++ (link:{java9-javadoc}/java/lang/Character.html#isMirrored%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isSpaceChar-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isSpaceChar%2Dint%2D[isSpaceChar](int)++ (link:{java9-javadoc}/java/lang/Character.html#isSpaceChar%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isSupplementaryCodePoint-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isSupplementaryCodePoint%2Dint%2D[isSupplementaryCodePoint](int)++ (link:{java9-javadoc}/java/lang/Character.html#isSupplementaryCodePoint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isSurrogate-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isSurrogate%2Dchar%2D[isSurrogate](char)++ (link:{java9-javadoc}/java/lang/Character.html#isSurrogate%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-isSurrogatePair-2]]static boolean link:{java8-javadoc}/java/lang/Character.html#isSurrogatePair%2Dchar%2Dchar%2D[isSurrogatePair](char, char)++ (link:{java9-javadoc}/java/lang/Character.html#isSurrogatePair%2Dchar%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-isTitleCase-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isTitleCase%2Dint%2D[isTitleCase](int)++ (link:{java9-javadoc}/java/lang/Character.html#isTitleCase%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isUnicodeIdentifierPart-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isUnicodeIdentifierPart%2Dint%2D[isUnicodeIdentifierPart](int)++ (link:{java9-javadoc}/java/lang/Character.html#isUnicodeIdentifierPart%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isUnicodeIdentifierStart-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isUnicodeIdentifierStart%2Dint%2D[isUnicodeIdentifierStart](int)++ (link:{java9-javadoc}/java/lang/Character.html#isUnicodeIdentifierStart%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isUpperCase-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isUpperCase%2Dint%2D[isUpperCase](int)++ (link:{java9-javadoc}/java/lang/Character.html#isUpperCase%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isValidCodePoint-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isValidCodePoint%2Dint%2D[isValidCodePoint](int)++ (link:{java9-javadoc}/java/lang/Character.html#isValidCodePoint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-isWhitespace-1]]static boolean link:{java8-javadoc}/java/lang/Character.html#isWhitespace%2Dint%2D[isWhitespace](int)++ (link:{java9-javadoc}/java/lang/Character.html#isWhitespace%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-lowSurrogate-1]]static char link:{java8-javadoc}/java/lang/Character.html#lowSurrogate%2Dint%2D[lowSurrogate](int)++ (link:{java9-javadoc}/java/lang/Character.html#lowSurrogate%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-offsetByCodePoints-3]]static int link:{java8-javadoc}/java/lang/Character.html#offsetByCodePoints%2Djava.lang.CharSequence%2Dint%2Dint%2D[offsetByCodePoints](<>, int, int)++ (link:{java9-javadoc}/java/lang/Character.html#offsetByCodePoints%2Djava.lang.CharSequence%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-offsetByCodePoints-5]]static int link:{java8-javadoc}/java/lang/Character.html#offsetByCodePoints%2Dchar:A%2Dint%2Dint%2Dint%2Dint%2D[offsetByCodePoints](char[], int, int, int, int)++ (link:{java9-javadoc}/java/lang/Character.html#offsetByCodePoints%2Dchar:A%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-reverseBytes-1]]static char link:{java8-javadoc}/java/lang/Character.html#reverseBytes%2Dchar%2D[reverseBytes](char)++ (link:{java9-javadoc}/java/lang/Character.html#reverseBytes%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-toChars-1]]static char[] link:{java8-javadoc}/java/lang/Character.html#toChars%2Dint%2D[toChars](int)++ (link:{java9-javadoc}/java/lang/Character.html#toChars%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-toChars-3]]static int link:{java8-javadoc}/java/lang/Character.html#toChars%2Dint%2Dchar:A%2Dint%2D[toChars](int, char[], int)++ (link:{java9-javadoc}/java/lang/Character.html#toChars%2Dint%2Dchar:A%2Dint%2D[java 9]) -* ++[[painless-api-reference-Character-toCodePoint-2]]static int link:{java8-javadoc}/java/lang/Character.html#toCodePoint%2Dchar%2Dchar%2D[toCodePoint](char, char)++ (link:{java9-javadoc}/java/lang/Character.html#toCodePoint%2Dchar%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-toLowerCase-1]]static char link:{java8-javadoc}/java/lang/Character.html#toLowerCase%2Dchar%2D[toLowerCase](char)++ (link:{java9-javadoc}/java/lang/Character.html#toLowerCase%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-toString-1]]static <> link:{java8-javadoc}/java/lang/Character.html#toString%2Dchar%2D[toString](char)++ (link:{java9-javadoc}/java/lang/Character.html#toString%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-toTitleCase-1]]static char link:{java8-javadoc}/java/lang/Character.html#toTitleCase%2Dchar%2D[toTitleCase](char)++ (link:{java9-javadoc}/java/lang/Character.html#toTitleCase%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-toUpperCase-1]]static char link:{java8-javadoc}/java/lang/Character.html#toUpperCase%2Dchar%2D[toUpperCase](char)++ (link:{java9-javadoc}/java/lang/Character.html#toUpperCase%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Character.html#valueOf%2Dchar%2D[valueOf](char)++ (link:{java9-javadoc}/java/lang/Character.html#valueOf%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Character-charValue-0]]char link:{java8-javadoc}/java/lang/Character.html#charValue%2D%2D[charValue]()++ (link:{java9-javadoc}/java/lang/Character.html#charValue%2D%2D[java 9]) -* ++[[painless-api-reference-Character-compareTo-1]]int link:{java8-javadoc}/java/lang/Character.html#compareTo%2Djava.lang.Character%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Character.html#compareTo%2Djava.lang.Character%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/CharacterIterator.asciidoc b/docs/painless/painless-api-reference/CharacterIterator.asciidoc deleted file mode 100644 index 1303e2d47396a..0000000000000 --- a/docs/painless/painless-api-reference/CharacterIterator.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-CharacterIterator]]++CharacterIterator++:: -** [[painless-api-reference-CharacterIterator-DONE]]static char link:{java8-javadoc}/java/text/CharacterIterator.html#DONE[DONE] (link:{java9-javadoc}/java/text/CharacterIterator.html#DONE[java 9]) -* ++[[painless-api-reference-CharacterIterator-clone-0]]def link:{java8-javadoc}/java/text/CharacterIterator.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-current-0]]char link:{java8-javadoc}/java/text/CharacterIterator.html#current%2D%2D[current]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#current%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-first-0]]char link:{java8-javadoc}/java/text/CharacterIterator.html#first%2D%2D[first]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#first%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-getBeginIndex-0]]int link:{java8-javadoc}/java/text/CharacterIterator.html#getBeginIndex%2D%2D[getBeginIndex]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#getBeginIndex%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-getEndIndex-0]]int link:{java8-javadoc}/java/text/CharacterIterator.html#getEndIndex%2D%2D[getEndIndex]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#getEndIndex%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-getIndex-0]]int link:{java8-javadoc}/java/text/CharacterIterator.html#getIndex%2D%2D[getIndex]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#getIndex%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-last-0]]char link:{java8-javadoc}/java/text/CharacterIterator.html#last%2D%2D[last]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#last%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-next-0]]char link:{java8-javadoc}/java/text/CharacterIterator.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-previous-0]]char link:{java8-javadoc}/java/text/CharacterIterator.html#previous%2D%2D[previous]()++ (link:{java9-javadoc}/java/text/CharacterIterator.html#previous%2D%2D[java 9]) -* ++[[painless-api-reference-CharacterIterator-setIndex-1]]char link:{java8-javadoc}/java/text/CharacterIterator.html#setIndex%2Dint%2D[setIndex](int)++ (link:{java9-javadoc}/java/text/CharacterIterator.html#setIndex%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ChoiceFormat.asciidoc b/docs/painless/painless-api-reference/ChoiceFormat.asciidoc deleted file mode 100644 index 426120a194f73..0000000000000 --- a/docs/painless/painless-api-reference/ChoiceFormat.asciidoc +++ /dev/null @@ -1,17 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChoiceFormat]]++ChoiceFormat++:: -* ++[[painless-api-reference-ChoiceFormat-nextDouble-1]]static double link:{java8-javadoc}/java/text/ChoiceFormat.html#nextDouble%2Ddouble%2D[nextDouble](double)++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#nextDouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-nextDouble-2]]static double link:{java8-javadoc}/java/text/ChoiceFormat.html#nextDouble%2Ddouble%2Dboolean%2D[nextDouble](double, boolean)++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#nextDouble%2Ddouble%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-previousDouble-1]]static double link:{java8-javadoc}/java/text/ChoiceFormat.html#previousDouble%2Ddouble%2D[previousDouble](double)++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#previousDouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-ChoiceFormat-1]]link:{java8-javadoc}/java/text/ChoiceFormat.html#ChoiceFormat%2Djava.lang.String%2D[ChoiceFormat](<>)++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#ChoiceFormat%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-ChoiceFormat-2]]link:{java8-javadoc}/java/text/ChoiceFormat.html#ChoiceFormat%2Ddouble:A%2Djava.lang.String:A%2D[ChoiceFormat](double[], <>[])++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#ChoiceFormat%2Ddouble:A%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-applyPattern-1]]void link:{java8-javadoc}/java/text/ChoiceFormat.html#applyPattern%2Djava.lang.String%2D[applyPattern](<>)++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#applyPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-getFormats-0]]def[] link:{java8-javadoc}/java/text/ChoiceFormat.html#getFormats%2D%2D[getFormats]()++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#getFormats%2D%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-getLimits-0]]double[] link:{java8-javadoc}/java/text/ChoiceFormat.html#getLimits%2D%2D[getLimits]()++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#getLimits%2D%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-setChoices-2]]void link:{java8-javadoc}/java/text/ChoiceFormat.html#setChoices%2Ddouble:A%2Djava.lang.String:A%2D[setChoices](double[], <>[])++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#setChoices%2Ddouble:A%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-ChoiceFormat-toPattern-0]]<> link:{java8-javadoc}/java/text/ChoiceFormat.html#toPattern%2D%2D[toPattern]()++ (link:{java9-javadoc}/java/text/ChoiceFormat.html#toPattern%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ChronoField.asciidoc b/docs/painless/painless-api-reference/ChronoField.asciidoc deleted file mode 100644 index 33143a52eaf63..0000000000000 --- a/docs/painless/painless-api-reference/ChronoField.asciidoc +++ /dev/null @@ -1,41 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChronoField]]++ChronoField++:: -** [[painless-api-reference-ChronoField-ALIGNED_DAY_OF_WEEK_IN_MONTH]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_DAY_OF_WEEK_IN_MONTH[ALIGNED_DAY_OF_WEEK_IN_MONTH] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_DAY_OF_WEEK_IN_MONTH[java 9]) -** [[painless-api-reference-ChronoField-ALIGNED_DAY_OF_WEEK_IN_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_DAY_OF_WEEK_IN_YEAR[ALIGNED_DAY_OF_WEEK_IN_YEAR] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_DAY_OF_WEEK_IN_YEAR[java 9]) -** [[painless-api-reference-ChronoField-ALIGNED_WEEK_OF_MONTH]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_WEEK_OF_MONTH[ALIGNED_WEEK_OF_MONTH] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_WEEK_OF_MONTH[java 9]) -** [[painless-api-reference-ChronoField-ALIGNED_WEEK_OF_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_WEEK_OF_YEAR[ALIGNED_WEEK_OF_YEAR] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#ALIGNED_WEEK_OF_YEAR[java 9]) -** [[painless-api-reference-ChronoField-AMPM_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#AMPM_OF_DAY[AMPM_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#AMPM_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-CLOCK_HOUR_OF_AMPM]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#CLOCK_HOUR_OF_AMPM[CLOCK_HOUR_OF_AMPM] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#CLOCK_HOUR_OF_AMPM[java 9]) -** [[painless-api-reference-ChronoField-CLOCK_HOUR_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#CLOCK_HOUR_OF_DAY[CLOCK_HOUR_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#CLOCK_HOUR_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-DAY_OF_MONTH]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#DAY_OF_MONTH[DAY_OF_MONTH] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#DAY_OF_MONTH[java 9]) -** [[painless-api-reference-ChronoField-DAY_OF_WEEK]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#DAY_OF_WEEK[DAY_OF_WEEK] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#DAY_OF_WEEK[java 9]) -** [[painless-api-reference-ChronoField-DAY_OF_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#DAY_OF_YEAR[DAY_OF_YEAR] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#DAY_OF_YEAR[java 9]) -** [[painless-api-reference-ChronoField-EPOCH_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#EPOCH_DAY[EPOCH_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#EPOCH_DAY[java 9]) -** [[painless-api-reference-ChronoField-ERA]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#ERA[ERA] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#ERA[java 9]) -** [[painless-api-reference-ChronoField-HOUR_OF_AMPM]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#HOUR_OF_AMPM[HOUR_OF_AMPM] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#HOUR_OF_AMPM[java 9]) -** [[painless-api-reference-ChronoField-HOUR_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#HOUR_OF_DAY[HOUR_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#HOUR_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-INSTANT_SECONDS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#INSTANT_SECONDS[INSTANT_SECONDS] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#INSTANT_SECONDS[java 9]) -** [[painless-api-reference-ChronoField-MICRO_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MICRO_OF_DAY[MICRO_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MICRO_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-MICRO_OF_SECOND]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MICRO_OF_SECOND[MICRO_OF_SECOND] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MICRO_OF_SECOND[java 9]) -** [[painless-api-reference-ChronoField-MILLI_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MILLI_OF_DAY[MILLI_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MILLI_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-MILLI_OF_SECOND]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MILLI_OF_SECOND[MILLI_OF_SECOND] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MILLI_OF_SECOND[java 9]) -** [[painless-api-reference-ChronoField-MINUTE_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MINUTE_OF_DAY[MINUTE_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MINUTE_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-MINUTE_OF_HOUR]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MINUTE_OF_HOUR[MINUTE_OF_HOUR] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MINUTE_OF_HOUR[java 9]) -** [[painless-api-reference-ChronoField-MONTH_OF_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#MONTH_OF_YEAR[MONTH_OF_YEAR] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#MONTH_OF_YEAR[java 9]) -** [[painless-api-reference-ChronoField-NANO_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#NANO_OF_DAY[NANO_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#NANO_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-NANO_OF_SECOND]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#NANO_OF_SECOND[NANO_OF_SECOND] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#NANO_OF_SECOND[java 9]) -** [[painless-api-reference-ChronoField-OFFSET_SECONDS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#OFFSET_SECONDS[OFFSET_SECONDS] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#OFFSET_SECONDS[java 9]) -** [[painless-api-reference-ChronoField-PROLEPTIC_MONTH]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#PROLEPTIC_MONTH[PROLEPTIC_MONTH] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#PROLEPTIC_MONTH[java 9]) -** [[painless-api-reference-ChronoField-SECOND_OF_DAY]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#SECOND_OF_DAY[SECOND_OF_DAY] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#SECOND_OF_DAY[java 9]) -** [[painless-api-reference-ChronoField-SECOND_OF_MINUTE]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#SECOND_OF_MINUTE[SECOND_OF_MINUTE] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#SECOND_OF_MINUTE[java 9]) -** [[painless-api-reference-ChronoField-YEAR]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#YEAR[YEAR] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#YEAR[java 9]) -** [[painless-api-reference-ChronoField-YEAR_OF_ERA]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#YEAR_OF_ERA[YEAR_OF_ERA] (link:{java9-javadoc}/java/time/temporal/ChronoField.html#YEAR_OF_ERA[java 9]) -* ++[[painless-api-reference-ChronoField-valueOf-1]]static <> link:{java8-javadoc}/java/time/temporal/ChronoField.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/temporal/ChronoField.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ChronoField-values-0]]static <>[] link:{java8-javadoc}/java/time/temporal/ChronoField.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/temporal/ChronoField.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoField-checkValidIntValue-1]]int link:{java8-javadoc}/java/time/temporal/ChronoField.html#checkValidIntValue%2Dlong%2D[checkValidIntValue](long)++ (link:{java9-javadoc}/java/time/temporal/ChronoField.html#checkValidIntValue%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ChronoField-checkValidValue-1]]long link:{java8-javadoc}/java/time/temporal/ChronoField.html#checkValidValue%2Dlong%2D[checkValidValue](long)++ (link:{java9-javadoc}/java/time/temporal/ChronoField.html#checkValidValue%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ChronoLocalDate.asciidoc b/docs/painless/painless-api-reference/ChronoLocalDate.asciidoc deleted file mode 100644 index 2c483c27bd8d9..0000000000000 --- a/docs/painless/painless-api-reference/ChronoLocalDate.asciidoc +++ /dev/null @@ -1,31 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChronoLocalDate]]++ChronoLocalDate++:: -* ++[[painless-api-reference-ChronoLocalDate-from-1]]static <> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-timeLineOrder-0]]static <> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#timeLineOrder%2D%2D[timeLineOrder]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#timeLineOrder%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-atTime-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#atTime%2Djava.time.LocalTime%2D[atTime](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#atTime%2Djava.time.LocalTime%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-compareTo-1]]int link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#compareTo%2Djava.time.chrono.ChronoLocalDate%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#compareTo%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-equals-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-format-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-getEra-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#getEra%2D%2D[getEra]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#getEra%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-hashCode-0]]int link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-isAfter-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#isAfter%2Djava.time.chrono.ChronoLocalDate%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#isAfter%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-isBefore-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#isBefore%2Djava.time.chrono.ChronoLocalDate%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#isBefore%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-isEqual-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#isEqual%2Djava.time.chrono.ChronoLocalDate%2D[isEqual](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#isEqual%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-isLeapYear-0]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#isLeapYear%2D%2D[isLeapYear]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#isLeapYear%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-lengthOfMonth-0]]int link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#lengthOfMonth%2D%2D[lengthOfMonth]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#lengthOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-lengthOfYear-0]]int link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#lengthOfYear%2D%2D[lengthOfYear]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#lengthOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-minus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-minus-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-plus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-plus-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-toEpochDay-0]]long link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#toEpochDay%2D%2D[toEpochDay]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#toEpochDay%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-toString-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#toString%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-until-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#until%2Djava.time.chrono.ChronoLocalDate%2D[until](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#until%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-with-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDate-with-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ChronoLocalDateTime.asciidoc b/docs/painless/painless-api-reference/ChronoLocalDateTime.asciidoc deleted file mode 100644 index 7b49712ee0985..0000000000000 --- a/docs/painless/painless-api-reference/ChronoLocalDateTime.asciidoc +++ /dev/null @@ -1,29 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChronoLocalDateTime]]++ChronoLocalDateTime++:: -* ++[[painless-api-reference-ChronoLocalDateTime-from-1]]static <> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-timeLineOrder-0]]static <> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#timeLineOrder%2D%2D[timeLineOrder]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#timeLineOrder%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-atZone-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#atZone%2Djava.time.ZoneId%2D[atZone](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#atZone%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-compareTo-1]]int link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#compareTo%2Djava.time.chrono.ChronoLocalDateTime%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#compareTo%2Djava.time.chrono.ChronoLocalDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-equals-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-format-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-hashCode-0]]int link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-isAfter-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#isAfter%2Djava.time.chrono.ChronoLocalDateTime%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#isAfter%2Djava.time.chrono.ChronoLocalDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-isBefore-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#isBefore%2Djava.time.chrono.ChronoLocalDateTime%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#isBefore%2Djava.time.chrono.ChronoLocalDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-isEqual-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#isEqual%2Djava.time.chrono.ChronoLocalDateTime%2D[isEqual](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#isEqual%2Djava.time.chrono.ChronoLocalDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-minus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-minus-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-plus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-plus-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-toEpochSecond-1]]long link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toEpochSecond%2Djava.time.ZoneOffset%2D[toEpochSecond](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toEpochSecond%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-toInstant-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toInstant%2Djava.time.ZoneOffset%2D[toInstant](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toInstant%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-toLocalDate-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toLocalDate%2D%2D[toLocalDate]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toLocalDate%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-toLocalTime-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toLocalTime%2D%2D[toLocalTime]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toLocalTime%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-toString-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#toString%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-with-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-ChronoLocalDateTime-with-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoLocalDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/ChronoLocalDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ChronoPeriod.asciidoc b/docs/painless/painless-api-reference/ChronoPeriod.asciidoc deleted file mode 100644 index 1b15df31c7667..0000000000000 --- a/docs/painless/painless-api-reference/ChronoPeriod.asciidoc +++ /dev/null @@ -1,20 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChronoPeriod]]++ChronoPeriod++:: -* ++[[painless-api-reference-ChronoPeriod-between-2]]static <> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#between%2Djava.time.chrono.ChronoLocalDate%2Djava.time.chrono.ChronoLocalDate%2D[between](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#between%2Djava.time.chrono.ChronoLocalDate%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-equals-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-getUnits-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#getUnits%2D%2D[getUnits]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#getUnits%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-hashCode-0]]int link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-isNegative-0]]boolean link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#isNegative%2D%2D[isNegative]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#isNegative%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-isZero-0]]boolean link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#isZero%2D%2D[isZero]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#isZero%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-minus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-multipliedBy-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#multipliedBy%2Dint%2D[multipliedBy](int)++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#multipliedBy%2Dint%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-negated-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#negated%2D%2D[negated]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#negated%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-normalized-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#normalized%2D%2D[normalized]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#normalized%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-plus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoPeriod-toString-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoPeriod.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/chrono/ChronoPeriod.html#toString%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ChronoUnit.asciidoc b/docs/painless/painless-api-reference/ChronoUnit.asciidoc deleted file mode 100644 index 297fbf61835f9..0000000000000 --- a/docs/painless/painless-api-reference/ChronoUnit.asciidoc +++ /dev/null @@ -1,25 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChronoUnit]]++ChronoUnit++:: -** [[painless-api-reference-ChronoUnit-CENTURIES]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#CENTURIES[CENTURIES] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#CENTURIES[java 9]) -** [[painless-api-reference-ChronoUnit-DAYS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#DAYS[DAYS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#DAYS[java 9]) -** [[painless-api-reference-ChronoUnit-DECADES]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#DECADES[DECADES] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#DECADES[java 9]) -** [[painless-api-reference-ChronoUnit-ERAS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#ERAS[ERAS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#ERAS[java 9]) -** [[painless-api-reference-ChronoUnit-FOREVER]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#FOREVER[FOREVER] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#FOREVER[java 9]) -** [[painless-api-reference-ChronoUnit-HALF_DAYS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#HALF_DAYS[HALF_DAYS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#HALF_DAYS[java 9]) -** [[painless-api-reference-ChronoUnit-HOURS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#HOURS[HOURS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#HOURS[java 9]) -** [[painless-api-reference-ChronoUnit-MICROS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#MICROS[MICROS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#MICROS[java 9]) -** [[painless-api-reference-ChronoUnit-MILLENNIA]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#MILLENNIA[MILLENNIA] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#MILLENNIA[java 9]) -** [[painless-api-reference-ChronoUnit-MILLIS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#MILLIS[MILLIS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#MILLIS[java 9]) -** [[painless-api-reference-ChronoUnit-MINUTES]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#MINUTES[MINUTES] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#MINUTES[java 9]) -** [[painless-api-reference-ChronoUnit-MONTHS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#MONTHS[MONTHS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#MONTHS[java 9]) -** [[painless-api-reference-ChronoUnit-NANOS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#NANOS[NANOS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#NANOS[java 9]) -** [[painless-api-reference-ChronoUnit-SECONDS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#SECONDS[SECONDS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#SECONDS[java 9]) -** [[painless-api-reference-ChronoUnit-WEEKS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#WEEKS[WEEKS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#WEEKS[java 9]) -** [[painless-api-reference-ChronoUnit-YEARS]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#YEARS[YEARS] (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#YEARS[java 9]) -* ++[[painless-api-reference-ChronoUnit-valueOf-1]]static <> link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ChronoUnit-values-0]]static <>[] link:{java8-javadoc}/java/time/temporal/ChronoUnit.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/temporal/ChronoUnit.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ChronoZonedDateTime.asciidoc b/docs/painless/painless-api-reference/ChronoZonedDateTime.asciidoc deleted file mode 100644 index 0684b8b541e02..0000000000000 --- a/docs/painless/painless-api-reference/ChronoZonedDateTime.asciidoc +++ /dev/null @@ -1,35 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ChronoZonedDateTime]]++ChronoZonedDateTime++:: -* ++[[painless-api-reference-ChronoZonedDateTime-from-1]]static <> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-timeLineOrder-0]]static <> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#timeLineOrder%2D%2D[timeLineOrder]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#timeLineOrder%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-compareTo-1]]int link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#compareTo%2Djava.time.chrono.ChronoZonedDateTime%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#compareTo%2Djava.time.chrono.ChronoZonedDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-equals-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-format-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-getOffset-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#getOffset%2D%2D[getOffset]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#getOffset%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-getZone-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#getZone%2D%2D[getZone]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#getZone%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-hashCode-0]]int link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-isAfter-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#isAfter%2Djava.time.chrono.ChronoZonedDateTime%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#isAfter%2Djava.time.chrono.ChronoZonedDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-isBefore-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#isBefore%2Djava.time.chrono.ChronoZonedDateTime%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#isBefore%2Djava.time.chrono.ChronoZonedDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-isEqual-1]]boolean link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#isEqual%2Djava.time.chrono.ChronoZonedDateTime%2D[isEqual](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#isEqual%2Djava.time.chrono.ChronoZonedDateTime%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-minus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-minus-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-plus-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-plus-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-toEpochSecond-0]]long link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toEpochSecond%2D%2D[toEpochSecond]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toEpochSecond%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-toInstant-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toInstant%2D%2D[toInstant]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toInstant%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-toLocalDate-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toLocalDate%2D%2D[toLocalDate]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toLocalDate%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-toLocalDateTime-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toLocalDateTime%2D%2D[toLocalDateTime]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toLocalDateTime%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-toLocalTime-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toLocalTime%2D%2D[toLocalTime]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toLocalTime%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-toString-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#toString%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-with-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-with-2]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-withEarlierOffsetAtOverlap-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withEarlierOffsetAtOverlap%2D%2D[withEarlierOffsetAtOverlap]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withEarlierOffsetAtOverlap%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-withLaterOffsetAtOverlap-0]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withLaterOffsetAtOverlap%2D%2D[withLaterOffsetAtOverlap]()++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withLaterOffsetAtOverlap%2D%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-withZoneSameInstant-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withZoneSameInstant%2Djava.time.ZoneId%2D[withZoneSameInstant](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withZoneSameInstant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ChronoZonedDateTime-withZoneSameLocal-1]]<> link:{java8-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withZoneSameLocal%2Djava.time.ZoneId%2D[withZoneSameLocal](<>)++ (link:{java9-javadoc}/java/time/chrono/ChronoZonedDateTime.html#withZoneSameLocal%2Djava.time.ZoneId%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Chronology.asciidoc b/docs/painless/painless-api-reference/Chronology.asciidoc deleted file mode 100644 index 42b395d930a40..0000000000000 --- a/docs/painless/painless-api-reference/Chronology.asciidoc +++ /dev/null @@ -1,33 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Chronology]]++Chronology++:: -* ++[[painless-api-reference-Chronology-from-1]]static <> link:{java8-javadoc}/java/time/chrono/Chronology.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Chronology-getAvailableChronologies-0]]static <> link:{java8-javadoc}/java/time/chrono/Chronology.html#getAvailableChronologies%2D%2D[getAvailableChronologies]()++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#getAvailableChronologies%2D%2D[java 9]) -* ++[[painless-api-reference-Chronology-of-1]]static <> link:{java8-javadoc}/java/time/chrono/Chronology.html#of%2Djava.lang.String%2D[of](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#of%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Chronology-ofLocale-1]]static <> link:{java8-javadoc}/java/time/chrono/Chronology.html#ofLocale%2Djava.util.Locale%2D[ofLocale](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#ofLocale%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Chronology-compareTo-1]]int link:{java8-javadoc}/java/time/chrono/Chronology.html#compareTo%2Djava.time.chrono.Chronology%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#compareTo%2Djava.time.chrono.Chronology%2D[java 9]) -* ++[[painless-api-reference-Chronology-date-1]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[date](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Chronology-date-3]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#date%2Dint%2Dint%2Dint%2D[date](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#date%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-date-4]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[date](<>, int, int, int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-dateEpochDay-1]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#dateEpochDay%2Dlong%2D[dateEpochDay](long)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#dateEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Chronology-dateYearDay-2]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#dateYearDay%2Dint%2Dint%2D[dateYearDay](int, int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#dateYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-dateYearDay-3]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[dateYearDay](<>, int, int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-equals-1]]boolean link:{java8-javadoc}/java/time/chrono/Chronology.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Chronology-eraOf-1]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#eraOf%2Dint%2D[eraOf](int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#eraOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-eras-0]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#eras%2D%2D[eras]()++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#eras%2D%2D[java 9]) -* ++[[painless-api-reference-Chronology-getCalendarType-0]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#getCalendarType%2D%2D[getCalendarType]()++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#getCalendarType%2D%2D[java 9]) -* ++[[painless-api-reference-Chronology-getDisplayName-2]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[getDisplayName](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Chronology-getId-0]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#getId%2D%2D[getId]()++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#getId%2D%2D[java 9]) -* ++[[painless-api-reference-Chronology-hashCode-0]]int link:{java8-javadoc}/java/time/chrono/Chronology.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-Chronology-isLeapYear-1]]boolean link:{java8-javadoc}/java/time/chrono/Chronology.html#isLeapYear%2Dlong%2D[isLeapYear](long)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#isLeapYear%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Chronology-localDateTime-1]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#localDateTime%2Djava.time.temporal.TemporalAccessor%2D[localDateTime](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#localDateTime%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Chronology-period-3]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#period%2Dint%2Dint%2Dint%2D[period](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#period%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-prolepticYear-2]]int link:{java8-javadoc}/java/time/chrono/Chronology.html#prolepticYear%2Djava.time.chrono.Era%2Dint%2D[prolepticYear](<>, int)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#prolepticYear%2Djava.time.chrono.Era%2Dint%2D[java 9]) -* ++[[painless-api-reference-Chronology-range-1]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#range%2Djava.time.temporal.ChronoField%2D[range](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#range%2Djava.time.temporal.ChronoField%2D[java 9]) -* ++[[painless-api-reference-Chronology-resolveDate-2]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[resolveDate](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[java 9]) -* ++[[painless-api-reference-Chronology-toString-0]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#toString%2D%2D[java 9]) -* ++[[painless-api-reference-Chronology-zonedDateTime-1]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#zonedDateTime%2Djava.time.temporal.TemporalAccessor%2D[zonedDateTime](<>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#zonedDateTime%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Chronology-zonedDateTime-2]]<> link:{java8-javadoc}/java/time/chrono/Chronology.html#zonedDateTime%2Djava.time.Instant%2Djava.time.ZoneId%2D[zonedDateTime](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/Chronology.html#zonedDateTime%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) diff --git a/docs/painless/painless-api-reference/ClassCastException.asciidoc b/docs/painless/painless-api-reference/ClassCastException.asciidoc deleted file mode 100644 index f837c66f9db11..0000000000000 --- a/docs/painless/painless-api-reference/ClassCastException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ClassCastException]]++ClassCastException++:: -* ++[[painless-api-reference-ClassCastException-ClassCastException-0]]link:{java8-javadoc}/java/lang/ClassCastException.html#ClassCastException%2D%2D[ClassCastException]()++ (link:{java9-javadoc}/java/lang/ClassCastException.html#ClassCastException%2D%2D[java 9]) -* ++[[painless-api-reference-ClassCastException-ClassCastException-1]]link:{java8-javadoc}/java/lang/ClassCastException.html#ClassCastException%2Djava.lang.String%2D[ClassCastException](<>)++ (link:{java9-javadoc}/java/lang/ClassCastException.html#ClassCastException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ClassNotFoundException.asciidoc b/docs/painless/painless-api-reference/ClassNotFoundException.asciidoc deleted file mode 100644 index f4d0d63aa0605..0000000000000 --- a/docs/painless/painless-api-reference/ClassNotFoundException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ClassNotFoundException]]++ClassNotFoundException++:: -* ++[[painless-api-reference-ClassNotFoundException-ClassNotFoundException-0]]link:{java8-javadoc}/java/lang/ClassNotFoundException.html#ClassNotFoundException%2D%2D[ClassNotFoundException]()++ (link:{java9-javadoc}/java/lang/ClassNotFoundException.html#ClassNotFoundException%2D%2D[java 9]) -* ++[[painless-api-reference-ClassNotFoundException-ClassNotFoundException-1]]link:{java8-javadoc}/java/lang/ClassNotFoundException.html#ClassNotFoundException%2Djava.lang.String%2D[ClassNotFoundException](<>)++ (link:{java9-javadoc}/java/lang/ClassNotFoundException.html#ClassNotFoundException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Clock.asciidoc b/docs/painless/painless-api-reference/Clock.asciidoc deleted file mode 100644 index c40a09a2fe92a..0000000000000 --- a/docs/painless/painless-api-reference/Clock.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Clock]]++Clock++:: -* ++[[painless-api-reference-Clock-fixed-2]]static <> link:{java8-javadoc}/java/time/Clock.html#fixed%2Djava.time.Instant%2Djava.time.ZoneId%2D[fixed](<>, <>)++ (link:{java9-javadoc}/java/time/Clock.html#fixed%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-Clock-offset-2]]static <> link:{java8-javadoc}/java/time/Clock.html#offset%2Djava.time.Clock%2Djava.time.Duration%2D[offset](<>, <>)++ (link:{java9-javadoc}/java/time/Clock.html#offset%2Djava.time.Clock%2Djava.time.Duration%2D[java 9]) -* ++[[painless-api-reference-Clock-tick-2]]static <> link:{java8-javadoc}/java/time/Clock.html#tick%2Djava.time.Clock%2Djava.time.Duration%2D[tick](<>, <>)++ (link:{java9-javadoc}/java/time/Clock.html#tick%2Djava.time.Clock%2Djava.time.Duration%2D[java 9]) -* ++[[painless-api-reference-Clock-getZone-0]]<> link:{java8-javadoc}/java/time/Clock.html#getZone%2D%2D[getZone]()++ (link:{java9-javadoc}/java/time/Clock.html#getZone%2D%2D[java 9]) -* ++[[painless-api-reference-Clock-instant-0]]<> link:{java8-javadoc}/java/time/Clock.html#instant%2D%2D[instant]()++ (link:{java9-javadoc}/java/time/Clock.html#instant%2D%2D[java 9]) -* ++[[painless-api-reference-Clock-millis-0]]long link:{java8-javadoc}/java/time/Clock.html#millis%2D%2D[millis]()++ (link:{java9-javadoc}/java/time/Clock.html#millis%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/CloneNotSupportedException.asciidoc b/docs/painless/painless-api-reference/CloneNotSupportedException.asciidoc deleted file mode 100644 index deeb87d599b5c..0000000000000 --- a/docs/painless/painless-api-reference/CloneNotSupportedException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-CloneNotSupportedException]]++CloneNotSupportedException++:: -* ++[[painless-api-reference-CloneNotSupportedException-CloneNotSupportedException-0]]link:{java8-javadoc}/java/lang/CloneNotSupportedException.html#CloneNotSupportedException%2D%2D[CloneNotSupportedException]()++ (link:{java9-javadoc}/java/lang/CloneNotSupportedException.html#CloneNotSupportedException%2D%2D[java 9]) -* ++[[painless-api-reference-CloneNotSupportedException-CloneNotSupportedException-1]]link:{java8-javadoc}/java/lang/CloneNotSupportedException.html#CloneNotSupportedException%2Djava.lang.String%2D[CloneNotSupportedException](<>)++ (link:{java9-javadoc}/java/lang/CloneNotSupportedException.html#CloneNotSupportedException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/CollationElementIterator.asciidoc b/docs/painless/painless-api-reference/CollationElementIterator.asciidoc deleted file mode 100644 index f277709123d8a..0000000000000 --- a/docs/painless/painless-api-reference/CollationElementIterator.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-CollationElementIterator]]++CollationElementIterator++:: -** [[painless-api-reference-CollationElementIterator-NULLORDER]]static int link:{java8-javadoc}/java/text/CollationElementIterator.html#NULLORDER[NULLORDER] (link:{java9-javadoc}/java/text/CollationElementIterator.html#NULLORDER[java 9]) -* ++[[painless-api-reference-CollationElementIterator-primaryOrder-1]]static int link:{java8-javadoc}/java/text/CollationElementIterator.html#primaryOrder%2Dint%2D[primaryOrder](int)++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#primaryOrder%2Dint%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-secondaryOrder-1]]static short link:{java8-javadoc}/java/text/CollationElementIterator.html#secondaryOrder%2Dint%2D[secondaryOrder](int)++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#secondaryOrder%2Dint%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-tertiaryOrder-1]]static short link:{java8-javadoc}/java/text/CollationElementIterator.html#tertiaryOrder%2Dint%2D[tertiaryOrder](int)++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#tertiaryOrder%2Dint%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-getMaxExpansion-1]]int link:{java8-javadoc}/java/text/CollationElementIterator.html#getMaxExpansion%2Dint%2D[getMaxExpansion](int)++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#getMaxExpansion%2Dint%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-getOffset-0]]int link:{java8-javadoc}/java/text/CollationElementIterator.html#getOffset%2D%2D[getOffset]()++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#getOffset%2D%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-next-0]]int link:{java8-javadoc}/java/text/CollationElementIterator.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-previous-0]]int link:{java8-javadoc}/java/text/CollationElementIterator.html#previous%2D%2D[previous]()++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#previous%2D%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-reset-0]]void link:{java8-javadoc}/java/text/CollationElementIterator.html#reset%2D%2D[reset]()++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#reset%2D%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-setOffset-1]]void link:{java8-javadoc}/java/text/CollationElementIterator.html#setOffset%2Dint%2D[setOffset](int)++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#setOffset%2Dint%2D[java 9]) -* ++[[painless-api-reference-CollationElementIterator-setText-1]]void link:{java8-javadoc}/java/text/CollationElementIterator.html#setText%2Djava.lang.String%2D[setText](<>)++ (link:{java9-javadoc}/java/text/CollationElementIterator.html#setText%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/CollationKey.asciidoc b/docs/painless/painless-api-reference/CollationKey.asciidoc deleted file mode 100644 index adb6d29cf81bc..0000000000000 --- a/docs/painless/painless-api-reference/CollationKey.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-CollationKey]]++CollationKey++:: -* ++[[painless-api-reference-CollationKey-compareTo-1]]int link:{java8-javadoc}/java/text/CollationKey.html#compareTo%2Djava.text.CollationKey%2D[compareTo](<>)++ (link:{java9-javadoc}/java/text/CollationKey.html#compareTo%2Djava.text.CollationKey%2D[java 9]) -* ++[[painless-api-reference-CollationKey-getSourceString-0]]<> link:{java8-javadoc}/java/text/CollationKey.html#getSourceString%2D%2D[getSourceString]()++ (link:{java9-javadoc}/java/text/CollationKey.html#getSourceString%2D%2D[java 9]) -* ++[[painless-api-reference-CollationKey-toByteArray-0]]byte[] link:{java8-javadoc}/java/text/CollationKey.html#toByteArray%2D%2D[toByteArray]()++ (link:{java9-javadoc}/java/text/CollationKey.html#toByteArray%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Collator.asciidoc b/docs/painless/painless-api-reference/Collator.asciidoc deleted file mode 100644 index a14f29d986851..0000000000000 --- a/docs/painless/painless-api-reference/Collator.asciidoc +++ /dev/null @@ -1,24 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Collator]]++Collator++:: -** [[painless-api-reference-Collator-CANONICAL_DECOMPOSITION]]static int link:{java8-javadoc}/java/text/Collator.html#CANONICAL_DECOMPOSITION[CANONICAL_DECOMPOSITION] (link:{java9-javadoc}/java/text/Collator.html#CANONICAL_DECOMPOSITION[java 9]) -** [[painless-api-reference-Collator-FULL_DECOMPOSITION]]static int link:{java8-javadoc}/java/text/Collator.html#FULL_DECOMPOSITION[FULL_DECOMPOSITION] (link:{java9-javadoc}/java/text/Collator.html#FULL_DECOMPOSITION[java 9]) -** [[painless-api-reference-Collator-IDENTICAL]]static int link:{java8-javadoc}/java/text/Collator.html#IDENTICAL[IDENTICAL] (link:{java9-javadoc}/java/text/Collator.html#IDENTICAL[java 9]) -** [[painless-api-reference-Collator-NO_DECOMPOSITION]]static int link:{java8-javadoc}/java/text/Collator.html#NO_DECOMPOSITION[NO_DECOMPOSITION] (link:{java9-javadoc}/java/text/Collator.html#NO_DECOMPOSITION[java 9]) -** [[painless-api-reference-Collator-PRIMARY]]static int link:{java8-javadoc}/java/text/Collator.html#PRIMARY[PRIMARY] (link:{java9-javadoc}/java/text/Collator.html#PRIMARY[java 9]) -** [[painless-api-reference-Collator-SECONDARY]]static int link:{java8-javadoc}/java/text/Collator.html#SECONDARY[SECONDARY] (link:{java9-javadoc}/java/text/Collator.html#SECONDARY[java 9]) -** [[painless-api-reference-Collator-TERTIARY]]static int link:{java8-javadoc}/java/text/Collator.html#TERTIARY[TERTIARY] (link:{java9-javadoc}/java/text/Collator.html#TERTIARY[java 9]) -* ++[[painless-api-reference-Collator-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/text/Collator.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/text/Collator.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-Collator-getInstance-0]]static <> link:{java8-javadoc}/java/text/Collator.html#getInstance%2D%2D[getInstance]()++ (link:{java9-javadoc}/java/text/Collator.html#getInstance%2D%2D[java 9]) -* ++[[painless-api-reference-Collator-getInstance-1]]static <> link:{java8-javadoc}/java/text/Collator.html#getInstance%2Djava.util.Locale%2D[getInstance](<>)++ (link:{java9-javadoc}/java/text/Collator.html#getInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Collator-clone-0]]def link:{java8-javadoc}/java/text/Collator.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/text/Collator.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-Collator-equals-2]]boolean link:{java8-javadoc}/java/text/Collator.html#equals%2Djava.lang.String%2Djava.lang.String%2D[equals](<>, <>)++ (link:{java9-javadoc}/java/text/Collator.html#equals%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Collator-getCollationKey-1]]<> link:{java8-javadoc}/java/text/Collator.html#getCollationKey%2Djava.lang.String%2D[getCollationKey](<>)++ (link:{java9-javadoc}/java/text/Collator.html#getCollationKey%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Collator-getDecomposition-0]]int link:{java8-javadoc}/java/text/Collator.html#getDecomposition%2D%2D[getDecomposition]()++ (link:{java9-javadoc}/java/text/Collator.html#getDecomposition%2D%2D[java 9]) -* ++[[painless-api-reference-Collator-getStrength-0]]int link:{java8-javadoc}/java/text/Collator.html#getStrength%2D%2D[getStrength]()++ (link:{java9-javadoc}/java/text/Collator.html#getStrength%2D%2D[java 9]) -* ++[[painless-api-reference-Collator-setDecomposition-1]]void link:{java8-javadoc}/java/text/Collator.html#setDecomposition%2Dint%2D[setDecomposition](int)++ (link:{java9-javadoc}/java/text/Collator.html#setDecomposition%2Dint%2D[java 9]) -* ++[[painless-api-reference-Collator-setStrength-1]]void link:{java8-javadoc}/java/text/Collator.html#setStrength%2Dint%2D[setStrength](int)++ (link:{java9-javadoc}/java/text/Collator.html#setStrength%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Collection.asciidoc b/docs/painless/painless-api-reference/Collection.asciidoc deleted file mode 100644 index e8faba23a6109..0000000000000 --- a/docs/painless/painless-api-reference/Collection.asciidoc +++ /dev/null @@ -1,28 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Collection]]++Collection++:: -* ++[[painless-api-reference-Collection-add-1]]boolean link:{java8-javadoc}/java/util/Collection.html#add%2Djava.lang.Object%2D[add](def)++ (link:{java9-javadoc}/java/util/Collection.html#add%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collection-addAll-1]]boolean link:{java8-javadoc}/java/util/Collection.html#addAll%2Djava.util.Collection%2D[addAll](<>)++ (link:{java9-javadoc}/java/util/Collection.html#addAll%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collection-clear-0]]void link:{java8-javadoc}/java/util/Collection.html#clear%2D%2D[clear]()++ (link:{java9-javadoc}/java/util/Collection.html#clear%2D%2D[java 9]) -* ++[[painless-api-reference-Collection-collect-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#collect%2Djava.util.Collection%2Djava.util.function.Function%2D[collect](<>)++ -* ++[[painless-api-reference-Collection-collect-2]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#collect%2Djava.util.Collection%2Djava.util.Collection%2Djava.util.function.Function%2D[collect](<>, <>)++ -* ++[[painless-api-reference-Collection-contains-1]]boolean link:{java8-javadoc}/java/util/Collection.html#contains%2Djava.lang.Object%2D[contains](def)++ (link:{java9-javadoc}/java/util/Collection.html#contains%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collection-containsAll-1]]boolean link:{java8-javadoc}/java/util/Collection.html#containsAll%2Djava.util.Collection%2D[containsAll](<>)++ (link:{java9-javadoc}/java/util/Collection.html#containsAll%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collection-find-1]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#find%2Djava.util.Collection%2Djava.util.function.Predicate%2D[find](<>)++ -* ++[[painless-api-reference-Collection-findAll-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findAll%2Djava.util.Collection%2Djava.util.function.Predicate%2D[findAll](<>)++ -* ++[[painless-api-reference-Collection-findResult-1]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findResult%2Djava.util.Collection%2Djava.util.function.Function%2D[findResult](<>)++ -* ++[[painless-api-reference-Collection-findResult-2]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findResult%2Djava.util.Collection%2Djava.lang.Object%2Djava.util.function.Function%2D[findResult](def, <>)++ -* ++[[painless-api-reference-Collection-isEmpty-0]]boolean link:{java8-javadoc}/java/util/Collection.html#isEmpty%2D%2D[isEmpty]()++ (link:{java9-javadoc}/java/util/Collection.html#isEmpty%2D%2D[java 9]) -* ++[[painless-api-reference-Collection-removeAll-1]]boolean link:{java8-javadoc}/java/util/Collection.html#removeAll%2Djava.util.Collection%2D[removeAll](<>)++ (link:{java9-javadoc}/java/util/Collection.html#removeAll%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collection-removeIf-1]]boolean link:{java8-javadoc}/java/util/Collection.html#removeIf%2Djava.util.function.Predicate%2D[removeIf](<>)++ (link:{java9-javadoc}/java/util/Collection.html#removeIf%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Collection-retainAll-1]]boolean link:{java8-javadoc}/java/util/Collection.html#retainAll%2Djava.util.Collection%2D[retainAll](<>)++ (link:{java9-javadoc}/java/util/Collection.html#retainAll%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collection-size-0]]int link:{java8-javadoc}/java/util/Collection.html#size%2D%2D[size]()++ (link:{java9-javadoc}/java/util/Collection.html#size%2D%2D[java 9]) -* ++[[painless-api-reference-Collection-split-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#split%2Djava.util.Collection%2Djava.util.function.Predicate%2D[split](<>)++ -* ++[[painless-api-reference-Collection-spliterator-0]]<> link:{java8-javadoc}/java/util/Collection.html#spliterator%2D%2D[spliterator]()++ (link:{java9-javadoc}/java/util/Collection.html#spliterator%2D%2D[java 9]) -* ++[[painless-api-reference-Collection-stream-0]]<> link:{java8-javadoc}/java/util/Collection.html#stream%2D%2D[stream]()++ (link:{java9-javadoc}/java/util/Collection.html#stream%2D%2D[java 9]) -* ++[[painless-api-reference-Collection-toArray-0]]def[] link:{java8-javadoc}/java/util/Collection.html#toArray%2D%2D[toArray]()++ (link:{java9-javadoc}/java/util/Collection.html#toArray%2D%2D[java 9]) -* ++[[painless-api-reference-Collection-toArray-1]]def[] link:{java8-javadoc}/java/util/Collection.html#toArray%2Djava.lang.Object:A%2D[toArray](def[])++ (link:{java9-javadoc}/java/util/Collection.html#toArray%2Djava.lang.Object:A%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Collections.asciidoc b/docs/painless/painless-api-reference/Collections.asciidoc deleted file mode 100644 index db6d87e85210e..0000000000000 --- a/docs/painless/painless-api-reference/Collections.asciidoc +++ /dev/null @@ -1,59 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Collections]]++Collections++:: -** [[painless-api-reference-Collections-EMPTY_LIST]]static <> link:{java8-javadoc}/java/util/Collections.html#EMPTY_LIST[EMPTY_LIST] (link:{java9-javadoc}/java/util/Collections.html#EMPTY_LIST[java 9]) -** [[painless-api-reference-Collections-EMPTY_MAP]]static <> link:{java8-javadoc}/java/util/Collections.html#EMPTY_MAP[EMPTY_MAP] (link:{java9-javadoc}/java/util/Collections.html#EMPTY_MAP[java 9]) -** [[painless-api-reference-Collections-EMPTY_SET]]static <> link:{java8-javadoc}/java/util/Collections.html#EMPTY_SET[EMPTY_SET] (link:{java9-javadoc}/java/util/Collections.html#EMPTY_SET[java 9]) -* ++[[painless-api-reference-Collections-addAll-2]]static boolean link:{java8-javadoc}/java/util/Collections.html#addAll%2Djava.util.Collection%2Djava.lang.Object:A%2D[addAll](<>, def[])++ (link:{java9-javadoc}/java/util/Collections.html#addAll%2Djava.util.Collection%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Collections-asLifoQueue-1]]static <> link:{java8-javadoc}/java/util/Collections.html#asLifoQueue%2Djava.util.Deque%2D[asLifoQueue](<>)++ (link:{java9-javadoc}/java/util/Collections.html#asLifoQueue%2Djava.util.Deque%2D[java 9]) -* ++[[painless-api-reference-Collections-binarySearch-2]]static int link:{java8-javadoc}/java/util/Collections.html#binarySearch%2Djava.util.List%2Djava.lang.Object%2D[binarySearch](<>, def)++ (link:{java9-javadoc}/java/util/Collections.html#binarySearch%2Djava.util.List%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-binarySearch-3]]static int link:{java8-javadoc}/java/util/Collections.html#binarySearch%2Djava.util.List%2Djava.lang.Object%2Djava.util.Comparator%2D[binarySearch](<>, def, <>)++ (link:{java9-javadoc}/java/util/Collections.html#binarySearch%2Djava.util.List%2Djava.lang.Object%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collections-copy-2]]static void link:{java8-javadoc}/java/util/Collections.html#copy%2Djava.util.List%2Djava.util.List%2D[copy](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#copy%2Djava.util.List%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-disjoint-2]]static boolean link:{java8-javadoc}/java/util/Collections.html#disjoint%2Djava.util.Collection%2Djava.util.Collection%2D[disjoint](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#disjoint%2Djava.util.Collection%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyEnumeration-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyEnumeration%2D%2D[emptyEnumeration]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyEnumeration%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyIterator-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyIterator%2D%2D[emptyIterator]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyIterator%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyList-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyList%2D%2D[emptyList]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyList%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyListIterator-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyListIterator%2D%2D[emptyListIterator]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyListIterator%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyMap-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyMap%2D%2D[emptyMap]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyMap%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyNavigableMap-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyNavigableMap%2D%2D[emptyNavigableMap]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyNavigableMap%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptyNavigableSet-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptyNavigableSet%2D%2D[emptyNavigableSet]()++ (link:{java9-javadoc}/java/util/Collections.html#emptyNavigableSet%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptySet-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptySet%2D%2D[emptySet]()++ (link:{java9-javadoc}/java/util/Collections.html#emptySet%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptySortedMap-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptySortedMap%2D%2D[emptySortedMap]()++ (link:{java9-javadoc}/java/util/Collections.html#emptySortedMap%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-emptySortedSet-0]]static <> link:{java8-javadoc}/java/util/Collections.html#emptySortedSet%2D%2D[emptySortedSet]()++ (link:{java9-javadoc}/java/util/Collections.html#emptySortedSet%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-enumeration-1]]static <> link:{java8-javadoc}/java/util/Collections.html#enumeration%2Djava.util.Collection%2D[enumeration](<>)++ (link:{java9-javadoc}/java/util/Collections.html#enumeration%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collections-fill-2]]static void link:{java8-javadoc}/java/util/Collections.html#fill%2Djava.util.List%2Djava.lang.Object%2D[fill](<>, def)++ (link:{java9-javadoc}/java/util/Collections.html#fill%2Djava.util.List%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-frequency-2]]static int link:{java8-javadoc}/java/util/Collections.html#frequency%2Djava.util.Collection%2Djava.lang.Object%2D[frequency](<>, def)++ (link:{java9-javadoc}/java/util/Collections.html#frequency%2Djava.util.Collection%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-indexOfSubList-2]]static int link:{java8-javadoc}/java/util/Collections.html#indexOfSubList%2Djava.util.List%2Djava.util.List%2D[indexOfSubList](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#indexOfSubList%2Djava.util.List%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-lastIndexOfSubList-2]]static int link:{java8-javadoc}/java/util/Collections.html#lastIndexOfSubList%2Djava.util.List%2Djava.util.List%2D[lastIndexOfSubList](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#lastIndexOfSubList%2Djava.util.List%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-list-1]]static <> link:{java8-javadoc}/java/util/Collections.html#list%2Djava.util.Enumeration%2D[list](<>)++ (link:{java9-javadoc}/java/util/Collections.html#list%2Djava.util.Enumeration%2D[java 9]) -* ++[[painless-api-reference-Collections-max-1]]static def link:{java8-javadoc}/java/util/Collections.html#max%2Djava.util.Collection%2D[max](<>)++ (link:{java9-javadoc}/java/util/Collections.html#max%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collections-max-2]]static def link:{java8-javadoc}/java/util/Collections.html#max%2Djava.util.Collection%2Djava.util.Comparator%2D[max](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#max%2Djava.util.Collection%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collections-min-1]]static def link:{java8-javadoc}/java/util/Collections.html#min%2Djava.util.Collection%2D[min](<>)++ (link:{java9-javadoc}/java/util/Collections.html#min%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collections-min-2]]static def link:{java8-javadoc}/java/util/Collections.html#min%2Djava.util.Collection%2Djava.util.Comparator%2D[min](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#min%2Djava.util.Collection%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collections-nCopies-2]]static <> link:{java8-javadoc}/java/util/Collections.html#nCopies%2Dint%2Djava.lang.Object%2D[nCopies](int, def)++ (link:{java9-javadoc}/java/util/Collections.html#nCopies%2Dint%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-newSetFromMap-1]]static <> link:{java8-javadoc}/java/util/Collections.html#newSetFromMap%2Djava.util.Map%2D[newSetFromMap](<>)++ (link:{java9-javadoc}/java/util/Collections.html#newSetFromMap%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-Collections-replaceAll-3]]static boolean link:{java8-javadoc}/java/util/Collections.html#replaceAll%2Djava.util.List%2Djava.lang.Object%2Djava.lang.Object%2D[replaceAll](<>, def, def)++ (link:{java9-javadoc}/java/util/Collections.html#replaceAll%2Djava.util.List%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-reverse-1]]static void link:{java8-javadoc}/java/util/Collections.html#reverse%2Djava.util.List%2D[reverse](<>)++ (link:{java9-javadoc}/java/util/Collections.html#reverse%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-reverseOrder-0]]static <> link:{java8-javadoc}/java/util/Collections.html#reverseOrder%2D%2D[reverseOrder]()++ (link:{java9-javadoc}/java/util/Collections.html#reverseOrder%2D%2D[java 9]) -* ++[[painless-api-reference-Collections-reverseOrder-1]]static <> link:{java8-javadoc}/java/util/Collections.html#reverseOrder%2Djava.util.Comparator%2D[reverseOrder](<>)++ (link:{java9-javadoc}/java/util/Collections.html#reverseOrder%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collections-rotate-2]]static void link:{java8-javadoc}/java/util/Collections.html#rotate%2Djava.util.List%2Dint%2D[rotate](<>, int)++ (link:{java9-javadoc}/java/util/Collections.html#rotate%2Djava.util.List%2Dint%2D[java 9]) -* ++[[painless-api-reference-Collections-shuffle-1]]static void link:{java8-javadoc}/java/util/Collections.html#shuffle%2Djava.util.List%2D[shuffle](<>)++ (link:{java9-javadoc}/java/util/Collections.html#shuffle%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-shuffle-2]]static void link:{java8-javadoc}/java/util/Collections.html#shuffle%2Djava.util.List%2Djava.util.Random%2D[shuffle](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#shuffle%2Djava.util.List%2Djava.util.Random%2D[java 9]) -* ++[[painless-api-reference-Collections-singleton-1]]static <> link:{java8-javadoc}/java/util/Collections.html#singleton%2Djava.lang.Object%2D[singleton](def)++ (link:{java9-javadoc}/java/util/Collections.html#singleton%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-singletonList-1]]static <> link:{java8-javadoc}/java/util/Collections.html#singletonList%2Djava.lang.Object%2D[singletonList](def)++ (link:{java9-javadoc}/java/util/Collections.html#singletonList%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-singletonMap-2]]static <> link:{java8-javadoc}/java/util/Collections.html#singletonMap%2Djava.lang.Object%2Djava.lang.Object%2D[singletonMap](def, def)++ (link:{java9-javadoc}/java/util/Collections.html#singletonMap%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Collections-sort-1]]static void link:{java8-javadoc}/java/util/Collections.html#sort%2Djava.util.List%2D[sort](<>)++ (link:{java9-javadoc}/java/util/Collections.html#sort%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-sort-2]]static void link:{java8-javadoc}/java/util/Collections.html#sort%2Djava.util.List%2Djava.util.Comparator%2D[sort](<>, <>)++ (link:{java9-javadoc}/java/util/Collections.html#sort%2Djava.util.List%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collections-swap-3]]static void link:{java8-javadoc}/java/util/Collections.html#swap%2Djava.util.List%2Dint%2Dint%2D[swap](<>, int, int)++ (link:{java9-javadoc}/java/util/Collections.html#swap%2Djava.util.List%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableCollection-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableCollection%2Djava.util.Collection%2D[unmodifiableCollection](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableCollection%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableList-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableList%2Djava.util.List%2D[unmodifiableList](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableList%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableMap-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableMap%2Djava.util.Map%2D[unmodifiableMap](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableMap%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableNavigableMap-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableNavigableMap%2Djava.util.NavigableMap%2D[unmodifiableNavigableMap](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableNavigableMap%2Djava.util.NavigableMap%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableNavigableSet-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableNavigableSet%2Djava.util.NavigableSet%2D[unmodifiableNavigableSet](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableNavigableSet%2Djava.util.NavigableSet%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableSet-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableSet%2Djava.util.Set%2D[unmodifiableSet](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableSet%2Djava.util.Set%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableSortedMap-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableSortedMap%2Djava.util.SortedMap%2D[unmodifiableSortedMap](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableSortedMap%2Djava.util.SortedMap%2D[java 9]) -* ++[[painless-api-reference-Collections-unmodifiableSortedSet-1]]static <> link:{java8-javadoc}/java/util/Collections.html#unmodifiableSortedSet%2Djava.util.SortedSet%2D[unmodifiableSortedSet](<>)++ (link:{java9-javadoc}/java/util/Collections.html#unmodifiableSortedSet%2Djava.util.SortedSet%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Collector.Characteristics.asciidoc b/docs/painless/painless-api-reference/Collector.Characteristics.asciidoc deleted file mode 100644 index a0c95223d57d0..0000000000000 --- a/docs/painless/painless-api-reference/Collector.Characteristics.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Collector-Characteristics]]++Collector.Characteristics++:: -** [[painless-api-reference-Collector-Characteristics-CONCURRENT]]static <> link:{java8-javadoc}/java/util/stream/Collector.Characteristics.html#CONCURRENT[CONCURRENT] (link:{java9-javadoc}/java/util/stream/Collector.Characteristics.html#CONCURRENT[java 9]) -** [[painless-api-reference-Collector-Characteristics-IDENTITY_FINISH]]static <> link:{java8-javadoc}/java/util/stream/Collector.Characteristics.html#IDENTITY_FINISH[IDENTITY_FINISH] (link:{java9-javadoc}/java/util/stream/Collector.Characteristics.html#IDENTITY_FINISH[java 9]) -** [[painless-api-reference-Collector-Characteristics-UNORDERED]]static <> link:{java8-javadoc}/java/util/stream/Collector.Characteristics.html#UNORDERED[UNORDERED] (link:{java9-javadoc}/java/util/stream/Collector.Characteristics.html#UNORDERED[java 9]) -* ++[[painless-api-reference-Collector-Characteristics-valueOf-1]]static <> link:{java8-javadoc}/java/util/stream/Collector.Characteristics.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/util/stream/Collector.Characteristics.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Collector-Characteristics-values-0]]static <>[] link:{java8-javadoc}/java/util/stream/Collector.Characteristics.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/util/stream/Collector.Characteristics.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Collector.asciidoc b/docs/painless/painless-api-reference/Collector.asciidoc deleted file mode 100644 index 1d1b92f1e0911..0000000000000 --- a/docs/painless/painless-api-reference/Collector.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Collector]]++Collector++:: -* ++[[painless-api-reference-Collector-of-4]]static <> link:{java8-javadoc}/java/util/stream/Collector.html#of%2Djava.util.function.Supplier%2Djava.util.function.BiConsumer%2Djava.util.function.BinaryOperator%2Djava.util.stream.Collector$Characteristics:A%2D[of](<>, <>, <>, <>[])++ (link:{java9-javadoc}/java/util/stream/Collector.html#of%2Djava.util.function.Supplier%2Djava.util.function.BiConsumer%2Djava.util.function.BinaryOperator%2Djava.util.stream.Collector$Characteristics:A%2D[java 9]) -* ++[[painless-api-reference-Collector-of-5]]static <> link:{java8-javadoc}/java/util/stream/Collector.html#of%2Djava.util.function.Supplier%2Djava.util.function.BiConsumer%2Djava.util.function.BinaryOperator%2Djava.util.function.Function%2Djava.util.stream.Collector$Characteristics:A%2D[of](<>, <>, <>, <>, <>[])++ (link:{java9-javadoc}/java/util/stream/Collector.html#of%2Djava.util.function.Supplier%2Djava.util.function.BiConsumer%2Djava.util.function.BinaryOperator%2Djava.util.function.Function%2Djava.util.stream.Collector$Characteristics:A%2D[java 9]) -* ++[[painless-api-reference-Collector-accumulator-0]]<> link:{java8-javadoc}/java/util/stream/Collector.html#accumulator%2D%2D[accumulator]()++ (link:{java9-javadoc}/java/util/stream/Collector.html#accumulator%2D%2D[java 9]) -* ++[[painless-api-reference-Collector-characteristics-0]]<> link:{java8-javadoc}/java/util/stream/Collector.html#characteristics%2D%2D[characteristics]()++ (link:{java9-javadoc}/java/util/stream/Collector.html#characteristics%2D%2D[java 9]) -* ++[[painless-api-reference-Collector-combiner-0]]<> link:{java8-javadoc}/java/util/stream/Collector.html#combiner%2D%2D[combiner]()++ (link:{java9-javadoc}/java/util/stream/Collector.html#combiner%2D%2D[java 9]) -* ++[[painless-api-reference-Collector-finisher-0]]<> link:{java8-javadoc}/java/util/stream/Collector.html#finisher%2D%2D[finisher]()++ (link:{java9-javadoc}/java/util/stream/Collector.html#finisher%2D%2D[java 9]) -* ++[[painless-api-reference-Collector-supplier-0]]<> link:{java8-javadoc}/java/util/stream/Collector.html#supplier%2D%2D[supplier]()++ (link:{java9-javadoc}/java/util/stream/Collector.html#supplier%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Collectors.asciidoc b/docs/painless/painless-api-reference/Collectors.asciidoc deleted file mode 100644 index ef4d0199386e1..0000000000000 --- a/docs/painless/painless-api-reference/Collectors.asciidoc +++ /dev/null @@ -1,38 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Collectors]]++Collectors++:: -* ++[[painless-api-reference-Collectors-averagingDouble-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#averagingDouble%2Djava.util.function.ToDoubleFunction%2D[averagingDouble](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#averagingDouble%2Djava.util.function.ToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-averagingInt-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#averagingInt%2Djava.util.function.ToIntFunction%2D[averagingInt](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#averagingInt%2Djava.util.function.ToIntFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-averagingLong-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#averagingLong%2Djava.util.function.ToLongFunction%2D[averagingLong](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#averagingLong%2Djava.util.function.ToLongFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-collectingAndThen-2]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#collectingAndThen%2Djava.util.stream.Collector%2Djava.util.function.Function%2D[collectingAndThen](<>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#collectingAndThen%2Djava.util.stream.Collector%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Collectors-counting-0]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#counting%2D%2D[counting]()++ (link:{java9-javadoc}/java/util/stream/Collectors.html#counting%2D%2D[java 9]) -* ++[[painless-api-reference-Collectors-groupingBy-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#groupingBy%2Djava.util.function.Function%2D[groupingBy](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#groupingBy%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Collectors-groupingBy-2]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#groupingBy%2Djava.util.function.Function%2Djava.util.stream.Collector%2D[groupingBy](<>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#groupingBy%2Djava.util.function.Function%2Djava.util.stream.Collector%2D[java 9]) -* ++[[painless-api-reference-Collectors-groupingBy-3]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#groupingBy%2Djava.util.function.Function%2Djava.util.function.Supplier%2Djava.util.stream.Collector%2D[groupingBy](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#groupingBy%2Djava.util.function.Function%2Djava.util.function.Supplier%2Djava.util.stream.Collector%2D[java 9]) -* ++[[painless-api-reference-Collectors-joining-0]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#joining%2D%2D[joining]()++ (link:{java9-javadoc}/java/util/stream/Collectors.html#joining%2D%2D[java 9]) -* ++[[painless-api-reference-Collectors-joining-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#joining%2Djava.lang.CharSequence%2D[joining](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#joining%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Collectors-joining-3]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#joining%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2D[joining](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#joining%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Collectors-mapping-2]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#mapping%2Djava.util.function.Function%2Djava.util.stream.Collector%2D[mapping](<>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#mapping%2Djava.util.function.Function%2Djava.util.stream.Collector%2D[java 9]) -* ++[[painless-api-reference-Collectors-maxBy-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#maxBy%2Djava.util.Comparator%2D[maxBy](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#maxBy%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collectors-minBy-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#minBy%2Djava.util.Comparator%2D[minBy](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#minBy%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Collectors-partitioningBy-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#partitioningBy%2Djava.util.function.Predicate%2D[partitioningBy](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#partitioningBy%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Collectors-partitioningBy-2]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#partitioningBy%2Djava.util.function.Predicate%2Djava.util.stream.Collector%2D[partitioningBy](<>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#partitioningBy%2Djava.util.function.Predicate%2Djava.util.stream.Collector%2D[java 9]) -* ++[[painless-api-reference-Collectors-reducing-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#reducing%2Djava.util.function.BinaryOperator%2D[reducing](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#reducing%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Collectors-reducing-2]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#reducing%2Djava.lang.Object%2Djava.util.function.BinaryOperator%2D[reducing](def, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#reducing%2Djava.lang.Object%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Collectors-reducing-3]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#reducing%2Djava.lang.Object%2Djava.util.function.Function%2Djava.util.function.BinaryOperator%2D[reducing](def, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#reducing%2Djava.lang.Object%2Djava.util.function.Function%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Collectors-summarizingDouble-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#summarizingDouble%2Djava.util.function.ToDoubleFunction%2D[summarizingDouble](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#summarizingDouble%2Djava.util.function.ToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-summarizingInt-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#summarizingInt%2Djava.util.function.ToIntFunction%2D[summarizingInt](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#summarizingInt%2Djava.util.function.ToIntFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-summarizingLong-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#summarizingLong%2Djava.util.function.ToLongFunction%2D[summarizingLong](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#summarizingLong%2Djava.util.function.ToLongFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-summingDouble-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#summingDouble%2Djava.util.function.ToDoubleFunction%2D[summingDouble](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#summingDouble%2Djava.util.function.ToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-summingInt-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#summingInt%2Djava.util.function.ToIntFunction%2D[summingInt](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#summingInt%2Djava.util.function.ToIntFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-summingLong-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#summingLong%2Djava.util.function.ToLongFunction%2D[summingLong](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#summingLong%2Djava.util.function.ToLongFunction%2D[java 9]) -* ++[[painless-api-reference-Collectors-toCollection-1]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#toCollection%2Djava.util.function.Supplier%2D[toCollection](<>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#toCollection%2Djava.util.function.Supplier%2D[java 9]) -* ++[[painless-api-reference-Collectors-toList-0]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#toList%2D%2D[toList]()++ (link:{java9-javadoc}/java/util/stream/Collectors.html#toList%2D%2D[java 9]) -* ++[[painless-api-reference-Collectors-toMap-2]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#toMap%2Djava.util.function.Function%2Djava.util.function.Function%2D[toMap](<>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#toMap%2Djava.util.function.Function%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Collectors-toMap-3]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#toMap%2Djava.util.function.Function%2Djava.util.function.Function%2Djava.util.function.BinaryOperator%2D[toMap](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#toMap%2Djava.util.function.Function%2Djava.util.function.Function%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Collectors-toMap-4]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#toMap%2Djava.util.function.Function%2Djava.util.function.Function%2Djava.util.function.BinaryOperator%2Djava.util.function.Supplier%2D[toMap](<>, <>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Collectors.html#toMap%2Djava.util.function.Function%2Djava.util.function.Function%2Djava.util.function.BinaryOperator%2Djava.util.function.Supplier%2D[java 9]) -* ++[[painless-api-reference-Collectors-toSet-0]]static <> link:{java8-javadoc}/java/util/stream/Collectors.html#toSet%2D%2D[toSet]()++ (link:{java9-javadoc}/java/util/stream/Collectors.html#toSet%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Comparable.asciidoc b/docs/painless/painless-api-reference/Comparable.asciidoc deleted file mode 100644 index 1550d93b51144..0000000000000 --- a/docs/painless/painless-api-reference/Comparable.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Comparable]]++Comparable++:: -* ++[[painless-api-reference-Comparable-compareTo-1]]int link:{java8-javadoc}/java/lang/Comparable.html#compareTo%2Djava.lang.Object%2D[compareTo](def)++ (link:{java9-javadoc}/java/lang/Comparable.html#compareTo%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Comparator.asciidoc b/docs/painless/painless-api-reference/Comparator.asciidoc deleted file mode 100644 index 25545376942cc..0000000000000 --- a/docs/painless/painless-api-reference/Comparator.asciidoc +++ /dev/null @@ -1,24 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Comparator]]++Comparator++:: -* ++[[painless-api-reference-Comparator-comparing-1]]static <> link:{java8-javadoc}/java/util/Comparator.html#comparing%2Djava.util.function.Function%2D[comparing](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#comparing%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Comparator-comparing-2]]static <> link:{java8-javadoc}/java/util/Comparator.html#comparing%2Djava.util.function.Function%2Djava.util.Comparator%2D[comparing](<>, <>)++ (link:{java9-javadoc}/java/util/Comparator.html#comparing%2Djava.util.function.Function%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Comparator-comparingDouble-1]]static <> link:{java8-javadoc}/java/util/Comparator.html#comparingDouble%2Djava.util.function.ToDoubleFunction%2D[comparingDouble](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#comparingDouble%2Djava.util.function.ToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-Comparator-comparingInt-1]]static <> link:{java8-javadoc}/java/util/Comparator.html#comparingInt%2Djava.util.function.ToIntFunction%2D[comparingInt](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#comparingInt%2Djava.util.function.ToIntFunction%2D[java 9]) -* ++[[painless-api-reference-Comparator-comparingLong-1]]static <> link:{java8-javadoc}/java/util/Comparator.html#comparingLong%2Djava.util.function.ToLongFunction%2D[comparingLong](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#comparingLong%2Djava.util.function.ToLongFunction%2D[java 9]) -* ++[[painless-api-reference-Comparator-naturalOrder-0]]static <> link:{java8-javadoc}/java/util/Comparator.html#naturalOrder%2D%2D[naturalOrder]()++ (link:{java9-javadoc}/java/util/Comparator.html#naturalOrder%2D%2D[java 9]) -* ++[[painless-api-reference-Comparator-nullsFirst-1]]static <> link:{java8-javadoc}/java/util/Comparator.html#nullsFirst%2Djava.util.Comparator%2D[nullsFirst](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#nullsFirst%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Comparator-nullsLast-1]]static <> link:{java8-javadoc}/java/util/Comparator.html#nullsLast%2Djava.util.Comparator%2D[nullsLast](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#nullsLast%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Comparator-reverseOrder-0]]static <> link:{java8-javadoc}/java/util/Comparator.html#reverseOrder%2D%2D[reverseOrder]()++ (link:{java9-javadoc}/java/util/Comparator.html#reverseOrder%2D%2D[java 9]) -* ++[[painless-api-reference-Comparator-compare-2]]int link:{java8-javadoc}/java/util/Comparator.html#compare%2Djava.lang.Object%2Djava.lang.Object%2D[compare](def, def)++ (link:{java9-javadoc}/java/util/Comparator.html#compare%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Comparator-equals-1]]boolean link:{java8-javadoc}/java/util/Comparator.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Comparator-reversed-0]]<> link:{java8-javadoc}/java/util/Comparator.html#reversed%2D%2D[reversed]()++ (link:{java9-javadoc}/java/util/Comparator.html#reversed%2D%2D[java 9]) -* ++[[painless-api-reference-Comparator-thenComparing-1]]<> link:{java8-javadoc}/java/util/Comparator.html#thenComparing%2Djava.util.Comparator%2D[thenComparing](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#thenComparing%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Comparator-thenComparing-2]]<> link:{java8-javadoc}/java/util/Comparator.html#thenComparing%2Djava.util.function.Function%2Djava.util.Comparator%2D[thenComparing](<>, <>)++ (link:{java9-javadoc}/java/util/Comparator.html#thenComparing%2Djava.util.function.Function%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Comparator-thenComparingDouble-1]]<> link:{java8-javadoc}/java/util/Comparator.html#thenComparingDouble%2Djava.util.function.ToDoubleFunction%2D[thenComparingDouble](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#thenComparingDouble%2Djava.util.function.ToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-Comparator-thenComparingInt-1]]<> link:{java8-javadoc}/java/util/Comparator.html#thenComparingInt%2Djava.util.function.ToIntFunction%2D[thenComparingInt](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#thenComparingInt%2Djava.util.function.ToIntFunction%2D[java 9]) -* ++[[painless-api-reference-Comparator-thenComparingLong-1]]<> link:{java8-javadoc}/java/util/Comparator.html#thenComparingLong%2Djava.util.function.ToLongFunction%2D[thenComparingLong](<>)++ (link:{java9-javadoc}/java/util/Comparator.html#thenComparingLong%2Djava.util.function.ToLongFunction%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ConcurrentModificationException.asciidoc b/docs/painless/painless-api-reference/ConcurrentModificationException.asciidoc deleted file mode 100644 index b33eefee1e5ad..0000000000000 --- a/docs/painless/painless-api-reference/ConcurrentModificationException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ConcurrentModificationException]]++ConcurrentModificationException++:: -* ++[[painless-api-reference-ConcurrentModificationException-ConcurrentModificationException-0]]link:{java8-javadoc}/java/util/ConcurrentModificationException.html#ConcurrentModificationException%2D%2D[ConcurrentModificationException]()++ (link:{java9-javadoc}/java/util/ConcurrentModificationException.html#ConcurrentModificationException%2D%2D[java 9]) -* ++[[painless-api-reference-ConcurrentModificationException-ConcurrentModificationException-1]]link:{java8-javadoc}/java/util/ConcurrentModificationException.html#ConcurrentModificationException%2Djava.lang.String%2D[ConcurrentModificationException](<>)++ (link:{java9-javadoc}/java/util/ConcurrentModificationException.html#ConcurrentModificationException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Consumer.asciidoc b/docs/painless/painless-api-reference/Consumer.asciidoc deleted file mode 100644 index a93f47e1a1a7c..0000000000000 --- a/docs/painless/painless-api-reference/Consumer.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Consumer]]++Consumer++:: -* ++[[painless-api-reference-Consumer-accept-1]]void link:{java8-javadoc}/java/util/function/Consumer.html#accept%2Djava.lang.Object%2D[accept](def)++ (link:{java9-javadoc}/java/util/function/Consumer.html#accept%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Consumer-andThen-1]]<> link:{java8-javadoc}/java/util/function/Consumer.html#andThen%2Djava.util.function.Consumer%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/Consumer.html#andThen%2Djava.util.function.Consumer%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Currency.asciidoc b/docs/painless/painless-api-reference/Currency.asciidoc deleted file mode 100644 index 12e172e01eeea..0000000000000 --- a/docs/painless/painless-api-reference/Currency.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Currency]]++Currency++:: -* ++[[painless-api-reference-Currency-getAvailableCurrencies-0]]static <> link:{java8-javadoc}/java/util/Currency.html#getAvailableCurrencies%2D%2D[getAvailableCurrencies]()++ (link:{java9-javadoc}/java/util/Currency.html#getAvailableCurrencies%2D%2D[java 9]) -* ++[[painless-api-reference-Currency-getInstance-1]]static <> link:{java8-javadoc}/java/util/Currency.html#getInstance%2Djava.lang.String%2D[getInstance](<>)++ (link:{java9-javadoc}/java/util/Currency.html#getInstance%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Currency-getCurrencyCode-0]]<> link:{java8-javadoc}/java/util/Currency.html#getCurrencyCode%2D%2D[getCurrencyCode]()++ (link:{java9-javadoc}/java/util/Currency.html#getCurrencyCode%2D%2D[java 9]) -* ++[[painless-api-reference-Currency-getDefaultFractionDigits-0]]int link:{java8-javadoc}/java/util/Currency.html#getDefaultFractionDigits%2D%2D[getDefaultFractionDigits]()++ (link:{java9-javadoc}/java/util/Currency.html#getDefaultFractionDigits%2D%2D[java 9]) -* ++[[painless-api-reference-Currency-getDisplayName-0]]<> link:{java8-javadoc}/java/util/Currency.html#getDisplayName%2D%2D[getDisplayName]()++ (link:{java9-javadoc}/java/util/Currency.html#getDisplayName%2D%2D[java 9]) -* ++[[painless-api-reference-Currency-getDisplayName-1]]<> link:{java8-javadoc}/java/util/Currency.html#getDisplayName%2Djava.util.Locale%2D[getDisplayName](<>)++ (link:{java9-javadoc}/java/util/Currency.html#getDisplayName%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Currency-getNumericCode-0]]int link:{java8-javadoc}/java/util/Currency.html#getNumericCode%2D%2D[getNumericCode]()++ (link:{java9-javadoc}/java/util/Currency.html#getNumericCode%2D%2D[java 9]) -* ++[[painless-api-reference-Currency-getSymbol-0]]<> link:{java8-javadoc}/java/util/Currency.html#getSymbol%2D%2D[getSymbol]()++ (link:{java9-javadoc}/java/util/Currency.html#getSymbol%2D%2D[java 9]) -* ++[[painless-api-reference-Currency-getSymbol-1]]<> link:{java8-javadoc}/java/util/Currency.html#getSymbol%2Djava.util.Locale%2D[getSymbol](<>)++ (link:{java9-javadoc}/java/util/Currency.html#getSymbol%2Djava.util.Locale%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Date.asciidoc b/docs/painless/painless-api-reference/Date.asciidoc deleted file mode 100644 index 5b83f70a0eaae..0000000000000 --- a/docs/painless/painless-api-reference/Date.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Date]]++Date++:: -* ++[[painless-api-reference-Date-from-1]]static <> link:{java8-javadoc}/java/util/Date.html#from%2Djava.time.Instant%2D[from](<>)++ (link:{java9-javadoc}/java/util/Date.html#from%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-Date-Date-0]]link:{java8-javadoc}/java/util/Date.html#Date%2D%2D[Date]()++ (link:{java9-javadoc}/java/util/Date.html#Date%2D%2D[java 9]) -* ++[[painless-api-reference-Date-Date-1]]link:{java8-javadoc}/java/util/Date.html#Date%2Dlong%2D[Date](long)++ (link:{java9-javadoc}/java/util/Date.html#Date%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Date-after-1]]boolean link:{java8-javadoc}/java/util/Date.html#after%2Djava.util.Date%2D[after](<>)++ (link:{java9-javadoc}/java/util/Date.html#after%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-Date-before-1]]boolean link:{java8-javadoc}/java/util/Date.html#before%2Djava.util.Date%2D[before](<>)++ (link:{java9-javadoc}/java/util/Date.html#before%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-Date-clone-0]]def link:{java8-javadoc}/java/util/Date.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/Date.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-Date-compareTo-1]]int link:{java8-javadoc}/java/util/Date.html#compareTo%2Djava.util.Date%2D[compareTo](<>)++ (link:{java9-javadoc}/java/util/Date.html#compareTo%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-Date-getTime-0]]long link:{java8-javadoc}/java/util/Date.html#getTime%2D%2D[getTime]()++ (link:{java9-javadoc}/java/util/Date.html#getTime%2D%2D[java 9]) -* ++[[painless-api-reference-Date-setTime-1]]void link:{java8-javadoc}/java/util/Date.html#setTime%2Dlong%2D[setTime](long)++ (link:{java9-javadoc}/java/util/Date.html#setTime%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DateFormat.Field.asciidoc b/docs/painless/painless-api-reference/DateFormat.Field.asciidoc deleted file mode 100644 index b949cfe830cdc..0000000000000 --- a/docs/painless/painless-api-reference/DateFormat.Field.asciidoc +++ /dev/null @@ -1,27 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateFormat-Field]]++DateFormat.Field++:: -** [[painless-api-reference-DateFormat-Field-AM_PM]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#AM_PM[AM_PM] (link:{java9-javadoc}/java/text/DateFormat.Field.html#AM_PM[java 9]) -** [[painless-api-reference-DateFormat-Field-DAY_OF_MONTH]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#DAY_OF_MONTH[DAY_OF_MONTH] (link:{java9-javadoc}/java/text/DateFormat.Field.html#DAY_OF_MONTH[java 9]) -** [[painless-api-reference-DateFormat-Field-DAY_OF_WEEK]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#DAY_OF_WEEK[DAY_OF_WEEK] (link:{java9-javadoc}/java/text/DateFormat.Field.html#DAY_OF_WEEK[java 9]) -** [[painless-api-reference-DateFormat-Field-DAY_OF_WEEK_IN_MONTH]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#DAY_OF_WEEK_IN_MONTH[DAY_OF_WEEK_IN_MONTH] (link:{java9-javadoc}/java/text/DateFormat.Field.html#DAY_OF_WEEK_IN_MONTH[java 9]) -** [[painless-api-reference-DateFormat-Field-DAY_OF_YEAR]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#DAY_OF_YEAR[DAY_OF_YEAR] (link:{java9-javadoc}/java/text/DateFormat.Field.html#DAY_OF_YEAR[java 9]) -** [[painless-api-reference-DateFormat-Field-ERA]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#ERA[ERA] (link:{java9-javadoc}/java/text/DateFormat.Field.html#ERA[java 9]) -** [[painless-api-reference-DateFormat-Field-HOUR0]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#HOUR0[HOUR0] (link:{java9-javadoc}/java/text/DateFormat.Field.html#HOUR0[java 9]) -** [[painless-api-reference-DateFormat-Field-HOUR1]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#HOUR1[HOUR1] (link:{java9-javadoc}/java/text/DateFormat.Field.html#HOUR1[java 9]) -** [[painless-api-reference-DateFormat-Field-HOUR_OF_DAY0]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#HOUR_OF_DAY0[HOUR_OF_DAY0] (link:{java9-javadoc}/java/text/DateFormat.Field.html#HOUR_OF_DAY0[java 9]) -** [[painless-api-reference-DateFormat-Field-HOUR_OF_DAY1]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#HOUR_OF_DAY1[HOUR_OF_DAY1] (link:{java9-javadoc}/java/text/DateFormat.Field.html#HOUR_OF_DAY1[java 9]) -** [[painless-api-reference-DateFormat-Field-MILLISECOND]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#MILLISECOND[MILLISECOND] (link:{java9-javadoc}/java/text/DateFormat.Field.html#MILLISECOND[java 9]) -** [[painless-api-reference-DateFormat-Field-MINUTE]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#MINUTE[MINUTE] (link:{java9-javadoc}/java/text/DateFormat.Field.html#MINUTE[java 9]) -** [[painless-api-reference-DateFormat-Field-MONTH]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#MONTH[MONTH] (link:{java9-javadoc}/java/text/DateFormat.Field.html#MONTH[java 9]) -** [[painless-api-reference-DateFormat-Field-SECOND]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#SECOND[SECOND] (link:{java9-javadoc}/java/text/DateFormat.Field.html#SECOND[java 9]) -** [[painless-api-reference-DateFormat-Field-TIME_ZONE]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#TIME_ZONE[TIME_ZONE] (link:{java9-javadoc}/java/text/DateFormat.Field.html#TIME_ZONE[java 9]) -** [[painless-api-reference-DateFormat-Field-WEEK_OF_MONTH]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#WEEK_OF_MONTH[WEEK_OF_MONTH] (link:{java9-javadoc}/java/text/DateFormat.Field.html#WEEK_OF_MONTH[java 9]) -** [[painless-api-reference-DateFormat-Field-WEEK_OF_YEAR]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#WEEK_OF_YEAR[WEEK_OF_YEAR] (link:{java9-javadoc}/java/text/DateFormat.Field.html#WEEK_OF_YEAR[java 9]) -** [[painless-api-reference-DateFormat-Field-YEAR]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#YEAR[YEAR] (link:{java9-javadoc}/java/text/DateFormat.Field.html#YEAR[java 9]) -* ++[[painless-api-reference-DateFormat-Field-ofCalendarField-1]]static <> link:{java8-javadoc}/java/text/DateFormat.Field.html#ofCalendarField%2Dint%2D[ofCalendarField](int)++ (link:{java9-javadoc}/java/text/DateFormat.Field.html#ofCalendarField%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateFormat-Field-getCalendarField-0]]int link:{java8-javadoc}/java/text/DateFormat.Field.html#getCalendarField%2D%2D[getCalendarField]()++ (link:{java9-javadoc}/java/text/DateFormat.Field.html#getCalendarField%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DateFormat.asciidoc b/docs/painless/painless-api-reference/DateFormat.asciidoc deleted file mode 100644 index 8bb4fdf2eff71..0000000000000 --- a/docs/painless/painless-api-reference/DateFormat.asciidoc +++ /dev/null @@ -1,51 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateFormat]]++DateFormat++:: -** [[painless-api-reference-DateFormat-AM_PM_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#AM_PM_FIELD[AM_PM_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#AM_PM_FIELD[java 9]) -** [[painless-api-reference-DateFormat-DATE_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#DATE_FIELD[DATE_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#DATE_FIELD[java 9]) -** [[painless-api-reference-DateFormat-DAY_OF_WEEK_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#DAY_OF_WEEK_FIELD[DAY_OF_WEEK_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#DAY_OF_WEEK_FIELD[java 9]) -** [[painless-api-reference-DateFormat-DAY_OF_WEEK_IN_MONTH_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#DAY_OF_WEEK_IN_MONTH_FIELD[DAY_OF_WEEK_IN_MONTH_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#DAY_OF_WEEK_IN_MONTH_FIELD[java 9]) -** [[painless-api-reference-DateFormat-DAY_OF_YEAR_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#DAY_OF_YEAR_FIELD[DAY_OF_YEAR_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#DAY_OF_YEAR_FIELD[java 9]) -** [[painless-api-reference-DateFormat-DEFAULT]]static int link:{java8-javadoc}/java/text/DateFormat.html#DEFAULT[DEFAULT] (link:{java9-javadoc}/java/text/DateFormat.html#DEFAULT[java 9]) -** [[painless-api-reference-DateFormat-ERA_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#ERA_FIELD[ERA_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#ERA_FIELD[java 9]) -** [[painless-api-reference-DateFormat-FULL]]static int link:{java8-javadoc}/java/text/DateFormat.html#FULL[FULL] (link:{java9-javadoc}/java/text/DateFormat.html#FULL[java 9]) -** [[painless-api-reference-DateFormat-HOUR0_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#HOUR0_FIELD[HOUR0_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#HOUR0_FIELD[java 9]) -** [[painless-api-reference-DateFormat-HOUR1_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#HOUR1_FIELD[HOUR1_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#HOUR1_FIELD[java 9]) -** [[painless-api-reference-DateFormat-HOUR_OF_DAY0_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#HOUR_OF_DAY0_FIELD[HOUR_OF_DAY0_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#HOUR_OF_DAY0_FIELD[java 9]) -** [[painless-api-reference-DateFormat-HOUR_OF_DAY1_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#HOUR_OF_DAY1_FIELD[HOUR_OF_DAY1_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#HOUR_OF_DAY1_FIELD[java 9]) -** [[painless-api-reference-DateFormat-LONG]]static int link:{java8-javadoc}/java/text/DateFormat.html#LONG[LONG] (link:{java9-javadoc}/java/text/DateFormat.html#LONG[java 9]) -** [[painless-api-reference-DateFormat-MEDIUM]]static int link:{java8-javadoc}/java/text/DateFormat.html#MEDIUM[MEDIUM] (link:{java9-javadoc}/java/text/DateFormat.html#MEDIUM[java 9]) -** [[painless-api-reference-DateFormat-MILLISECOND_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#MILLISECOND_FIELD[MILLISECOND_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#MILLISECOND_FIELD[java 9]) -** [[painless-api-reference-DateFormat-MINUTE_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#MINUTE_FIELD[MINUTE_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#MINUTE_FIELD[java 9]) -** [[painless-api-reference-DateFormat-MONTH_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#MONTH_FIELD[MONTH_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#MONTH_FIELD[java 9]) -** [[painless-api-reference-DateFormat-SECOND_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#SECOND_FIELD[SECOND_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#SECOND_FIELD[java 9]) -** [[painless-api-reference-DateFormat-SHORT]]static int link:{java8-javadoc}/java/text/DateFormat.html#SHORT[SHORT] (link:{java9-javadoc}/java/text/DateFormat.html#SHORT[java 9]) -** [[painless-api-reference-DateFormat-TIMEZONE_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#TIMEZONE_FIELD[TIMEZONE_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#TIMEZONE_FIELD[java 9]) -** [[painless-api-reference-DateFormat-WEEK_OF_MONTH_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#WEEK_OF_MONTH_FIELD[WEEK_OF_MONTH_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#WEEK_OF_MONTH_FIELD[java 9]) -** [[painless-api-reference-DateFormat-WEEK_OF_YEAR_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#WEEK_OF_YEAR_FIELD[WEEK_OF_YEAR_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#WEEK_OF_YEAR_FIELD[java 9]) -** [[painless-api-reference-DateFormat-YEAR_FIELD]]static int link:{java8-javadoc}/java/text/DateFormat.html#YEAR_FIELD[YEAR_FIELD] (link:{java9-javadoc}/java/text/DateFormat.html#YEAR_FIELD[java 9]) -* ++[[painless-api-reference-DateFormat-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/text/DateFormat.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getDateInstance-0]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getDateInstance%2D%2D[getDateInstance]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getDateInstance%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getDateInstance-1]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getDateInstance%2Dint%2D[getDateInstance](int)++ (link:{java9-javadoc}/java/text/DateFormat.html#getDateInstance%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getDateInstance-2]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getDateInstance%2Dint%2Djava.util.Locale%2D[getDateInstance](int, <>)++ (link:{java9-javadoc}/java/text/DateFormat.html#getDateInstance%2Dint%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getDateTimeInstance-0]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getDateTimeInstance%2D%2D[getDateTimeInstance]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getDateTimeInstance%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getDateTimeInstance-2]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getDateTimeInstance%2Dint%2Dint%2D[getDateTimeInstance](int, int)++ (link:{java9-javadoc}/java/text/DateFormat.html#getDateTimeInstance%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getDateTimeInstance-3]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getDateTimeInstance%2Dint%2Dint%2Djava.util.Locale%2D[getDateTimeInstance](int, int, <>)++ (link:{java9-javadoc}/java/text/DateFormat.html#getDateTimeInstance%2Dint%2Dint%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getInstance-0]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getInstance%2D%2D[getInstance]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getInstance%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getTimeInstance-0]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getTimeInstance%2D%2D[getTimeInstance]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getTimeInstance%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getTimeInstance-1]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getTimeInstance%2Dint%2D[getTimeInstance](int)++ (link:{java9-javadoc}/java/text/DateFormat.html#getTimeInstance%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getTimeInstance-2]]static <> link:{java8-javadoc}/java/text/DateFormat.html#getTimeInstance%2Dint%2Djava.util.Locale%2D[getTimeInstance](int, <>)++ (link:{java9-javadoc}/java/text/DateFormat.html#getTimeInstance%2Dint%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getCalendar-0]]<> link:{java8-javadoc}/java/text/DateFormat.html#getCalendar%2D%2D[getCalendar]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getCalendar%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getNumberFormat-0]]<> link:{java8-javadoc}/java/text/DateFormat.html#getNumberFormat%2D%2D[getNumberFormat]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getNumberFormat%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-getTimeZone-0]]<> link:{java8-javadoc}/java/text/DateFormat.html#getTimeZone%2D%2D[getTimeZone]()++ (link:{java9-javadoc}/java/text/DateFormat.html#getTimeZone%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-isLenient-0]]boolean link:{java8-javadoc}/java/text/DateFormat.html#isLenient%2D%2D[isLenient]()++ (link:{java9-javadoc}/java/text/DateFormat.html#isLenient%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormat-parse-1]]<> link:{java8-javadoc}/java/text/DateFormat.html#parse%2Djava.lang.String%2D[parse](<>)++ (link:{java9-javadoc}/java/text/DateFormat.html#parse%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DateFormat-parse-2]]<> link:{java8-javadoc}/java/text/DateFormat.html#parse%2Djava.lang.String%2Djava.text.ParsePosition%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/text/DateFormat.html#parse%2Djava.lang.String%2Djava.text.ParsePosition%2D[java 9]) -* ++[[painless-api-reference-DateFormat-setCalendar-1]]void link:{java8-javadoc}/java/text/DateFormat.html#setCalendar%2Djava.util.Calendar%2D[setCalendar](<>)++ (link:{java9-javadoc}/java/text/DateFormat.html#setCalendar%2Djava.util.Calendar%2D[java 9]) -* ++[[painless-api-reference-DateFormat-setLenient-1]]void link:{java8-javadoc}/java/text/DateFormat.html#setLenient%2Dboolean%2D[setLenient](boolean)++ (link:{java9-javadoc}/java/text/DateFormat.html#setLenient%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-DateFormat-setNumberFormat-1]]void link:{java8-javadoc}/java/text/DateFormat.html#setNumberFormat%2Djava.text.NumberFormat%2D[setNumberFormat](<>)++ (link:{java9-javadoc}/java/text/DateFormat.html#setNumberFormat%2Djava.text.NumberFormat%2D[java 9]) -* ++[[painless-api-reference-DateFormat-setTimeZone-1]]void link:{java8-javadoc}/java/text/DateFormat.html#setTimeZone%2Djava.util.TimeZone%2D[setTimeZone](<>)++ (link:{java9-javadoc}/java/text/DateFormat.html#setTimeZone%2Djava.util.TimeZone%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DateFormatSymbols.asciidoc b/docs/painless/painless-api-reference/DateFormatSymbols.asciidoc deleted file mode 100644 index 31cf39bbc4586..0000000000000 --- a/docs/painless/painless-api-reference/DateFormatSymbols.asciidoc +++ /dev/null @@ -1,30 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateFormatSymbols]]++DateFormatSymbols++:: -* ++[[painless-api-reference-DateFormatSymbols-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getInstance-0]]static <> link:{java8-javadoc}/java/text/DateFormatSymbols.html#getInstance%2D%2D[getInstance]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getInstance%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getInstance-1]]static <> link:{java8-javadoc}/java/text/DateFormatSymbols.html#getInstance%2Djava.util.Locale%2D[getInstance](<>)++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-DateFormatSymbols-0]]link:{java8-javadoc}/java/text/DateFormatSymbols.html#DateFormatSymbols%2D%2D[DateFormatSymbols]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#DateFormatSymbols%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-DateFormatSymbols-1]]link:{java8-javadoc}/java/text/DateFormatSymbols.html#DateFormatSymbols%2Djava.util.Locale%2D[DateFormatSymbols](<>)++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#DateFormatSymbols%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-clone-0]]def link:{java8-javadoc}/java/text/DateFormatSymbols.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getAmPmStrings-0]]<>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getAmPmStrings%2D%2D[getAmPmStrings]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getAmPmStrings%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getEras-0]]<>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getEras%2D%2D[getEras]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getEras%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getLocalPatternChars-0]]<> link:{java8-javadoc}/java/text/DateFormatSymbols.html#getLocalPatternChars%2D%2D[getLocalPatternChars]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getLocalPatternChars%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getMonths-0]]<>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getMonths%2D%2D[getMonths]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getMonths%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getShortMonths-0]]<>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getShortMonths%2D%2D[getShortMonths]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getShortMonths%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getShortWeekdays-0]]<>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getShortWeekdays%2D%2D[getShortWeekdays]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getShortWeekdays%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getWeekdays-0]]<>[] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getWeekdays%2D%2D[getWeekdays]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getWeekdays%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-getZoneStrings-0]]<>[][] link:{java8-javadoc}/java/text/DateFormatSymbols.html#getZoneStrings%2D%2D[getZoneStrings]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#getZoneStrings%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-hashCode-0]]int link:{java8-javadoc}/java/text/DateFormatSymbols.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setAmPmStrings-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setAmPmStrings%2Djava.lang.String:A%2D[setAmPmStrings](<>[])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setAmPmStrings%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setEras-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setEras%2Djava.lang.String:A%2D[setEras](<>[])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setEras%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setLocalPatternChars-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setLocalPatternChars%2Djava.lang.String%2D[setLocalPatternChars](<>)++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setLocalPatternChars%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setMonths-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setMonths%2Djava.lang.String:A%2D[setMonths](<>[])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setMonths%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setShortMonths-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setShortMonths%2Djava.lang.String:A%2D[setShortMonths](<>[])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setShortMonths%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setShortWeekdays-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setShortWeekdays%2Djava.lang.String:A%2D[setShortWeekdays](<>[])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setShortWeekdays%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setWeekdays-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setWeekdays%2Djava.lang.String:A%2D[setWeekdays](<>[])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setWeekdays%2Djava.lang.String:A%2D[java 9]) -* ++[[painless-api-reference-DateFormatSymbols-setZoneStrings-1]]void link:{java8-javadoc}/java/text/DateFormatSymbols.html#setZoneStrings%2Djava.lang.String:A%2D[setZoneStrings](<>[][])++ (link:{java9-javadoc}/java/text/DateFormatSymbols.html#setZoneStrings%2Djava.lang.String:A%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DateTimeException.asciidoc b/docs/painless/painless-api-reference/DateTimeException.asciidoc deleted file mode 100644 index 5a9953c5bd582..0000000000000 --- a/docs/painless/painless-api-reference/DateTimeException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateTimeException]]++DateTimeException++:: -* ++[[painless-api-reference-DateTimeException-DateTimeException-1]]link:{java8-javadoc}/java/time/DateTimeException.html#DateTimeException%2Djava.lang.String%2D[DateTimeException](<>)++ (link:{java9-javadoc}/java/time/DateTimeException.html#DateTimeException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DateTimeFormatter.asciidoc b/docs/painless/painless-api-reference/DateTimeFormatter.asciidoc deleted file mode 100644 index ad104dfe5d6d2..0000000000000 --- a/docs/painless/painless-api-reference/DateTimeFormatter.asciidoc +++ /dev/null @@ -1,50 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateTimeFormatter]]++DateTimeFormatter++:: -** [[painless-api-reference-DateTimeFormatter-BASIC_ISO_DATE]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#BASIC_ISO_DATE[BASIC_ISO_DATE] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#BASIC_ISO_DATE[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_DATE]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_DATE[ISO_DATE] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_DATE[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_DATE_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_DATE_TIME[ISO_DATE_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_DATE_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_INSTANT]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_INSTANT[ISO_INSTANT] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_INSTANT[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_LOCAL_DATE]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE[ISO_LOCAL_DATE] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_LOCAL_DATE_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE_TIME[ISO_LOCAL_DATE_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_LOCAL_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_LOCAL_TIME[ISO_LOCAL_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_LOCAL_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_OFFSET_DATE]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_OFFSET_DATE[ISO_OFFSET_DATE] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_OFFSET_DATE[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_OFFSET_DATE_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_OFFSET_DATE_TIME[ISO_OFFSET_DATE_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_OFFSET_DATE_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_OFFSET_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_OFFSET_TIME[ISO_OFFSET_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_OFFSET_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_ORDINAL_DATE]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_ORDINAL_DATE[ISO_ORDINAL_DATE] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_ORDINAL_DATE[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_TIME[ISO_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_WEEK_DATE]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_WEEK_DATE[ISO_WEEK_DATE] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_WEEK_DATE[java 9]) -** [[painless-api-reference-DateTimeFormatter-ISO_ZONED_DATE_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ISO_ZONED_DATE_TIME[ISO_ZONED_DATE_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ISO_ZONED_DATE_TIME[java 9]) -** [[painless-api-reference-DateTimeFormatter-RFC_1123_DATE_TIME]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#RFC_1123_DATE_TIME[RFC_1123_DATE_TIME] (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#RFC_1123_DATE_TIME[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-ofLocalizedDate-1]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedDate%2Djava.time.format.FormatStyle%2D[ofLocalizedDate](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedDate%2Djava.time.format.FormatStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-ofLocalizedDateTime-1]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedDateTime%2Djava.time.format.FormatStyle%2D[ofLocalizedDateTime](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedDateTime%2Djava.time.format.FormatStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-ofLocalizedDateTime-2]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedDateTime%2Djava.time.format.FormatStyle%2Djava.time.format.FormatStyle%2D[ofLocalizedDateTime](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedDateTime%2Djava.time.format.FormatStyle%2Djava.time.format.FormatStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-ofLocalizedTime-1]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedTime%2Djava.time.format.FormatStyle%2D[ofLocalizedTime](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ofLocalizedTime%2Djava.time.format.FormatStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-ofPattern-1]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ofPattern%2Djava.lang.String%2D[ofPattern](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ofPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-ofPattern-2]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#ofPattern%2Djava.lang.String%2Djava.util.Locale%2D[ofPattern](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#ofPattern%2Djava.lang.String%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-parsedExcessDays-0]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#parsedExcessDays%2D%2D[parsedExcessDays]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#parsedExcessDays%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-parsedLeapSecond-0]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#parsedLeapSecond%2D%2D[parsedLeapSecond]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#parsedLeapSecond%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-format-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#format%2Djava.time.temporal.TemporalAccessor%2D[format](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#format%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-formatTo-2]]void link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#formatTo%2Djava.time.temporal.TemporalAccessor%2Djava.lang.Appendable%2D[formatTo](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#formatTo%2Djava.time.temporal.TemporalAccessor%2Djava.lang.Appendable%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-getChronology-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-getDecimalStyle-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#getDecimalStyle%2D%2D[getDecimalStyle]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#getDecimalStyle%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-getLocale-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#getLocale%2D%2D[getLocale]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#getLocale%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-getResolverFields-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#getResolverFields%2D%2D[getResolverFields]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#getResolverFields%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-getResolverStyle-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#getResolverStyle%2D%2D[getResolverStyle]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#getResolverStyle%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-getZone-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#getZone%2D%2D[getZone]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#getZone%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-parse-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-parse-2]]def link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#parse%2Djava.lang.CharSequence%2Djava.time.temporal.TemporalQuery%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#parse%2Djava.lang.CharSequence%2Djava.time.temporal.TemporalQuery%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-parseBest-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#parseBest%2Djava.lang.CharSequence%2Djava.time.temporal.TemporalQuery:A%2D[parseBest](<>, <>[])++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#parseBest%2Djava.lang.CharSequence%2Djava.time.temporal.TemporalQuery:A%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-parseUnresolved-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#parseUnresolved%2Djava.lang.CharSequence%2Djava.text.ParsePosition%2D[parseUnresolved](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#parseUnresolved%2Djava.lang.CharSequence%2Djava.text.ParsePosition%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-toFormat-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#toFormat%2D%2D[toFormat]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#toFormat%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-toFormat-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#toFormat%2Djava.time.temporal.TemporalQuery%2D[toFormat](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#toFormat%2Djava.time.temporal.TemporalQuery%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-withChronology-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#withChronology%2Djava.time.chrono.Chronology%2D[withChronology](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#withChronology%2Djava.time.chrono.Chronology%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-withDecimalStyle-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#withDecimalStyle%2Djava.time.format.DecimalStyle%2D[withDecimalStyle](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#withDecimalStyle%2Djava.time.format.DecimalStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-withLocale-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#withLocale%2Djava.util.Locale%2D[withLocale](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#withLocale%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-withResolverFields-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#withResolverFields%2Djava.util.Set%2D[withResolverFields](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#withResolverFields%2Djava.util.Set%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-withResolverStyle-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#withResolverStyle%2Djava.time.format.ResolverStyle%2D[withResolverStyle](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#withResolverStyle%2Djava.time.format.ResolverStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatter-withZone-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatter.html#withZone%2Djava.time.ZoneId%2D[withZone](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatter.html#withZone%2Djava.time.ZoneId%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DateTimeFormatterBuilder.asciidoc b/docs/painless/painless-api-reference/DateTimeFormatterBuilder.asciidoc deleted file mode 100644 index cc352560f41a9..0000000000000 --- a/docs/painless/painless-api-reference/DateTimeFormatterBuilder.asciidoc +++ /dev/null @@ -1,44 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateTimeFormatterBuilder]]++DateTimeFormatterBuilder++:: -* ++[[painless-api-reference-DateTimeFormatterBuilder-getLocalizedDateTimePattern-4]]static <> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#getLocalizedDateTimePattern%2Djava.time.format.FormatStyle%2Djava.time.format.FormatStyle%2Djava.time.chrono.Chronology%2Djava.util.Locale%2D[getLocalizedDateTimePattern](<>, <>, <>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#getLocalizedDateTimePattern%2Djava.time.format.FormatStyle%2Djava.time.format.FormatStyle%2Djava.time.chrono.Chronology%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-DateTimeFormatterBuilder-0]]link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#DateTimeFormatterBuilder%2D%2D[DateTimeFormatterBuilder]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#DateTimeFormatterBuilder%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-append-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#append%2Djava.time.format.DateTimeFormatter%2D[append](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#append%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendChronologyId-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendChronologyId%2D%2D[appendChronologyId]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendChronologyId%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendChronologyText-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendChronologyText%2Djava.time.format.TextStyle%2D[appendChronologyText](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendChronologyText%2Djava.time.format.TextStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendFraction-4]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendFraction%2Djava.time.temporal.TemporalField%2Dint%2Dint%2Dboolean%2D[appendFraction](<>, int, int, boolean)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendFraction%2Djava.time.temporal.TemporalField%2Dint%2Dint%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendInstant-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendInstant%2D%2D[appendInstant]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendInstant%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendInstant-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendInstant%2Dint%2D[appendInstant](int)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendInstant%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendLiteral-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendLiteral%2Djava.lang.String%2D[appendLiteral](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendLiteral%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendLocalized-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendLocalized%2Djava.time.format.FormatStyle%2Djava.time.format.FormatStyle%2D[appendLocalized](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendLocalized%2Djava.time.format.FormatStyle%2Djava.time.format.FormatStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendLocalizedOffset-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendLocalizedOffset%2Djava.time.format.TextStyle%2D[appendLocalizedOffset](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendLocalizedOffset%2Djava.time.format.TextStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendOffset-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendOffset%2Djava.lang.String%2Djava.lang.String%2D[appendOffset](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendOffset%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendOffsetId-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendOffsetId%2D%2D[appendOffsetId]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendOffsetId%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendOptional-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendOptional%2Djava.time.format.DateTimeFormatter%2D[appendOptional](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendOptional%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendPattern-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendPattern%2Djava.lang.String%2D[appendPattern](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendText-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendText%2Djava.time.temporal.TemporalField%2D[appendText](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendText%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendText-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendText%2Djava.time.temporal.TemporalField%2Djava.time.format.TextStyle%2D[appendText](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendText%2Djava.time.temporal.TemporalField%2Djava.time.format.TextStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendValue-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValue%2Djava.time.temporal.TemporalField%2D[appendValue](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValue%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendValue-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValue%2Djava.time.temporal.TemporalField%2Dint%2D[appendValue](<>, int)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValue%2Djava.time.temporal.TemporalField%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendValue-4]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValue%2Djava.time.temporal.TemporalField%2Dint%2Dint%2Djava.time.format.SignStyle%2D[appendValue](<>, int, int, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValue%2Djava.time.temporal.TemporalField%2Dint%2Dint%2Djava.time.format.SignStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendValueReduced-4]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValueReduced%2Djava.time.temporal.TemporalField%2Dint%2Dint%2Dint%2D[appendValueReduced](<>, int, int, int)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendValueReduced%2Djava.time.temporal.TemporalField%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendZoneId-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneId%2D%2D[appendZoneId]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneId%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendZoneOrOffsetId-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneOrOffsetId%2D%2D[appendZoneOrOffsetId]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneOrOffsetId%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendZoneRegionId-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneRegionId%2D%2D[appendZoneRegionId]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneRegionId%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendZoneText-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneText%2Djava.time.format.TextStyle%2D[appendZoneText](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneText%2Djava.time.format.TextStyle%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-appendZoneText-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneText%2Djava.time.format.TextStyle%2Djava.util.Set%2D[appendZoneText](<>, <>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#appendZoneText%2Djava.time.format.TextStyle%2Djava.util.Set%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-optionalEnd-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#optionalEnd%2D%2D[optionalEnd]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#optionalEnd%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-optionalStart-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#optionalStart%2D%2D[optionalStart]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#optionalStart%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-padNext-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#padNext%2Dint%2D[padNext](int)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#padNext%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-padNext-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#padNext%2Dint%2Dchar%2D[padNext](int, char)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#padNext%2Dint%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-parseCaseInsensitive-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseCaseInsensitive%2D%2D[parseCaseInsensitive]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseCaseInsensitive%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-parseCaseSensitive-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseCaseSensitive%2D%2D[parseCaseSensitive]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseCaseSensitive%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-parseDefaulting-2]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseDefaulting%2Djava.time.temporal.TemporalField%2Dlong%2D[parseDefaulting](<>, long)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseDefaulting%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-parseLenient-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseLenient%2D%2D[parseLenient]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseLenient%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-parseStrict-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseStrict%2D%2D[parseStrict]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#parseStrict%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-toFormatter-0]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#toFormatter%2D%2D[toFormatter]()++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#toFormatter%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeFormatterBuilder-toFormatter-1]]<> link:{java8-javadoc}/java/time/format/DateTimeFormatterBuilder.html#toFormatter%2Djava.util.Locale%2D[toFormatter](<>)++ (link:{java9-javadoc}/java/time/format/DateTimeFormatterBuilder.html#toFormatter%2Djava.util.Locale%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DateTimeParseException.asciidoc b/docs/painless/painless-api-reference/DateTimeParseException.asciidoc deleted file mode 100644 index 8a9626b64052a..0000000000000 --- a/docs/painless/painless-api-reference/DateTimeParseException.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DateTimeParseException]]++DateTimeParseException++:: -* ++[[painless-api-reference-DateTimeParseException-DateTimeParseException-3]]link:{java8-javadoc}/java/time/format/DateTimeParseException.html#DateTimeParseException%2Djava.lang.String%2Djava.lang.CharSequence%2Dint%2D[DateTimeParseException](<>, <>, int)++ (link:{java9-javadoc}/java/time/format/DateTimeParseException.html#DateTimeParseException%2Djava.lang.String%2Djava.lang.CharSequence%2Dint%2D[java 9]) -* ++[[painless-api-reference-DateTimeParseException-getErrorIndex-0]]int link:{java8-javadoc}/java/time/format/DateTimeParseException.html#getErrorIndex%2D%2D[getErrorIndex]()++ (link:{java9-javadoc}/java/time/format/DateTimeParseException.html#getErrorIndex%2D%2D[java 9]) -* ++[[painless-api-reference-DateTimeParseException-getParsedString-0]]<> link:{java8-javadoc}/java/time/format/DateTimeParseException.html#getParsedString%2D%2D[getParsedString]()++ (link:{java9-javadoc}/java/time/format/DateTimeParseException.html#getParsedString%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DayOfWeek.asciidoc b/docs/painless/painless-api-reference/DayOfWeek.asciidoc deleted file mode 100644 index fd847f3d019ab..0000000000000 --- a/docs/painless/painless-api-reference/DayOfWeek.asciidoc +++ /dev/null @@ -1,22 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DayOfWeek]]++DayOfWeek++:: -** [[painless-api-reference-DayOfWeek-FRIDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#FRIDAY[FRIDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#FRIDAY[java 9]) -** [[painless-api-reference-DayOfWeek-MONDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#MONDAY[MONDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#MONDAY[java 9]) -** [[painless-api-reference-DayOfWeek-SATURDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#SATURDAY[SATURDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#SATURDAY[java 9]) -** [[painless-api-reference-DayOfWeek-SUNDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#SUNDAY[SUNDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#SUNDAY[java 9]) -** [[painless-api-reference-DayOfWeek-THURSDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#THURSDAY[THURSDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#THURSDAY[java 9]) -** [[painless-api-reference-DayOfWeek-TUESDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#TUESDAY[TUESDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#TUESDAY[java 9]) -** [[painless-api-reference-DayOfWeek-WEDNESDAY]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#WEDNESDAY[WEDNESDAY] (link:{java9-javadoc}/java/time/DayOfWeek.html#WEDNESDAY[java 9]) -* ++[[painless-api-reference-DayOfWeek-from-1]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/DayOfWeek.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-of-1]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/DayOfWeek.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-valueOf-1]]static <> link:{java8-javadoc}/java/time/DayOfWeek.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/DayOfWeek.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-values-0]]static <>[] link:{java8-javadoc}/java/time/DayOfWeek.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/DayOfWeek.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-getDisplayName-2]]<> link:{java8-javadoc}/java/time/DayOfWeek.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[getDisplayName](<>, <>)++ (link:{java9-javadoc}/java/time/DayOfWeek.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-getValue-0]]int link:{java8-javadoc}/java/time/DayOfWeek.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/DayOfWeek.html#getValue%2D%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-minus-1]]<> link:{java8-javadoc}/java/time/DayOfWeek.html#minus%2Dlong%2D[minus](long)++ (link:{java9-javadoc}/java/time/DayOfWeek.html#minus%2Dlong%2D[java 9]) -* ++[[painless-api-reference-DayOfWeek-plus-1]]<> link:{java8-javadoc}/java/time/DayOfWeek.html#plus%2Dlong%2D[plus](long)++ (link:{java9-javadoc}/java/time/DayOfWeek.html#plus%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Debug.asciidoc b/docs/painless/painless-api-reference/Debug.asciidoc deleted file mode 100644 index 8c64ed6bcc4ad..0000000000000 --- a/docs/painless/painless-api-reference/Debug.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Debug]]++Debug++:: -* ++[[painless-api-reference-Debug-explain-1]]static void link:{painless-javadoc}/org/elasticsearch/painless/api/Debug.html#explain%2Djava.lang.Object%2D[explain](<>)++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DecimalFormat.asciidoc b/docs/painless/painless-api-reference/DecimalFormat.asciidoc deleted file mode 100644 index 96768b77273f8..0000000000000 --- a/docs/painless/painless-api-reference/DecimalFormat.asciidoc +++ /dev/null @@ -1,32 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DecimalFormat]]++DecimalFormat++:: -* ++[[painless-api-reference-DecimalFormat-DecimalFormat-0]]link:{java8-javadoc}/java/text/DecimalFormat.html#DecimalFormat%2D%2D[DecimalFormat]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#DecimalFormat%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-DecimalFormat-1]]link:{java8-javadoc}/java/text/DecimalFormat.html#DecimalFormat%2Djava.lang.String%2D[DecimalFormat](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#DecimalFormat%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-DecimalFormat-2]]link:{java8-javadoc}/java/text/DecimalFormat.html#DecimalFormat%2Djava.lang.String%2Djava.text.DecimalFormatSymbols%2D[DecimalFormat](<>, <>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#DecimalFormat%2Djava.lang.String%2Djava.text.DecimalFormatSymbols%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-applyLocalizedPattern-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#applyLocalizedPattern%2Djava.lang.String%2D[applyLocalizedPattern](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#applyLocalizedPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-applyPattern-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#applyPattern%2Djava.lang.String%2D[applyPattern](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#applyPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getDecimalFormatSymbols-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#getDecimalFormatSymbols%2D%2D[getDecimalFormatSymbols]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getDecimalFormatSymbols%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getGroupingSize-0]]int link:{java8-javadoc}/java/text/DecimalFormat.html#getGroupingSize%2D%2D[getGroupingSize]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getGroupingSize%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getMultiplier-0]]int link:{java8-javadoc}/java/text/DecimalFormat.html#getMultiplier%2D%2D[getMultiplier]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getMultiplier%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getNegativePrefix-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#getNegativePrefix%2D%2D[getNegativePrefix]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getNegativePrefix%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getNegativeSuffix-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#getNegativeSuffix%2D%2D[getNegativeSuffix]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getNegativeSuffix%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getPositivePrefix-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#getPositivePrefix%2D%2D[getPositivePrefix]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getPositivePrefix%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-getPositiveSuffix-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#getPositiveSuffix%2D%2D[getPositiveSuffix]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#getPositiveSuffix%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-isDecimalSeparatorAlwaysShown-0]]boolean link:{java8-javadoc}/java/text/DecimalFormat.html#isDecimalSeparatorAlwaysShown%2D%2D[isDecimalSeparatorAlwaysShown]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#isDecimalSeparatorAlwaysShown%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-isParseBigDecimal-0]]boolean link:{java8-javadoc}/java/text/DecimalFormat.html#isParseBigDecimal%2D%2D[isParseBigDecimal]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#isParseBigDecimal%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setDecimalFormatSymbols-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setDecimalFormatSymbols%2Djava.text.DecimalFormatSymbols%2D[setDecimalFormatSymbols](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setDecimalFormatSymbols%2Djava.text.DecimalFormatSymbols%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setDecimalSeparatorAlwaysShown-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setDecimalSeparatorAlwaysShown%2Dboolean%2D[setDecimalSeparatorAlwaysShown](boolean)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setDecimalSeparatorAlwaysShown%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setGroupingSize-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setGroupingSize%2Dint%2D[setGroupingSize](int)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setGroupingSize%2Dint%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setMultiplier-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setMultiplier%2Dint%2D[setMultiplier](int)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setMultiplier%2Dint%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setNegativePrefix-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setNegativePrefix%2Djava.lang.String%2D[setNegativePrefix](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setNegativePrefix%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setNegativeSuffix-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setNegativeSuffix%2Djava.lang.String%2D[setNegativeSuffix](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setNegativeSuffix%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setParseBigDecimal-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setParseBigDecimal%2Dboolean%2D[setParseBigDecimal](boolean)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setParseBigDecimal%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setPositivePrefix-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setPositivePrefix%2Djava.lang.String%2D[setPositivePrefix](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setPositivePrefix%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-setPositiveSuffix-1]]void link:{java8-javadoc}/java/text/DecimalFormat.html#setPositiveSuffix%2Djava.lang.String%2D[setPositiveSuffix](<>)++ (link:{java9-javadoc}/java/text/DecimalFormat.html#setPositiveSuffix%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-toLocalizedPattern-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#toLocalizedPattern%2D%2D[toLocalizedPattern]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#toLocalizedPattern%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormat-toPattern-0]]<> link:{java8-javadoc}/java/text/DecimalFormat.html#toPattern%2D%2D[toPattern]()++ (link:{java9-javadoc}/java/text/DecimalFormat.html#toPattern%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DecimalFormatSymbols.asciidoc b/docs/painless/painless-api-reference/DecimalFormatSymbols.asciidoc deleted file mode 100644 index 37f6fc090d105..0000000000000 --- a/docs/painless/painless-api-reference/DecimalFormatSymbols.asciidoc +++ /dev/null @@ -1,43 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DecimalFormatSymbols]]++DecimalFormatSymbols++:: -* ++[[painless-api-reference-DecimalFormatSymbols-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getInstance-0]]static <> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getInstance%2D%2D[getInstance]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getInstance%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getInstance-1]]static <> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getInstance%2Djava.util.Locale%2D[getInstance](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-DecimalFormatSymbols-0]]link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#DecimalFormatSymbols%2D%2D[DecimalFormatSymbols]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#DecimalFormatSymbols%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-DecimalFormatSymbols-1]]link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#DecimalFormatSymbols%2Djava.util.Locale%2D[DecimalFormatSymbols](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#DecimalFormatSymbols%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-clone-0]]def link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getCurrency-0]]<> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getCurrency%2D%2D[getCurrency]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getCurrency%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getCurrencySymbol-0]]<> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getCurrencySymbol%2D%2D[getCurrencySymbol]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getCurrencySymbol%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getDecimalSeparator-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getDecimalSeparator%2D%2D[getDecimalSeparator]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getDecimalSeparator%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getDigit-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getDigit%2D%2D[getDigit]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getDigit%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getExponentSeparator-0]]<> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getExponentSeparator%2D%2D[getExponentSeparator]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getExponentSeparator%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getGroupingSeparator-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getGroupingSeparator%2D%2D[getGroupingSeparator]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getGroupingSeparator%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getInfinity-0]]<> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getInfinity%2D%2D[getInfinity]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getInfinity%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getInternationalCurrencySymbol-0]]<> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getInternationalCurrencySymbol%2D%2D[getInternationalCurrencySymbol]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getInternationalCurrencySymbol%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getMinusSign-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getMinusSign%2D%2D[getMinusSign]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getMinusSign%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getMonetaryDecimalSeparator-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getMonetaryDecimalSeparator%2D%2D[getMonetaryDecimalSeparator]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getMonetaryDecimalSeparator%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getNaN-0]]<> link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getNaN%2D%2D[getNaN]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getNaN%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getPatternSeparator-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getPatternSeparator%2D%2D[getPatternSeparator]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getPatternSeparator%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getPerMill-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getPerMill%2D%2D[getPerMill]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getPerMill%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getPercent-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getPercent%2D%2D[getPercent]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getPercent%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-getZeroDigit-0]]char link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#getZeroDigit%2D%2D[getZeroDigit]()++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#getZeroDigit%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setCurrency-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setCurrency%2Djava.util.Currency%2D[setCurrency](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setCurrency%2Djava.util.Currency%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setCurrencySymbol-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setCurrencySymbol%2Djava.lang.String%2D[setCurrencySymbol](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setCurrencySymbol%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setDecimalSeparator-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setDecimalSeparator%2Dchar%2D[setDecimalSeparator](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setDecimalSeparator%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setDigit-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setDigit%2Dchar%2D[setDigit](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setDigit%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setExponentSeparator-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setExponentSeparator%2Djava.lang.String%2D[setExponentSeparator](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setExponentSeparator%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setGroupingSeparator-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setGroupingSeparator%2Dchar%2D[setGroupingSeparator](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setGroupingSeparator%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setInfinity-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setInfinity%2Djava.lang.String%2D[setInfinity](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setInfinity%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setInternationalCurrencySymbol-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setInternationalCurrencySymbol%2Djava.lang.String%2D[setInternationalCurrencySymbol](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setInternationalCurrencySymbol%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setMinusSign-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setMinusSign%2Dchar%2D[setMinusSign](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setMinusSign%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setMonetaryDecimalSeparator-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setMonetaryDecimalSeparator%2Dchar%2D[setMonetaryDecimalSeparator](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setMonetaryDecimalSeparator%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setNaN-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setNaN%2Djava.lang.String%2D[setNaN](<>)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setNaN%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setPatternSeparator-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setPatternSeparator%2Dchar%2D[setPatternSeparator](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setPatternSeparator%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setPerMill-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setPerMill%2Dchar%2D[setPerMill](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setPerMill%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setPercent-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setPercent%2Dchar%2D[setPercent](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setPercent%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalFormatSymbols-setZeroDigit-1]]void link:{java8-javadoc}/java/text/DecimalFormatSymbols.html#setZeroDigit%2Dchar%2D[setZeroDigit](char)++ (link:{java9-javadoc}/java/text/DecimalFormatSymbols.html#setZeroDigit%2Dchar%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DecimalStyle.asciidoc b/docs/painless/painless-api-reference/DecimalStyle.asciidoc deleted file mode 100644 index 8dac3a7ae2440..0000000000000 --- a/docs/painless/painless-api-reference/DecimalStyle.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DecimalStyle]]++DecimalStyle++:: -** [[painless-api-reference-DecimalStyle-STANDARD]]static <> link:{java8-javadoc}/java/time/format/DecimalStyle.html#STANDARD[STANDARD] (link:{java9-javadoc}/java/time/format/DecimalStyle.html#STANDARD[java 9]) -* ++[[painless-api-reference-DecimalStyle-getAvailableLocales-0]]static <> link:{java8-javadoc}/java/time/format/DecimalStyle.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-of-1]]static <> link:{java8-javadoc}/java/time/format/DecimalStyle.html#of%2Djava.util.Locale%2D[of](<>)++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#of%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-ofDefaultLocale-0]]static <> link:{java8-javadoc}/java/time/format/DecimalStyle.html#ofDefaultLocale%2D%2D[ofDefaultLocale]()++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#ofDefaultLocale%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-getDecimalSeparator-0]]char link:{java8-javadoc}/java/time/format/DecimalStyle.html#getDecimalSeparator%2D%2D[getDecimalSeparator]()++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#getDecimalSeparator%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-getNegativeSign-0]]char link:{java8-javadoc}/java/time/format/DecimalStyle.html#getNegativeSign%2D%2D[getNegativeSign]()++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#getNegativeSign%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-getPositiveSign-0]]char link:{java8-javadoc}/java/time/format/DecimalStyle.html#getPositiveSign%2D%2D[getPositiveSign]()++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#getPositiveSign%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-getZeroDigit-0]]char link:{java8-javadoc}/java/time/format/DecimalStyle.html#getZeroDigit%2D%2D[getZeroDigit]()++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#getZeroDigit%2D%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-withDecimalSeparator-1]]<> link:{java8-javadoc}/java/time/format/DecimalStyle.html#withDecimalSeparator%2Dchar%2D[withDecimalSeparator](char)++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#withDecimalSeparator%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-withNegativeSign-1]]<> link:{java8-javadoc}/java/time/format/DecimalStyle.html#withNegativeSign%2Dchar%2D[withNegativeSign](char)++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#withNegativeSign%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-withPositiveSign-1]]<> link:{java8-javadoc}/java/time/format/DecimalStyle.html#withPositiveSign%2Dchar%2D[withPositiveSign](char)++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#withPositiveSign%2Dchar%2D[java 9]) -* ++[[painless-api-reference-DecimalStyle-withZeroDigit-1]]<> link:{java8-javadoc}/java/time/format/DecimalStyle.html#withZeroDigit%2Dchar%2D[withZeroDigit](char)++ (link:{java9-javadoc}/java/time/format/DecimalStyle.html#withZeroDigit%2Dchar%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Deque.asciidoc b/docs/painless/painless-api-reference/Deque.asciidoc deleted file mode 100644 index 4dc6b5c06fc84..0000000000000 --- a/docs/painless/painless-api-reference/Deque.asciidoc +++ /dev/null @@ -1,25 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Deque]]++Deque++:: -* ++[[painless-api-reference-Deque-addFirst-1]]void link:{java8-javadoc}/java/util/Deque.html#addFirst%2Djava.lang.Object%2D[addFirst](def)++ (link:{java9-javadoc}/java/util/Deque.html#addFirst%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-addLast-1]]void link:{java8-javadoc}/java/util/Deque.html#addLast%2Djava.lang.Object%2D[addLast](def)++ (link:{java9-javadoc}/java/util/Deque.html#addLast%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-descendingIterator-0]]<> link:{java8-javadoc}/java/util/Deque.html#descendingIterator%2D%2D[descendingIterator]()++ (link:{java9-javadoc}/java/util/Deque.html#descendingIterator%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-getFirst-0]]def link:{java8-javadoc}/java/util/Deque.html#getFirst%2D%2D[getFirst]()++ (link:{java9-javadoc}/java/util/Deque.html#getFirst%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-getLast-0]]def link:{java8-javadoc}/java/util/Deque.html#getLast%2D%2D[getLast]()++ (link:{java9-javadoc}/java/util/Deque.html#getLast%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-offerFirst-1]]boolean link:{java8-javadoc}/java/util/Deque.html#offerFirst%2Djava.lang.Object%2D[offerFirst](def)++ (link:{java9-javadoc}/java/util/Deque.html#offerFirst%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-offerLast-1]]boolean link:{java8-javadoc}/java/util/Deque.html#offerLast%2Djava.lang.Object%2D[offerLast](def)++ (link:{java9-javadoc}/java/util/Deque.html#offerLast%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-peekFirst-0]]def link:{java8-javadoc}/java/util/Deque.html#peekFirst%2D%2D[peekFirst]()++ (link:{java9-javadoc}/java/util/Deque.html#peekFirst%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-peekLast-0]]def link:{java8-javadoc}/java/util/Deque.html#peekLast%2D%2D[peekLast]()++ (link:{java9-javadoc}/java/util/Deque.html#peekLast%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-pollFirst-0]]def link:{java8-javadoc}/java/util/Deque.html#pollFirst%2D%2D[pollFirst]()++ (link:{java9-javadoc}/java/util/Deque.html#pollFirst%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-pollLast-0]]def link:{java8-javadoc}/java/util/Deque.html#pollLast%2D%2D[pollLast]()++ (link:{java9-javadoc}/java/util/Deque.html#pollLast%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-pop-0]]def link:{java8-javadoc}/java/util/Deque.html#pop%2D%2D[pop]()++ (link:{java9-javadoc}/java/util/Deque.html#pop%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-push-1]]void link:{java8-javadoc}/java/util/Deque.html#push%2Djava.lang.Object%2D[push](def)++ (link:{java9-javadoc}/java/util/Deque.html#push%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-remove-1]]boolean link:{java8-javadoc}/java/util/Deque.html#remove%2Djava.lang.Object%2D[remove](def)++ (link:{java9-javadoc}/java/util/Deque.html#remove%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-removeFirst-0]]def link:{java8-javadoc}/java/util/Deque.html#removeFirst%2D%2D[removeFirst]()++ (link:{java9-javadoc}/java/util/Deque.html#removeFirst%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-removeFirstOccurrence-1]]boolean link:{java8-javadoc}/java/util/Deque.html#removeFirstOccurrence%2Djava.lang.Object%2D[removeFirstOccurrence](def)++ (link:{java9-javadoc}/java/util/Deque.html#removeFirstOccurrence%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Deque-removeLast-0]]def link:{java8-javadoc}/java/util/Deque.html#removeLast%2D%2D[removeLast]()++ (link:{java9-javadoc}/java/util/Deque.html#removeLast%2D%2D[java 9]) -* ++[[painless-api-reference-Deque-removeLastOccurrence-1]]boolean link:{java8-javadoc}/java/util/Deque.html#removeLastOccurrence%2Djava.lang.Object%2D[removeLastOccurrence](def)++ (link:{java9-javadoc}/java/util/Deque.html#removeLastOccurrence%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Dictionary.asciidoc b/docs/painless/painless-api-reference/Dictionary.asciidoc deleted file mode 100644 index bfa8f2c495e19..0000000000000 --- a/docs/painless/painless-api-reference/Dictionary.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Dictionary]]++Dictionary++:: -* ++[[painless-api-reference-Dictionary-elements-0]]<> link:{java8-javadoc}/java/util/Dictionary.html#elements%2D%2D[elements]()++ (link:{java9-javadoc}/java/util/Dictionary.html#elements%2D%2D[java 9]) -* ++[[painless-api-reference-Dictionary-get-1]]def link:{java8-javadoc}/java/util/Dictionary.html#get%2Djava.lang.Object%2D[get](def)++ (link:{java9-javadoc}/java/util/Dictionary.html#get%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Dictionary-isEmpty-0]]boolean link:{java8-javadoc}/java/util/Dictionary.html#isEmpty%2D%2D[isEmpty]()++ (link:{java9-javadoc}/java/util/Dictionary.html#isEmpty%2D%2D[java 9]) -* ++[[painless-api-reference-Dictionary-keys-0]]<> link:{java8-javadoc}/java/util/Dictionary.html#keys%2D%2D[keys]()++ (link:{java9-javadoc}/java/util/Dictionary.html#keys%2D%2D[java 9]) -* ++[[painless-api-reference-Dictionary-put-2]]def link:{java8-javadoc}/java/util/Dictionary.html#put%2Djava.lang.Object%2Djava.lang.Object%2D[put](def, def)++ (link:{java9-javadoc}/java/util/Dictionary.html#put%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Dictionary-remove-1]]def link:{java8-javadoc}/java/util/Dictionary.html#remove%2Djava.lang.Object%2D[remove](def)++ (link:{java9-javadoc}/java/util/Dictionary.html#remove%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Dictionary-size-0]]int link:{java8-javadoc}/java/util/Dictionary.html#size%2D%2D[size]()++ (link:{java9-javadoc}/java/util/Dictionary.html#size%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Double.asciidoc b/docs/painless/painless-api-reference/Double.asciidoc deleted file mode 100644 index 50d1d8c217cba..0000000000000 --- a/docs/painless/painless-api-reference/Double.asciidoc +++ /dev/null @@ -1,35 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Double]]++Double++:: -** [[painless-api-reference-Double-BYTES]]static int link:{java8-javadoc}/java/lang/Double.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Double.html#BYTES[java 9]) -** [[painless-api-reference-Double-MAX_EXPONENT]]static int link:{java8-javadoc}/java/lang/Double.html#MAX_EXPONENT[MAX_EXPONENT] (link:{java9-javadoc}/java/lang/Double.html#MAX_EXPONENT[java 9]) -** [[painless-api-reference-Double-MAX_VALUE]]static double link:{java8-javadoc}/java/lang/Double.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Double.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Double-MIN_EXPONENT]]static int link:{java8-javadoc}/java/lang/Double.html#MIN_EXPONENT[MIN_EXPONENT] (link:{java9-javadoc}/java/lang/Double.html#MIN_EXPONENT[java 9]) -** [[painless-api-reference-Double-MIN_NORMAL]]static double link:{java8-javadoc}/java/lang/Double.html#MIN_NORMAL[MIN_NORMAL] (link:{java9-javadoc}/java/lang/Double.html#MIN_NORMAL[java 9]) -** [[painless-api-reference-Double-MIN_VALUE]]static double link:{java8-javadoc}/java/lang/Double.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Double.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Double-NEGATIVE_INFINITY]]static double link:{java8-javadoc}/java/lang/Double.html#NEGATIVE_INFINITY[NEGATIVE_INFINITY] (link:{java9-javadoc}/java/lang/Double.html#NEGATIVE_INFINITY[java 9]) -** [[painless-api-reference-Double-NaN]]static double link:{java8-javadoc}/java/lang/Double.html#NaN[NaN] (link:{java9-javadoc}/java/lang/Double.html#NaN[java 9]) -** [[painless-api-reference-Double-POSITIVE_INFINITY]]static double link:{java8-javadoc}/java/lang/Double.html#POSITIVE_INFINITY[POSITIVE_INFINITY] (link:{java9-javadoc}/java/lang/Double.html#POSITIVE_INFINITY[java 9]) -** [[painless-api-reference-Double-SIZE]]static int link:{java8-javadoc}/java/lang/Double.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Double.html#SIZE[java 9]) -* ++[[painless-api-reference-Double-compare-2]]static int link:{java8-javadoc}/java/lang/Double.html#compare%2Ddouble%2Ddouble%2D[compare](double, double)++ (link:{java9-javadoc}/java/lang/Double.html#compare%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-doubleToLongBits-1]]static long link:{java8-javadoc}/java/lang/Double.html#doubleToLongBits%2Ddouble%2D[doubleToLongBits](double)++ (link:{java9-javadoc}/java/lang/Double.html#doubleToLongBits%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-doubleToRawLongBits-1]]static long link:{java8-javadoc}/java/lang/Double.html#doubleToRawLongBits%2Ddouble%2D[doubleToRawLongBits](double)++ (link:{java9-javadoc}/java/lang/Double.html#doubleToRawLongBits%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-hashCode-1]]static int link:{java8-javadoc}/java/lang/Double.html#hashCode%2Ddouble%2D[hashCode](double)++ (link:{java9-javadoc}/java/lang/Double.html#hashCode%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-isFinite-1]]static boolean link:{java8-javadoc}/java/lang/Double.html#isFinite%2Ddouble%2D[isFinite](double)++ (link:{java9-javadoc}/java/lang/Double.html#isFinite%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-isInfinite-1]]static boolean link:{java8-javadoc}/java/lang/Double.html#isInfinite%2Ddouble%2D[isInfinite](double)++ (link:{java9-javadoc}/java/lang/Double.html#isInfinite%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-isNaN-1]]static boolean link:{java8-javadoc}/java/lang/Double.html#isNaN%2Ddouble%2D[isNaN](double)++ (link:{java9-javadoc}/java/lang/Double.html#isNaN%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-longBitsToDouble-1]]static double link:{java8-javadoc}/java/lang/Double.html#longBitsToDouble%2Dlong%2D[longBitsToDouble](long)++ (link:{java9-javadoc}/java/lang/Double.html#longBitsToDouble%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Double-max-2]]static double link:{java8-javadoc}/java/lang/Double.html#max%2Ddouble%2Ddouble%2D[max](double, double)++ (link:{java9-javadoc}/java/lang/Double.html#max%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-min-2]]static double link:{java8-javadoc}/java/lang/Double.html#min%2Ddouble%2Ddouble%2D[min](double, double)++ (link:{java9-javadoc}/java/lang/Double.html#min%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-parseDouble-1]]static double link:{java8-javadoc}/java/lang/Double.html#parseDouble%2Djava.lang.String%2D[parseDouble](<>)++ (link:{java9-javadoc}/java/lang/Double.html#parseDouble%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Double-sum-2]]static double link:{java8-javadoc}/java/lang/Double.html#sum%2Ddouble%2Ddouble%2D[sum](double, double)++ (link:{java9-javadoc}/java/lang/Double.html#sum%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-toHexString-1]]static <> link:{java8-javadoc}/java/lang/Double.html#toHexString%2Ddouble%2D[toHexString](double)++ (link:{java9-javadoc}/java/lang/Double.html#toHexString%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-toString-1]]static <> link:{java8-javadoc}/java/lang/Double.html#toString%2Ddouble%2D[toString](double)++ (link:{java9-javadoc}/java/lang/Double.html#toString%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Double.html#valueOf%2Ddouble%2D[valueOf](double)++ (link:{java9-javadoc}/java/lang/Double.html#valueOf%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Double-compareTo-1]]int link:{java8-javadoc}/java/lang/Double.html#compareTo%2Djava.lang.Double%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Double.html#compareTo%2Djava.lang.Double%2D[java 9]) -* ++[[painless-api-reference-Double-isInfinite-0]]boolean link:{java8-javadoc}/java/lang/Double.html#isInfinite%2D%2D[isInfinite]()++ (link:{java9-javadoc}/java/lang/Double.html#isInfinite%2D%2D[java 9]) -* ++[[painless-api-reference-Double-isNaN-0]]boolean link:{java8-javadoc}/java/lang/Double.html#isNaN%2D%2D[isNaN]()++ (link:{java9-javadoc}/java/lang/Double.html#isNaN%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleBinaryOperator.asciidoc b/docs/painless/painless-api-reference/DoubleBinaryOperator.asciidoc deleted file mode 100644 index deec8c5337c81..0000000000000 --- a/docs/painless/painless-api-reference/DoubleBinaryOperator.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleBinaryOperator]]++DoubleBinaryOperator++:: -* ++[[painless-api-reference-DoubleBinaryOperator-applyAsDouble-2]]double link:{java8-javadoc}/java/util/function/DoubleBinaryOperator.html#applyAsDouble%2Ddouble%2Ddouble%2D[applyAsDouble](double, double)++ (link:{java9-javadoc}/java/util/function/DoubleBinaryOperator.html#applyAsDouble%2Ddouble%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleConsumer.asciidoc b/docs/painless/painless-api-reference/DoubleConsumer.asciidoc deleted file mode 100644 index 3c078a4d4fb1c..0000000000000 --- a/docs/painless/painless-api-reference/DoubleConsumer.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleConsumer]]++DoubleConsumer++:: -* ++[[painless-api-reference-DoubleConsumer-accept-1]]void link:{java8-javadoc}/java/util/function/DoubleConsumer.html#accept%2Ddouble%2D[accept](double)++ (link:{java9-javadoc}/java/util/function/DoubleConsumer.html#accept%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-DoubleConsumer-andThen-1]]<> link:{java8-javadoc}/java/util/function/DoubleConsumer.html#andThen%2Djava.util.function.DoubleConsumer%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/DoubleConsumer.html#andThen%2Djava.util.function.DoubleConsumer%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleFunction.asciidoc b/docs/painless/painless-api-reference/DoubleFunction.asciidoc deleted file mode 100644 index 082a70001328e..0000000000000 --- a/docs/painless/painless-api-reference/DoubleFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleFunction]]++DoubleFunction++:: -* ++[[painless-api-reference-DoubleFunction-apply-1]]def link:{java8-javadoc}/java/util/function/DoubleFunction.html#apply%2Ddouble%2D[apply](double)++ (link:{java9-javadoc}/java/util/function/DoubleFunction.html#apply%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoublePredicate.asciidoc b/docs/painless/painless-api-reference/DoublePredicate.asciidoc deleted file mode 100644 index 48cfd3c59296c..0000000000000 --- a/docs/painless/painless-api-reference/DoublePredicate.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoublePredicate]]++DoublePredicate++:: -* ++[[painless-api-reference-DoublePredicate-and-1]]<> link:{java8-javadoc}/java/util/function/DoublePredicate.html#and%2Djava.util.function.DoublePredicate%2D[and](<>)++ (link:{java9-javadoc}/java/util/function/DoublePredicate.html#and%2Djava.util.function.DoublePredicate%2D[java 9]) -* ++[[painless-api-reference-DoublePredicate-negate-0]]<> link:{java8-javadoc}/java/util/function/DoublePredicate.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/util/function/DoublePredicate.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-DoublePredicate-or-1]]<> link:{java8-javadoc}/java/util/function/DoublePredicate.html#or%2Djava.util.function.DoublePredicate%2D[or](<>)++ (link:{java9-javadoc}/java/util/function/DoublePredicate.html#or%2Djava.util.function.DoublePredicate%2D[java 9]) -* ++[[painless-api-reference-DoublePredicate-test-1]]boolean link:{java8-javadoc}/java/util/function/DoublePredicate.html#test%2Ddouble%2D[test](double)++ (link:{java9-javadoc}/java/util/function/DoublePredicate.html#test%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleStream.Builder.asciidoc b/docs/painless/painless-api-reference/DoubleStream.Builder.asciidoc deleted file mode 100644 index 8f59249115f8f..0000000000000 --- a/docs/painless/painless-api-reference/DoubleStream.Builder.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleStream-Builder]]++DoubleStream.Builder++:: -* ++[[painless-api-reference-DoubleStream-Builder-add-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.Builder.html#add%2Ddouble%2D[add](double)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.Builder.html#add%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-Builder-build-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.Builder.html#build%2D%2D[build]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.Builder.html#build%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleStream.asciidoc b/docs/painless/painless-api-reference/DoubleStream.asciidoc deleted file mode 100644 index d3f517e29e40b..0000000000000 --- a/docs/painless/painless-api-reference/DoubleStream.asciidoc +++ /dev/null @@ -1,43 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleStream]]++DoubleStream++:: -* ++[[painless-api-reference-DoubleStream-builder-0]]static <> link:{java8-javadoc}/java/util/stream/DoubleStream.html#builder%2D%2D[builder]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#builder%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-concat-2]]static <> link:{java8-javadoc}/java/util/stream/DoubleStream.html#concat%2Djava.util.stream.DoubleStream%2Djava.util.stream.DoubleStream%2D[concat](<>, <>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#concat%2Djava.util.stream.DoubleStream%2Djava.util.stream.DoubleStream%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-empty-0]]static <> link:{java8-javadoc}/java/util/stream/DoubleStream.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-of-1]]static <> link:{java8-javadoc}/java/util/stream/DoubleStream.html#of%2Ddouble:A%2D[of](double[])++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#of%2Ddouble:A%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-allMatch-1]]boolean link:{java8-javadoc}/java/util/stream/DoubleStream.html#allMatch%2Djava.util.function.DoublePredicate%2D[allMatch](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#allMatch%2Djava.util.function.DoublePredicate%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-anyMatch-1]]boolean link:{java8-javadoc}/java/util/stream/DoubleStream.html#anyMatch%2Djava.util.function.DoublePredicate%2D[anyMatch](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#anyMatch%2Djava.util.function.DoublePredicate%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-average-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#average%2D%2D[average]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#average%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-boxed-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#boxed%2D%2D[boxed]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#boxed%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-collect-3]]def link:{java8-javadoc}/java/util/stream/DoubleStream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.ObjDoubleConsumer%2Djava.util.function.BiConsumer%2D[collect](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.ObjDoubleConsumer%2Djava.util.function.BiConsumer%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-count-0]]long link:{java8-javadoc}/java/util/stream/DoubleStream.html#count%2D%2D[count]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#count%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-distinct-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#distinct%2D%2D[distinct]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#distinct%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-filter-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#filter%2Djava.util.function.DoublePredicate%2D[filter](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#filter%2Djava.util.function.DoublePredicate%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-findAny-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#findAny%2D%2D[findAny]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#findAny%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-findFirst-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#findFirst%2D%2D[findFirst]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#findFirst%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-flatMap-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#flatMap%2Djava.util.function.DoubleFunction%2D[flatMap](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#flatMap%2Djava.util.function.DoubleFunction%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-forEach-1]]void link:{java8-javadoc}/java/util/stream/DoubleStream.html#forEach%2Djava.util.function.DoubleConsumer%2D[forEach](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#forEach%2Djava.util.function.DoubleConsumer%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-forEachOrdered-1]]void link:{java8-javadoc}/java/util/stream/DoubleStream.html#forEachOrdered%2Djava.util.function.DoubleConsumer%2D[forEachOrdered](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#forEachOrdered%2Djava.util.function.DoubleConsumer%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-iterator-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#iterator%2D%2D[iterator]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#iterator%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-limit-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#limit%2Dlong%2D[limit](long)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#limit%2Dlong%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-map-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#map%2Djava.util.function.DoubleUnaryOperator%2D[map](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#map%2Djava.util.function.DoubleUnaryOperator%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-mapToInt-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#mapToInt%2Djava.util.function.DoubleToIntFunction%2D[mapToInt](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#mapToInt%2Djava.util.function.DoubleToIntFunction%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-mapToLong-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#mapToLong%2Djava.util.function.DoubleToLongFunction%2D[mapToLong](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#mapToLong%2Djava.util.function.DoubleToLongFunction%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-mapToObj-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#mapToObj%2Djava.util.function.DoubleFunction%2D[mapToObj](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#mapToObj%2Djava.util.function.DoubleFunction%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-max-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#max%2D%2D[max]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#max%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-min-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#min%2D%2D[min]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#min%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-noneMatch-1]]boolean link:{java8-javadoc}/java/util/stream/DoubleStream.html#noneMatch%2Djava.util.function.DoublePredicate%2D[noneMatch](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#noneMatch%2Djava.util.function.DoublePredicate%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-peek-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#peek%2Djava.util.function.DoubleConsumer%2D[peek](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#peek%2Djava.util.function.DoubleConsumer%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-reduce-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#reduce%2Djava.util.function.DoubleBinaryOperator%2D[reduce](<>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#reduce%2Djava.util.function.DoubleBinaryOperator%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-reduce-2]]double link:{java8-javadoc}/java/util/stream/DoubleStream.html#reduce%2Ddouble%2Djava.util.function.DoubleBinaryOperator%2D[reduce](double, <>)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#reduce%2Ddouble%2Djava.util.function.DoubleBinaryOperator%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-sequential-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#sequential%2D%2D[sequential]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#sequential%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-skip-1]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#skip%2Dlong%2D[skip](long)++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#skip%2Dlong%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-sorted-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#sorted%2D%2D[sorted]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#sorted%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-spliterator-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#spliterator%2D%2D[spliterator]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#spliterator%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-sum-0]]double link:{java8-javadoc}/java/util/stream/DoubleStream.html#sum%2D%2D[sum]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#sum%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-summaryStatistics-0]]<> link:{java8-javadoc}/java/util/stream/DoubleStream.html#summaryStatistics%2D%2D[summaryStatistics]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#summaryStatistics%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleStream-toArray-0]]double[] link:{java8-javadoc}/java/util/stream/DoubleStream.html#toArray%2D%2D[toArray]()++ (link:{java9-javadoc}/java/util/stream/DoubleStream.html#toArray%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleSummaryStatistics.asciidoc b/docs/painless/painless-api-reference/DoubleSummaryStatistics.asciidoc deleted file mode 100644 index 976eb98440410..0000000000000 --- a/docs/painless/painless-api-reference/DoubleSummaryStatistics.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleSummaryStatistics]]++DoubleSummaryStatistics++:: -* ++[[painless-api-reference-DoubleSummaryStatistics-DoubleSummaryStatistics-0]]link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#DoubleSummaryStatistics%2D%2D[DoubleSummaryStatistics]()++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#DoubleSummaryStatistics%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleSummaryStatistics-combine-1]]void link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#combine%2Djava.util.DoubleSummaryStatistics%2D[combine](<>)++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#combine%2Djava.util.DoubleSummaryStatistics%2D[java 9]) -* ++[[painless-api-reference-DoubleSummaryStatistics-getAverage-0]]double link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#getAverage%2D%2D[getAverage]()++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#getAverage%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleSummaryStatistics-getCount-0]]long link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#getCount%2D%2D[getCount]()++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#getCount%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleSummaryStatistics-getMax-0]]double link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#getMax%2D%2D[getMax]()++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#getMax%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleSummaryStatistics-getMin-0]]double link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#getMin%2D%2D[getMin]()++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#getMin%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleSummaryStatistics-getSum-0]]double link:{java8-javadoc}/java/util/DoubleSummaryStatistics.html#getSum%2D%2D[getSum]()++ (link:{java9-javadoc}/java/util/DoubleSummaryStatistics.html#getSum%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleSupplier.asciidoc b/docs/painless/painless-api-reference/DoubleSupplier.asciidoc deleted file mode 100644 index 1de478ab50f95..0000000000000 --- a/docs/painless/painless-api-reference/DoubleSupplier.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleSupplier]]++DoubleSupplier++:: -* ++[[painless-api-reference-DoubleSupplier-getAsDouble-0]]double link:{java8-javadoc}/java/util/function/DoubleSupplier.html#getAsDouble%2D%2D[getAsDouble]()++ (link:{java9-javadoc}/java/util/function/DoubleSupplier.html#getAsDouble%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleToIntFunction.asciidoc b/docs/painless/painless-api-reference/DoubleToIntFunction.asciidoc deleted file mode 100644 index 9448a88faf35a..0000000000000 --- a/docs/painless/painless-api-reference/DoubleToIntFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleToIntFunction]]++DoubleToIntFunction++:: -* ++[[painless-api-reference-DoubleToIntFunction-applyAsInt-1]]int link:{java8-javadoc}/java/util/function/DoubleToIntFunction.html#applyAsInt%2Ddouble%2D[applyAsInt](double)++ (link:{java9-javadoc}/java/util/function/DoubleToIntFunction.html#applyAsInt%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleToLongFunction.asciidoc b/docs/painless/painless-api-reference/DoubleToLongFunction.asciidoc deleted file mode 100644 index 62a3f841b3391..0000000000000 --- a/docs/painless/painless-api-reference/DoubleToLongFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleToLongFunction]]++DoubleToLongFunction++:: -* ++[[painless-api-reference-DoubleToLongFunction-applyAsLong-1]]long link:{java8-javadoc}/java/util/function/DoubleToLongFunction.html#applyAsLong%2Ddouble%2D[applyAsLong](double)++ (link:{java9-javadoc}/java/util/function/DoubleToLongFunction.html#applyAsLong%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DoubleUnaryOperator.asciidoc b/docs/painless/painless-api-reference/DoubleUnaryOperator.asciidoc deleted file mode 100644 index a78b8ac5d0b1c..0000000000000 --- a/docs/painless/painless-api-reference/DoubleUnaryOperator.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DoubleUnaryOperator]]++DoubleUnaryOperator++:: -* ++[[painless-api-reference-DoubleUnaryOperator-identity-0]]static <> link:{java8-javadoc}/java/util/function/DoubleUnaryOperator.html#identity%2D%2D[identity]()++ (link:{java9-javadoc}/java/util/function/DoubleUnaryOperator.html#identity%2D%2D[java 9]) -* ++[[painless-api-reference-DoubleUnaryOperator-andThen-1]]<> link:{java8-javadoc}/java/util/function/DoubleUnaryOperator.html#andThen%2Djava.util.function.DoubleUnaryOperator%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/DoubleUnaryOperator.html#andThen%2Djava.util.function.DoubleUnaryOperator%2D[java 9]) -* ++[[painless-api-reference-DoubleUnaryOperator-applyAsDouble-1]]double link:{java8-javadoc}/java/util/function/DoubleUnaryOperator.html#applyAsDouble%2Ddouble%2D[applyAsDouble](double)++ (link:{java9-javadoc}/java/util/function/DoubleUnaryOperator.html#applyAsDouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-DoubleUnaryOperator-compose-1]]<> link:{java8-javadoc}/java/util/function/DoubleUnaryOperator.html#compose%2Djava.util.function.DoubleUnaryOperator%2D[compose](<>)++ (link:{java9-javadoc}/java/util/function/DoubleUnaryOperator.html#compose%2Djava.util.function.DoubleUnaryOperator%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/DuplicateFormatFlagsException.asciidoc b/docs/painless/painless-api-reference/DuplicateFormatFlagsException.asciidoc deleted file mode 100644 index f26103fdc198f..0000000000000 --- a/docs/painless/painless-api-reference/DuplicateFormatFlagsException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-DuplicateFormatFlagsException]]++DuplicateFormatFlagsException++:: -* ++[[painless-api-reference-DuplicateFormatFlagsException-DuplicateFormatFlagsException-1]]link:{java8-javadoc}/java/util/DuplicateFormatFlagsException.html#DuplicateFormatFlagsException%2Djava.lang.String%2D[DuplicateFormatFlagsException](<>)++ (link:{java9-javadoc}/java/util/DuplicateFormatFlagsException.html#DuplicateFormatFlagsException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-DuplicateFormatFlagsException-getFlags-0]]<> link:{java8-javadoc}/java/util/DuplicateFormatFlagsException.html#getFlags%2D%2D[getFlags]()++ (link:{java9-javadoc}/java/util/DuplicateFormatFlagsException.html#getFlags%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Duration.asciidoc b/docs/painless/painless-api-reference/Duration.asciidoc deleted file mode 100644 index ef9ecf720a0f2..0000000000000 --- a/docs/painless/painless-api-reference/Duration.asciidoc +++ /dev/null @@ -1,51 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Duration]]++Duration++:: -** [[painless-api-reference-Duration-ZERO]]static <> link:{java8-javadoc}/java/time/Duration.html#ZERO[ZERO] (link:{java9-javadoc}/java/time/Duration.html#ZERO[java 9]) -* ++[[painless-api-reference-Duration-between-2]]static <> link:{java8-javadoc}/java/time/Duration.html#between%2Djava.time.temporal.Temporal%2Djava.time.temporal.Temporal%2D[between](<>, <>)++ (link:{java9-javadoc}/java/time/Duration.html#between%2Djava.time.temporal.Temporal%2Djava.time.temporal.Temporal%2D[java 9]) -* ++[[painless-api-reference-Duration-from-1]]static <> link:{java8-javadoc}/java/time/Duration.html#from%2Djava.time.temporal.TemporalAmount%2D[from](<>)++ (link:{java9-javadoc}/java/time/Duration.html#from%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Duration-of-2]]static <> link:{java8-javadoc}/java/time/Duration.html#of%2Dlong%2Djava.time.temporal.TemporalUnit%2D[of](long, <>)++ (link:{java9-javadoc}/java/time/Duration.html#of%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Duration-ofDays-1]]static <> link:{java8-javadoc}/java/time/Duration.html#ofDays%2Dlong%2D[ofDays](long)++ (link:{java9-javadoc}/java/time/Duration.html#ofDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-ofHours-1]]static <> link:{java8-javadoc}/java/time/Duration.html#ofHours%2Dlong%2D[ofHours](long)++ (link:{java9-javadoc}/java/time/Duration.html#ofHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-ofMillis-1]]static <> link:{java8-javadoc}/java/time/Duration.html#ofMillis%2Dlong%2D[ofMillis](long)++ (link:{java9-javadoc}/java/time/Duration.html#ofMillis%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-ofMinutes-1]]static <> link:{java8-javadoc}/java/time/Duration.html#ofMinutes%2Dlong%2D[ofMinutes](long)++ (link:{java9-javadoc}/java/time/Duration.html#ofMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-ofNanos-1]]static <> link:{java8-javadoc}/java/time/Duration.html#ofNanos%2Dlong%2D[ofNanos](long)++ (link:{java9-javadoc}/java/time/Duration.html#ofNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-ofSeconds-1]]static <> link:{java8-javadoc}/java/time/Duration.html#ofSeconds%2Dlong%2D[ofSeconds](long)++ (link:{java9-javadoc}/java/time/Duration.html#ofSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-ofSeconds-2]]static <> link:{java8-javadoc}/java/time/Duration.html#ofSeconds%2Dlong%2Dlong%2D[ofSeconds](long, long)++ (link:{java9-javadoc}/java/time/Duration.html#ofSeconds%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-parse-1]]static <> link:{java8-javadoc}/java/time/Duration.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/Duration.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Duration-abs-0]]<> link:{java8-javadoc}/java/time/Duration.html#abs%2D%2D[abs]()++ (link:{java9-javadoc}/java/time/Duration.html#abs%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-compareTo-1]]int link:{java8-javadoc}/java/time/Duration.html#compareTo%2Djava.time.Duration%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/Duration.html#compareTo%2Djava.time.Duration%2D[java 9]) -* ++[[painless-api-reference-Duration-dividedBy-1]]<> link:{java8-javadoc}/java/time/Duration.html#dividedBy%2Dlong%2D[dividedBy](long)++ (link:{java9-javadoc}/java/time/Duration.html#dividedBy%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-getNano-0]]int link:{java8-javadoc}/java/time/Duration.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/Duration.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-getSeconds-0]]long link:{java8-javadoc}/java/time/Duration.html#getSeconds%2D%2D[getSeconds]()++ (link:{java9-javadoc}/java/time/Duration.html#getSeconds%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-isNegative-0]]boolean link:{java8-javadoc}/java/time/Duration.html#isNegative%2D%2D[isNegative]()++ (link:{java9-javadoc}/java/time/Duration.html#isNegative%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-isZero-0]]boolean link:{java8-javadoc}/java/time/Duration.html#isZero%2D%2D[isZero]()++ (link:{java9-javadoc}/java/time/Duration.html#isZero%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-minus-1]]<> link:{java8-javadoc}/java/time/Duration.html#minus%2Djava.time.Duration%2D[minus](<>)++ (link:{java9-javadoc}/java/time/Duration.html#minus%2Djava.time.Duration%2D[java 9]) -* ++[[painless-api-reference-Duration-minus-2]]<> link:{java8-javadoc}/java/time/Duration.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/Duration.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Duration-minusDays-1]]<> link:{java8-javadoc}/java/time/Duration.html#minusDays%2Dlong%2D[minusDays](long)++ (link:{java9-javadoc}/java/time/Duration.html#minusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-minusHours-1]]<> link:{java8-javadoc}/java/time/Duration.html#minusHours%2Dlong%2D[minusHours](long)++ (link:{java9-javadoc}/java/time/Duration.html#minusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-minusMillis-1]]<> link:{java8-javadoc}/java/time/Duration.html#minusMillis%2Dlong%2D[minusMillis](long)++ (link:{java9-javadoc}/java/time/Duration.html#minusMillis%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-minusMinutes-1]]<> link:{java8-javadoc}/java/time/Duration.html#minusMinutes%2Dlong%2D[minusMinutes](long)++ (link:{java9-javadoc}/java/time/Duration.html#minusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-minusNanos-1]]<> link:{java8-javadoc}/java/time/Duration.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/Duration.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-minusSeconds-1]]<> link:{java8-javadoc}/java/time/Duration.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/Duration.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-multipliedBy-1]]<> link:{java8-javadoc}/java/time/Duration.html#multipliedBy%2Dlong%2D[multipliedBy](long)++ (link:{java9-javadoc}/java/time/Duration.html#multipliedBy%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-negated-0]]<> link:{java8-javadoc}/java/time/Duration.html#negated%2D%2D[negated]()++ (link:{java9-javadoc}/java/time/Duration.html#negated%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-plus-1]]<> link:{java8-javadoc}/java/time/Duration.html#plus%2Djava.time.Duration%2D[plus](<>)++ (link:{java9-javadoc}/java/time/Duration.html#plus%2Djava.time.Duration%2D[java 9]) -* ++[[painless-api-reference-Duration-plus-2]]<> link:{java8-javadoc}/java/time/Duration.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/Duration.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Duration-plusDays-1]]<> link:{java8-javadoc}/java/time/Duration.html#plusDays%2Dlong%2D[plusDays](long)++ (link:{java9-javadoc}/java/time/Duration.html#plusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-plusHours-1]]<> link:{java8-javadoc}/java/time/Duration.html#plusHours%2Dlong%2D[plusHours](long)++ (link:{java9-javadoc}/java/time/Duration.html#plusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-plusMillis-1]]<> link:{java8-javadoc}/java/time/Duration.html#plusMillis%2Dlong%2D[plusMillis](long)++ (link:{java9-javadoc}/java/time/Duration.html#plusMillis%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-plusMinutes-1]]<> link:{java8-javadoc}/java/time/Duration.html#plusMinutes%2Dlong%2D[plusMinutes](long)++ (link:{java9-javadoc}/java/time/Duration.html#plusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-plusNanos-1]]<> link:{java8-javadoc}/java/time/Duration.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/Duration.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-plusSeconds-1]]<> link:{java8-javadoc}/java/time/Duration.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/Duration.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Duration-toDays-0]]long link:{java8-javadoc}/java/time/Duration.html#toDays%2D%2D[toDays]()++ (link:{java9-javadoc}/java/time/Duration.html#toDays%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-toHours-0]]long link:{java8-javadoc}/java/time/Duration.html#toHours%2D%2D[toHours]()++ (link:{java9-javadoc}/java/time/Duration.html#toHours%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-toMillis-0]]long link:{java8-javadoc}/java/time/Duration.html#toMillis%2D%2D[toMillis]()++ (link:{java9-javadoc}/java/time/Duration.html#toMillis%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-toMinutes-0]]long link:{java8-javadoc}/java/time/Duration.html#toMinutes%2D%2D[toMinutes]()++ (link:{java9-javadoc}/java/time/Duration.html#toMinutes%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-toNanos-0]]long link:{java8-javadoc}/java/time/Duration.html#toNanos%2D%2D[toNanos]()++ (link:{java9-javadoc}/java/time/Duration.html#toNanos%2D%2D[java 9]) -* ++[[painless-api-reference-Duration-withNanos-1]]<> link:{java8-javadoc}/java/time/Duration.html#withNanos%2Dint%2D[withNanos](int)++ (link:{java9-javadoc}/java/time/Duration.html#withNanos%2Dint%2D[java 9]) -* ++[[painless-api-reference-Duration-withSeconds-1]]<> link:{java8-javadoc}/java/time/Duration.html#withSeconds%2Dlong%2D[withSeconds](long)++ (link:{java9-javadoc}/java/time/Duration.html#withSeconds%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/EmptyStackException.asciidoc b/docs/painless/painless-api-reference/EmptyStackException.asciidoc deleted file mode 100644 index 09ca4ca96be20..0000000000000 --- a/docs/painless/painless-api-reference/EmptyStackException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-EmptyStackException]]++EmptyStackException++:: -* ++[[painless-api-reference-EmptyStackException-EmptyStackException-0]]link:{java8-javadoc}/java/util/EmptyStackException.html#EmptyStackException%2D%2D[EmptyStackException]()++ (link:{java9-javadoc}/java/util/EmptyStackException.html#EmptyStackException%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Enum.asciidoc b/docs/painless/painless-api-reference/Enum.asciidoc deleted file mode 100644 index 077044dbfb9cd..0000000000000 --- a/docs/painless/painless-api-reference/Enum.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Enum]]++Enum++:: -* ++[[painless-api-reference-Enum-compareTo-1]]int link:{java8-javadoc}/java/lang/Enum.html#compareTo%2Djava.lang.Enum%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Enum.html#compareTo%2Djava.lang.Enum%2D[java 9]) -* ++[[painless-api-reference-Enum-name-0]]<> link:{java8-javadoc}/java/lang/Enum.html#name%2D%2D[name]()++ (link:{java9-javadoc}/java/lang/Enum.html#name%2D%2D[java 9]) -* ++[[painless-api-reference-Enum-ordinal-0]]int link:{java8-javadoc}/java/lang/Enum.html#ordinal%2D%2D[ordinal]()++ (link:{java9-javadoc}/java/lang/Enum.html#ordinal%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/EnumConstantNotPresentException.asciidoc b/docs/painless/painless-api-reference/EnumConstantNotPresentException.asciidoc deleted file mode 100644 index 66a709379d797..0000000000000 --- a/docs/painless/painless-api-reference/EnumConstantNotPresentException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-EnumConstantNotPresentException]]++EnumConstantNotPresentException++:: -* ++[[painless-api-reference-EnumConstantNotPresentException-constantName-0]]<> link:{java8-javadoc}/java/lang/EnumConstantNotPresentException.html#constantName%2D%2D[constantName]()++ (link:{java9-javadoc}/java/lang/EnumConstantNotPresentException.html#constantName%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Enumeration.asciidoc b/docs/painless/painless-api-reference/Enumeration.asciidoc deleted file mode 100644 index 51849ec27de0f..0000000000000 --- a/docs/painless/painless-api-reference/Enumeration.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Enumeration]]++Enumeration++:: -* ++[[painless-api-reference-Enumeration-hasMoreElements-0]]boolean link:{java8-javadoc}/java/util/Enumeration.html#hasMoreElements%2D%2D[hasMoreElements]()++ (link:{java9-javadoc}/java/util/Enumeration.html#hasMoreElements%2D%2D[java 9]) -* ++[[painless-api-reference-Enumeration-nextElement-0]]def link:{java8-javadoc}/java/util/Enumeration.html#nextElement%2D%2D[nextElement]()++ (link:{java9-javadoc}/java/util/Enumeration.html#nextElement%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Era.asciidoc b/docs/painless/painless-api-reference/Era.asciidoc deleted file mode 100644 index db5ee6aa86b0d..0000000000000 --- a/docs/painless/painless-api-reference/Era.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Era]]++Era++:: -* ++[[painless-api-reference-Era-getDisplayName-2]]<> link:{java8-javadoc}/java/time/chrono/Era.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[getDisplayName](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/Era.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Era-getValue-0]]int link:{java8-javadoc}/java/time/chrono/Era.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/chrono/Era.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/EventListener.asciidoc b/docs/painless/painless-api-reference/EventListener.asciidoc deleted file mode 100644 index 499095ae020a0..0000000000000 --- a/docs/painless/painless-api-reference/EventListener.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-EventListener]]++EventListener++:: -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/EventListenerProxy.asciidoc b/docs/painless/painless-api-reference/EventListenerProxy.asciidoc deleted file mode 100644 index a3a4958ec7115..0000000000000 --- a/docs/painless/painless-api-reference/EventListenerProxy.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-EventListenerProxy]]++EventListenerProxy++:: -* ++[[painless-api-reference-EventListenerProxy-getListener-0]]<> link:{java8-javadoc}/java/util/EventListenerProxy.html#getListener%2D%2D[getListener]()++ (link:{java9-javadoc}/java/util/EventListenerProxy.html#getListener%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/EventObject.asciidoc b/docs/painless/painless-api-reference/EventObject.asciidoc deleted file mode 100644 index 4ac6b14150cb1..0000000000000 --- a/docs/painless/painless-api-reference/EventObject.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-EventObject]]++EventObject++:: -* ++[[painless-api-reference-EventObject-EventObject-1]]link:{java8-javadoc}/java/util/EventObject.html#EventObject%2Djava.lang.Object%2D[EventObject](<>)++ (link:{java9-javadoc}/java/util/EventObject.html#EventObject%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-EventObject-getSource-0]]<> link:{java8-javadoc}/java/util/EventObject.html#getSource%2D%2D[getSource]()++ (link:{java9-javadoc}/java/util/EventObject.html#getSource%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Exception.asciidoc b/docs/painless/painless-api-reference/Exception.asciidoc deleted file mode 100644 index d6ff7b0a7e53e..0000000000000 --- a/docs/painless/painless-api-reference/Exception.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Exception]]++Exception++:: -* ++[[painless-api-reference-Exception-Exception-0]]link:{java8-javadoc}/java/lang/Exception.html#Exception%2D%2D[Exception]()++ (link:{java9-javadoc}/java/lang/Exception.html#Exception%2D%2D[java 9]) -* ++[[painless-api-reference-Exception-Exception-1]]link:{java8-javadoc}/java/lang/Exception.html#Exception%2Djava.lang.String%2D[Exception](<>)++ (link:{java9-javadoc}/java/lang/Exception.html#Exception%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Exception-getLocalizedMessage-0]]<> link:{java8-javadoc}/java/lang/Exception.html#getLocalizedMessage%2D%2D[getLocalizedMessage]()++ (link:{java9-javadoc}/java/lang/Exception.html#getLocalizedMessage%2D%2D[java 9]) -* ++[[painless-api-reference-Exception-getMessage-0]]<> link:{java8-javadoc}/java/lang/Exception.html#getMessage%2D%2D[getMessage]()++ (link:{java9-javadoc}/java/lang/Exception.html#getMessage%2D%2D[java 9]) -* ++[[painless-api-reference-Exception-getStackTrace-0]]<>[] link:{java8-javadoc}/java/lang/Exception.html#getStackTrace%2D%2D[getStackTrace]()++ (link:{java9-javadoc}/java/lang/Exception.html#getStackTrace%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/FieldPosition.asciidoc b/docs/painless/painless-api-reference/FieldPosition.asciidoc deleted file mode 100644 index 7a4b5d78407ae..0000000000000 --- a/docs/painless/painless-api-reference/FieldPosition.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-FieldPosition]]++FieldPosition++:: -* ++[[painless-api-reference-FieldPosition-FieldPosition-1]]link:{java8-javadoc}/java/text/FieldPosition.html#FieldPosition%2Dint%2D[FieldPosition](int)++ (link:{java9-javadoc}/java/text/FieldPosition.html#FieldPosition%2Dint%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-FieldPosition-2]]link:{java8-javadoc}/java/text/FieldPosition.html#FieldPosition%2Djava.text.Format$Field%2Dint%2D[FieldPosition](<>, int)++ (link:{java9-javadoc}/java/text/FieldPosition.html#FieldPosition%2Djava.text.Format$Field%2Dint%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-getBeginIndex-0]]int link:{java8-javadoc}/java/text/FieldPosition.html#getBeginIndex%2D%2D[getBeginIndex]()++ (link:{java9-javadoc}/java/text/FieldPosition.html#getBeginIndex%2D%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-getEndIndex-0]]int link:{java8-javadoc}/java/text/FieldPosition.html#getEndIndex%2D%2D[getEndIndex]()++ (link:{java9-javadoc}/java/text/FieldPosition.html#getEndIndex%2D%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-getField-0]]int link:{java8-javadoc}/java/text/FieldPosition.html#getField%2D%2D[getField]()++ (link:{java9-javadoc}/java/text/FieldPosition.html#getField%2D%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-getFieldAttribute-0]]<> link:{java8-javadoc}/java/text/FieldPosition.html#getFieldAttribute%2D%2D[getFieldAttribute]()++ (link:{java9-javadoc}/java/text/FieldPosition.html#getFieldAttribute%2D%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-setBeginIndex-1]]void link:{java8-javadoc}/java/text/FieldPosition.html#setBeginIndex%2Dint%2D[setBeginIndex](int)++ (link:{java9-javadoc}/java/text/FieldPosition.html#setBeginIndex%2Dint%2D[java 9]) -* ++[[painless-api-reference-FieldPosition-setEndIndex-1]]void link:{java8-javadoc}/java/text/FieldPosition.html#setEndIndex%2Dint%2D[setEndIndex](int)++ (link:{java9-javadoc}/java/text/FieldPosition.html#setEndIndex%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Float.asciidoc b/docs/painless/painless-api-reference/Float.asciidoc deleted file mode 100644 index 1d20357842cfc..0000000000000 --- a/docs/painless/painless-api-reference/Float.asciidoc +++ /dev/null @@ -1,35 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Float]]++Float++:: -** [[painless-api-reference-Float-BYTES]]static int link:{java8-javadoc}/java/lang/Float.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Float.html#BYTES[java 9]) -** [[painless-api-reference-Float-MAX_EXPONENT]]static int link:{java8-javadoc}/java/lang/Float.html#MAX_EXPONENT[MAX_EXPONENT] (link:{java9-javadoc}/java/lang/Float.html#MAX_EXPONENT[java 9]) -** [[painless-api-reference-Float-MAX_VALUE]]static float link:{java8-javadoc}/java/lang/Float.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Float.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Float-MIN_EXPONENT]]static int link:{java8-javadoc}/java/lang/Float.html#MIN_EXPONENT[MIN_EXPONENT] (link:{java9-javadoc}/java/lang/Float.html#MIN_EXPONENT[java 9]) -** [[painless-api-reference-Float-MIN_NORMAL]]static float link:{java8-javadoc}/java/lang/Float.html#MIN_NORMAL[MIN_NORMAL] (link:{java9-javadoc}/java/lang/Float.html#MIN_NORMAL[java 9]) -** [[painless-api-reference-Float-MIN_VALUE]]static float link:{java8-javadoc}/java/lang/Float.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Float.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Float-NEGATIVE_INFINITY]]static float link:{java8-javadoc}/java/lang/Float.html#NEGATIVE_INFINITY[NEGATIVE_INFINITY] (link:{java9-javadoc}/java/lang/Float.html#NEGATIVE_INFINITY[java 9]) -** [[painless-api-reference-Float-NaN]]static float link:{java8-javadoc}/java/lang/Float.html#NaN[NaN] (link:{java9-javadoc}/java/lang/Float.html#NaN[java 9]) -** [[painless-api-reference-Float-POSITIVE_INFINITY]]static float link:{java8-javadoc}/java/lang/Float.html#POSITIVE_INFINITY[POSITIVE_INFINITY] (link:{java9-javadoc}/java/lang/Float.html#POSITIVE_INFINITY[java 9]) -** [[painless-api-reference-Float-SIZE]]static int link:{java8-javadoc}/java/lang/Float.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Float.html#SIZE[java 9]) -* ++[[painless-api-reference-Float-compare-2]]static int link:{java8-javadoc}/java/lang/Float.html#compare%2Dfloat%2Dfloat%2D[compare](float, float)++ (link:{java9-javadoc}/java/lang/Float.html#compare%2Dfloat%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-floatToIntBits-1]]static int link:{java8-javadoc}/java/lang/Float.html#floatToIntBits%2Dfloat%2D[floatToIntBits](float)++ (link:{java9-javadoc}/java/lang/Float.html#floatToIntBits%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-floatToRawIntBits-1]]static int link:{java8-javadoc}/java/lang/Float.html#floatToRawIntBits%2Dfloat%2D[floatToRawIntBits](float)++ (link:{java9-javadoc}/java/lang/Float.html#floatToRawIntBits%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-hashCode-1]]static int link:{java8-javadoc}/java/lang/Float.html#hashCode%2Dfloat%2D[hashCode](float)++ (link:{java9-javadoc}/java/lang/Float.html#hashCode%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-intBitsToFloat-1]]static float link:{java8-javadoc}/java/lang/Float.html#intBitsToFloat%2Dint%2D[intBitsToFloat](int)++ (link:{java9-javadoc}/java/lang/Float.html#intBitsToFloat%2Dint%2D[java 9]) -* ++[[painless-api-reference-Float-isFinite-1]]static boolean link:{java8-javadoc}/java/lang/Float.html#isFinite%2Dfloat%2D[isFinite](float)++ (link:{java9-javadoc}/java/lang/Float.html#isFinite%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-isInfinite-1]]static boolean link:{java8-javadoc}/java/lang/Float.html#isInfinite%2Dfloat%2D[isInfinite](float)++ (link:{java9-javadoc}/java/lang/Float.html#isInfinite%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-isNaN-1]]static boolean link:{java8-javadoc}/java/lang/Float.html#isNaN%2Dfloat%2D[isNaN](float)++ (link:{java9-javadoc}/java/lang/Float.html#isNaN%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-max-2]]static float link:{java8-javadoc}/java/lang/Float.html#max%2Dfloat%2Dfloat%2D[max](float, float)++ (link:{java9-javadoc}/java/lang/Float.html#max%2Dfloat%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-min-2]]static float link:{java8-javadoc}/java/lang/Float.html#min%2Dfloat%2Dfloat%2D[min](float, float)++ (link:{java9-javadoc}/java/lang/Float.html#min%2Dfloat%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-parseFloat-1]]static float link:{java8-javadoc}/java/lang/Float.html#parseFloat%2Djava.lang.String%2D[parseFloat](<>)++ (link:{java9-javadoc}/java/lang/Float.html#parseFloat%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Float-sum-2]]static float link:{java8-javadoc}/java/lang/Float.html#sum%2Dfloat%2Dfloat%2D[sum](float, float)++ (link:{java9-javadoc}/java/lang/Float.html#sum%2Dfloat%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-toHexString-1]]static <> link:{java8-javadoc}/java/lang/Float.html#toHexString%2Dfloat%2D[toHexString](float)++ (link:{java9-javadoc}/java/lang/Float.html#toHexString%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-toString-1]]static <> link:{java8-javadoc}/java/lang/Float.html#toString%2Dfloat%2D[toString](float)++ (link:{java9-javadoc}/java/lang/Float.html#toString%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Float.html#valueOf%2Dfloat%2D[valueOf](float)++ (link:{java9-javadoc}/java/lang/Float.html#valueOf%2Dfloat%2D[java 9]) -* ++[[painless-api-reference-Float-compareTo-1]]int link:{java8-javadoc}/java/lang/Float.html#compareTo%2Djava.lang.Float%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Float.html#compareTo%2Djava.lang.Float%2D[java 9]) -* ++[[painless-api-reference-Float-isInfinite-0]]boolean link:{java8-javadoc}/java/lang/Float.html#isInfinite%2D%2D[isInfinite]()++ (link:{java9-javadoc}/java/lang/Float.html#isInfinite%2D%2D[java 9]) -* ++[[painless-api-reference-Float-isNaN-0]]boolean link:{java8-javadoc}/java/lang/Float.html#isNaN%2D%2D[isNaN]()++ (link:{java9-javadoc}/java/lang/Float.html#isNaN%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Format.Field.asciidoc b/docs/painless/painless-api-reference/Format.Field.asciidoc deleted file mode 100644 index 3ddf6ec6d2f15..0000000000000 --- a/docs/painless/painless-api-reference/Format.Field.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Format-Field]]++Format.Field++:: -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Format.asciidoc b/docs/painless/painless-api-reference/Format.asciidoc deleted file mode 100644 index afa42a0598dce..0000000000000 --- a/docs/painless/painless-api-reference/Format.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Format]]++Format++:: -* ++[[painless-api-reference-Format-clone-0]]def link:{java8-javadoc}/java/text/Format.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/text/Format.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-Format-format-1]]<> link:{java8-javadoc}/java/text/Format.html#format%2Djava.lang.Object%2D[format](<>)++ (link:{java9-javadoc}/java/text/Format.html#format%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Format-format-3]]<> link:{java8-javadoc}/java/text/Format.html#format%2Djava.lang.Object%2Djava.lang.StringBuffer%2Djava.text.FieldPosition%2D[format](<>, <>, <>)++ (link:{java9-javadoc}/java/text/Format.html#format%2Djava.lang.Object%2Djava.lang.StringBuffer%2Djava.text.FieldPosition%2D[java 9]) -* ++[[painless-api-reference-Format-formatToCharacterIterator-1]]<> link:{java8-javadoc}/java/text/Format.html#formatToCharacterIterator%2Djava.lang.Object%2D[formatToCharacterIterator](<>)++ (link:{java9-javadoc}/java/text/Format.html#formatToCharacterIterator%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Format-parseObject-1]]<> link:{java8-javadoc}/java/text/Format.html#parseObject%2Djava.lang.String%2D[parseObject](<>)++ (link:{java9-javadoc}/java/text/Format.html#parseObject%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Format-parseObject-2]]<> link:{java8-javadoc}/java/text/Format.html#parseObject%2Djava.lang.String%2Djava.text.ParsePosition%2D[parseObject](<>, <>)++ (link:{java9-javadoc}/java/text/Format.html#parseObject%2Djava.lang.String%2Djava.text.ParsePosition%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/FormatFlagsConversionMismatchException.asciidoc b/docs/painless/painless-api-reference/FormatFlagsConversionMismatchException.asciidoc deleted file mode 100644 index b72d827b196e8..0000000000000 --- a/docs/painless/painless-api-reference/FormatFlagsConversionMismatchException.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-FormatFlagsConversionMismatchException]]++FormatFlagsConversionMismatchException++:: -* ++[[painless-api-reference-FormatFlagsConversionMismatchException-FormatFlagsConversionMismatchException-2]]link:{java8-javadoc}/java/util/FormatFlagsConversionMismatchException.html#FormatFlagsConversionMismatchException%2Djava.lang.String%2Dchar%2D[FormatFlagsConversionMismatchException](<>, char)++ (link:{java9-javadoc}/java/util/FormatFlagsConversionMismatchException.html#FormatFlagsConversionMismatchException%2Djava.lang.String%2Dchar%2D[java 9]) -* ++[[painless-api-reference-FormatFlagsConversionMismatchException-getConversion-0]]char link:{java8-javadoc}/java/util/FormatFlagsConversionMismatchException.html#getConversion%2D%2D[getConversion]()++ (link:{java9-javadoc}/java/util/FormatFlagsConversionMismatchException.html#getConversion%2D%2D[java 9]) -* ++[[painless-api-reference-FormatFlagsConversionMismatchException-getFlags-0]]<> link:{java8-javadoc}/java/util/FormatFlagsConversionMismatchException.html#getFlags%2D%2D[getFlags]()++ (link:{java9-javadoc}/java/util/FormatFlagsConversionMismatchException.html#getFlags%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/FormatStyle.asciidoc b/docs/painless/painless-api-reference/FormatStyle.asciidoc deleted file mode 100644 index 4db787c051ae6..0000000000000 --- a/docs/painless/painless-api-reference/FormatStyle.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-FormatStyle]]++FormatStyle++:: -** [[painless-api-reference-FormatStyle-FULL]]static <> link:{java8-javadoc}/java/time/format/FormatStyle.html#FULL[FULL] (link:{java9-javadoc}/java/time/format/FormatStyle.html#FULL[java 9]) -** [[painless-api-reference-FormatStyle-LONG]]static <> link:{java8-javadoc}/java/time/format/FormatStyle.html#LONG[LONG] (link:{java9-javadoc}/java/time/format/FormatStyle.html#LONG[java 9]) -** [[painless-api-reference-FormatStyle-MEDIUM]]static <> link:{java8-javadoc}/java/time/format/FormatStyle.html#MEDIUM[MEDIUM] (link:{java9-javadoc}/java/time/format/FormatStyle.html#MEDIUM[java 9]) -** [[painless-api-reference-FormatStyle-SHORT]]static <> link:{java8-javadoc}/java/time/format/FormatStyle.html#SHORT[SHORT] (link:{java9-javadoc}/java/time/format/FormatStyle.html#SHORT[java 9]) -* ++[[painless-api-reference-FormatStyle-valueOf-1]]static <> link:{java8-javadoc}/java/time/format/FormatStyle.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/format/FormatStyle.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-FormatStyle-values-0]]static <>[] link:{java8-javadoc}/java/time/format/FormatStyle.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/format/FormatStyle.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Formattable.asciidoc b/docs/painless/painless-api-reference/Formattable.asciidoc deleted file mode 100644 index ab70d1319e939..0000000000000 --- a/docs/painless/painless-api-reference/Formattable.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Formattable]]++Formattable++:: -* ++[[painless-api-reference-Formattable-formatTo-4]]void link:{java8-javadoc}/java/util/Formattable.html#formatTo%2Djava.util.Formatter%2Dint%2Dint%2Dint%2D[formatTo](<>, int, int, int)++ (link:{java9-javadoc}/java/util/Formattable.html#formatTo%2Djava.util.Formatter%2Dint%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/FormattableFlags.asciidoc b/docs/painless/painless-api-reference/FormattableFlags.asciidoc deleted file mode 100644 index dec639fe70b3e..0000000000000 --- a/docs/painless/painless-api-reference/FormattableFlags.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-FormattableFlags]]++FormattableFlags++:: -** [[painless-api-reference-FormattableFlags-ALTERNATE]]static int link:{java8-javadoc}/java/util/FormattableFlags.html#ALTERNATE[ALTERNATE] (link:{java9-javadoc}/java/util/FormattableFlags.html#ALTERNATE[java 9]) -** [[painless-api-reference-FormattableFlags-LEFT_JUSTIFY]]static int link:{java8-javadoc}/java/util/FormattableFlags.html#LEFT_JUSTIFY[LEFT_JUSTIFY] (link:{java9-javadoc}/java/util/FormattableFlags.html#LEFT_JUSTIFY[java 9]) -** [[painless-api-reference-FormattableFlags-UPPERCASE]]static int link:{java8-javadoc}/java/util/FormattableFlags.html#UPPERCASE[UPPERCASE] (link:{java9-javadoc}/java/util/FormattableFlags.html#UPPERCASE[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Formatter.BigDecimalLayoutForm.asciidoc b/docs/painless/painless-api-reference/Formatter.BigDecimalLayoutForm.asciidoc deleted file mode 100644 index 4fb9ce48062e6..0000000000000 --- a/docs/painless/painless-api-reference/Formatter.BigDecimalLayoutForm.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Formatter-BigDecimalLayoutForm]]++Formatter.BigDecimalLayoutForm++:: -** [[painless-api-reference-Formatter-BigDecimalLayoutForm-DECIMAL_FLOAT]]static <> link:{java8-javadoc}/java/util/Formatter.BigDecimalLayoutForm.html#DECIMAL_FLOAT[DECIMAL_FLOAT] (link:{java9-javadoc}/java/util/Formatter.BigDecimalLayoutForm.html#DECIMAL_FLOAT[java 9]) -** [[painless-api-reference-Formatter-BigDecimalLayoutForm-SCIENTIFIC]]static <> link:{java8-javadoc}/java/util/Formatter.BigDecimalLayoutForm.html#SCIENTIFIC[SCIENTIFIC] (link:{java9-javadoc}/java/util/Formatter.BigDecimalLayoutForm.html#SCIENTIFIC[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Formatter.asciidoc b/docs/painless/painless-api-reference/Formatter.asciidoc deleted file mode 100644 index 4cad434f1308f..0000000000000 --- a/docs/painless/painless-api-reference/Formatter.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Formatter]]++Formatter++:: -* ++[[painless-api-reference-Formatter-Formatter-0]]link:{java8-javadoc}/java/util/Formatter.html#Formatter%2D%2D[Formatter]()++ (link:{java9-javadoc}/java/util/Formatter.html#Formatter%2D%2D[java 9]) -* ++[[painless-api-reference-Formatter-Formatter-1]]link:{java8-javadoc}/java/util/Formatter.html#Formatter%2Djava.lang.Appendable%2D[Formatter](<>)++ (link:{java9-javadoc}/java/util/Formatter.html#Formatter%2Djava.lang.Appendable%2D[java 9]) -* ++[[painless-api-reference-Formatter-Formatter-2]]link:{java8-javadoc}/java/util/Formatter.html#Formatter%2Djava.lang.Appendable%2Djava.util.Locale%2D[Formatter](<>, <>)++ (link:{java9-javadoc}/java/util/Formatter.html#Formatter%2Djava.lang.Appendable%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Formatter-format-2]]<> link:{java8-javadoc}/java/util/Formatter.html#format%2Djava.lang.String%2Djava.lang.Object:A%2D[format](<>, def[])++ (link:{java9-javadoc}/java/util/Formatter.html#format%2Djava.lang.String%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Formatter-format-3]]<> link:{java8-javadoc}/java/util/Formatter.html#format%2Djava.util.Locale%2Djava.lang.String%2Djava.lang.Object:A%2D[format](<>, <>, def[])++ (link:{java9-javadoc}/java/util/Formatter.html#format%2Djava.util.Locale%2Djava.lang.String%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Formatter-locale-0]]<> link:{java8-javadoc}/java/util/Formatter.html#locale%2D%2D[locale]()++ (link:{java9-javadoc}/java/util/Formatter.html#locale%2D%2D[java 9]) -* ++[[painless-api-reference-Formatter-out-0]]<> link:{java8-javadoc}/java/util/Formatter.html#out%2D%2D[out]()++ (link:{java9-javadoc}/java/util/Formatter.html#out%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/FormatterClosedException.asciidoc b/docs/painless/painless-api-reference/FormatterClosedException.asciidoc deleted file mode 100644 index 00c4f5971d335..0000000000000 --- a/docs/painless/painless-api-reference/FormatterClosedException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-FormatterClosedException]]++FormatterClosedException++:: -* ++[[painless-api-reference-FormatterClosedException-FormatterClosedException-0]]link:{java8-javadoc}/java/util/FormatterClosedException.html#FormatterClosedException%2D%2D[FormatterClosedException]()++ (link:{java9-javadoc}/java/util/FormatterClosedException.html#FormatterClosedException%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Function.asciidoc b/docs/painless/painless-api-reference/Function.asciidoc deleted file mode 100644 index 61fe3b6b5da70..0000000000000 --- a/docs/painless/painless-api-reference/Function.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Function]]++Function++:: -* ++[[painless-api-reference-Function-identity-0]]static <> link:{java8-javadoc}/java/util/function/Function.html#identity%2D%2D[identity]()++ (link:{java9-javadoc}/java/util/function/Function.html#identity%2D%2D[java 9]) -* ++[[painless-api-reference-Function-andThen-1]]<> link:{java8-javadoc}/java/util/function/Function.html#andThen%2Djava.util.function.Function%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/Function.html#andThen%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Function-apply-1]]def link:{java8-javadoc}/java/util/function/Function.html#apply%2Djava.lang.Object%2D[apply](def)++ (link:{java9-javadoc}/java/util/function/Function.html#apply%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Function-compose-1]]<> link:{java8-javadoc}/java/util/function/Function.html#compose%2Djava.util.function.Function%2D[compose](<>)++ (link:{java9-javadoc}/java/util/function/Function.html#compose%2Djava.util.function.Function%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/GregorianCalendar.asciidoc b/docs/painless/painless-api-reference/GregorianCalendar.asciidoc deleted file mode 100644 index 8f7b9ccecc7e6..0000000000000 --- a/docs/painless/painless-api-reference/GregorianCalendar.asciidoc +++ /dev/null @@ -1,20 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-GregorianCalendar]]++GregorianCalendar++:: -** [[painless-api-reference-GregorianCalendar-AD]]static int link:{java8-javadoc}/java/util/GregorianCalendar.html#AD[AD] (link:{java9-javadoc}/java/util/GregorianCalendar.html#AD[java 9]) -** [[painless-api-reference-GregorianCalendar-BC]]static int link:{java8-javadoc}/java/util/GregorianCalendar.html#BC[BC] (link:{java9-javadoc}/java/util/GregorianCalendar.html#BC[java 9]) -* ++[[painless-api-reference-GregorianCalendar-from-1]]static <> link:{java8-javadoc}/java/util/GregorianCalendar.html#from%2Djava.time.ZonedDateTime%2D[from](<>)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#from%2Djava.time.ZonedDateTime%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-GregorianCalendar-0]]link:{java8-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2D%2D[GregorianCalendar]()++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2D%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-GregorianCalendar-1]]link:{java8-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Djava.util.TimeZone%2D[GregorianCalendar](<>)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Djava.util.TimeZone%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-GregorianCalendar-2]]link:{java8-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Djava.util.TimeZone%2Djava.util.Locale%2D[GregorianCalendar](<>, <>)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Djava.util.TimeZone%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-GregorianCalendar-3]]link:{java8-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Dint%2Dint%2Dint%2D[GregorianCalendar](int, int, int)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-GregorianCalendar-5]]link:{java8-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Dint%2Dint%2Dint%2Dint%2Dint%2D[GregorianCalendar](int, int, int, int, int)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-GregorianCalendar-6]]link:{java8-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[GregorianCalendar](int, int, int, int, int, int)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#GregorianCalendar%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-getGregorianChange-0]]<> link:{java8-javadoc}/java/util/GregorianCalendar.html#getGregorianChange%2D%2D[getGregorianChange]()++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#getGregorianChange%2D%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-isLeapYear-1]]boolean link:{java8-javadoc}/java/util/GregorianCalendar.html#isLeapYear%2Dint%2D[isLeapYear](int)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#isLeapYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-setGregorianChange-1]]void link:{java8-javadoc}/java/util/GregorianCalendar.html#setGregorianChange%2Djava.util.Date%2D[setGregorianChange](<>)++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#setGregorianChange%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-GregorianCalendar-toZonedDateTime-0]]<> link:{java8-javadoc}/java/util/GregorianCalendar.html#toZonedDateTime%2D%2D[toZonedDateTime]()++ (link:{java9-javadoc}/java/util/GregorianCalendar.html#toZonedDateTime%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/HashMap.asciidoc b/docs/painless/painless-api-reference/HashMap.asciidoc deleted file mode 100644 index f67e75d702f0c..0000000000000 --- a/docs/painless/painless-api-reference/HashMap.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-HashMap]]++HashMap++:: -* ++[[painless-api-reference-HashMap-HashMap-0]]link:{java8-javadoc}/java/util/HashMap.html#HashMap%2D%2D[HashMap]()++ (link:{java9-javadoc}/java/util/HashMap.html#HashMap%2D%2D[java 9]) -* ++[[painless-api-reference-HashMap-HashMap-1]]link:{java8-javadoc}/java/util/HashMap.html#HashMap%2Djava.util.Map%2D[HashMap](<>)++ (link:{java9-javadoc}/java/util/HashMap.html#HashMap%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-HashMap-clone-0]]def link:{java8-javadoc}/java/util/HashMap.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/HashMap.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/HashSet.asciidoc b/docs/painless/painless-api-reference/HashSet.asciidoc deleted file mode 100644 index af4171def2262..0000000000000 --- a/docs/painless/painless-api-reference/HashSet.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-HashSet]]++HashSet++:: -* ++[[painless-api-reference-HashSet-HashSet-0]]link:{java8-javadoc}/java/util/HashSet.html#HashSet%2D%2D[HashSet]()++ (link:{java9-javadoc}/java/util/HashSet.html#HashSet%2D%2D[java 9]) -* ++[[painless-api-reference-HashSet-HashSet-1]]link:{java8-javadoc}/java/util/HashSet.html#HashSet%2Djava.util.Collection%2D[HashSet](<>)++ (link:{java9-javadoc}/java/util/HashSet.html#HashSet%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-HashSet-clone-0]]def link:{java8-javadoc}/java/util/HashSet.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/HashSet.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Hashtable.asciidoc b/docs/painless/painless-api-reference/Hashtable.asciidoc deleted file mode 100644 index 34702e8466065..0000000000000 --- a/docs/painless/painless-api-reference/Hashtable.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Hashtable]]++Hashtable++:: -* ++[[painless-api-reference-Hashtable-Hashtable-0]]link:{java8-javadoc}/java/util/Hashtable.html#Hashtable%2D%2D[Hashtable]()++ (link:{java9-javadoc}/java/util/Hashtable.html#Hashtable%2D%2D[java 9]) -* ++[[painless-api-reference-Hashtable-Hashtable-1]]link:{java8-javadoc}/java/util/Hashtable.html#Hashtable%2Djava.util.Map%2D[Hashtable](<>)++ (link:{java9-javadoc}/java/util/Hashtable.html#Hashtable%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-Hashtable-clone-0]]def link:{java8-javadoc}/java/util/Hashtable.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/Hashtable.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/HijrahChronology.asciidoc b/docs/painless/painless-api-reference/HijrahChronology.asciidoc deleted file mode 100644 index b7577fbdfd720..0000000000000 --- a/docs/painless/painless-api-reference/HijrahChronology.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-HijrahChronology]]++HijrahChronology++:: -** [[painless-api-reference-HijrahChronology-INSTANCE]]static <> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#INSTANCE[INSTANCE] (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#INSTANCE[java 9]) -* ++[[painless-api-reference-HijrahChronology-date-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[date](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-date-3]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#date%2Dint%2Dint%2Dint%2D[date](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#date%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-date-4]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[date](<>, int, int, int)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-dateEpochDay-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#dateEpochDay%2Dlong%2D[dateEpochDay](long)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#dateEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-dateYearDay-2]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#dateYearDay%2Dint%2Dint%2D[dateYearDay](int, int)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#dateYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-dateYearDay-3]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[dateYearDay](<>, int, int)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-eraOf-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#eraOf%2Dint%2D[eraOf](int)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#eraOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahChronology-resolveDate-2]]<> link:{java8-javadoc}/java/time/chrono/HijrahChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[resolveDate](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/HijrahChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/HijrahDate.asciidoc b/docs/painless/painless-api-reference/HijrahDate.asciidoc deleted file mode 100644 index 09ebf31682a13..0000000000000 --- a/docs/painless/painless-api-reference/HijrahDate.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-HijrahDate]]++HijrahDate++:: -* ++[[painless-api-reference-HijrahDate-from-1]]static <> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-of-3]]static <> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-getEra-0]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#getEra%2D%2D[getEra]()++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#getEra%2D%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-minus-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-minus-2]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-plus-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-plus-2]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-with-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-with-2]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-HijrahDate-withVariant-1]]<> link:{java8-javadoc}/java/time/chrono/HijrahDate.html#withVariant%2Djava.time.chrono.HijrahChronology%2D[withVariant](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahDate.html#withVariant%2Djava.time.chrono.HijrahChronology%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/HijrahEra.asciidoc b/docs/painless/painless-api-reference/HijrahEra.asciidoc deleted file mode 100644 index 9bfe561d2fa58..0000000000000 --- a/docs/painless/painless-api-reference/HijrahEra.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-HijrahEra]]++HijrahEra++:: -** [[painless-api-reference-HijrahEra-AH]]static <> link:{java8-javadoc}/java/time/chrono/HijrahEra.html#AH[AH] (link:{java9-javadoc}/java/time/chrono/HijrahEra.html#AH[java 9]) -* ++[[painless-api-reference-HijrahEra-of-1]]static <> link:{java8-javadoc}/java/time/chrono/HijrahEra.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/chrono/HijrahEra.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-HijrahEra-valueOf-1]]static <> link:{java8-javadoc}/java/time/chrono/HijrahEra.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/chrono/HijrahEra.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-HijrahEra-values-0]]static <>[] link:{java8-javadoc}/java/time/chrono/HijrahEra.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/chrono/HijrahEra.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-HijrahEra-getValue-0]]int link:{java8-javadoc}/java/time/chrono/HijrahEra.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/chrono/HijrahEra.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IdentityHashMap.asciidoc b/docs/painless/painless-api-reference/IdentityHashMap.asciidoc deleted file mode 100644 index 050d0eaeb1c8e..0000000000000 --- a/docs/painless/painless-api-reference/IdentityHashMap.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IdentityHashMap]]++IdentityHashMap++:: -* ++[[painless-api-reference-IdentityHashMap-IdentityHashMap-0]]link:{java8-javadoc}/java/util/IdentityHashMap.html#IdentityHashMap%2D%2D[IdentityHashMap]()++ (link:{java9-javadoc}/java/util/IdentityHashMap.html#IdentityHashMap%2D%2D[java 9]) -* ++[[painless-api-reference-IdentityHashMap-IdentityHashMap-1]]link:{java8-javadoc}/java/util/IdentityHashMap.html#IdentityHashMap%2Djava.util.Map%2D[IdentityHashMap](<>)++ (link:{java9-javadoc}/java/util/IdentityHashMap.html#IdentityHashMap%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-IdentityHashMap-clone-0]]def link:{java8-javadoc}/java/util/IdentityHashMap.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/IdentityHashMap.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalAccessException.asciidoc b/docs/painless/painless-api-reference/IllegalAccessException.asciidoc deleted file mode 100644 index 30c8ba4536d02..0000000000000 --- a/docs/painless/painless-api-reference/IllegalAccessException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalAccessException]]++IllegalAccessException++:: -* ++[[painless-api-reference-IllegalAccessException-IllegalAccessException-0]]link:{java8-javadoc}/java/lang/IllegalAccessException.html#IllegalAccessException%2D%2D[IllegalAccessException]()++ (link:{java9-javadoc}/java/lang/IllegalAccessException.html#IllegalAccessException%2D%2D[java 9]) -* ++[[painless-api-reference-IllegalAccessException-IllegalAccessException-1]]link:{java8-javadoc}/java/lang/IllegalAccessException.html#IllegalAccessException%2Djava.lang.String%2D[IllegalAccessException](<>)++ (link:{java9-javadoc}/java/lang/IllegalAccessException.html#IllegalAccessException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalArgumentException.asciidoc b/docs/painless/painless-api-reference/IllegalArgumentException.asciidoc deleted file mode 100644 index 538fa46a656e2..0000000000000 --- a/docs/painless/painless-api-reference/IllegalArgumentException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalArgumentException]]++IllegalArgumentException++:: -* ++[[painless-api-reference-IllegalArgumentException-IllegalArgumentException-0]]link:{java8-javadoc}/java/lang/IllegalArgumentException.html#IllegalArgumentException%2D%2D[IllegalArgumentException]()++ (link:{java9-javadoc}/java/lang/IllegalArgumentException.html#IllegalArgumentException%2D%2D[java 9]) -* ++[[painless-api-reference-IllegalArgumentException-IllegalArgumentException-1]]link:{java8-javadoc}/java/lang/IllegalArgumentException.html#IllegalArgumentException%2Djava.lang.String%2D[IllegalArgumentException](<>)++ (link:{java9-javadoc}/java/lang/IllegalArgumentException.html#IllegalArgumentException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalFormatCodePointException.asciidoc b/docs/painless/painless-api-reference/IllegalFormatCodePointException.asciidoc deleted file mode 100644 index 2a533384198d1..0000000000000 --- a/docs/painless/painless-api-reference/IllegalFormatCodePointException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalFormatCodePointException]]++IllegalFormatCodePointException++:: -* ++[[painless-api-reference-IllegalFormatCodePointException-IllegalFormatCodePointException-1]]link:{java8-javadoc}/java/util/IllegalFormatCodePointException.html#IllegalFormatCodePointException%2Dint%2D[IllegalFormatCodePointException](int)++ (link:{java9-javadoc}/java/util/IllegalFormatCodePointException.html#IllegalFormatCodePointException%2Dint%2D[java 9]) -* ++[[painless-api-reference-IllegalFormatCodePointException-getCodePoint-0]]int link:{java8-javadoc}/java/util/IllegalFormatCodePointException.html#getCodePoint%2D%2D[getCodePoint]()++ (link:{java9-javadoc}/java/util/IllegalFormatCodePointException.html#getCodePoint%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalFormatConversionException.asciidoc b/docs/painless/painless-api-reference/IllegalFormatConversionException.asciidoc deleted file mode 100644 index 2a36b5367a339..0000000000000 --- a/docs/painless/painless-api-reference/IllegalFormatConversionException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalFormatConversionException]]++IllegalFormatConversionException++:: -* ++[[painless-api-reference-IllegalFormatConversionException-getConversion-0]]char link:{java8-javadoc}/java/util/IllegalFormatConversionException.html#getConversion%2D%2D[getConversion]()++ (link:{java9-javadoc}/java/util/IllegalFormatConversionException.html#getConversion%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalFormatException.asciidoc b/docs/painless/painless-api-reference/IllegalFormatException.asciidoc deleted file mode 100644 index 98a58ceddd265..0000000000000 --- a/docs/painless/painless-api-reference/IllegalFormatException.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalFormatException]]++IllegalFormatException++:: -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalFormatFlagsException.asciidoc b/docs/painless/painless-api-reference/IllegalFormatFlagsException.asciidoc deleted file mode 100644 index 9ed7810f0e60f..0000000000000 --- a/docs/painless/painless-api-reference/IllegalFormatFlagsException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalFormatFlagsException]]++IllegalFormatFlagsException++:: -* ++[[painless-api-reference-IllegalFormatFlagsException-IllegalFormatFlagsException-1]]link:{java8-javadoc}/java/util/IllegalFormatFlagsException.html#IllegalFormatFlagsException%2Djava.lang.String%2D[IllegalFormatFlagsException](<>)++ (link:{java9-javadoc}/java/util/IllegalFormatFlagsException.html#IllegalFormatFlagsException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-IllegalFormatFlagsException-getFlags-0]]<> link:{java8-javadoc}/java/util/IllegalFormatFlagsException.html#getFlags%2D%2D[getFlags]()++ (link:{java9-javadoc}/java/util/IllegalFormatFlagsException.html#getFlags%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalFormatPrecisionException.asciidoc b/docs/painless/painless-api-reference/IllegalFormatPrecisionException.asciidoc deleted file mode 100644 index 318b80dcfaa6c..0000000000000 --- a/docs/painless/painless-api-reference/IllegalFormatPrecisionException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalFormatPrecisionException]]++IllegalFormatPrecisionException++:: -* ++[[painless-api-reference-IllegalFormatPrecisionException-IllegalFormatPrecisionException-1]]link:{java8-javadoc}/java/util/IllegalFormatPrecisionException.html#IllegalFormatPrecisionException%2Dint%2D[IllegalFormatPrecisionException](int)++ (link:{java9-javadoc}/java/util/IllegalFormatPrecisionException.html#IllegalFormatPrecisionException%2Dint%2D[java 9]) -* ++[[painless-api-reference-IllegalFormatPrecisionException-getPrecision-0]]int link:{java8-javadoc}/java/util/IllegalFormatPrecisionException.html#getPrecision%2D%2D[getPrecision]()++ (link:{java9-javadoc}/java/util/IllegalFormatPrecisionException.html#getPrecision%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalFormatWidthException.asciidoc b/docs/painless/painless-api-reference/IllegalFormatWidthException.asciidoc deleted file mode 100644 index 96f008b4a5334..0000000000000 --- a/docs/painless/painless-api-reference/IllegalFormatWidthException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalFormatWidthException]]++IllegalFormatWidthException++:: -* ++[[painless-api-reference-IllegalFormatWidthException-IllegalFormatWidthException-1]]link:{java8-javadoc}/java/util/IllegalFormatWidthException.html#IllegalFormatWidthException%2Dint%2D[IllegalFormatWidthException](int)++ (link:{java9-javadoc}/java/util/IllegalFormatWidthException.html#IllegalFormatWidthException%2Dint%2D[java 9]) -* ++[[painless-api-reference-IllegalFormatWidthException-getWidth-0]]int link:{java8-javadoc}/java/util/IllegalFormatWidthException.html#getWidth%2D%2D[getWidth]()++ (link:{java9-javadoc}/java/util/IllegalFormatWidthException.html#getWidth%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalMonitorStateException.asciidoc b/docs/painless/painless-api-reference/IllegalMonitorStateException.asciidoc deleted file mode 100644 index 057c58f948b73..0000000000000 --- a/docs/painless/painless-api-reference/IllegalMonitorStateException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalMonitorStateException]]++IllegalMonitorStateException++:: -* ++[[painless-api-reference-IllegalMonitorStateException-IllegalMonitorStateException-0]]link:{java8-javadoc}/java/lang/IllegalMonitorStateException.html#IllegalMonitorStateException%2D%2D[IllegalMonitorStateException]()++ (link:{java9-javadoc}/java/lang/IllegalMonitorStateException.html#IllegalMonitorStateException%2D%2D[java 9]) -* ++[[painless-api-reference-IllegalMonitorStateException-IllegalMonitorStateException-1]]link:{java8-javadoc}/java/lang/IllegalMonitorStateException.html#IllegalMonitorStateException%2Djava.lang.String%2D[IllegalMonitorStateException](<>)++ (link:{java9-javadoc}/java/lang/IllegalMonitorStateException.html#IllegalMonitorStateException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalStateException.asciidoc b/docs/painless/painless-api-reference/IllegalStateException.asciidoc deleted file mode 100644 index fa8ddf72fa5bd..0000000000000 --- a/docs/painless/painless-api-reference/IllegalStateException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalStateException]]++IllegalStateException++:: -* ++[[painless-api-reference-IllegalStateException-IllegalStateException-0]]link:{java8-javadoc}/java/lang/IllegalStateException.html#IllegalStateException%2D%2D[IllegalStateException]()++ (link:{java9-javadoc}/java/lang/IllegalStateException.html#IllegalStateException%2D%2D[java 9]) -* ++[[painless-api-reference-IllegalStateException-IllegalStateException-1]]link:{java8-javadoc}/java/lang/IllegalStateException.html#IllegalStateException%2Djava.lang.String%2D[IllegalStateException](<>)++ (link:{java9-javadoc}/java/lang/IllegalStateException.html#IllegalStateException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllegalThreadStateException.asciidoc b/docs/painless/painless-api-reference/IllegalThreadStateException.asciidoc deleted file mode 100644 index 5fc8293fe0933..0000000000000 --- a/docs/painless/painless-api-reference/IllegalThreadStateException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllegalThreadStateException]]++IllegalThreadStateException++:: -* ++[[painless-api-reference-IllegalThreadStateException-IllegalThreadStateException-0]]link:{java8-javadoc}/java/lang/IllegalThreadStateException.html#IllegalThreadStateException%2D%2D[IllegalThreadStateException]()++ (link:{java9-javadoc}/java/lang/IllegalThreadStateException.html#IllegalThreadStateException%2D%2D[java 9]) -* ++[[painless-api-reference-IllegalThreadStateException-IllegalThreadStateException-1]]link:{java8-javadoc}/java/lang/IllegalThreadStateException.html#IllegalThreadStateException%2Djava.lang.String%2D[IllegalThreadStateException](<>)++ (link:{java9-javadoc}/java/lang/IllegalThreadStateException.html#IllegalThreadStateException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IllformedLocaleException.asciidoc b/docs/painless/painless-api-reference/IllformedLocaleException.asciidoc deleted file mode 100644 index 59a814d9520dd..0000000000000 --- a/docs/painless/painless-api-reference/IllformedLocaleException.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IllformedLocaleException]]++IllformedLocaleException++:: -* ++[[painless-api-reference-IllformedLocaleException-IllformedLocaleException-0]]link:{java8-javadoc}/java/util/IllformedLocaleException.html#IllformedLocaleException%2D%2D[IllformedLocaleException]()++ (link:{java9-javadoc}/java/util/IllformedLocaleException.html#IllformedLocaleException%2D%2D[java 9]) -* ++[[painless-api-reference-IllformedLocaleException-IllformedLocaleException-1]]link:{java8-javadoc}/java/util/IllformedLocaleException.html#IllformedLocaleException%2Djava.lang.String%2D[IllformedLocaleException](<>)++ (link:{java9-javadoc}/java/util/IllformedLocaleException.html#IllformedLocaleException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-IllformedLocaleException-IllformedLocaleException-2]]link:{java8-javadoc}/java/util/IllformedLocaleException.html#IllformedLocaleException%2Djava.lang.String%2Dint%2D[IllformedLocaleException](<>, int)++ (link:{java9-javadoc}/java/util/IllformedLocaleException.html#IllformedLocaleException%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-IllformedLocaleException-getErrorIndex-0]]int link:{java8-javadoc}/java/util/IllformedLocaleException.html#getErrorIndex%2D%2D[getErrorIndex]()++ (link:{java9-javadoc}/java/util/IllformedLocaleException.html#getErrorIndex%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IndexOutOfBoundsException.asciidoc b/docs/painless/painless-api-reference/IndexOutOfBoundsException.asciidoc deleted file mode 100644 index a141ec6b04152..0000000000000 --- a/docs/painless/painless-api-reference/IndexOutOfBoundsException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IndexOutOfBoundsException]]++IndexOutOfBoundsException++:: -* ++[[painless-api-reference-IndexOutOfBoundsException-IndexOutOfBoundsException-0]]link:{java8-javadoc}/java/lang/IndexOutOfBoundsException.html#IndexOutOfBoundsException%2D%2D[IndexOutOfBoundsException]()++ (link:{java9-javadoc}/java/lang/IndexOutOfBoundsException.html#IndexOutOfBoundsException%2D%2D[java 9]) -* ++[[painless-api-reference-IndexOutOfBoundsException-IndexOutOfBoundsException-1]]link:{java8-javadoc}/java/lang/IndexOutOfBoundsException.html#IndexOutOfBoundsException%2Djava.lang.String%2D[IndexOutOfBoundsException](<>)++ (link:{java9-javadoc}/java/lang/IndexOutOfBoundsException.html#IndexOutOfBoundsException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/InputMismatchException.asciidoc b/docs/painless/painless-api-reference/InputMismatchException.asciidoc deleted file mode 100644 index 9497cf9486e03..0000000000000 --- a/docs/painless/painless-api-reference/InputMismatchException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-InputMismatchException]]++InputMismatchException++:: -* ++[[painless-api-reference-InputMismatchException-InputMismatchException-0]]link:{java8-javadoc}/java/util/InputMismatchException.html#InputMismatchException%2D%2D[InputMismatchException]()++ (link:{java9-javadoc}/java/util/InputMismatchException.html#InputMismatchException%2D%2D[java 9]) -* ++[[painless-api-reference-InputMismatchException-InputMismatchException-1]]link:{java8-javadoc}/java/util/InputMismatchException.html#InputMismatchException%2Djava.lang.String%2D[InputMismatchException](<>)++ (link:{java9-javadoc}/java/util/InputMismatchException.html#InputMismatchException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Instant.asciidoc b/docs/painless/painless-api-reference/Instant.asciidoc deleted file mode 100644 index 4a7ffe45d991c..0000000000000 --- a/docs/painless/painless-api-reference/Instant.asciidoc +++ /dev/null @@ -1,36 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Instant]]++Instant++:: -** [[painless-api-reference-Instant-EPOCH]]static <> link:{java8-javadoc}/java/time/Instant.html#EPOCH[EPOCH] (link:{java9-javadoc}/java/time/Instant.html#EPOCH[java 9]) -** [[painless-api-reference-Instant-MAX]]static <> link:{java8-javadoc}/java/time/Instant.html#MAX[MAX] (link:{java9-javadoc}/java/time/Instant.html#MAX[java 9]) -** [[painless-api-reference-Instant-MIN]]static <> link:{java8-javadoc}/java/time/Instant.html#MIN[MIN] (link:{java9-javadoc}/java/time/Instant.html#MIN[java 9]) -* ++[[painless-api-reference-Instant-from-1]]static <> link:{java8-javadoc}/java/time/Instant.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/Instant.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Instant-ofEpochMilli-1]]static <> link:{java8-javadoc}/java/time/Instant.html#ofEpochMilli%2Dlong%2D[ofEpochMilli](long)++ (link:{java9-javadoc}/java/time/Instant.html#ofEpochMilli%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-ofEpochSecond-1]]static <> link:{java8-javadoc}/java/time/Instant.html#ofEpochSecond%2Dlong%2D[ofEpochSecond](long)++ (link:{java9-javadoc}/java/time/Instant.html#ofEpochSecond%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-ofEpochSecond-2]]static <> link:{java8-javadoc}/java/time/Instant.html#ofEpochSecond%2Dlong%2Dlong%2D[ofEpochSecond](long, long)++ (link:{java9-javadoc}/java/time/Instant.html#ofEpochSecond%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-parse-1]]static <> link:{java8-javadoc}/java/time/Instant.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/Instant.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Instant-atOffset-1]]<> link:{java8-javadoc}/java/time/Instant.html#atOffset%2Djava.time.ZoneOffset%2D[atOffset](<>)++ (link:{java9-javadoc}/java/time/Instant.html#atOffset%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-Instant-atZone-1]]<> link:{java8-javadoc}/java/time/Instant.html#atZone%2Djava.time.ZoneId%2D[atZone](<>)++ (link:{java9-javadoc}/java/time/Instant.html#atZone%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-Instant-compareTo-1]]int link:{java8-javadoc}/java/time/Instant.html#compareTo%2Djava.time.Instant%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/Instant.html#compareTo%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-Instant-getEpochSecond-0]]long link:{java8-javadoc}/java/time/Instant.html#getEpochSecond%2D%2D[getEpochSecond]()++ (link:{java9-javadoc}/java/time/Instant.html#getEpochSecond%2D%2D[java 9]) -* ++[[painless-api-reference-Instant-getNano-0]]int link:{java8-javadoc}/java/time/Instant.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/Instant.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-Instant-isAfter-1]]boolean link:{java8-javadoc}/java/time/Instant.html#isAfter%2Djava.time.Instant%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/Instant.html#isAfter%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-Instant-isBefore-1]]boolean link:{java8-javadoc}/java/time/Instant.html#isBefore%2Djava.time.Instant%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/Instant.html#isBefore%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-Instant-minus-1]]<> link:{java8-javadoc}/java/time/Instant.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/Instant.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Instant-minus-2]]<> link:{java8-javadoc}/java/time/Instant.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/Instant.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Instant-minusMillis-1]]<> link:{java8-javadoc}/java/time/Instant.html#minusMillis%2Dlong%2D[minusMillis](long)++ (link:{java9-javadoc}/java/time/Instant.html#minusMillis%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-minusNanos-1]]<> link:{java8-javadoc}/java/time/Instant.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/Instant.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-minusSeconds-1]]<> link:{java8-javadoc}/java/time/Instant.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/Instant.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-plus-1]]<> link:{java8-javadoc}/java/time/Instant.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/Instant.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Instant-plus-2]]<> link:{java8-javadoc}/java/time/Instant.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/Instant.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Instant-plusMillis-1]]<> link:{java8-javadoc}/java/time/Instant.html#plusMillis%2Dlong%2D[plusMillis](long)++ (link:{java9-javadoc}/java/time/Instant.html#plusMillis%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-plusNanos-1]]<> link:{java8-javadoc}/java/time/Instant.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/Instant.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-plusSeconds-1]]<> link:{java8-javadoc}/java/time/Instant.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/Instant.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Instant-toEpochMilli-0]]long link:{java8-javadoc}/java/time/Instant.html#toEpochMilli%2D%2D[toEpochMilli]()++ (link:{java9-javadoc}/java/time/Instant.html#toEpochMilli%2D%2D[java 9]) -* ++[[painless-api-reference-Instant-truncatedTo-1]]<> link:{java8-javadoc}/java/time/Instant.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[truncatedTo](<>)++ (link:{java9-javadoc}/java/time/Instant.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Instant-with-1]]<> link:{java8-javadoc}/java/time/Instant.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/Instant.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-Instant-with-2]]<> link:{java8-javadoc}/java/time/Instant.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/Instant.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/InstantiationException.asciidoc b/docs/painless/painless-api-reference/InstantiationException.asciidoc deleted file mode 100644 index 06d88eb654f00..0000000000000 --- a/docs/painless/painless-api-reference/InstantiationException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-InstantiationException]]++InstantiationException++:: -* ++[[painless-api-reference-InstantiationException-InstantiationException-0]]link:{java8-javadoc}/java/lang/InstantiationException.html#InstantiationException%2D%2D[InstantiationException]()++ (link:{java9-javadoc}/java/lang/InstantiationException.html#InstantiationException%2D%2D[java 9]) -* ++[[painless-api-reference-InstantiationException-InstantiationException-1]]link:{java8-javadoc}/java/lang/InstantiationException.html#InstantiationException%2Djava.lang.String%2D[InstantiationException](<>)++ (link:{java9-javadoc}/java/lang/InstantiationException.html#InstantiationException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IntBinaryOperator.asciidoc b/docs/painless/painless-api-reference/IntBinaryOperator.asciidoc deleted file mode 100644 index ca3c64ab3d898..0000000000000 --- a/docs/painless/painless-api-reference/IntBinaryOperator.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntBinaryOperator]]++IntBinaryOperator++:: -* ++[[painless-api-reference-IntBinaryOperator-applyAsInt-2]]int link:{java8-javadoc}/java/util/function/IntBinaryOperator.html#applyAsInt%2Dint%2Dint%2D[applyAsInt](int, int)++ (link:{java9-javadoc}/java/util/function/IntBinaryOperator.html#applyAsInt%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntConsumer.asciidoc b/docs/painless/painless-api-reference/IntConsumer.asciidoc deleted file mode 100644 index e8325838dac51..0000000000000 --- a/docs/painless/painless-api-reference/IntConsumer.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntConsumer]]++IntConsumer++:: -* ++[[painless-api-reference-IntConsumer-accept-1]]void link:{java8-javadoc}/java/util/function/IntConsumer.html#accept%2Dint%2D[accept](int)++ (link:{java9-javadoc}/java/util/function/IntConsumer.html#accept%2Dint%2D[java 9]) -* ++[[painless-api-reference-IntConsumer-andThen-1]]<> link:{java8-javadoc}/java/util/function/IntConsumer.html#andThen%2Djava.util.function.IntConsumer%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/IntConsumer.html#andThen%2Djava.util.function.IntConsumer%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntFunction.asciidoc b/docs/painless/painless-api-reference/IntFunction.asciidoc deleted file mode 100644 index 336c1f7c3cb70..0000000000000 --- a/docs/painless/painless-api-reference/IntFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntFunction]]++IntFunction++:: -* ++[[painless-api-reference-IntFunction-apply-1]]def link:{java8-javadoc}/java/util/function/IntFunction.html#apply%2Dint%2D[apply](int)++ (link:{java9-javadoc}/java/util/function/IntFunction.html#apply%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntPredicate.asciidoc b/docs/painless/painless-api-reference/IntPredicate.asciidoc deleted file mode 100644 index a9c45c95af855..0000000000000 --- a/docs/painless/painless-api-reference/IntPredicate.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntPredicate]]++IntPredicate++:: -* ++[[painless-api-reference-IntPredicate-and-1]]<> link:{java8-javadoc}/java/util/function/IntPredicate.html#and%2Djava.util.function.IntPredicate%2D[and](<>)++ (link:{java9-javadoc}/java/util/function/IntPredicate.html#and%2Djava.util.function.IntPredicate%2D[java 9]) -* ++[[painless-api-reference-IntPredicate-negate-0]]<> link:{java8-javadoc}/java/util/function/IntPredicate.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/util/function/IntPredicate.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-IntPredicate-or-1]]<> link:{java8-javadoc}/java/util/function/IntPredicate.html#or%2Djava.util.function.IntPredicate%2D[or](<>)++ (link:{java9-javadoc}/java/util/function/IntPredicate.html#or%2Djava.util.function.IntPredicate%2D[java 9]) -* ++[[painless-api-reference-IntPredicate-test-1]]boolean link:{java8-javadoc}/java/util/function/IntPredicate.html#test%2Dint%2D[test](int)++ (link:{java9-javadoc}/java/util/function/IntPredicate.html#test%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntStream.Builder.asciidoc b/docs/painless/painless-api-reference/IntStream.Builder.asciidoc deleted file mode 100644 index 0db51e7164b22..0000000000000 --- a/docs/painless/painless-api-reference/IntStream.Builder.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntStream-Builder]]++IntStream.Builder++:: -* ++[[painless-api-reference-IntStream-Builder-add-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.Builder.html#add%2Dint%2D[add](int)++ (link:{java9-javadoc}/java/util/stream/IntStream.Builder.html#add%2Dint%2D[java 9]) -* ++[[painless-api-reference-IntStream-Builder-build-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.Builder.html#build%2D%2D[build]()++ (link:{java9-javadoc}/java/util/stream/IntStream.Builder.html#build%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IntStream.asciidoc b/docs/painless/painless-api-reference/IntStream.asciidoc deleted file mode 100644 index 10e1149268986..0000000000000 --- a/docs/painless/painless-api-reference/IntStream.asciidoc +++ /dev/null @@ -1,47 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntStream]]++IntStream++:: -* ++[[painless-api-reference-IntStream-builder-0]]static <> link:{java8-javadoc}/java/util/stream/IntStream.html#builder%2D%2D[builder]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#builder%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-concat-2]]static <> link:{java8-javadoc}/java/util/stream/IntStream.html#concat%2Djava.util.stream.IntStream%2Djava.util.stream.IntStream%2D[concat](<>, <>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#concat%2Djava.util.stream.IntStream%2Djava.util.stream.IntStream%2D[java 9]) -* ++[[painless-api-reference-IntStream-empty-0]]static <> link:{java8-javadoc}/java/util/stream/IntStream.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-of-1]]static <> link:{java8-javadoc}/java/util/stream/IntStream.html#of%2Dint:A%2D[of](int[])++ (link:{java9-javadoc}/java/util/stream/IntStream.html#of%2Dint:A%2D[java 9]) -* ++[[painless-api-reference-IntStream-range-2]]static <> link:{java8-javadoc}/java/util/stream/IntStream.html#range%2Dint%2Dint%2D[range](int, int)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#range%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IntStream-rangeClosed-2]]static <> link:{java8-javadoc}/java/util/stream/IntStream.html#rangeClosed%2Dint%2Dint%2D[rangeClosed](int, int)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#rangeClosed%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IntStream-allMatch-1]]boolean link:{java8-javadoc}/java/util/stream/IntStream.html#allMatch%2Djava.util.function.IntPredicate%2D[allMatch](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#allMatch%2Djava.util.function.IntPredicate%2D[java 9]) -* ++[[painless-api-reference-IntStream-anyMatch-1]]boolean link:{java8-javadoc}/java/util/stream/IntStream.html#anyMatch%2Djava.util.function.IntPredicate%2D[anyMatch](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#anyMatch%2Djava.util.function.IntPredicate%2D[java 9]) -* ++[[painless-api-reference-IntStream-asDoubleStream-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#asDoubleStream%2D%2D[asDoubleStream]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#asDoubleStream%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-asLongStream-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#asLongStream%2D%2D[asLongStream]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#asLongStream%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-average-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#average%2D%2D[average]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#average%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-boxed-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#boxed%2D%2D[boxed]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#boxed%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-collect-3]]def link:{java8-javadoc}/java/util/stream/IntStream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.ObjIntConsumer%2Djava.util.function.BiConsumer%2D[collect](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.ObjIntConsumer%2Djava.util.function.BiConsumer%2D[java 9]) -* ++[[painless-api-reference-IntStream-count-0]]long link:{java8-javadoc}/java/util/stream/IntStream.html#count%2D%2D[count]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#count%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-distinct-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#distinct%2D%2D[distinct]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#distinct%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-filter-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#filter%2Djava.util.function.IntPredicate%2D[filter](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#filter%2Djava.util.function.IntPredicate%2D[java 9]) -* ++[[painless-api-reference-IntStream-findAny-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#findAny%2D%2D[findAny]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#findAny%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-findFirst-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#findFirst%2D%2D[findFirst]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#findFirst%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-flatMap-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#flatMap%2Djava.util.function.IntFunction%2D[flatMap](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#flatMap%2Djava.util.function.IntFunction%2D[java 9]) -* ++[[painless-api-reference-IntStream-forEach-1]]void link:{java8-javadoc}/java/util/stream/IntStream.html#forEach%2Djava.util.function.IntConsumer%2D[forEach](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#forEach%2Djava.util.function.IntConsumer%2D[java 9]) -* ++[[painless-api-reference-IntStream-forEachOrdered-1]]void link:{java8-javadoc}/java/util/stream/IntStream.html#forEachOrdered%2Djava.util.function.IntConsumer%2D[forEachOrdered](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#forEachOrdered%2Djava.util.function.IntConsumer%2D[java 9]) -* ++[[painless-api-reference-IntStream-iterator-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#iterator%2D%2D[iterator]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#iterator%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-limit-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#limit%2Dlong%2D[limit](long)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#limit%2Dlong%2D[java 9]) -* ++[[painless-api-reference-IntStream-map-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#map%2Djava.util.function.IntUnaryOperator%2D[map](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#map%2Djava.util.function.IntUnaryOperator%2D[java 9]) -* ++[[painless-api-reference-IntStream-mapToDouble-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#mapToDouble%2Djava.util.function.IntToDoubleFunction%2D[mapToDouble](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#mapToDouble%2Djava.util.function.IntToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-IntStream-mapToLong-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#mapToLong%2Djava.util.function.IntToLongFunction%2D[mapToLong](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#mapToLong%2Djava.util.function.IntToLongFunction%2D[java 9]) -* ++[[painless-api-reference-IntStream-mapToObj-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#mapToObj%2Djava.util.function.IntFunction%2D[mapToObj](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#mapToObj%2Djava.util.function.IntFunction%2D[java 9]) -* ++[[painless-api-reference-IntStream-max-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#max%2D%2D[max]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#max%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-min-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#min%2D%2D[min]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#min%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-noneMatch-1]]boolean link:{java8-javadoc}/java/util/stream/IntStream.html#noneMatch%2Djava.util.function.IntPredicate%2D[noneMatch](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#noneMatch%2Djava.util.function.IntPredicate%2D[java 9]) -* ++[[painless-api-reference-IntStream-peek-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#peek%2Djava.util.function.IntConsumer%2D[peek](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#peek%2Djava.util.function.IntConsumer%2D[java 9]) -* ++[[painless-api-reference-IntStream-reduce-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#reduce%2Djava.util.function.IntBinaryOperator%2D[reduce](<>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#reduce%2Djava.util.function.IntBinaryOperator%2D[java 9]) -* ++[[painless-api-reference-IntStream-reduce-2]]int link:{java8-javadoc}/java/util/stream/IntStream.html#reduce%2Dint%2Djava.util.function.IntBinaryOperator%2D[reduce](int, <>)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#reduce%2Dint%2Djava.util.function.IntBinaryOperator%2D[java 9]) -* ++[[painless-api-reference-IntStream-sequential-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#sequential%2D%2D[sequential]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#sequential%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-skip-1]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#skip%2Dlong%2D[skip](long)++ (link:{java9-javadoc}/java/util/stream/IntStream.html#skip%2Dlong%2D[java 9]) -* ++[[painless-api-reference-IntStream-sorted-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#sorted%2D%2D[sorted]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#sorted%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-spliterator-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#spliterator%2D%2D[spliterator]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#spliterator%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-sum-0]]int link:{java8-javadoc}/java/util/stream/IntStream.html#sum%2D%2D[sum]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#sum%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-summaryStatistics-0]]<> link:{java8-javadoc}/java/util/stream/IntStream.html#summaryStatistics%2D%2D[summaryStatistics]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#summaryStatistics%2D%2D[java 9]) -* ++[[painless-api-reference-IntStream-toArray-0]]int[] link:{java8-javadoc}/java/util/stream/IntStream.html#toArray%2D%2D[toArray]()++ (link:{java9-javadoc}/java/util/stream/IntStream.html#toArray%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IntSummaryStatistics.asciidoc b/docs/painless/painless-api-reference/IntSummaryStatistics.asciidoc deleted file mode 100644 index e13d47ad7c4aa..0000000000000 --- a/docs/painless/painless-api-reference/IntSummaryStatistics.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntSummaryStatistics]]++IntSummaryStatistics++:: -* ++[[painless-api-reference-IntSummaryStatistics-IntSummaryStatistics-0]]link:{java8-javadoc}/java/util/IntSummaryStatistics.html#IntSummaryStatistics%2D%2D[IntSummaryStatistics]()++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#IntSummaryStatistics%2D%2D[java 9]) -* ++[[painless-api-reference-IntSummaryStatistics-combine-1]]void link:{java8-javadoc}/java/util/IntSummaryStatistics.html#combine%2Djava.util.IntSummaryStatistics%2D[combine](<>)++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#combine%2Djava.util.IntSummaryStatistics%2D[java 9]) -* ++[[painless-api-reference-IntSummaryStatistics-getAverage-0]]double link:{java8-javadoc}/java/util/IntSummaryStatistics.html#getAverage%2D%2D[getAverage]()++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#getAverage%2D%2D[java 9]) -* ++[[painless-api-reference-IntSummaryStatistics-getCount-0]]long link:{java8-javadoc}/java/util/IntSummaryStatistics.html#getCount%2D%2D[getCount]()++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#getCount%2D%2D[java 9]) -* ++[[painless-api-reference-IntSummaryStatistics-getMax-0]]int link:{java8-javadoc}/java/util/IntSummaryStatistics.html#getMax%2D%2D[getMax]()++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#getMax%2D%2D[java 9]) -* ++[[painless-api-reference-IntSummaryStatistics-getMin-0]]int link:{java8-javadoc}/java/util/IntSummaryStatistics.html#getMin%2D%2D[getMin]()++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#getMin%2D%2D[java 9]) -* ++[[painless-api-reference-IntSummaryStatistics-getSum-0]]long link:{java8-javadoc}/java/util/IntSummaryStatistics.html#getSum%2D%2D[getSum]()++ (link:{java9-javadoc}/java/util/IntSummaryStatistics.html#getSum%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IntSupplier.asciidoc b/docs/painless/painless-api-reference/IntSupplier.asciidoc deleted file mode 100644 index bca332f7e6cf4..0000000000000 --- a/docs/painless/painless-api-reference/IntSupplier.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntSupplier]]++IntSupplier++:: -* ++[[painless-api-reference-IntSupplier-getAsInt-0]]int link:{java8-javadoc}/java/util/function/IntSupplier.html#getAsInt%2D%2D[getAsInt]()++ (link:{java9-javadoc}/java/util/function/IntSupplier.html#getAsInt%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntToDoubleFunction.asciidoc b/docs/painless/painless-api-reference/IntToDoubleFunction.asciidoc deleted file mode 100644 index 1cfadec16b2d8..0000000000000 --- a/docs/painless/painless-api-reference/IntToDoubleFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntToDoubleFunction]]++IntToDoubleFunction++:: -* ++[[painless-api-reference-IntToDoubleFunction-applyAsDouble-1]]double link:{java8-javadoc}/java/util/function/IntToDoubleFunction.html#applyAsDouble%2Dint%2D[applyAsDouble](int)++ (link:{java9-javadoc}/java/util/function/IntToDoubleFunction.html#applyAsDouble%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntToLongFunction.asciidoc b/docs/painless/painless-api-reference/IntToLongFunction.asciidoc deleted file mode 100644 index 3cffe6cddea16..0000000000000 --- a/docs/painless/painless-api-reference/IntToLongFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntToLongFunction]]++IntToLongFunction++:: -* ++[[painless-api-reference-IntToLongFunction-applyAsLong-1]]long link:{java8-javadoc}/java/util/function/IntToLongFunction.html#applyAsLong%2Dint%2D[applyAsLong](int)++ (link:{java9-javadoc}/java/util/function/IntToLongFunction.html#applyAsLong%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IntUnaryOperator.asciidoc b/docs/painless/painless-api-reference/IntUnaryOperator.asciidoc deleted file mode 100644 index a1bb1651fee56..0000000000000 --- a/docs/painless/painless-api-reference/IntUnaryOperator.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IntUnaryOperator]]++IntUnaryOperator++:: -* ++[[painless-api-reference-IntUnaryOperator-identity-0]]static <> link:{java8-javadoc}/java/util/function/IntUnaryOperator.html#identity%2D%2D[identity]()++ (link:{java9-javadoc}/java/util/function/IntUnaryOperator.html#identity%2D%2D[java 9]) -* ++[[painless-api-reference-IntUnaryOperator-andThen-1]]<> link:{java8-javadoc}/java/util/function/IntUnaryOperator.html#andThen%2Djava.util.function.IntUnaryOperator%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/IntUnaryOperator.html#andThen%2Djava.util.function.IntUnaryOperator%2D[java 9]) -* ++[[painless-api-reference-IntUnaryOperator-applyAsInt-1]]int link:{java8-javadoc}/java/util/function/IntUnaryOperator.html#applyAsInt%2Dint%2D[applyAsInt](int)++ (link:{java9-javadoc}/java/util/function/IntUnaryOperator.html#applyAsInt%2Dint%2D[java 9]) -* ++[[painless-api-reference-IntUnaryOperator-compose-1]]<> link:{java8-javadoc}/java/util/function/IntUnaryOperator.html#compose%2Djava.util.function.IntUnaryOperator%2D[compose](<>)++ (link:{java9-javadoc}/java/util/function/IntUnaryOperator.html#compose%2Djava.util.function.IntUnaryOperator%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Integer.asciidoc b/docs/painless/painless-api-reference/Integer.asciidoc deleted file mode 100644 index 2f608bcc662f7..0000000000000 --- a/docs/painless/painless-api-reference/Integer.asciidoc +++ /dev/null @@ -1,44 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Integer]]++Integer++:: -** [[painless-api-reference-Integer-BYTES]]static int link:{java8-javadoc}/java/lang/Integer.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Integer.html#BYTES[java 9]) -** [[painless-api-reference-Integer-MAX_VALUE]]static int link:{java8-javadoc}/java/lang/Integer.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Integer.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Integer-MIN_VALUE]]static int link:{java8-javadoc}/java/lang/Integer.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Integer.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Integer-SIZE]]static int link:{java8-javadoc}/java/lang/Integer.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Integer.html#SIZE[java 9]) -* ++[[painless-api-reference-Integer-bitCount-1]]static int link:{java8-javadoc}/java/lang/Integer.html#bitCount%2Dint%2D[bitCount](int)++ (link:{java9-javadoc}/java/lang/Integer.html#bitCount%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-compare-2]]static int link:{java8-javadoc}/java/lang/Integer.html#compare%2Dint%2Dint%2D[compare](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#compare%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-compareUnsigned-2]]static int link:{java8-javadoc}/java/lang/Integer.html#compareUnsigned%2Dint%2Dint%2D[compareUnsigned](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#compareUnsigned%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-decode-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#decode%2Djava.lang.String%2D[decode](<>)++ (link:{java9-javadoc}/java/lang/Integer.html#decode%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Integer-divideUnsigned-2]]static int link:{java8-javadoc}/java/lang/Integer.html#divideUnsigned%2Dint%2Dint%2D[divideUnsigned](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#divideUnsigned%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-hashCode-1]]static int link:{java8-javadoc}/java/lang/Integer.html#hashCode%2Dint%2D[hashCode](int)++ (link:{java9-javadoc}/java/lang/Integer.html#hashCode%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-highestOneBit-1]]static int link:{java8-javadoc}/java/lang/Integer.html#highestOneBit%2Dint%2D[highestOneBit](int)++ (link:{java9-javadoc}/java/lang/Integer.html#highestOneBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-lowestOneBit-1]]static int link:{java8-javadoc}/java/lang/Integer.html#lowestOneBit%2Dint%2D[lowestOneBit](int)++ (link:{java9-javadoc}/java/lang/Integer.html#lowestOneBit%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-max-2]]static int link:{java8-javadoc}/java/lang/Integer.html#max%2Dint%2Dint%2D[max](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#max%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-min-2]]static int link:{java8-javadoc}/java/lang/Integer.html#min%2Dint%2Dint%2D[min](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#min%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-numberOfLeadingZeros-1]]static int link:{java8-javadoc}/java/lang/Integer.html#numberOfLeadingZeros%2Dint%2D[numberOfLeadingZeros](int)++ (link:{java9-javadoc}/java/lang/Integer.html#numberOfLeadingZeros%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-numberOfTrailingZeros-1]]static int link:{java8-javadoc}/java/lang/Integer.html#numberOfTrailingZeros%2Dint%2D[numberOfTrailingZeros](int)++ (link:{java9-javadoc}/java/lang/Integer.html#numberOfTrailingZeros%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-parseInt-1]]static int link:{java8-javadoc}/java/lang/Integer.html#parseInt%2Djava.lang.String%2D[parseInt](<>)++ (link:{java9-javadoc}/java/lang/Integer.html#parseInt%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Integer-parseInt-2]]static int link:{java8-javadoc}/java/lang/Integer.html#parseInt%2Djava.lang.String%2Dint%2D[parseInt](<>, int)++ (link:{java9-javadoc}/java/lang/Integer.html#parseInt%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-parseUnsignedInt-1]]static int link:{java8-javadoc}/java/lang/Integer.html#parseUnsignedInt%2Djava.lang.String%2D[parseUnsignedInt](<>)++ (link:{java9-javadoc}/java/lang/Integer.html#parseUnsignedInt%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Integer-parseUnsignedInt-2]]static int link:{java8-javadoc}/java/lang/Integer.html#parseUnsignedInt%2Djava.lang.String%2Dint%2D[parseUnsignedInt](<>, int)++ (link:{java9-javadoc}/java/lang/Integer.html#parseUnsignedInt%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-remainderUnsigned-2]]static int link:{java8-javadoc}/java/lang/Integer.html#remainderUnsigned%2Dint%2Dint%2D[remainderUnsigned](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#remainderUnsigned%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-reverse-1]]static int link:{java8-javadoc}/java/lang/Integer.html#reverse%2Dint%2D[reverse](int)++ (link:{java9-javadoc}/java/lang/Integer.html#reverse%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-reverseBytes-1]]static int link:{java8-javadoc}/java/lang/Integer.html#reverseBytes%2Dint%2D[reverseBytes](int)++ (link:{java9-javadoc}/java/lang/Integer.html#reverseBytes%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-rotateLeft-2]]static int link:{java8-javadoc}/java/lang/Integer.html#rotateLeft%2Dint%2Dint%2D[rotateLeft](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#rotateLeft%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-rotateRight-2]]static int link:{java8-javadoc}/java/lang/Integer.html#rotateRight%2Dint%2Dint%2D[rotateRight](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#rotateRight%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-signum-1]]static int link:{java8-javadoc}/java/lang/Integer.html#signum%2Dint%2D[signum](int)++ (link:{java9-javadoc}/java/lang/Integer.html#signum%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toBinaryString-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#toBinaryString%2Dint%2D[toBinaryString](int)++ (link:{java9-javadoc}/java/lang/Integer.html#toBinaryString%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toHexString-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#toHexString%2Dint%2D[toHexString](int)++ (link:{java9-javadoc}/java/lang/Integer.html#toHexString%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toOctalString-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#toOctalString%2Dint%2D[toOctalString](int)++ (link:{java9-javadoc}/java/lang/Integer.html#toOctalString%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toString-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#toString%2Dint%2D[toString](int)++ (link:{java9-javadoc}/java/lang/Integer.html#toString%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toString-2]]static <> link:{java8-javadoc}/java/lang/Integer.html#toString%2Dint%2Dint%2D[toString](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#toString%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toUnsignedLong-1]]static long link:{java8-javadoc}/java/lang/Integer.html#toUnsignedLong%2Dint%2D[toUnsignedLong](int)++ (link:{java9-javadoc}/java/lang/Integer.html#toUnsignedLong%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toUnsignedString-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#toUnsignedString%2Dint%2D[toUnsignedString](int)++ (link:{java9-javadoc}/java/lang/Integer.html#toUnsignedString%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-toUnsignedString-2]]static <> link:{java8-javadoc}/java/lang/Integer.html#toUnsignedString%2Dint%2Dint%2D[toUnsignedString](int, int)++ (link:{java9-javadoc}/java/lang/Integer.html#toUnsignedString%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Integer.html#valueOf%2Dint%2D[valueOf](int)++ (link:{java9-javadoc}/java/lang/Integer.html#valueOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-valueOf-2]]static <> link:{java8-javadoc}/java/lang/Integer.html#valueOf%2Djava.lang.String%2Dint%2D[valueOf](<>, int)++ (link:{java9-javadoc}/java/lang/Integer.html#valueOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Integer-compareTo-1]]int link:{java8-javadoc}/java/lang/Integer.html#compareTo%2Djava.lang.Integer%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Integer.html#compareTo%2Djava.lang.Integer%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/InterruptedException.asciidoc b/docs/painless/painless-api-reference/InterruptedException.asciidoc deleted file mode 100644 index cd0ef457f76ab..0000000000000 --- a/docs/painless/painless-api-reference/InterruptedException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-InterruptedException]]++InterruptedException++:: -* ++[[painless-api-reference-InterruptedException-InterruptedException-0]]link:{java8-javadoc}/java/lang/InterruptedException.html#InterruptedException%2D%2D[InterruptedException]()++ (link:{java9-javadoc}/java/lang/InterruptedException.html#InterruptedException%2D%2D[java 9]) -* ++[[painless-api-reference-InterruptedException-InterruptedException-1]]link:{java8-javadoc}/java/lang/InterruptedException.html#InterruptedException%2Djava.lang.String%2D[InterruptedException](<>)++ (link:{java9-javadoc}/java/lang/InterruptedException.html#InterruptedException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IsoChronology.asciidoc b/docs/painless/painless-api-reference/IsoChronology.asciidoc deleted file mode 100644 index 3990839b041fc..0000000000000 --- a/docs/painless/painless-api-reference/IsoChronology.asciidoc +++ /dev/null @@ -1,20 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IsoChronology]]++IsoChronology++:: -** [[painless-api-reference-IsoChronology-INSTANCE]]static <> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#INSTANCE[INSTANCE] (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#INSTANCE[java 9]) -* ++[[painless-api-reference-IsoChronology-date-1]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[date](<>)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-date-3]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#date%2Dint%2Dint%2Dint%2D[date](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#date%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-date-4]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[date](<>, int, int, int)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-dateEpochDay-1]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#dateEpochDay%2Dlong%2D[dateEpochDay](long)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#dateEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-dateYearDay-2]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#dateYearDay%2Dint%2Dint%2D[dateYearDay](int, int)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#dateYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-dateYearDay-3]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[dateYearDay](<>, int, int)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-eraOf-1]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#eraOf%2Dint%2D[eraOf](int)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#eraOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-localDateTime-1]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#localDateTime%2Djava.time.temporal.TemporalAccessor%2D[localDateTime](<>)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#localDateTime%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-period-3]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#period%2Dint%2Dint%2Dint%2D[period](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#period%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-resolveDate-2]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[resolveDate](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-zonedDateTime-1]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#zonedDateTime%2Djava.time.temporal.TemporalAccessor%2D[zonedDateTime](<>)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#zonedDateTime%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-IsoChronology-zonedDateTime-2]]<> link:{java8-javadoc}/java/time/chrono/IsoChronology.html#zonedDateTime%2Djava.time.Instant%2Djava.time.ZoneId%2D[zonedDateTime](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/IsoChronology.html#zonedDateTime%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/IsoEra.asciidoc b/docs/painless/painless-api-reference/IsoEra.asciidoc deleted file mode 100644 index 975d7d1153aef..0000000000000 --- a/docs/painless/painless-api-reference/IsoEra.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IsoEra]]++IsoEra++:: -** [[painless-api-reference-IsoEra-BCE]]static <> link:{java8-javadoc}/java/time/chrono/IsoEra.html#BCE[BCE] (link:{java9-javadoc}/java/time/chrono/IsoEra.html#BCE[java 9]) -** [[painless-api-reference-IsoEra-CE]]static <> link:{java8-javadoc}/java/time/chrono/IsoEra.html#CE[CE] (link:{java9-javadoc}/java/time/chrono/IsoEra.html#CE[java 9]) -* ++[[painless-api-reference-IsoEra-of-1]]static <> link:{java8-javadoc}/java/time/chrono/IsoEra.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/chrono/IsoEra.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-IsoEra-valueOf-1]]static <> link:{java8-javadoc}/java/time/chrono/IsoEra.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/chrono/IsoEra.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-IsoEra-values-0]]static <>[] link:{java8-javadoc}/java/time/chrono/IsoEra.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/chrono/IsoEra.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-IsoEra-getValue-0]]int link:{java8-javadoc}/java/time/chrono/IsoEra.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/chrono/IsoEra.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/IsoFields.asciidoc b/docs/painless/painless-api-reference/IsoFields.asciidoc deleted file mode 100644 index db3f5bbefbb5e..0000000000000 --- a/docs/painless/painless-api-reference/IsoFields.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-IsoFields]]++IsoFields++:: -** [[painless-api-reference-IsoFields-DAY_OF_QUARTER]]static <> link:{java8-javadoc}/java/time/temporal/IsoFields.html#DAY_OF_QUARTER[DAY_OF_QUARTER] (link:{java9-javadoc}/java/time/temporal/IsoFields.html#DAY_OF_QUARTER[java 9]) -** [[painless-api-reference-IsoFields-QUARTER_OF_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/IsoFields.html#QUARTER_OF_YEAR[QUARTER_OF_YEAR] (link:{java9-javadoc}/java/time/temporal/IsoFields.html#QUARTER_OF_YEAR[java 9]) -** [[painless-api-reference-IsoFields-QUARTER_YEARS]]static <> link:{java8-javadoc}/java/time/temporal/IsoFields.html#QUARTER_YEARS[QUARTER_YEARS] (link:{java9-javadoc}/java/time/temporal/IsoFields.html#QUARTER_YEARS[java 9]) -** [[painless-api-reference-IsoFields-WEEK_BASED_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/IsoFields.html#WEEK_BASED_YEAR[WEEK_BASED_YEAR] (link:{java9-javadoc}/java/time/temporal/IsoFields.html#WEEK_BASED_YEAR[java 9]) -** [[painless-api-reference-IsoFields-WEEK_BASED_YEARS]]static <> link:{java8-javadoc}/java/time/temporal/IsoFields.html#WEEK_BASED_YEARS[WEEK_BASED_YEARS] (link:{java9-javadoc}/java/time/temporal/IsoFields.html#WEEK_BASED_YEARS[java 9]) -** [[painless-api-reference-IsoFields-WEEK_OF_WEEK_BASED_YEAR]]static <> link:{java8-javadoc}/java/time/temporal/IsoFields.html#WEEK_OF_WEEK_BASED_YEAR[WEEK_OF_WEEK_BASED_YEAR] (link:{java9-javadoc}/java/time/temporal/IsoFields.html#WEEK_OF_WEEK_BASED_YEAR[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Iterable.asciidoc b/docs/painless/painless-api-reference/Iterable.asciidoc deleted file mode 100644 index b7d626767c710..0000000000000 --- a/docs/painless/painless-api-reference/Iterable.asciidoc +++ /dev/null @@ -1,21 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Iterable]]++Iterable++:: -* ++[[painless-api-reference-Iterable-any-1]]boolean link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#any%2Djava.lang.Iterable%2Djava.util.function.Predicate%2D[any](<>)++ -* ++[[painless-api-reference-Iterable-asCollection-0]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#asCollection%2Djava.lang.Iterable%2D[asCollection]()++ -* ++[[painless-api-reference-Iterable-asList-0]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#asList%2Djava.lang.Iterable%2D[asList]()++ -* ++[[painless-api-reference-Iterable-each-1]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#each%2Djava.lang.Iterable%2Djava.util.function.Consumer%2D[each](<>)++ -* ++[[painless-api-reference-Iterable-eachWithIndex-1]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#eachWithIndex%2Djava.lang.Iterable%2Djava.util.function.ObjIntConsumer%2D[eachWithIndex](<>)++ -* ++[[painless-api-reference-Iterable-every-1]]boolean link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#every%2Djava.lang.Iterable%2Djava.util.function.Predicate%2D[every](<>)++ -* ++[[painless-api-reference-Iterable-findResults-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findResults%2Djava.lang.Iterable%2Djava.util.function.Function%2D[findResults](<>)++ -* ++[[painless-api-reference-Iterable-forEach-1]]void link:{java8-javadoc}/java/lang/Iterable.html#forEach%2Djava.util.function.Consumer%2D[forEach](<>)++ (link:{java9-javadoc}/java/lang/Iterable.html#forEach%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Iterable-groupBy-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#groupBy%2Djava.lang.Iterable%2Djava.util.function.Function%2D[groupBy](<>)++ -* ++[[painless-api-reference-Iterable-iterator-0]]<> link:{java8-javadoc}/java/lang/Iterable.html#iterator%2D%2D[iterator]()++ (link:{java9-javadoc}/java/lang/Iterable.html#iterator%2D%2D[java 9]) -* ++[[painless-api-reference-Iterable-join-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#join%2Djava.lang.Iterable%2Djava.lang.String%2D[join](<>)++ -* ++[[painless-api-reference-Iterable-spliterator-0]]<> link:{java8-javadoc}/java/lang/Iterable.html#spliterator%2D%2D[spliterator]()++ (link:{java9-javadoc}/java/lang/Iterable.html#spliterator%2D%2D[java 9]) -* ++[[painless-api-reference-Iterable-sum-0]]double link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#sum%2Djava.lang.Iterable%2D[sum]()++ -* ++[[painless-api-reference-Iterable-sum-1]]double link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#sum%2Djava.lang.Iterable%2Djava.util.function.ToDoubleFunction%2D[sum](<>)++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Iterator.asciidoc b/docs/painless/painless-api-reference/Iterator.asciidoc deleted file mode 100644 index 9dd3f5601d54b..0000000000000 --- a/docs/painless/painless-api-reference/Iterator.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Iterator]]++Iterator++:: -* ++[[painless-api-reference-Iterator-forEachRemaining-1]]void link:{java8-javadoc}/java/util/Iterator.html#forEachRemaining%2Djava.util.function.Consumer%2D[forEachRemaining](<>)++ (link:{java9-javadoc}/java/util/Iterator.html#forEachRemaining%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Iterator-hasNext-0]]boolean link:{java8-javadoc}/java/util/Iterator.html#hasNext%2D%2D[hasNext]()++ (link:{java9-javadoc}/java/util/Iterator.html#hasNext%2D%2D[java 9]) -* ++[[painless-api-reference-Iterator-next-0]]def link:{java8-javadoc}/java/util/Iterator.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/util/Iterator.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-Iterator-remove-0]]void link:{java8-javadoc}/java/util/Iterator.html#remove%2D%2D[remove]()++ (link:{java9-javadoc}/java/util/Iterator.html#remove%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/JapaneseChronology.asciidoc b/docs/painless/painless-api-reference/JapaneseChronology.asciidoc deleted file mode 100644 index 166f3ca176d09..0000000000000 --- a/docs/painless/painless-api-reference/JapaneseChronology.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-JapaneseChronology]]++JapaneseChronology++:: -** [[painless-api-reference-JapaneseChronology-INSTANCE]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#INSTANCE[INSTANCE] (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#INSTANCE[java 9]) -* ++[[painless-api-reference-JapaneseChronology-date-1]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[date](<>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-date-3]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#date%2Dint%2Dint%2Dint%2D[date](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#date%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-date-4]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[date](<>, int, int, int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-dateEpochDay-1]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#dateEpochDay%2Dlong%2D[dateEpochDay](long)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#dateEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-dateYearDay-2]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#dateYearDay%2Dint%2Dint%2D[dateYearDay](int, int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#dateYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-dateYearDay-3]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[dateYearDay](<>, int, int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-eraOf-1]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#eraOf%2Dint%2D[eraOf](int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#eraOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseChronology-resolveDate-2]]<> link:{java8-javadoc}/java/time/chrono/JapaneseChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[resolveDate](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/JapaneseDate.asciidoc b/docs/painless/painless-api-reference/JapaneseDate.asciidoc deleted file mode 100644 index 560ebc7a664bc..0000000000000 --- a/docs/painless/painless-api-reference/JapaneseDate.asciidoc +++ /dev/null @@ -1,17 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-JapaneseDate]]++JapaneseDate++:: -* ++[[painless-api-reference-JapaneseDate-from-1]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-of-3]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-getEra-0]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#getEra%2D%2D[getEra]()++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#getEra%2D%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-minus-1]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-minus-2]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-plus-1]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-plus-2]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-with-1]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-JapaneseDate-with-2]]<> link:{java8-javadoc}/java/time/chrono/JapaneseDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/JapaneseDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/JapaneseEra.asciidoc b/docs/painless/painless-api-reference/JapaneseEra.asciidoc deleted file mode 100644 index b5a9c7f63bc04..0000000000000 --- a/docs/painless/painless-api-reference/JapaneseEra.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-JapaneseEra]]++JapaneseEra++:: -** [[painless-api-reference-JapaneseEra-HEISEI]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#HEISEI[HEISEI] (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#HEISEI[java 9]) -** [[painless-api-reference-JapaneseEra-MEIJI]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#MEIJI[MEIJI] (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#MEIJI[java 9]) -** [[painless-api-reference-JapaneseEra-SHOWA]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#SHOWA[SHOWA] (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#SHOWA[java 9]) -** [[painless-api-reference-JapaneseEra-TAISHO]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#TAISHO[TAISHO] (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#TAISHO[java 9]) -* ++[[painless-api-reference-JapaneseEra-of-1]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-JapaneseEra-valueOf-1]]static <> link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-JapaneseEra-values-0]]static <>[] link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-JapaneseEra-getValue-0]]int link:{java8-javadoc}/java/time/chrono/JapaneseEra.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/chrono/JapaneseEra.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/JulianFields.asciidoc b/docs/painless/painless-api-reference/JulianFields.asciidoc deleted file mode 100644 index 19ee6c5e2ebd6..0000000000000 --- a/docs/painless/painless-api-reference/JulianFields.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-JulianFields]]++JulianFields++:: -** [[painless-api-reference-JulianFields-JULIAN_DAY]]static <> link:{java8-javadoc}/java/time/temporal/JulianFields.html#JULIAN_DAY[JULIAN_DAY] (link:{java9-javadoc}/java/time/temporal/JulianFields.html#JULIAN_DAY[java 9]) -** [[painless-api-reference-JulianFields-MODIFIED_JULIAN_DAY]]static <> link:{java8-javadoc}/java/time/temporal/JulianFields.html#MODIFIED_JULIAN_DAY[MODIFIED_JULIAN_DAY] (link:{java9-javadoc}/java/time/temporal/JulianFields.html#MODIFIED_JULIAN_DAY[java 9]) -** [[painless-api-reference-JulianFields-RATA_DIE]]static <> link:{java8-javadoc}/java/time/temporal/JulianFields.html#RATA_DIE[RATA_DIE] (link:{java9-javadoc}/java/time/temporal/JulianFields.html#RATA_DIE[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LinkedHashMap.asciidoc b/docs/painless/painless-api-reference/LinkedHashMap.asciidoc deleted file mode 100644 index bda2d45f22cb6..0000000000000 --- a/docs/painless/painless-api-reference/LinkedHashMap.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LinkedHashMap]]++LinkedHashMap++:: -* ++[[painless-api-reference-LinkedHashMap-LinkedHashMap-0]]link:{java8-javadoc}/java/util/LinkedHashMap.html#LinkedHashMap%2D%2D[LinkedHashMap]()++ (link:{java9-javadoc}/java/util/LinkedHashMap.html#LinkedHashMap%2D%2D[java 9]) -* ++[[painless-api-reference-LinkedHashMap-LinkedHashMap-1]]link:{java8-javadoc}/java/util/LinkedHashMap.html#LinkedHashMap%2Djava.util.Map%2D[LinkedHashMap](<>)++ (link:{java9-javadoc}/java/util/LinkedHashMap.html#LinkedHashMap%2Djava.util.Map%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LinkedHashSet.asciidoc b/docs/painless/painless-api-reference/LinkedHashSet.asciidoc deleted file mode 100644 index 3a7710771e8ba..0000000000000 --- a/docs/painless/painless-api-reference/LinkedHashSet.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LinkedHashSet]]++LinkedHashSet++:: -* ++[[painless-api-reference-LinkedHashSet-LinkedHashSet-0]]link:{java8-javadoc}/java/util/LinkedHashSet.html#LinkedHashSet%2D%2D[LinkedHashSet]()++ (link:{java9-javadoc}/java/util/LinkedHashSet.html#LinkedHashSet%2D%2D[java 9]) -* ++[[painless-api-reference-LinkedHashSet-LinkedHashSet-1]]link:{java8-javadoc}/java/util/LinkedHashSet.html#LinkedHashSet%2Djava.util.Collection%2D[LinkedHashSet](<>)++ (link:{java9-javadoc}/java/util/LinkedHashSet.html#LinkedHashSet%2Djava.util.Collection%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LinkedList.asciidoc b/docs/painless/painless-api-reference/LinkedList.asciidoc deleted file mode 100644 index 5d18b373ba360..0000000000000 --- a/docs/painless/painless-api-reference/LinkedList.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LinkedList]]++LinkedList++:: -* ++[[painless-api-reference-LinkedList-LinkedList-0]]link:{java8-javadoc}/java/util/LinkedList.html#LinkedList%2D%2D[LinkedList]()++ (link:{java9-javadoc}/java/util/LinkedList.html#LinkedList%2D%2D[java 9]) -* ++[[painless-api-reference-LinkedList-LinkedList-1]]link:{java8-javadoc}/java/util/LinkedList.html#LinkedList%2Djava.util.Collection%2D[LinkedList](<>)++ (link:{java9-javadoc}/java/util/LinkedList.html#LinkedList%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-LinkedList-clone-0]]def link:{java8-javadoc}/java/util/LinkedList.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/LinkedList.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/List.asciidoc b/docs/painless/painless-api-reference/List.asciidoc deleted file mode 100644 index abe438451b86b..0000000000000 --- a/docs/painless/painless-api-reference/List.asciidoc +++ /dev/null @@ -1,22 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-List]]++List++:: -* ++[[painless-api-reference-List-add-2]]void link:{java8-javadoc}/java/util/List.html#add%2Dint%2Djava.lang.Object%2D[add](int, def)++ (link:{java9-javadoc}/java/util/List.html#add%2Dint%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-List-addAll-2]]boolean link:{java8-javadoc}/java/util/List.html#addAll%2Dint%2Djava.util.Collection%2D[addAll](int, <>)++ (link:{java9-javadoc}/java/util/List.html#addAll%2Dint%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-List-equals-1]]boolean link:{java8-javadoc}/java/util/List.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/util/List.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-List-get-1]]def link:{java8-javadoc}/java/util/List.html#get%2Dint%2D[get](int)++ (link:{java9-javadoc}/java/util/List.html#get%2Dint%2D[java 9]) -* ++[[painless-api-reference-List-getLength-0]]int link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#getLength%2Djava.util.List%2D[getLength]()++ -* ++[[painless-api-reference-List-hashCode-0]]int link:{java8-javadoc}/java/util/List.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/util/List.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-List-indexOf-1]]int link:{java8-javadoc}/java/util/List.html#indexOf%2Djava.lang.Object%2D[indexOf](def)++ (link:{java9-javadoc}/java/util/List.html#indexOf%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-List-lastIndexOf-1]]int link:{java8-javadoc}/java/util/List.html#lastIndexOf%2Djava.lang.Object%2D[lastIndexOf](def)++ (link:{java9-javadoc}/java/util/List.html#lastIndexOf%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-List-listIterator-0]]<> link:{java8-javadoc}/java/util/List.html#listIterator%2D%2D[listIterator]()++ (link:{java9-javadoc}/java/util/List.html#listIterator%2D%2D[java 9]) -* ++[[painless-api-reference-List-listIterator-1]]<> link:{java8-javadoc}/java/util/List.html#listIterator%2Dint%2D[listIterator](int)++ (link:{java9-javadoc}/java/util/List.html#listIterator%2Dint%2D[java 9]) -* ++[[painless-api-reference-List-remove-1]]def link:{java8-javadoc}/java/util/List.html#remove%2Dint%2D[remove](int)++ (link:{java9-javadoc}/java/util/List.html#remove%2Dint%2D[java 9]) -* ++[[painless-api-reference-List-replaceAll-1]]void link:{java8-javadoc}/java/util/List.html#replaceAll%2Djava.util.function.UnaryOperator%2D[replaceAll](<>)++ (link:{java9-javadoc}/java/util/List.html#replaceAll%2Djava.util.function.UnaryOperator%2D[java 9]) -* ++[[painless-api-reference-List-set-2]]def link:{java8-javadoc}/java/util/List.html#set%2Dint%2Djava.lang.Object%2D[set](int, def)++ (link:{java9-javadoc}/java/util/List.html#set%2Dint%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-List-sort-1]]void link:{java8-javadoc}/java/util/List.html#sort%2Djava.util.Comparator%2D[sort](<>)++ (link:{java9-javadoc}/java/util/List.html#sort%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-List-subList-2]]<> link:{java8-javadoc}/java/util/List.html#subList%2Dint%2Dint%2D[subList](int, int)++ (link:{java9-javadoc}/java/util/List.html#subList%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ListIterator.asciidoc b/docs/painless/painless-api-reference/ListIterator.asciidoc deleted file mode 100644 index 37d04b2a755cf..0000000000000 --- a/docs/painless/painless-api-reference/ListIterator.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ListIterator]]++ListIterator++:: -* ++[[painless-api-reference-ListIterator-add-1]]void link:{java8-javadoc}/java/util/ListIterator.html#add%2Djava.lang.Object%2D[add](def)++ (link:{java9-javadoc}/java/util/ListIterator.html#add%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-ListIterator-hasPrevious-0]]boolean link:{java8-javadoc}/java/util/ListIterator.html#hasPrevious%2D%2D[hasPrevious]()++ (link:{java9-javadoc}/java/util/ListIterator.html#hasPrevious%2D%2D[java 9]) -* ++[[painless-api-reference-ListIterator-nextIndex-0]]int link:{java8-javadoc}/java/util/ListIterator.html#nextIndex%2D%2D[nextIndex]()++ (link:{java9-javadoc}/java/util/ListIterator.html#nextIndex%2D%2D[java 9]) -* ++[[painless-api-reference-ListIterator-previousIndex-0]]int link:{java8-javadoc}/java/util/ListIterator.html#previousIndex%2D%2D[previousIndex]()++ (link:{java9-javadoc}/java/util/ListIterator.html#previousIndex%2D%2D[java 9]) -* ++[[painless-api-reference-ListIterator-set-1]]void link:{java8-javadoc}/java/util/ListIterator.html#set%2Djava.lang.Object%2D[set](def)++ (link:{java9-javadoc}/java/util/ListIterator.html#set%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LocalDate.asciidoc b/docs/painless/painless-api-reference/LocalDate.asciidoc deleted file mode 100644 index 8498474d02a0f..0000000000000 --- a/docs/painless/painless-api-reference/LocalDate.asciidoc +++ /dev/null @@ -1,47 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LocalDate]]++LocalDate++:: -** [[painless-api-reference-LocalDate-MAX]]static <> link:{java8-javadoc}/java/time/LocalDate.html#MAX[MAX] (link:{java9-javadoc}/java/time/LocalDate.html#MAX[java 9]) -** [[painless-api-reference-LocalDate-MIN]]static <> link:{java8-javadoc}/java/time/LocalDate.html#MIN[MIN] (link:{java9-javadoc}/java/time/LocalDate.html#MIN[java 9]) -* ++[[painless-api-reference-LocalDate-from-1]]static <> link:{java8-javadoc}/java/time/LocalDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-LocalDate-of-3]]static <> link:{java8-javadoc}/java/time/LocalDate.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/LocalDate.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-ofEpochDay-1]]static <> link:{java8-javadoc}/java/time/LocalDate.html#ofEpochDay%2Dlong%2D[ofEpochDay](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#ofEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-ofYearDay-2]]static <> link:{java8-javadoc}/java/time/LocalDate.html#ofYearDay%2Dint%2Dint%2D[ofYearDay](int, int)++ (link:{java9-javadoc}/java/time/LocalDate.html#ofYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-parse-1]]static <> link:{java8-javadoc}/java/time/LocalDate.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-LocalDate-parse-2]]static <> link:{java8-javadoc}/java/time/LocalDate.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/LocalDate.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-LocalDate-atStartOfDay-0]]<> link:{java8-javadoc}/java/time/LocalDate.html#atStartOfDay%2D%2D[atStartOfDay]()++ (link:{java9-javadoc}/java/time/LocalDate.html#atStartOfDay%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-atStartOfDay-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#atStartOfDay%2Djava.time.ZoneId%2D[atStartOfDay](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#atStartOfDay%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-LocalDate-atTime-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#atTime%2Djava.time.LocalTime%2D[atTime](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#atTime%2Djava.time.LocalTime%2D[java 9]) -* ++[[painless-api-reference-LocalDate-atTime-2]]<> link:{java8-javadoc}/java/time/LocalDate.html#atTime%2Dint%2Dint%2D[atTime](int, int)++ (link:{java9-javadoc}/java/time/LocalDate.html#atTime%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-atTime-3]]<> link:{java8-javadoc}/java/time/LocalDate.html#atTime%2Dint%2Dint%2Dint%2D[atTime](int, int, int)++ (link:{java9-javadoc}/java/time/LocalDate.html#atTime%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-atTime-4]]<> link:{java8-javadoc}/java/time/LocalDate.html#atTime%2Dint%2Dint%2Dint%2Dint%2D[atTime](int, int, int, int)++ (link:{java9-javadoc}/java/time/LocalDate.html#atTime%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getChronology-0]]<> link:{java8-javadoc}/java/time/LocalDate.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getDayOfMonth-0]]int link:{java8-javadoc}/java/time/LocalDate.html#getDayOfMonth%2D%2D[getDayOfMonth]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getDayOfWeek-0]]<> link:{java8-javadoc}/java/time/LocalDate.html#getDayOfWeek%2D%2D[getDayOfWeek]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getDayOfYear-0]]int link:{java8-javadoc}/java/time/LocalDate.html#getDayOfYear%2D%2D[getDayOfYear]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getDayOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getMonth-0]]<> link:{java8-javadoc}/java/time/LocalDate.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getMonthValue-0]]int link:{java8-javadoc}/java/time/LocalDate.html#getMonthValue%2D%2D[getMonthValue]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getMonthValue%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-getYear-0]]int link:{java8-javadoc}/java/time/LocalDate.html#getYear%2D%2D[getYear]()++ (link:{java9-javadoc}/java/time/LocalDate.html#getYear%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDate-minus-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-LocalDate-minus-2]]<> link:{java8-javadoc}/java/time/LocalDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/LocalDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalDate-minusDays-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#minusDays%2Dlong%2D[minusDays](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#minusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-minusMonths-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#minusMonths%2Dlong%2D[minusMonths](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#minusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-minusWeeks-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#minusWeeks%2Dlong%2D[minusWeeks](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#minusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-minusYears-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-plus-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-LocalDate-plus-2]]<> link:{java8-javadoc}/java/time/LocalDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/LocalDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalDate-plusDays-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#plusDays%2Dlong%2D[plusDays](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#plusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-plusMonths-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#plusMonths%2Dlong%2D[plusMonths](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#plusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-plusWeeks-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#plusWeeks%2Dlong%2D[plusWeeks](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#plusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-plusYears-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/LocalDate.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-until-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#until%2Djava.time.chrono.ChronoLocalDate%2D[until](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#until%2Djava.time.chrono.ChronoLocalDate%2D[java 9]) -* ++[[painless-api-reference-LocalDate-with-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/LocalDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-LocalDate-with-2]]<> link:{java8-javadoc}/java/time/LocalDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/LocalDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDate-withDayOfMonth-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#withDayOfMonth%2Dint%2D[withDayOfMonth](int)++ (link:{java9-javadoc}/java/time/LocalDate.html#withDayOfMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-withDayOfYear-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#withDayOfYear%2Dint%2D[withDayOfYear](int)++ (link:{java9-javadoc}/java/time/LocalDate.html#withDayOfYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-withMonth-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#withMonth%2Dint%2D[withMonth](int)++ (link:{java9-javadoc}/java/time/LocalDate.html#withMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDate-withYear-1]]<> link:{java8-javadoc}/java/time/LocalDate.html#withYear%2Dint%2D[withYear](int)++ (link:{java9-javadoc}/java/time/LocalDate.html#withYear%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LocalDateTime.asciidoc b/docs/painless/painless-api-reference/LocalDateTime.asciidoc deleted file mode 100644 index 9507ff1dd5231..0000000000000 --- a/docs/painless/painless-api-reference/LocalDateTime.asciidoc +++ /dev/null @@ -1,61 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LocalDateTime]]++LocalDateTime++:: -** [[painless-api-reference-LocalDateTime-MAX]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#MAX[MAX] (link:{java9-javadoc}/java/time/LocalDateTime.html#MAX[java 9]) -** [[painless-api-reference-LocalDateTime-MIN]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#MIN[MIN] (link:{java9-javadoc}/java/time/LocalDateTime.html#MIN[java 9]) -* ++[[painless-api-reference-LocalDateTime-from-1]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-of-2]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#of%2Djava.time.LocalDate%2Djava.time.LocalTime%2D[of](<>, <>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#of%2Djava.time.LocalDate%2Djava.time.LocalTime%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-of-5]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2D[of](int, int, int, int, int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-of-6]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[of](int, int, int, int, int, int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-of-7]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[of](int, int, int, int, int, int, int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-ofEpochSecond-3]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#ofEpochSecond%2Dlong%2Dint%2Djava.time.ZoneOffset%2D[ofEpochSecond](long, int, <>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#ofEpochSecond%2Dlong%2Dint%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-ofInstant-2]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[ofInstant](<>, <>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-parse-1]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-parse-2]]static <> link:{java8-javadoc}/java/time/LocalDateTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-atOffset-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#atOffset%2Djava.time.ZoneOffset%2D[atOffset](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#atOffset%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-atZone-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#atZone%2Djava.time.ZoneId%2D[atZone](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#atZone%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getDayOfMonth-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getDayOfMonth%2D%2D[getDayOfMonth]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getDayOfWeek-0]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#getDayOfWeek%2D%2D[getDayOfWeek]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getDayOfYear-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getDayOfYear%2D%2D[getDayOfYear]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getDayOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getHour-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getHour%2D%2D[getHour]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getHour%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getMinute-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getMinute%2D%2D[getMinute]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getMinute%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getMonth-0]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getMonthValue-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getMonthValue%2D%2D[getMonthValue]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getMonthValue%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getNano-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getSecond-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getSecond%2D%2D[getSecond]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getSecond%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-getYear-0]]int link:{java8-javadoc}/java/time/LocalDateTime.html#getYear%2D%2D[getYear]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#getYear%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minus-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minus-2]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusDays-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusDays%2Dlong%2D[minusDays](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusHours-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusHours%2Dlong%2D[minusHours](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusMinutes-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusMinutes%2Dlong%2D[minusMinutes](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusMonths-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusMonths%2Dlong%2D[minusMonths](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusNanos-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusSeconds-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusWeeks-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusWeeks%2Dlong%2D[minusWeeks](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-minusYears-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plus-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plus-2]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusDays-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusDays%2Dlong%2D[plusDays](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusHours-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusHours%2Dlong%2D[plusHours](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusMinutes-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusMinutes%2Dlong%2D[plusMinutes](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusMonths-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusMonths%2Dlong%2D[plusMonths](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusNanos-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusSeconds-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusWeeks-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusWeeks%2Dlong%2D[plusWeeks](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-plusYears-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-toLocalDate-0]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#toLocalDate%2D%2D[toLocalDate]()++ (link:{java9-javadoc}/java/time/LocalDateTime.html#toLocalDate%2D%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-truncatedTo-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[truncatedTo](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-with-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-with-2]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withDayOfMonth-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withDayOfMonth%2Dint%2D[withDayOfMonth](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withDayOfMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withDayOfYear-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withDayOfYear%2Dint%2D[withDayOfYear](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withDayOfYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withHour-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withHour%2Dint%2D[withHour](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withHour%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withMinute-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withMinute%2Dint%2D[withMinute](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withMinute%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withMonth-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withMonth%2Dint%2D[withMonth](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withSecond-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withSecond%2Dint%2D[withSecond](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withSecond%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalDateTime-withYear-1]]<> link:{java8-javadoc}/java/time/LocalDateTime.html#withYear%2Dint%2D[withYear](int)++ (link:{java9-javadoc}/java/time/LocalDateTime.html#withYear%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LocalTime.asciidoc b/docs/painless/painless-api-reference/LocalTime.asciidoc deleted file mode 100644 index f12e13f4e7854..0000000000000 --- a/docs/painless/painless-api-reference/LocalTime.asciidoc +++ /dev/null @@ -1,50 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LocalTime]]++LocalTime++:: -** [[painless-api-reference-LocalTime-MAX]]static <> link:{java8-javadoc}/java/time/LocalTime.html#MAX[MAX] (link:{java9-javadoc}/java/time/LocalTime.html#MAX[java 9]) -** [[painless-api-reference-LocalTime-MIDNIGHT]]static <> link:{java8-javadoc}/java/time/LocalTime.html#MIDNIGHT[MIDNIGHT] (link:{java9-javadoc}/java/time/LocalTime.html#MIDNIGHT[java 9]) -** [[painless-api-reference-LocalTime-MIN]]static <> link:{java8-javadoc}/java/time/LocalTime.html#MIN[MIN] (link:{java9-javadoc}/java/time/LocalTime.html#MIN[java 9]) -** [[painless-api-reference-LocalTime-NOON]]static <> link:{java8-javadoc}/java/time/LocalTime.html#NOON[NOON] (link:{java9-javadoc}/java/time/LocalTime.html#NOON[java 9]) -* ++[[painless-api-reference-LocalTime-from-1]]static <> link:{java8-javadoc}/java/time/LocalTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-LocalTime-of-2]]static <> link:{java8-javadoc}/java/time/LocalTime.html#of%2Dint%2Dint%2D[of](int, int)++ (link:{java9-javadoc}/java/time/LocalTime.html#of%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalTime-of-3]]static <> link:{java8-javadoc}/java/time/LocalTime.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/LocalTime.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalTime-of-4]]static <> link:{java8-javadoc}/java/time/LocalTime.html#of%2Dint%2Dint%2Dint%2Dint%2D[of](int, int, int, int)++ (link:{java9-javadoc}/java/time/LocalTime.html#of%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalTime-ofNanoOfDay-1]]static <> link:{java8-javadoc}/java/time/LocalTime.html#ofNanoOfDay%2Dlong%2D[ofNanoOfDay](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#ofNanoOfDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-ofSecondOfDay-1]]static <> link:{java8-javadoc}/java/time/LocalTime.html#ofSecondOfDay%2Dlong%2D[ofSecondOfDay](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#ofSecondOfDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-parse-1]]static <> link:{java8-javadoc}/java/time/LocalTime.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-LocalTime-parse-2]]static <> link:{java8-javadoc}/java/time/LocalTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/LocalTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-LocalTime-atDate-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#atDate%2Djava.time.LocalDate%2D[atDate](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#atDate%2Djava.time.LocalDate%2D[java 9]) -* ++[[painless-api-reference-LocalTime-atOffset-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#atOffset%2Djava.time.ZoneOffset%2D[atOffset](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#atOffset%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-LocalTime-compareTo-1]]int link:{java8-javadoc}/java/time/LocalTime.html#compareTo%2Djava.time.LocalTime%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#compareTo%2Djava.time.LocalTime%2D[java 9]) -* ++[[painless-api-reference-LocalTime-format-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-LocalTime-getHour-0]]int link:{java8-javadoc}/java/time/LocalTime.html#getHour%2D%2D[getHour]()++ (link:{java9-javadoc}/java/time/LocalTime.html#getHour%2D%2D[java 9]) -* ++[[painless-api-reference-LocalTime-getMinute-0]]int link:{java8-javadoc}/java/time/LocalTime.html#getMinute%2D%2D[getMinute]()++ (link:{java9-javadoc}/java/time/LocalTime.html#getMinute%2D%2D[java 9]) -* ++[[painless-api-reference-LocalTime-getNano-0]]int link:{java8-javadoc}/java/time/LocalTime.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/LocalTime.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-LocalTime-getSecond-0]]int link:{java8-javadoc}/java/time/LocalTime.html#getSecond%2D%2D[getSecond]()++ (link:{java9-javadoc}/java/time/LocalTime.html#getSecond%2D%2D[java 9]) -* ++[[painless-api-reference-LocalTime-isAfter-1]]boolean link:{java8-javadoc}/java/time/LocalTime.html#isAfter%2Djava.time.LocalTime%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#isAfter%2Djava.time.LocalTime%2D[java 9]) -* ++[[painless-api-reference-LocalTime-isBefore-1]]boolean link:{java8-javadoc}/java/time/LocalTime.html#isBefore%2Djava.time.LocalTime%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#isBefore%2Djava.time.LocalTime%2D[java 9]) -* ++[[painless-api-reference-LocalTime-minus-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-LocalTime-minus-2]]<> link:{java8-javadoc}/java/time/LocalTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/LocalTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalTime-minusHours-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#minusHours%2Dlong%2D[minusHours](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#minusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-minusMinutes-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#minusMinutes%2Dlong%2D[minusMinutes](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#minusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-minusNanos-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-minusSeconds-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-plus-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-LocalTime-plus-2]]<> link:{java8-javadoc}/java/time/LocalTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/LocalTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalTime-plusHours-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#plusHours%2Dlong%2D[plusHours](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#plusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-plusMinutes-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#plusMinutes%2Dlong%2D[plusMinutes](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#plusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-plusNanos-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-plusSeconds-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/LocalTime.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-toNanoOfDay-0]]long link:{java8-javadoc}/java/time/LocalTime.html#toNanoOfDay%2D%2D[toNanoOfDay]()++ (link:{java9-javadoc}/java/time/LocalTime.html#toNanoOfDay%2D%2D[java 9]) -* ++[[painless-api-reference-LocalTime-toSecondOfDay-0]]int link:{java8-javadoc}/java/time/LocalTime.html#toSecondOfDay%2D%2D[toSecondOfDay]()++ (link:{java9-javadoc}/java/time/LocalTime.html#toSecondOfDay%2D%2D[java 9]) -* ++[[painless-api-reference-LocalTime-truncatedTo-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[truncatedTo](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-LocalTime-with-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/LocalTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-LocalTime-with-2]]<> link:{java8-javadoc}/java/time/LocalTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/LocalTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LocalTime-withHour-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#withHour%2Dint%2D[withHour](int)++ (link:{java9-javadoc}/java/time/LocalTime.html#withHour%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalTime-withMinute-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#withMinute%2Dint%2D[withMinute](int)++ (link:{java9-javadoc}/java/time/LocalTime.html#withMinute%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalTime-withNano-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#withNano%2Dint%2D[withNano](int)++ (link:{java9-javadoc}/java/time/LocalTime.html#withNano%2Dint%2D[java 9]) -* ++[[painless-api-reference-LocalTime-withSecond-1]]<> link:{java8-javadoc}/java/time/LocalTime.html#withSecond%2Dint%2D[withSecond](int)++ (link:{java9-javadoc}/java/time/LocalTime.html#withSecond%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Locale.Builder.asciidoc b/docs/painless/painless-api-reference/Locale.Builder.asciidoc deleted file mode 100644 index 3677c81e60411..0000000000000 --- a/docs/painless/painless-api-reference/Locale.Builder.asciidoc +++ /dev/null @@ -1,21 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Locale-Builder]]++Locale.Builder++:: -* ++[[painless-api-reference-Locale-Builder-Locale.Builder-0]]link:{java8-javadoc}/java/util/Locale.Builder.html#Locale.Builder%2D%2D[Locale.Builder]()++ (link:{java9-javadoc}/java/util/Locale.Builder.html#Locale.Builder%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-addUnicodeLocaleAttribute-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#addUnicodeLocaleAttribute%2Djava.lang.String%2D[addUnicodeLocaleAttribute](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#addUnicodeLocaleAttribute%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-build-0]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#build%2D%2D[build]()++ (link:{java9-javadoc}/java/util/Locale.Builder.html#build%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-clear-0]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#clear%2D%2D[clear]()++ (link:{java9-javadoc}/java/util/Locale.Builder.html#clear%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-clearExtensions-0]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#clearExtensions%2D%2D[clearExtensions]()++ (link:{java9-javadoc}/java/util/Locale.Builder.html#clearExtensions%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-removeUnicodeLocaleAttribute-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#removeUnicodeLocaleAttribute%2Djava.lang.String%2D[removeUnicodeLocaleAttribute](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#removeUnicodeLocaleAttribute%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setExtension-2]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setExtension%2Dchar%2Djava.lang.String%2D[setExtension](char, <>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setExtension%2Dchar%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setLanguage-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setLanguage%2Djava.lang.String%2D[setLanguage](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setLanguage%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setLanguageTag-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setLanguageTag%2Djava.lang.String%2D[setLanguageTag](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setLanguageTag%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setLocale-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setLocale%2Djava.util.Locale%2D[setLocale](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setLocale%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setRegion-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setRegion%2Djava.lang.String%2D[setRegion](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setRegion%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setScript-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setScript%2Djava.lang.String%2D[setScript](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setScript%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setUnicodeLocaleKeyword-2]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setUnicodeLocaleKeyword%2Djava.lang.String%2Djava.lang.String%2D[setUnicodeLocaleKeyword](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setUnicodeLocaleKeyword%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Builder-setVariant-1]]<> link:{java8-javadoc}/java/util/Locale.Builder.html#setVariant%2Djava.lang.String%2D[setVariant](<>)++ (link:{java9-javadoc}/java/util/Locale.Builder.html#setVariant%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Locale.Category.asciidoc b/docs/painless/painless-api-reference/Locale.Category.asciidoc deleted file mode 100644 index 96b1cd3fd1011..0000000000000 --- a/docs/painless/painless-api-reference/Locale.Category.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Locale-Category]]++Locale.Category++:: -** [[painless-api-reference-Locale-Category-DISPLAY]]static <> link:{java8-javadoc}/java/util/Locale.Category.html#DISPLAY[DISPLAY] (link:{java9-javadoc}/java/util/Locale.Category.html#DISPLAY[java 9]) -** [[painless-api-reference-Locale-Category-FORMAT]]static <> link:{java8-javadoc}/java/util/Locale.Category.html#FORMAT[FORMAT] (link:{java9-javadoc}/java/util/Locale.Category.html#FORMAT[java 9]) -* ++[[painless-api-reference-Locale-Category-valueOf-1]]static <> link:{java8-javadoc}/java/util/Locale.Category.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/util/Locale.Category.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Category-values-0]]static <>[] link:{java8-javadoc}/java/util/Locale.Category.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/util/Locale.Category.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Locale.FilteringMode.asciidoc b/docs/painless/painless-api-reference/Locale.FilteringMode.asciidoc deleted file mode 100644 index f513e4c239668..0000000000000 --- a/docs/painless/painless-api-reference/Locale.FilteringMode.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Locale-FilteringMode]]++Locale.FilteringMode++:: -** [[painless-api-reference-Locale-FilteringMode-AUTOSELECT_FILTERING]]static <> link:{java8-javadoc}/java/util/Locale.FilteringMode.html#AUTOSELECT_FILTERING[AUTOSELECT_FILTERING] (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#AUTOSELECT_FILTERING[java 9]) -** [[painless-api-reference-Locale-FilteringMode-EXTENDED_FILTERING]]static <> link:{java8-javadoc}/java/util/Locale.FilteringMode.html#EXTENDED_FILTERING[EXTENDED_FILTERING] (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#EXTENDED_FILTERING[java 9]) -** [[painless-api-reference-Locale-FilteringMode-IGNORE_EXTENDED_RANGES]]static <> link:{java8-javadoc}/java/util/Locale.FilteringMode.html#IGNORE_EXTENDED_RANGES[IGNORE_EXTENDED_RANGES] (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#IGNORE_EXTENDED_RANGES[java 9]) -** [[painless-api-reference-Locale-FilteringMode-MAP_EXTENDED_RANGES]]static <> link:{java8-javadoc}/java/util/Locale.FilteringMode.html#MAP_EXTENDED_RANGES[MAP_EXTENDED_RANGES] (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#MAP_EXTENDED_RANGES[java 9]) -** [[painless-api-reference-Locale-FilteringMode-REJECT_EXTENDED_RANGES]]static <> link:{java8-javadoc}/java/util/Locale.FilteringMode.html#REJECT_EXTENDED_RANGES[REJECT_EXTENDED_RANGES] (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#REJECT_EXTENDED_RANGES[java 9]) -* ++[[painless-api-reference-Locale-FilteringMode-valueOf-1]]static <> link:{java8-javadoc}/java/util/Locale.FilteringMode.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-FilteringMode-values-0]]static <>[] link:{java8-javadoc}/java/util/Locale.FilteringMode.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/util/Locale.FilteringMode.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Locale.LanguageRange.asciidoc b/docs/painless/painless-api-reference/Locale.LanguageRange.asciidoc deleted file mode 100644 index a0a76b3282021..0000000000000 --- a/docs/painless/painless-api-reference/Locale.LanguageRange.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Locale-LanguageRange]]++Locale.LanguageRange++:: -** [[painless-api-reference-Locale-LanguageRange-MAX_WEIGHT]]static double link:{java8-javadoc}/java/util/Locale.LanguageRange.html#MAX_WEIGHT[MAX_WEIGHT] (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#MAX_WEIGHT[java 9]) -** [[painless-api-reference-Locale-LanguageRange-MIN_WEIGHT]]static double link:{java8-javadoc}/java/util/Locale.LanguageRange.html#MIN_WEIGHT[MIN_WEIGHT] (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#MIN_WEIGHT[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-mapEquivalents-2]]static <> link:{java8-javadoc}/java/util/Locale.LanguageRange.html#mapEquivalents%2Djava.util.List%2Djava.util.Map%2D[mapEquivalents](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#mapEquivalents%2Djava.util.List%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-parse-1]]static <> link:{java8-javadoc}/java/util/Locale.LanguageRange.html#parse%2Djava.lang.String%2D[parse](<>)++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#parse%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-parse-2]]static <> link:{java8-javadoc}/java/util/Locale.LanguageRange.html#parse%2Djava.lang.String%2Djava.util.Map%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#parse%2Djava.lang.String%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-Locale.LanguageRange-1]]link:{java8-javadoc}/java/util/Locale.LanguageRange.html#Locale.LanguageRange%2Djava.lang.String%2D[Locale.LanguageRange](<>)++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#Locale.LanguageRange%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-Locale.LanguageRange-2]]link:{java8-javadoc}/java/util/Locale.LanguageRange.html#Locale.LanguageRange%2Djava.lang.String%2Ddouble%2D[Locale.LanguageRange](<>, double)++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#Locale.LanguageRange%2Djava.lang.String%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-getRange-0]]<> link:{java8-javadoc}/java/util/Locale.LanguageRange.html#getRange%2D%2D[getRange]()++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#getRange%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-LanguageRange-getWeight-0]]double link:{java8-javadoc}/java/util/Locale.LanguageRange.html#getWeight%2D%2D[getWeight]()++ (link:{java9-javadoc}/java/util/Locale.LanguageRange.html#getWeight%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Locale.asciidoc b/docs/painless/painless-api-reference/Locale.asciidoc deleted file mode 100644 index d041c6c1f7fb3..0000000000000 --- a/docs/painless/painless-api-reference/Locale.asciidoc +++ /dev/null @@ -1,69 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Locale]]++Locale++:: -** [[painless-api-reference-Locale-CANADA]]static <> link:{java8-javadoc}/java/util/Locale.html#CANADA[CANADA] (link:{java9-javadoc}/java/util/Locale.html#CANADA[java 9]) -** [[painless-api-reference-Locale-CANADA_FRENCH]]static <> link:{java8-javadoc}/java/util/Locale.html#CANADA_FRENCH[CANADA_FRENCH] (link:{java9-javadoc}/java/util/Locale.html#CANADA_FRENCH[java 9]) -** [[painless-api-reference-Locale-CHINA]]static <> link:{java8-javadoc}/java/util/Locale.html#CHINA[CHINA] (link:{java9-javadoc}/java/util/Locale.html#CHINA[java 9]) -** [[painless-api-reference-Locale-CHINESE]]static <> link:{java8-javadoc}/java/util/Locale.html#CHINESE[CHINESE] (link:{java9-javadoc}/java/util/Locale.html#CHINESE[java 9]) -** [[painless-api-reference-Locale-ENGLISH]]static <> link:{java8-javadoc}/java/util/Locale.html#ENGLISH[ENGLISH] (link:{java9-javadoc}/java/util/Locale.html#ENGLISH[java 9]) -** [[painless-api-reference-Locale-FRANCE]]static <> link:{java8-javadoc}/java/util/Locale.html#FRANCE[FRANCE] (link:{java9-javadoc}/java/util/Locale.html#FRANCE[java 9]) -** [[painless-api-reference-Locale-FRENCH]]static <> link:{java8-javadoc}/java/util/Locale.html#FRENCH[FRENCH] (link:{java9-javadoc}/java/util/Locale.html#FRENCH[java 9]) -** [[painless-api-reference-Locale-GERMAN]]static <> link:{java8-javadoc}/java/util/Locale.html#GERMAN[GERMAN] (link:{java9-javadoc}/java/util/Locale.html#GERMAN[java 9]) -** [[painless-api-reference-Locale-GERMANY]]static <> link:{java8-javadoc}/java/util/Locale.html#GERMANY[GERMANY] (link:{java9-javadoc}/java/util/Locale.html#GERMANY[java 9]) -** [[painless-api-reference-Locale-ITALIAN]]static <> link:{java8-javadoc}/java/util/Locale.html#ITALIAN[ITALIAN] (link:{java9-javadoc}/java/util/Locale.html#ITALIAN[java 9]) -** [[painless-api-reference-Locale-ITALY]]static <> link:{java8-javadoc}/java/util/Locale.html#ITALY[ITALY] (link:{java9-javadoc}/java/util/Locale.html#ITALY[java 9]) -** [[painless-api-reference-Locale-JAPAN]]static <> link:{java8-javadoc}/java/util/Locale.html#JAPAN[JAPAN] (link:{java9-javadoc}/java/util/Locale.html#JAPAN[java 9]) -** [[painless-api-reference-Locale-JAPANESE]]static <> link:{java8-javadoc}/java/util/Locale.html#JAPANESE[JAPANESE] (link:{java9-javadoc}/java/util/Locale.html#JAPANESE[java 9]) -** [[painless-api-reference-Locale-KOREA]]static <> link:{java8-javadoc}/java/util/Locale.html#KOREA[KOREA] (link:{java9-javadoc}/java/util/Locale.html#KOREA[java 9]) -** [[painless-api-reference-Locale-KOREAN]]static <> link:{java8-javadoc}/java/util/Locale.html#KOREAN[KOREAN] (link:{java9-javadoc}/java/util/Locale.html#KOREAN[java 9]) -** [[painless-api-reference-Locale-PRC]]static <> link:{java8-javadoc}/java/util/Locale.html#PRC[PRC] (link:{java9-javadoc}/java/util/Locale.html#PRC[java 9]) -** [[painless-api-reference-Locale-PRIVATE_USE_EXTENSION]]static char link:{java8-javadoc}/java/util/Locale.html#PRIVATE_USE_EXTENSION[PRIVATE_USE_EXTENSION] (link:{java9-javadoc}/java/util/Locale.html#PRIVATE_USE_EXTENSION[java 9]) -** [[painless-api-reference-Locale-ROOT]]static <> link:{java8-javadoc}/java/util/Locale.html#ROOT[ROOT] (link:{java9-javadoc}/java/util/Locale.html#ROOT[java 9]) -** [[painless-api-reference-Locale-SIMPLIFIED_CHINESE]]static <> link:{java8-javadoc}/java/util/Locale.html#SIMPLIFIED_CHINESE[SIMPLIFIED_CHINESE] (link:{java9-javadoc}/java/util/Locale.html#SIMPLIFIED_CHINESE[java 9]) -** [[painless-api-reference-Locale-TAIWAN]]static <> link:{java8-javadoc}/java/util/Locale.html#TAIWAN[TAIWAN] (link:{java9-javadoc}/java/util/Locale.html#TAIWAN[java 9]) -** [[painless-api-reference-Locale-TRADITIONAL_CHINESE]]static <> link:{java8-javadoc}/java/util/Locale.html#TRADITIONAL_CHINESE[TRADITIONAL_CHINESE] (link:{java9-javadoc}/java/util/Locale.html#TRADITIONAL_CHINESE[java 9]) -** [[painless-api-reference-Locale-UK]]static <> link:{java8-javadoc}/java/util/Locale.html#UK[UK] (link:{java9-javadoc}/java/util/Locale.html#UK[java 9]) -** [[painless-api-reference-Locale-UNICODE_LOCALE_EXTENSION]]static char link:{java8-javadoc}/java/util/Locale.html#UNICODE_LOCALE_EXTENSION[UNICODE_LOCALE_EXTENSION] (link:{java9-javadoc}/java/util/Locale.html#UNICODE_LOCALE_EXTENSION[java 9]) -** [[painless-api-reference-Locale-US]]static <> link:{java8-javadoc}/java/util/Locale.html#US[US] (link:{java9-javadoc}/java/util/Locale.html#US[java 9]) -* ++[[painless-api-reference-Locale-filter-2]]static <> link:{java8-javadoc}/java/util/Locale.html#filter%2Djava.util.List%2Djava.util.Collection%2D[filter](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.html#filter%2Djava.util.List%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Locale-filterTags-2]]static <> link:{java8-javadoc}/java/util/Locale.html#filterTags%2Djava.util.List%2Djava.util.Collection%2D[filterTags](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.html#filterTags%2Djava.util.List%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Locale-forLanguageTag-1]]static <> link:{java8-javadoc}/java/util/Locale.html#forLanguageTag%2Djava.lang.String%2D[forLanguageTag](<>)++ (link:{java9-javadoc}/java/util/Locale.html#forLanguageTag%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/util/Locale.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/util/Locale.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDefault-0]]static <> link:{java8-javadoc}/java/util/Locale.html#getDefault%2D%2D[getDefault]()++ (link:{java9-javadoc}/java/util/Locale.html#getDefault%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDefault-1]]static <> link:{java8-javadoc}/java/util/Locale.html#getDefault%2Djava.util.Locale$Category%2D[getDefault](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getDefault%2Djava.util.Locale$Category%2D[java 9]) -* ++[[painless-api-reference-Locale-getISOCountries-0]]static <>[] link:{java8-javadoc}/java/util/Locale.html#getISOCountries%2D%2D[getISOCountries]()++ (link:{java9-javadoc}/java/util/Locale.html#getISOCountries%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getISOLanguages-0]]static <>[] link:{java8-javadoc}/java/util/Locale.html#getISOLanguages%2D%2D[getISOLanguages]()++ (link:{java9-javadoc}/java/util/Locale.html#getISOLanguages%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-lookup-2]]static <> link:{java8-javadoc}/java/util/Locale.html#lookup%2Djava.util.List%2Djava.util.Collection%2D[lookup](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.html#lookup%2Djava.util.List%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Locale-lookupTag-2]]static <> link:{java8-javadoc}/java/util/Locale.html#lookupTag%2Djava.util.List%2Djava.util.Collection%2D[lookupTag](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.html#lookupTag%2Djava.util.List%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Locale-Locale-1]]link:{java8-javadoc}/java/util/Locale.html#Locale%2Djava.lang.String%2D[Locale](<>)++ (link:{java9-javadoc}/java/util/Locale.html#Locale%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Locale-2]]link:{java8-javadoc}/java/util/Locale.html#Locale%2Djava.lang.String%2Djava.lang.String%2D[Locale](<>, <>)++ (link:{java9-javadoc}/java/util/Locale.html#Locale%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-Locale-3]]link:{java8-javadoc}/java/util/Locale.html#Locale%2Djava.lang.String%2Djava.lang.String%2Djava.lang.String%2D[Locale](<>, <>, <>)++ (link:{java9-javadoc}/java/util/Locale.html#Locale%2Djava.lang.String%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-clone-0]]def link:{java8-javadoc}/java/util/Locale.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/Locale.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getCountry-0]]<> link:{java8-javadoc}/java/util/Locale.html#getCountry%2D%2D[getCountry]()++ (link:{java9-javadoc}/java/util/Locale.html#getCountry%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayCountry-0]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayCountry%2D%2D[getDisplayCountry]()++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayCountry%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayCountry-1]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayCountry%2Djava.util.Locale%2D[getDisplayCountry](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayCountry%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayLanguage-0]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayLanguage%2D%2D[getDisplayLanguage]()++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayLanguage%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayLanguage-1]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayLanguage%2Djava.util.Locale%2D[getDisplayLanguage](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayLanguage%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayName-0]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayName%2D%2D[getDisplayName]()++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayName%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayName-1]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayName%2Djava.util.Locale%2D[getDisplayName](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayName%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayScript-0]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayScript%2D%2D[getDisplayScript]()++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayScript%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayScript-1]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayScript%2Djava.util.Locale%2D[getDisplayScript](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayScript%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayVariant-0]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayVariant%2D%2D[getDisplayVariant]()++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayVariant%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getDisplayVariant-1]]<> link:{java8-javadoc}/java/util/Locale.html#getDisplayVariant%2Djava.util.Locale%2D[getDisplayVariant](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getDisplayVariant%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Locale-getExtension-1]]<> link:{java8-javadoc}/java/util/Locale.html#getExtension%2Dchar%2D[getExtension](char)++ (link:{java9-javadoc}/java/util/Locale.html#getExtension%2Dchar%2D[java 9]) -* ++[[painless-api-reference-Locale-getExtensionKeys-0]]<> link:{java8-javadoc}/java/util/Locale.html#getExtensionKeys%2D%2D[getExtensionKeys]()++ (link:{java9-javadoc}/java/util/Locale.html#getExtensionKeys%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getISO3Country-0]]<> link:{java8-javadoc}/java/util/Locale.html#getISO3Country%2D%2D[getISO3Country]()++ (link:{java9-javadoc}/java/util/Locale.html#getISO3Country%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getISO3Language-0]]<> link:{java8-javadoc}/java/util/Locale.html#getISO3Language%2D%2D[getISO3Language]()++ (link:{java9-javadoc}/java/util/Locale.html#getISO3Language%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getLanguage-0]]<> link:{java8-javadoc}/java/util/Locale.html#getLanguage%2D%2D[getLanguage]()++ (link:{java9-javadoc}/java/util/Locale.html#getLanguage%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getScript-0]]<> link:{java8-javadoc}/java/util/Locale.html#getScript%2D%2D[getScript]()++ (link:{java9-javadoc}/java/util/Locale.html#getScript%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getUnicodeLocaleAttributes-0]]<> link:{java8-javadoc}/java/util/Locale.html#getUnicodeLocaleAttributes%2D%2D[getUnicodeLocaleAttributes]()++ (link:{java9-javadoc}/java/util/Locale.html#getUnicodeLocaleAttributes%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getUnicodeLocaleKeys-0]]<> link:{java8-javadoc}/java/util/Locale.html#getUnicodeLocaleKeys%2D%2D[getUnicodeLocaleKeys]()++ (link:{java9-javadoc}/java/util/Locale.html#getUnicodeLocaleKeys%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-getUnicodeLocaleType-1]]<> link:{java8-javadoc}/java/util/Locale.html#getUnicodeLocaleType%2Djava.lang.String%2D[getUnicodeLocaleType](<>)++ (link:{java9-javadoc}/java/util/Locale.html#getUnicodeLocaleType%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Locale-getVariant-0]]<> link:{java8-javadoc}/java/util/Locale.html#getVariant%2D%2D[getVariant]()++ (link:{java9-javadoc}/java/util/Locale.html#getVariant%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-hasExtensions-0]]boolean link:{java8-javadoc}/java/util/Locale.html#hasExtensions%2D%2D[hasExtensions]()++ (link:{java9-javadoc}/java/util/Locale.html#hasExtensions%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-stripExtensions-0]]<> link:{java8-javadoc}/java/util/Locale.html#stripExtensions%2D%2D[stripExtensions]()++ (link:{java9-javadoc}/java/util/Locale.html#stripExtensions%2D%2D[java 9]) -* ++[[painless-api-reference-Locale-toLanguageTag-0]]<> link:{java8-javadoc}/java/util/Locale.html#toLanguageTag%2D%2D[toLanguageTag]()++ (link:{java9-javadoc}/java/util/Locale.html#toLanguageTag%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Long.asciidoc b/docs/painless/painless-api-reference/Long.asciidoc deleted file mode 100644 index e35c80f63b96d..0000000000000 --- a/docs/painless/painless-api-reference/Long.asciidoc +++ /dev/null @@ -1,44 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Long]]++Long++:: -** [[painless-api-reference-Long-BYTES]]static int link:{java8-javadoc}/java/lang/Long.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Long.html#BYTES[java 9]) -** [[painless-api-reference-Long-MAX_VALUE]]static long link:{java8-javadoc}/java/lang/Long.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Long.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Long-MIN_VALUE]]static long link:{java8-javadoc}/java/lang/Long.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Long.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Long-SIZE]]static int link:{java8-javadoc}/java/lang/Long.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Long.html#SIZE[java 9]) -* ++[[painless-api-reference-Long-bitCount-1]]static int link:{java8-javadoc}/java/lang/Long.html#bitCount%2Dlong%2D[bitCount](long)++ (link:{java9-javadoc}/java/lang/Long.html#bitCount%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-compare-2]]static int link:{java8-javadoc}/java/lang/Long.html#compare%2Dlong%2Dlong%2D[compare](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#compare%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-compareUnsigned-2]]static int link:{java8-javadoc}/java/lang/Long.html#compareUnsigned%2Dlong%2Dlong%2D[compareUnsigned](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#compareUnsigned%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-decode-1]]static <> link:{java8-javadoc}/java/lang/Long.html#decode%2Djava.lang.String%2D[decode](<>)++ (link:{java9-javadoc}/java/lang/Long.html#decode%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Long-divideUnsigned-2]]static long link:{java8-javadoc}/java/lang/Long.html#divideUnsigned%2Dlong%2Dlong%2D[divideUnsigned](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#divideUnsigned%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-hashCode-1]]static int link:{java8-javadoc}/java/lang/Long.html#hashCode%2Dlong%2D[hashCode](long)++ (link:{java9-javadoc}/java/lang/Long.html#hashCode%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-highestOneBit-1]]static long link:{java8-javadoc}/java/lang/Long.html#highestOneBit%2Dlong%2D[highestOneBit](long)++ (link:{java9-javadoc}/java/lang/Long.html#highestOneBit%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-lowestOneBit-1]]static long link:{java8-javadoc}/java/lang/Long.html#lowestOneBit%2Dlong%2D[lowestOneBit](long)++ (link:{java9-javadoc}/java/lang/Long.html#lowestOneBit%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-max-2]]static long link:{java8-javadoc}/java/lang/Long.html#max%2Dlong%2Dlong%2D[max](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#max%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-min-2]]static long link:{java8-javadoc}/java/lang/Long.html#min%2Dlong%2Dlong%2D[min](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#min%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-numberOfLeadingZeros-1]]static int link:{java8-javadoc}/java/lang/Long.html#numberOfLeadingZeros%2Dlong%2D[numberOfLeadingZeros](long)++ (link:{java9-javadoc}/java/lang/Long.html#numberOfLeadingZeros%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-numberOfTrailingZeros-1]]static int link:{java8-javadoc}/java/lang/Long.html#numberOfTrailingZeros%2Dlong%2D[numberOfTrailingZeros](long)++ (link:{java9-javadoc}/java/lang/Long.html#numberOfTrailingZeros%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-parseLong-1]]static long link:{java8-javadoc}/java/lang/Long.html#parseLong%2Djava.lang.String%2D[parseLong](<>)++ (link:{java9-javadoc}/java/lang/Long.html#parseLong%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Long-parseLong-2]]static long link:{java8-javadoc}/java/lang/Long.html#parseLong%2Djava.lang.String%2Dint%2D[parseLong](<>, int)++ (link:{java9-javadoc}/java/lang/Long.html#parseLong%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-parseUnsignedLong-1]]static long link:{java8-javadoc}/java/lang/Long.html#parseUnsignedLong%2Djava.lang.String%2D[parseUnsignedLong](<>)++ (link:{java9-javadoc}/java/lang/Long.html#parseUnsignedLong%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Long-parseUnsignedLong-2]]static long link:{java8-javadoc}/java/lang/Long.html#parseUnsignedLong%2Djava.lang.String%2Dint%2D[parseUnsignedLong](<>, int)++ (link:{java9-javadoc}/java/lang/Long.html#parseUnsignedLong%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-remainderUnsigned-2]]static long link:{java8-javadoc}/java/lang/Long.html#remainderUnsigned%2Dlong%2Dlong%2D[remainderUnsigned](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#remainderUnsigned%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-reverse-1]]static long link:{java8-javadoc}/java/lang/Long.html#reverse%2Dlong%2D[reverse](long)++ (link:{java9-javadoc}/java/lang/Long.html#reverse%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-reverseBytes-1]]static long link:{java8-javadoc}/java/lang/Long.html#reverseBytes%2Dlong%2D[reverseBytes](long)++ (link:{java9-javadoc}/java/lang/Long.html#reverseBytes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-rotateLeft-2]]static long link:{java8-javadoc}/java/lang/Long.html#rotateLeft%2Dlong%2Dint%2D[rotateLeft](long, int)++ (link:{java9-javadoc}/java/lang/Long.html#rotateLeft%2Dlong%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-rotateRight-2]]static long link:{java8-javadoc}/java/lang/Long.html#rotateRight%2Dlong%2Dint%2D[rotateRight](long, int)++ (link:{java9-javadoc}/java/lang/Long.html#rotateRight%2Dlong%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-signum-1]]static int link:{java8-javadoc}/java/lang/Long.html#signum%2Dlong%2D[signum](long)++ (link:{java9-javadoc}/java/lang/Long.html#signum%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-sum-2]]static long link:{java8-javadoc}/java/lang/Long.html#sum%2Dlong%2Dlong%2D[sum](long, long)++ (link:{java9-javadoc}/java/lang/Long.html#sum%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-toBinaryString-1]]static <> link:{java8-javadoc}/java/lang/Long.html#toBinaryString%2Dlong%2D[toBinaryString](long)++ (link:{java9-javadoc}/java/lang/Long.html#toBinaryString%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-toHexString-1]]static <> link:{java8-javadoc}/java/lang/Long.html#toHexString%2Dlong%2D[toHexString](long)++ (link:{java9-javadoc}/java/lang/Long.html#toHexString%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-toOctalString-1]]static <> link:{java8-javadoc}/java/lang/Long.html#toOctalString%2Dlong%2D[toOctalString](long)++ (link:{java9-javadoc}/java/lang/Long.html#toOctalString%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-toString-1]]static <> link:{java8-javadoc}/java/lang/Long.html#toString%2Dlong%2D[toString](long)++ (link:{java9-javadoc}/java/lang/Long.html#toString%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-toString-2]]static <> link:{java8-javadoc}/java/lang/Long.html#toString%2Dlong%2Dint%2D[toString](long, int)++ (link:{java9-javadoc}/java/lang/Long.html#toString%2Dlong%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-toUnsignedString-1]]static <> link:{java8-javadoc}/java/lang/Long.html#toUnsignedString%2Dlong%2D[toUnsignedString](long)++ (link:{java9-javadoc}/java/lang/Long.html#toUnsignedString%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-toUnsignedString-2]]static <> link:{java8-javadoc}/java/lang/Long.html#toUnsignedString%2Dlong%2Dint%2D[toUnsignedString](long, int)++ (link:{java9-javadoc}/java/lang/Long.html#toUnsignedString%2Dlong%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Long.html#valueOf%2Dlong%2D[valueOf](long)++ (link:{java9-javadoc}/java/lang/Long.html#valueOf%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Long-valueOf-2]]static <> link:{java8-javadoc}/java/lang/Long.html#valueOf%2Djava.lang.String%2Dint%2D[valueOf](<>, int)++ (link:{java9-javadoc}/java/lang/Long.html#valueOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Long-compareTo-1]]int link:{java8-javadoc}/java/lang/Long.html#compareTo%2Djava.lang.Long%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Long.html#compareTo%2Djava.lang.Long%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LongBinaryOperator.asciidoc b/docs/painless/painless-api-reference/LongBinaryOperator.asciidoc deleted file mode 100644 index 45a777a69715b..0000000000000 --- a/docs/painless/painless-api-reference/LongBinaryOperator.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongBinaryOperator]]++LongBinaryOperator++:: -* ++[[painless-api-reference-LongBinaryOperator-applyAsLong-2]]long link:{java8-javadoc}/java/util/function/LongBinaryOperator.html#applyAsLong%2Dlong%2Dlong%2D[applyAsLong](long, long)++ (link:{java9-javadoc}/java/util/function/LongBinaryOperator.html#applyAsLong%2Dlong%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongConsumer.asciidoc b/docs/painless/painless-api-reference/LongConsumer.asciidoc deleted file mode 100644 index 28bbc92c03121..0000000000000 --- a/docs/painless/painless-api-reference/LongConsumer.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongConsumer]]++LongConsumer++:: -* ++[[painless-api-reference-LongConsumer-accept-1]]void link:{java8-javadoc}/java/util/function/LongConsumer.html#accept%2Dlong%2D[accept](long)++ (link:{java9-javadoc}/java/util/function/LongConsumer.html#accept%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongConsumer-andThen-1]]<> link:{java8-javadoc}/java/util/function/LongConsumer.html#andThen%2Djava.util.function.LongConsumer%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/LongConsumer.html#andThen%2Djava.util.function.LongConsumer%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongFunction.asciidoc b/docs/painless/painless-api-reference/LongFunction.asciidoc deleted file mode 100644 index 033392914a5e3..0000000000000 --- a/docs/painless/painless-api-reference/LongFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongFunction]]++LongFunction++:: -* ++[[painless-api-reference-LongFunction-apply-1]]def link:{java8-javadoc}/java/util/function/LongFunction.html#apply%2Dlong%2D[apply](long)++ (link:{java9-javadoc}/java/util/function/LongFunction.html#apply%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongPredicate.asciidoc b/docs/painless/painless-api-reference/LongPredicate.asciidoc deleted file mode 100644 index 59b719572e3d1..0000000000000 --- a/docs/painless/painless-api-reference/LongPredicate.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongPredicate]]++LongPredicate++:: -* ++[[painless-api-reference-LongPredicate-and-1]]<> link:{java8-javadoc}/java/util/function/LongPredicate.html#and%2Djava.util.function.LongPredicate%2D[and](<>)++ (link:{java9-javadoc}/java/util/function/LongPredicate.html#and%2Djava.util.function.LongPredicate%2D[java 9]) -* ++[[painless-api-reference-LongPredicate-negate-0]]<> link:{java8-javadoc}/java/util/function/LongPredicate.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/util/function/LongPredicate.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-LongPredicate-or-1]]<> link:{java8-javadoc}/java/util/function/LongPredicate.html#or%2Djava.util.function.LongPredicate%2D[or](<>)++ (link:{java9-javadoc}/java/util/function/LongPredicate.html#or%2Djava.util.function.LongPredicate%2D[java 9]) -* ++[[painless-api-reference-LongPredicate-test-1]]boolean link:{java8-javadoc}/java/util/function/LongPredicate.html#test%2Dlong%2D[test](long)++ (link:{java9-javadoc}/java/util/function/LongPredicate.html#test%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongStream.Builder.asciidoc b/docs/painless/painless-api-reference/LongStream.Builder.asciidoc deleted file mode 100644 index a5d8d0a874cb0..0000000000000 --- a/docs/painless/painless-api-reference/LongStream.Builder.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongStream-Builder]]++LongStream.Builder++:: -* ++[[painless-api-reference-LongStream-Builder-add-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.Builder.html#add%2Dlong%2D[add](long)++ (link:{java9-javadoc}/java/util/stream/LongStream.Builder.html#add%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongStream-Builder-build-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.Builder.html#build%2D%2D[build]()++ (link:{java9-javadoc}/java/util/stream/LongStream.Builder.html#build%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LongStream.asciidoc b/docs/painless/painless-api-reference/LongStream.asciidoc deleted file mode 100644 index 08ea178451a0a..0000000000000 --- a/docs/painless/painless-api-reference/LongStream.asciidoc +++ /dev/null @@ -1,46 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongStream]]++LongStream++:: -* ++[[painless-api-reference-LongStream-builder-0]]static <> link:{java8-javadoc}/java/util/stream/LongStream.html#builder%2D%2D[builder]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#builder%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-concat-2]]static <> link:{java8-javadoc}/java/util/stream/LongStream.html#concat%2Djava.util.stream.LongStream%2Djava.util.stream.LongStream%2D[concat](<>, <>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#concat%2Djava.util.stream.LongStream%2Djava.util.stream.LongStream%2D[java 9]) -* ++[[painless-api-reference-LongStream-empty-0]]static <> link:{java8-javadoc}/java/util/stream/LongStream.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-of-1]]static <> link:{java8-javadoc}/java/util/stream/LongStream.html#of%2Dlong:A%2D[of](long[])++ (link:{java9-javadoc}/java/util/stream/LongStream.html#of%2Dlong:A%2D[java 9]) -* ++[[painless-api-reference-LongStream-range-2]]static <> link:{java8-javadoc}/java/util/stream/LongStream.html#range%2Dlong%2Dlong%2D[range](long, long)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#range%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongStream-rangeClosed-2]]static <> link:{java8-javadoc}/java/util/stream/LongStream.html#rangeClosed%2Dlong%2Dlong%2D[rangeClosed](long, long)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#rangeClosed%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongStream-allMatch-1]]boolean link:{java8-javadoc}/java/util/stream/LongStream.html#allMatch%2Djava.util.function.LongPredicate%2D[allMatch](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#allMatch%2Djava.util.function.LongPredicate%2D[java 9]) -* ++[[painless-api-reference-LongStream-anyMatch-1]]boolean link:{java8-javadoc}/java/util/stream/LongStream.html#anyMatch%2Djava.util.function.LongPredicate%2D[anyMatch](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#anyMatch%2Djava.util.function.LongPredicate%2D[java 9]) -* ++[[painless-api-reference-LongStream-asDoubleStream-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#asDoubleStream%2D%2D[asDoubleStream]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#asDoubleStream%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-average-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#average%2D%2D[average]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#average%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-boxed-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#boxed%2D%2D[boxed]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#boxed%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-collect-3]]def link:{java8-javadoc}/java/util/stream/LongStream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.ObjLongConsumer%2Djava.util.function.BiConsumer%2D[collect](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.ObjLongConsumer%2Djava.util.function.BiConsumer%2D[java 9]) -* ++[[painless-api-reference-LongStream-count-0]]long link:{java8-javadoc}/java/util/stream/LongStream.html#count%2D%2D[count]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#count%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-distinct-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#distinct%2D%2D[distinct]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#distinct%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-filter-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#filter%2Djava.util.function.LongPredicate%2D[filter](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#filter%2Djava.util.function.LongPredicate%2D[java 9]) -* ++[[painless-api-reference-LongStream-findAny-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#findAny%2D%2D[findAny]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#findAny%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-findFirst-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#findFirst%2D%2D[findFirst]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#findFirst%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-flatMap-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#flatMap%2Djava.util.function.LongFunction%2D[flatMap](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#flatMap%2Djava.util.function.LongFunction%2D[java 9]) -* ++[[painless-api-reference-LongStream-forEach-1]]void link:{java8-javadoc}/java/util/stream/LongStream.html#forEach%2Djava.util.function.LongConsumer%2D[forEach](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#forEach%2Djava.util.function.LongConsumer%2D[java 9]) -* ++[[painless-api-reference-LongStream-forEachOrdered-1]]void link:{java8-javadoc}/java/util/stream/LongStream.html#forEachOrdered%2Djava.util.function.LongConsumer%2D[forEachOrdered](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#forEachOrdered%2Djava.util.function.LongConsumer%2D[java 9]) -* ++[[painless-api-reference-LongStream-iterator-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#iterator%2D%2D[iterator]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#iterator%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-limit-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#limit%2Dlong%2D[limit](long)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#limit%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongStream-map-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#map%2Djava.util.function.LongUnaryOperator%2D[map](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#map%2Djava.util.function.LongUnaryOperator%2D[java 9]) -* ++[[painless-api-reference-LongStream-mapToDouble-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#mapToDouble%2Djava.util.function.LongToDoubleFunction%2D[mapToDouble](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#mapToDouble%2Djava.util.function.LongToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-LongStream-mapToInt-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#mapToInt%2Djava.util.function.LongToIntFunction%2D[mapToInt](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#mapToInt%2Djava.util.function.LongToIntFunction%2D[java 9]) -* ++[[painless-api-reference-LongStream-mapToObj-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#mapToObj%2Djava.util.function.LongFunction%2D[mapToObj](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#mapToObj%2Djava.util.function.LongFunction%2D[java 9]) -* ++[[painless-api-reference-LongStream-max-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#max%2D%2D[max]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#max%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-min-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#min%2D%2D[min]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#min%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-noneMatch-1]]boolean link:{java8-javadoc}/java/util/stream/LongStream.html#noneMatch%2Djava.util.function.LongPredicate%2D[noneMatch](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#noneMatch%2Djava.util.function.LongPredicate%2D[java 9]) -* ++[[painless-api-reference-LongStream-peek-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#peek%2Djava.util.function.LongConsumer%2D[peek](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#peek%2Djava.util.function.LongConsumer%2D[java 9]) -* ++[[painless-api-reference-LongStream-reduce-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#reduce%2Djava.util.function.LongBinaryOperator%2D[reduce](<>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#reduce%2Djava.util.function.LongBinaryOperator%2D[java 9]) -* ++[[painless-api-reference-LongStream-reduce-2]]long link:{java8-javadoc}/java/util/stream/LongStream.html#reduce%2Dlong%2Djava.util.function.LongBinaryOperator%2D[reduce](long, <>)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#reduce%2Dlong%2Djava.util.function.LongBinaryOperator%2D[java 9]) -* ++[[painless-api-reference-LongStream-sequential-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#sequential%2D%2D[sequential]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#sequential%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-skip-1]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#skip%2Dlong%2D[skip](long)++ (link:{java9-javadoc}/java/util/stream/LongStream.html#skip%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongStream-sorted-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#sorted%2D%2D[sorted]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#sorted%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-spliterator-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#spliterator%2D%2D[spliterator]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#spliterator%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-sum-0]]long link:{java8-javadoc}/java/util/stream/LongStream.html#sum%2D%2D[sum]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#sum%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-summaryStatistics-0]]<> link:{java8-javadoc}/java/util/stream/LongStream.html#summaryStatistics%2D%2D[summaryStatistics]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#summaryStatistics%2D%2D[java 9]) -* ++[[painless-api-reference-LongStream-toArray-0]]long[] link:{java8-javadoc}/java/util/stream/LongStream.html#toArray%2D%2D[toArray]()++ (link:{java9-javadoc}/java/util/stream/LongStream.html#toArray%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LongSummaryStatistics.asciidoc b/docs/painless/painless-api-reference/LongSummaryStatistics.asciidoc deleted file mode 100644 index b678d51efa4ac..0000000000000 --- a/docs/painless/painless-api-reference/LongSummaryStatistics.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongSummaryStatistics]]++LongSummaryStatistics++:: -* ++[[painless-api-reference-LongSummaryStatistics-LongSummaryStatistics-0]]link:{java8-javadoc}/java/util/LongSummaryStatistics.html#LongSummaryStatistics%2D%2D[LongSummaryStatistics]()++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#LongSummaryStatistics%2D%2D[java 9]) -* ++[[painless-api-reference-LongSummaryStatistics-combine-1]]void link:{java8-javadoc}/java/util/LongSummaryStatistics.html#combine%2Djava.util.LongSummaryStatistics%2D[combine](<>)++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#combine%2Djava.util.LongSummaryStatistics%2D[java 9]) -* ++[[painless-api-reference-LongSummaryStatistics-getAverage-0]]double link:{java8-javadoc}/java/util/LongSummaryStatistics.html#getAverage%2D%2D[getAverage]()++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#getAverage%2D%2D[java 9]) -* ++[[painless-api-reference-LongSummaryStatistics-getCount-0]]long link:{java8-javadoc}/java/util/LongSummaryStatistics.html#getCount%2D%2D[getCount]()++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#getCount%2D%2D[java 9]) -* ++[[painless-api-reference-LongSummaryStatistics-getMax-0]]long link:{java8-javadoc}/java/util/LongSummaryStatistics.html#getMax%2D%2D[getMax]()++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#getMax%2D%2D[java 9]) -* ++[[painless-api-reference-LongSummaryStatistics-getMin-0]]long link:{java8-javadoc}/java/util/LongSummaryStatistics.html#getMin%2D%2D[getMin]()++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#getMin%2D%2D[java 9]) -* ++[[painless-api-reference-LongSummaryStatistics-getSum-0]]long link:{java8-javadoc}/java/util/LongSummaryStatistics.html#getSum%2D%2D[getSum]()++ (link:{java9-javadoc}/java/util/LongSummaryStatistics.html#getSum%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/LongSupplier.asciidoc b/docs/painless/painless-api-reference/LongSupplier.asciidoc deleted file mode 100644 index 6693e51f17a47..0000000000000 --- a/docs/painless/painless-api-reference/LongSupplier.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongSupplier]]++LongSupplier++:: -* ++[[painless-api-reference-LongSupplier-getAsLong-0]]long link:{java8-javadoc}/java/util/function/LongSupplier.html#getAsLong%2D%2D[getAsLong]()++ (link:{java9-javadoc}/java/util/function/LongSupplier.html#getAsLong%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongToDoubleFunction.asciidoc b/docs/painless/painless-api-reference/LongToDoubleFunction.asciidoc deleted file mode 100644 index 0bed3833371c7..0000000000000 --- a/docs/painless/painless-api-reference/LongToDoubleFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongToDoubleFunction]]++LongToDoubleFunction++:: -* ++[[painless-api-reference-LongToDoubleFunction-applyAsDouble-1]]double link:{java8-javadoc}/java/util/function/LongToDoubleFunction.html#applyAsDouble%2Dlong%2D[applyAsDouble](long)++ (link:{java9-javadoc}/java/util/function/LongToDoubleFunction.html#applyAsDouble%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongToIntFunction.asciidoc b/docs/painless/painless-api-reference/LongToIntFunction.asciidoc deleted file mode 100644 index 2586ba85511b0..0000000000000 --- a/docs/painless/painless-api-reference/LongToIntFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongToIntFunction]]++LongToIntFunction++:: -* ++[[painless-api-reference-LongToIntFunction-applyAsInt-1]]int link:{java8-javadoc}/java/util/function/LongToIntFunction.html#applyAsInt%2Dlong%2D[applyAsInt](long)++ (link:{java9-javadoc}/java/util/function/LongToIntFunction.html#applyAsInt%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/LongUnaryOperator.asciidoc b/docs/painless/painless-api-reference/LongUnaryOperator.asciidoc deleted file mode 100644 index eb84a3688bdeb..0000000000000 --- a/docs/painless/painless-api-reference/LongUnaryOperator.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-LongUnaryOperator]]++LongUnaryOperator++:: -* ++[[painless-api-reference-LongUnaryOperator-identity-0]]static <> link:{java8-javadoc}/java/util/function/LongUnaryOperator.html#identity%2D%2D[identity]()++ (link:{java9-javadoc}/java/util/function/LongUnaryOperator.html#identity%2D%2D[java 9]) -* ++[[painless-api-reference-LongUnaryOperator-andThen-1]]<> link:{java8-javadoc}/java/util/function/LongUnaryOperator.html#andThen%2Djava.util.function.LongUnaryOperator%2D[andThen](<>)++ (link:{java9-javadoc}/java/util/function/LongUnaryOperator.html#andThen%2Djava.util.function.LongUnaryOperator%2D[java 9]) -* ++[[painless-api-reference-LongUnaryOperator-applyAsLong-1]]long link:{java8-javadoc}/java/util/function/LongUnaryOperator.html#applyAsLong%2Dlong%2D[applyAsLong](long)++ (link:{java9-javadoc}/java/util/function/LongUnaryOperator.html#applyAsLong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-LongUnaryOperator-compose-1]]<> link:{java8-javadoc}/java/util/function/LongUnaryOperator.html#compose%2Djava.util.function.LongUnaryOperator%2D[compose](<>)++ (link:{java9-javadoc}/java/util/function/LongUnaryOperator.html#compose%2Djava.util.function.LongUnaryOperator%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Map.Entry.asciidoc b/docs/painless/painless-api-reference/Map.Entry.asciidoc deleted file mode 100644 index bf40736759eea..0000000000000 --- a/docs/painless/painless-api-reference/Map.Entry.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Map-Entry]]++Map.Entry++:: -* ++[[painless-api-reference-Map-Entry-comparingByKey-0]]static <> link:{java8-javadoc}/java/util/Map.Entry.html#comparingByKey%2D%2D[comparingByKey]()++ (link:{java9-javadoc}/java/util/Map.Entry.html#comparingByKey%2D%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-comparingByKey-1]]static <> link:{java8-javadoc}/java/util/Map.Entry.html#comparingByKey%2Djava.util.Comparator%2D[comparingByKey](<>)++ (link:{java9-javadoc}/java/util/Map.Entry.html#comparingByKey%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-comparingByValue-0]]static <> link:{java8-javadoc}/java/util/Map.Entry.html#comparingByValue%2D%2D[comparingByValue]()++ (link:{java9-javadoc}/java/util/Map.Entry.html#comparingByValue%2D%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-comparingByValue-1]]static <> link:{java8-javadoc}/java/util/Map.Entry.html#comparingByValue%2Djava.util.Comparator%2D[comparingByValue](<>)++ (link:{java9-javadoc}/java/util/Map.Entry.html#comparingByValue%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-equals-1]]boolean link:{java8-javadoc}/java/util/Map.Entry.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/util/Map.Entry.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-getKey-0]]def link:{java8-javadoc}/java/util/Map.Entry.html#getKey%2D%2D[getKey]()++ (link:{java9-javadoc}/java/util/Map.Entry.html#getKey%2D%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-getValue-0]]def link:{java8-javadoc}/java/util/Map.Entry.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/util/Map.Entry.html#getValue%2D%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-hashCode-0]]int link:{java8-javadoc}/java/util/Map.Entry.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/util/Map.Entry.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-Map-Entry-setValue-1]]def link:{java8-javadoc}/java/util/Map.Entry.html#setValue%2Djava.lang.Object%2D[setValue](def)++ (link:{java9-javadoc}/java/util/Map.Entry.html#setValue%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Map.asciidoc b/docs/painless/painless-api-reference/Map.asciidoc deleted file mode 100644 index a32c9c137d8cd..0000000000000 --- a/docs/painless/painless-api-reference/Map.asciidoc +++ /dev/null @@ -1,42 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Map]]++Map++:: -* ++[[painless-api-reference-Map-clear-0]]void link:{java8-javadoc}/java/util/Map.html#clear%2D%2D[clear]()++ (link:{java9-javadoc}/java/util/Map.html#clear%2D%2D[java 9]) -* ++[[painless-api-reference-Map-collect-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#collect%2Djava.util.Map%2Djava.util.function.BiFunction%2D[collect](<>)++ -* ++[[painless-api-reference-Map-collect-2]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#collect%2Djava.util.Map%2Djava.util.Collection%2Djava.util.function.BiFunction%2D[collect](<>, <>)++ -* ++[[painless-api-reference-Map-compute-2]]def link:{java8-javadoc}/java/util/Map.html#compute%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[compute](def, <>)++ (link:{java9-javadoc}/java/util/Map.html#compute%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[java 9]) -* ++[[painless-api-reference-Map-computeIfAbsent-2]]def link:{java8-javadoc}/java/util/Map.html#computeIfAbsent%2Djava.lang.Object%2Djava.util.function.Function%2D[computeIfAbsent](def, <>)++ (link:{java9-javadoc}/java/util/Map.html#computeIfAbsent%2Djava.lang.Object%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Map-computeIfPresent-2]]def link:{java8-javadoc}/java/util/Map.html#computeIfPresent%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[computeIfPresent](def, <>)++ (link:{java9-javadoc}/java/util/Map.html#computeIfPresent%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[java 9]) -* ++[[painless-api-reference-Map-containsKey-1]]boolean link:{java8-javadoc}/java/util/Map.html#containsKey%2Djava.lang.Object%2D[containsKey](def)++ (link:{java9-javadoc}/java/util/Map.html#containsKey%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-containsValue-1]]boolean link:{java8-javadoc}/java/util/Map.html#containsValue%2Djava.lang.Object%2D[containsValue](def)++ (link:{java9-javadoc}/java/util/Map.html#containsValue%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-count-1]]int link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#count%2Djava.util.Map%2Djava.util.function.BiPredicate%2D[count](<>)++ -* ++[[painless-api-reference-Map-each-1]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#each%2Djava.util.Map%2Djava.util.function.BiConsumer%2D[each](<>)++ -* ++[[painless-api-reference-Map-entrySet-0]]<> link:{java8-javadoc}/java/util/Map.html#entrySet%2D%2D[entrySet]()++ (link:{java9-javadoc}/java/util/Map.html#entrySet%2D%2D[java 9]) -* ++[[painless-api-reference-Map-equals-1]]boolean link:{java8-javadoc}/java/util/Map.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/util/Map.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-every-1]]boolean link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#every%2Djava.util.Map%2Djava.util.function.BiPredicate%2D[every](<>)++ -* ++[[painless-api-reference-Map-find-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#find%2Djava.util.Map%2Djava.util.function.BiPredicate%2D[find](<>)++ -* ++[[painless-api-reference-Map-findAll-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findAll%2Djava.util.Map%2Djava.util.function.BiPredicate%2D[findAll](<>)++ -* ++[[painless-api-reference-Map-findResult-1]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findResult%2Djava.util.Map%2Djava.util.function.BiFunction%2D[findResult](<>)++ -* ++[[painless-api-reference-Map-findResult-2]]def link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findResult%2Djava.util.Map%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[findResult](def, <>)++ -* ++[[painless-api-reference-Map-findResults-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#findResults%2Djava.util.Map%2Djava.util.function.BiFunction%2D[findResults](<>)++ -* ++[[painless-api-reference-Map-forEach-1]]void link:{java8-javadoc}/java/util/Map.html#forEach%2Djava.util.function.BiConsumer%2D[forEach](<>)++ (link:{java9-javadoc}/java/util/Map.html#forEach%2Djava.util.function.BiConsumer%2D[java 9]) -* ++[[painless-api-reference-Map-get-1]]def link:{java8-javadoc}/java/util/Map.html#get%2Djava.lang.Object%2D[get](def)++ (link:{java9-javadoc}/java/util/Map.html#get%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-getOrDefault-2]]def link:{java8-javadoc}/java/util/Map.html#getOrDefault%2Djava.lang.Object%2Djava.lang.Object%2D[getOrDefault](def, def)++ (link:{java9-javadoc}/java/util/Map.html#getOrDefault%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-groupBy-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#groupBy%2Djava.util.Map%2Djava.util.function.BiFunction%2D[groupBy](<>)++ -* ++[[painless-api-reference-Map-isEmpty-0]]boolean link:{java8-javadoc}/java/util/Map.html#isEmpty%2D%2D[isEmpty]()++ (link:{java9-javadoc}/java/util/Map.html#isEmpty%2D%2D[java 9]) -* ++[[painless-api-reference-Map-keySet-0]]<> link:{java8-javadoc}/java/util/Map.html#keySet%2D%2D[keySet]()++ (link:{java9-javadoc}/java/util/Map.html#keySet%2D%2D[java 9]) -* ++[[painless-api-reference-Map-merge-3]]def link:{java8-javadoc}/java/util/Map.html#merge%2Djava.lang.Object%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[merge](def, def, <>)++ (link:{java9-javadoc}/java/util/Map.html#merge%2Djava.lang.Object%2Djava.lang.Object%2Djava.util.function.BiFunction%2D[java 9]) -* ++[[painless-api-reference-Map-put-2]]def link:{java8-javadoc}/java/util/Map.html#put%2Djava.lang.Object%2Djava.lang.Object%2D[put](def, def)++ (link:{java9-javadoc}/java/util/Map.html#put%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-putAll-1]]void link:{java8-javadoc}/java/util/Map.html#putAll%2Djava.util.Map%2D[putAll](<>)++ (link:{java9-javadoc}/java/util/Map.html#putAll%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-Map-putIfAbsent-2]]def link:{java8-javadoc}/java/util/Map.html#putIfAbsent%2Djava.lang.Object%2Djava.lang.Object%2D[putIfAbsent](def, def)++ (link:{java9-javadoc}/java/util/Map.html#putIfAbsent%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-remove-1]]def link:{java8-javadoc}/java/util/Map.html#remove%2Djava.lang.Object%2D[remove](def)++ (link:{java9-javadoc}/java/util/Map.html#remove%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-remove-2]]boolean link:{java8-javadoc}/java/util/Map.html#remove%2Djava.lang.Object%2Djava.lang.Object%2D[remove](def, def)++ (link:{java9-javadoc}/java/util/Map.html#remove%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-replace-2]]def link:{java8-javadoc}/java/util/Map.html#replace%2Djava.lang.Object%2Djava.lang.Object%2D[replace](def, def)++ (link:{java9-javadoc}/java/util/Map.html#replace%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-replace-3]]boolean link:{java8-javadoc}/java/util/Map.html#replace%2Djava.lang.Object%2Djava.lang.Object%2Djava.lang.Object%2D[replace](def, def, def)++ (link:{java9-javadoc}/java/util/Map.html#replace%2Djava.lang.Object%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Map-replaceAll-1]]void link:{java8-javadoc}/java/util/Map.html#replaceAll%2Djava.util.function.BiFunction%2D[replaceAll](<>)++ (link:{java9-javadoc}/java/util/Map.html#replaceAll%2Djava.util.function.BiFunction%2D[java 9]) -* ++[[painless-api-reference-Map-size-0]]int link:{java8-javadoc}/java/util/Map.html#size%2D%2D[size]()++ (link:{java9-javadoc}/java/util/Map.html#size%2D%2D[java 9]) -* ++[[painless-api-reference-Map-values-0]]<> link:{java8-javadoc}/java/util/Map.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/util/Map.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Matcher.asciidoc b/docs/painless/painless-api-reference/Matcher.asciidoc deleted file mode 100644 index 85198e0bc8db1..0000000000000 --- a/docs/painless/painless-api-reference/Matcher.asciidoc +++ /dev/null @@ -1,34 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Matcher]]++Matcher++:: -* ++[[painless-api-reference-Matcher-quoteReplacement-1]]static <> link:{java8-javadoc}/java/util/regex/Matcher.html#quoteReplacement%2Djava.lang.String%2D[quoteReplacement](<>)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#quoteReplacement%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Matcher-end-0]]int link:{java8-javadoc}/java/util/regex/Matcher.html#end%2D%2D[end]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#end%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-end-1]]int link:{java8-javadoc}/java/util/regex/Matcher.html#end%2Dint%2D[end](int)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#end%2Dint%2D[java 9]) -* ++[[painless-api-reference-Matcher-find-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#find%2D%2D[find]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#find%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-find-1]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#find%2Dint%2D[find](int)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#find%2Dint%2D[java 9]) -* ++[[painless-api-reference-Matcher-group-0]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#group%2D%2D[group]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#group%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-group-1]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#group%2Dint%2D[group](int)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#group%2Dint%2D[java 9]) -* ++[[painless-api-reference-Matcher-groupCount-0]]int link:{java8-javadoc}/java/util/regex/Matcher.html#groupCount%2D%2D[groupCount]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#groupCount%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-hasAnchoringBounds-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#hasAnchoringBounds%2D%2D[hasAnchoringBounds]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#hasAnchoringBounds%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-hasTransparentBounds-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#hasTransparentBounds%2D%2D[hasTransparentBounds]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#hasTransparentBounds%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-hitEnd-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#hitEnd%2D%2D[hitEnd]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#hitEnd%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-lookingAt-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#lookingAt%2D%2D[lookingAt]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#lookingAt%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-matches-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#matches%2D%2D[matches]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#matches%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-namedGroup-1]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#namedGroup%2Djava.util.regex.Matcher%2Djava.lang.String%2D[namedGroup](<>)++ -* ++[[painless-api-reference-Matcher-pattern-0]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#pattern%2D%2D[pattern]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#pattern%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-region-2]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#region%2Dint%2Dint%2D[region](int, int)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#region%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Matcher-regionEnd-0]]int link:{java8-javadoc}/java/util/regex/Matcher.html#regionEnd%2D%2D[regionEnd]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#regionEnd%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-regionStart-0]]int link:{java8-javadoc}/java/util/regex/Matcher.html#regionStart%2D%2D[regionStart]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#regionStart%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-replaceAll-1]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#replaceAll%2Djava.lang.String%2D[replaceAll](<>)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#replaceAll%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Matcher-replaceFirst-1]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#replaceFirst%2Djava.lang.String%2D[replaceFirst](<>)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#replaceFirst%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Matcher-requireEnd-0]]boolean link:{java8-javadoc}/java/util/regex/Matcher.html#requireEnd%2D%2D[requireEnd]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#requireEnd%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-reset-0]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#reset%2D%2D[reset]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#reset%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-start-0]]int link:{java8-javadoc}/java/util/regex/Matcher.html#start%2D%2D[start]()++ (link:{java9-javadoc}/java/util/regex/Matcher.html#start%2D%2D[java 9]) -* ++[[painless-api-reference-Matcher-start-1]]int link:{java8-javadoc}/java/util/regex/Matcher.html#start%2Dint%2D[start](int)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#start%2Dint%2D[java 9]) -* ++[[painless-api-reference-Matcher-useAnchoringBounds-1]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#useAnchoringBounds%2Dboolean%2D[useAnchoringBounds](boolean)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#useAnchoringBounds%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Matcher-usePattern-1]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#usePattern%2Djava.util.regex.Pattern%2D[usePattern](<>)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#usePattern%2Djava.util.regex.Pattern%2D[java 9]) -* ++[[painless-api-reference-Matcher-useTransparentBounds-1]]<> link:{java8-javadoc}/java/util/regex/Matcher.html#useTransparentBounds%2Dboolean%2D[useTransparentBounds](boolean)++ (link:{java9-javadoc}/java/util/regex/Matcher.html#useTransparentBounds%2Dboolean%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Math.asciidoc b/docs/painless/painless-api-reference/Math.asciidoc deleted file mode 100644 index 4b8322c15fc50..0000000000000 --- a/docs/painless/painless-api-reference/Math.asciidoc +++ /dev/null @@ -1,46 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Math]]++Math++:: -** [[painless-api-reference-Math-E]]static double link:{java8-javadoc}/java/lang/Math.html#E[E] (link:{java9-javadoc}/java/lang/Math.html#E[java 9]) -** [[painless-api-reference-Math-PI]]static double link:{java8-javadoc}/java/lang/Math.html#PI[PI] (link:{java9-javadoc}/java/lang/Math.html#PI[java 9]) -* ++[[painless-api-reference-Math-IEEEremainder-2]]static double link:{java8-javadoc}/java/lang/Math.html#IEEEremainder%2Ddouble%2Ddouble%2D[IEEEremainder](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#IEEEremainder%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-abs-1]]static double link:{java8-javadoc}/java/lang/Math.html#abs%2Ddouble%2D[abs](double)++ (link:{java9-javadoc}/java/lang/Math.html#abs%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-acos-1]]static double link:{java8-javadoc}/java/lang/Math.html#acos%2Ddouble%2D[acos](double)++ (link:{java9-javadoc}/java/lang/Math.html#acos%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-asin-1]]static double link:{java8-javadoc}/java/lang/Math.html#asin%2Ddouble%2D[asin](double)++ (link:{java9-javadoc}/java/lang/Math.html#asin%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-atan-1]]static double link:{java8-javadoc}/java/lang/Math.html#atan%2Ddouble%2D[atan](double)++ (link:{java9-javadoc}/java/lang/Math.html#atan%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-atan2-2]]static double link:{java8-javadoc}/java/lang/Math.html#atan2%2Ddouble%2Ddouble%2D[atan2](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#atan2%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-cbrt-1]]static double link:{java8-javadoc}/java/lang/Math.html#cbrt%2Ddouble%2D[cbrt](double)++ (link:{java9-javadoc}/java/lang/Math.html#cbrt%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-ceil-1]]static double link:{java8-javadoc}/java/lang/Math.html#ceil%2Ddouble%2D[ceil](double)++ (link:{java9-javadoc}/java/lang/Math.html#ceil%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-copySign-2]]static double link:{java8-javadoc}/java/lang/Math.html#copySign%2Ddouble%2Ddouble%2D[copySign](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#copySign%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-cos-1]]static double link:{java8-javadoc}/java/lang/Math.html#cos%2Ddouble%2D[cos](double)++ (link:{java9-javadoc}/java/lang/Math.html#cos%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-cosh-1]]static double link:{java8-javadoc}/java/lang/Math.html#cosh%2Ddouble%2D[cosh](double)++ (link:{java9-javadoc}/java/lang/Math.html#cosh%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-exp-1]]static double link:{java8-javadoc}/java/lang/Math.html#exp%2Ddouble%2D[exp](double)++ (link:{java9-javadoc}/java/lang/Math.html#exp%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-expm1-1]]static double link:{java8-javadoc}/java/lang/Math.html#expm1%2Ddouble%2D[expm1](double)++ (link:{java9-javadoc}/java/lang/Math.html#expm1%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-floor-1]]static double link:{java8-javadoc}/java/lang/Math.html#floor%2Ddouble%2D[floor](double)++ (link:{java9-javadoc}/java/lang/Math.html#floor%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-hypot-2]]static double link:{java8-javadoc}/java/lang/Math.html#hypot%2Ddouble%2Ddouble%2D[hypot](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#hypot%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-log-1]]static double link:{java8-javadoc}/java/lang/Math.html#log%2Ddouble%2D[log](double)++ (link:{java9-javadoc}/java/lang/Math.html#log%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-log10-1]]static double link:{java8-javadoc}/java/lang/Math.html#log10%2Ddouble%2D[log10](double)++ (link:{java9-javadoc}/java/lang/Math.html#log10%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-log1p-1]]static double link:{java8-javadoc}/java/lang/Math.html#log1p%2Ddouble%2D[log1p](double)++ (link:{java9-javadoc}/java/lang/Math.html#log1p%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-max-2]]static double link:{java8-javadoc}/java/lang/Math.html#max%2Ddouble%2Ddouble%2D[max](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#max%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-min-2]]static double link:{java8-javadoc}/java/lang/Math.html#min%2Ddouble%2Ddouble%2D[min](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#min%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-nextAfter-2]]static double link:{java8-javadoc}/java/lang/Math.html#nextAfter%2Ddouble%2Ddouble%2D[nextAfter](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#nextAfter%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-nextDown-1]]static double link:{java8-javadoc}/java/lang/Math.html#nextDown%2Ddouble%2D[nextDown](double)++ (link:{java9-javadoc}/java/lang/Math.html#nextDown%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-nextUp-1]]static double link:{java8-javadoc}/java/lang/Math.html#nextUp%2Ddouble%2D[nextUp](double)++ (link:{java9-javadoc}/java/lang/Math.html#nextUp%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-pow-2]]static double link:{java8-javadoc}/java/lang/Math.html#pow%2Ddouble%2Ddouble%2D[pow](double, double)++ (link:{java9-javadoc}/java/lang/Math.html#pow%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-random-0]]static double link:{java8-javadoc}/java/lang/Math.html#random%2D%2D[random]()++ (link:{java9-javadoc}/java/lang/Math.html#random%2D%2D[java 9]) -* ++[[painless-api-reference-Math-rint-1]]static double link:{java8-javadoc}/java/lang/Math.html#rint%2Ddouble%2D[rint](double)++ (link:{java9-javadoc}/java/lang/Math.html#rint%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-round-1]]static long link:{java8-javadoc}/java/lang/Math.html#round%2Ddouble%2D[round](double)++ (link:{java9-javadoc}/java/lang/Math.html#round%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-scalb-2]]static double link:{java8-javadoc}/java/lang/Math.html#scalb%2Ddouble%2Dint%2D[scalb](double, int)++ (link:{java9-javadoc}/java/lang/Math.html#scalb%2Ddouble%2Dint%2D[java 9]) -* ++[[painless-api-reference-Math-signum-1]]static double link:{java8-javadoc}/java/lang/Math.html#signum%2Ddouble%2D[signum](double)++ (link:{java9-javadoc}/java/lang/Math.html#signum%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-sin-1]]static double link:{java8-javadoc}/java/lang/Math.html#sin%2Ddouble%2D[sin](double)++ (link:{java9-javadoc}/java/lang/Math.html#sin%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-sinh-1]]static double link:{java8-javadoc}/java/lang/Math.html#sinh%2Ddouble%2D[sinh](double)++ (link:{java9-javadoc}/java/lang/Math.html#sinh%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-sqrt-1]]static double link:{java8-javadoc}/java/lang/Math.html#sqrt%2Ddouble%2D[sqrt](double)++ (link:{java9-javadoc}/java/lang/Math.html#sqrt%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-tan-1]]static double link:{java8-javadoc}/java/lang/Math.html#tan%2Ddouble%2D[tan](double)++ (link:{java9-javadoc}/java/lang/Math.html#tan%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-tanh-1]]static double link:{java8-javadoc}/java/lang/Math.html#tanh%2Ddouble%2D[tanh](double)++ (link:{java9-javadoc}/java/lang/Math.html#tanh%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-toDegrees-1]]static double link:{java8-javadoc}/java/lang/Math.html#toDegrees%2Ddouble%2D[toDegrees](double)++ (link:{java9-javadoc}/java/lang/Math.html#toDegrees%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-toRadians-1]]static double link:{java8-javadoc}/java/lang/Math.html#toRadians%2Ddouble%2D[toRadians](double)++ (link:{java9-javadoc}/java/lang/Math.html#toRadians%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Math-ulp-1]]static double link:{java8-javadoc}/java/lang/Math.html#ulp%2Ddouble%2D[ulp](double)++ (link:{java9-javadoc}/java/lang/Math.html#ulp%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/MathContext.asciidoc b/docs/painless/painless-api-reference/MathContext.asciidoc deleted file mode 100644 index c8167367fd59b..0000000000000 --- a/docs/painless/painless-api-reference/MathContext.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MathContext]]++MathContext++:: -** [[painless-api-reference-MathContext-DECIMAL128]]static <> link:{java8-javadoc}/java/math/MathContext.html#DECIMAL128[DECIMAL128] (link:{java9-javadoc}/java/math/MathContext.html#DECIMAL128[java 9]) -** [[painless-api-reference-MathContext-DECIMAL32]]static <> link:{java8-javadoc}/java/math/MathContext.html#DECIMAL32[DECIMAL32] (link:{java9-javadoc}/java/math/MathContext.html#DECIMAL32[java 9]) -** [[painless-api-reference-MathContext-DECIMAL64]]static <> link:{java8-javadoc}/java/math/MathContext.html#DECIMAL64[DECIMAL64] (link:{java9-javadoc}/java/math/MathContext.html#DECIMAL64[java 9]) -** [[painless-api-reference-MathContext-UNLIMITED]]static <> link:{java8-javadoc}/java/math/MathContext.html#UNLIMITED[UNLIMITED] (link:{java9-javadoc}/java/math/MathContext.html#UNLIMITED[java 9]) -* ++[[painless-api-reference-MathContext-MathContext-1]]link:{java8-javadoc}/java/math/MathContext.html#MathContext%2Dint%2D[MathContext](int)++ (link:{java9-javadoc}/java/math/MathContext.html#MathContext%2Dint%2D[java 9]) -* ++[[painless-api-reference-MathContext-MathContext-2]]link:{java8-javadoc}/java/math/MathContext.html#MathContext%2Dint%2Djava.math.RoundingMode%2D[MathContext](int, <>)++ (link:{java9-javadoc}/java/math/MathContext.html#MathContext%2Dint%2Djava.math.RoundingMode%2D[java 9]) -* ++[[painless-api-reference-MathContext-getPrecision-0]]int link:{java8-javadoc}/java/math/MathContext.html#getPrecision%2D%2D[getPrecision]()++ (link:{java9-javadoc}/java/math/MathContext.html#getPrecision%2D%2D[java 9]) -* ++[[painless-api-reference-MathContext-getRoundingMode-0]]<> link:{java8-javadoc}/java/math/MathContext.html#getRoundingMode%2D%2D[getRoundingMode]()++ (link:{java9-javadoc}/java/math/MathContext.html#getRoundingMode%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/MessageFormat.Field.asciidoc b/docs/painless/painless-api-reference/MessageFormat.Field.asciidoc deleted file mode 100644 index 200d59df17de9..0000000000000 --- a/docs/painless/painless-api-reference/MessageFormat.Field.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MessageFormat-Field]]++MessageFormat.Field++:: -** [[painless-api-reference-MessageFormat-Field-ARGUMENT]]static <> link:{java8-javadoc}/java/text/MessageFormat.Field.html#ARGUMENT[ARGUMENT] (link:{java9-javadoc}/java/text/MessageFormat.Field.html#ARGUMENT[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/MessageFormat.asciidoc b/docs/painless/painless-api-reference/MessageFormat.asciidoc deleted file mode 100644 index eaed777d9f0f7..0000000000000 --- a/docs/painless/painless-api-reference/MessageFormat.asciidoc +++ /dev/null @@ -1,20 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MessageFormat]]++MessageFormat++:: -* ++[[painless-api-reference-MessageFormat-format-2]]static <> link:{java8-javadoc}/java/text/MessageFormat.html#format%2Djava.lang.String%2Djava.lang.Object:A%2D[format](<>, <>[])++ (link:{java9-javadoc}/java/text/MessageFormat.html#format%2Djava.lang.String%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-applyPattern-1]]void link:{java8-javadoc}/java/text/MessageFormat.html#applyPattern%2Djava.lang.String%2D[applyPattern](<>)++ (link:{java9-javadoc}/java/text/MessageFormat.html#applyPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-getFormats-0]]<>[] link:{java8-javadoc}/java/text/MessageFormat.html#getFormats%2D%2D[getFormats]()++ (link:{java9-javadoc}/java/text/MessageFormat.html#getFormats%2D%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-getFormatsByArgumentIndex-0]]<>[] link:{java8-javadoc}/java/text/MessageFormat.html#getFormatsByArgumentIndex%2D%2D[getFormatsByArgumentIndex]()++ (link:{java9-javadoc}/java/text/MessageFormat.html#getFormatsByArgumentIndex%2D%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-getLocale-0]]<> link:{java8-javadoc}/java/text/MessageFormat.html#getLocale%2D%2D[getLocale]()++ (link:{java9-javadoc}/java/text/MessageFormat.html#getLocale%2D%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-parse-1]]<>[] link:{java8-javadoc}/java/text/MessageFormat.html#parse%2Djava.lang.String%2D[parse](<>)++ (link:{java9-javadoc}/java/text/MessageFormat.html#parse%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-parse-2]]<>[] link:{java8-javadoc}/java/text/MessageFormat.html#parse%2Djava.lang.String%2Djava.text.ParsePosition%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/text/MessageFormat.html#parse%2Djava.lang.String%2Djava.text.ParsePosition%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-setFormat-2]]void link:{java8-javadoc}/java/text/MessageFormat.html#setFormat%2Dint%2Djava.text.Format%2D[setFormat](int, <>)++ (link:{java9-javadoc}/java/text/MessageFormat.html#setFormat%2Dint%2Djava.text.Format%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-setFormatByArgumentIndex-2]]void link:{java8-javadoc}/java/text/MessageFormat.html#setFormatByArgumentIndex%2Dint%2Djava.text.Format%2D[setFormatByArgumentIndex](int, <>)++ (link:{java9-javadoc}/java/text/MessageFormat.html#setFormatByArgumentIndex%2Dint%2Djava.text.Format%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-setFormats-1]]void link:{java8-javadoc}/java/text/MessageFormat.html#setFormats%2Djava.text.Format:A%2D[setFormats](<>[])++ (link:{java9-javadoc}/java/text/MessageFormat.html#setFormats%2Djava.text.Format:A%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-setFormatsByArgumentIndex-1]]void link:{java8-javadoc}/java/text/MessageFormat.html#setFormatsByArgumentIndex%2Djava.text.Format:A%2D[setFormatsByArgumentIndex](<>[])++ (link:{java9-javadoc}/java/text/MessageFormat.html#setFormatsByArgumentIndex%2Djava.text.Format:A%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-setLocale-1]]void link:{java8-javadoc}/java/text/MessageFormat.html#setLocale%2Djava.util.Locale%2D[setLocale](<>)++ (link:{java9-javadoc}/java/text/MessageFormat.html#setLocale%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-MessageFormat-toPattern-0]]<> link:{java8-javadoc}/java/text/MessageFormat.html#toPattern%2D%2D[toPattern]()++ (link:{java9-javadoc}/java/text/MessageFormat.html#toPattern%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/MinguoChronology.asciidoc b/docs/painless/painless-api-reference/MinguoChronology.asciidoc deleted file mode 100644 index 9e9afb2cedfa2..0000000000000 --- a/docs/painless/painless-api-reference/MinguoChronology.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MinguoChronology]]++MinguoChronology++:: -** [[painless-api-reference-MinguoChronology-INSTANCE]]static <> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#INSTANCE[INSTANCE] (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#INSTANCE[java 9]) -* ++[[painless-api-reference-MinguoChronology-date-1]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[date](<>)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-date-3]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#date%2Dint%2Dint%2Dint%2D[date](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#date%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-date-4]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[date](<>, int, int, int)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-dateEpochDay-1]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#dateEpochDay%2Dlong%2D[dateEpochDay](long)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#dateEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-dateYearDay-2]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#dateYearDay%2Dint%2Dint%2D[dateYearDay](int, int)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#dateYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-dateYearDay-3]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[dateYearDay](<>, int, int)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-eraOf-1]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#eraOf%2Dint%2D[eraOf](int)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#eraOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoChronology-resolveDate-2]]<> link:{java8-javadoc}/java/time/chrono/MinguoChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[resolveDate](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/MinguoChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/MinguoDate.asciidoc b/docs/painless/painless-api-reference/MinguoDate.asciidoc deleted file mode 100644 index 38d0b87da762d..0000000000000 --- a/docs/painless/painless-api-reference/MinguoDate.asciidoc +++ /dev/null @@ -1,17 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MinguoDate]]++MinguoDate++:: -* ++[[painless-api-reference-MinguoDate-from-1]]static <> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-of-3]]static <> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-getEra-0]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#getEra%2D%2D[getEra]()++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#getEra%2D%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-minus-1]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-minus-2]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-plus-1]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-plus-2]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-with-1]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-MinguoDate-with-2]]<> link:{java8-javadoc}/java/time/chrono/MinguoDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/MinguoDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/MinguoEra.asciidoc b/docs/painless/painless-api-reference/MinguoEra.asciidoc deleted file mode 100644 index 0b32509913d91..0000000000000 --- a/docs/painless/painless-api-reference/MinguoEra.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MinguoEra]]++MinguoEra++:: -** [[painless-api-reference-MinguoEra-BEFORE_ROC]]static <> link:{java8-javadoc}/java/time/chrono/MinguoEra.html#BEFORE_ROC[BEFORE_ROC] (link:{java9-javadoc}/java/time/chrono/MinguoEra.html#BEFORE_ROC[java 9]) -** [[painless-api-reference-MinguoEra-ROC]]static <> link:{java8-javadoc}/java/time/chrono/MinguoEra.html#ROC[ROC] (link:{java9-javadoc}/java/time/chrono/MinguoEra.html#ROC[java 9]) -* ++[[painless-api-reference-MinguoEra-of-1]]static <> link:{java8-javadoc}/java/time/chrono/MinguoEra.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/chrono/MinguoEra.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-MinguoEra-valueOf-1]]static <> link:{java8-javadoc}/java/time/chrono/MinguoEra.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/chrono/MinguoEra.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-MinguoEra-values-0]]static <>[] link:{java8-javadoc}/java/time/chrono/MinguoEra.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/chrono/MinguoEra.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-MinguoEra-getValue-0]]int link:{java8-javadoc}/java/time/chrono/MinguoEra.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/chrono/MinguoEra.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/MissingFormatArgumentException.asciidoc b/docs/painless/painless-api-reference/MissingFormatArgumentException.asciidoc deleted file mode 100644 index 7c09ad6ba5cb7..0000000000000 --- a/docs/painless/painless-api-reference/MissingFormatArgumentException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MissingFormatArgumentException]]++MissingFormatArgumentException++:: -* ++[[painless-api-reference-MissingFormatArgumentException-MissingFormatArgumentException-1]]link:{java8-javadoc}/java/util/MissingFormatArgumentException.html#MissingFormatArgumentException%2Djava.lang.String%2D[MissingFormatArgumentException](<>)++ (link:{java9-javadoc}/java/util/MissingFormatArgumentException.html#MissingFormatArgumentException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-MissingFormatArgumentException-getFormatSpecifier-0]]<> link:{java8-javadoc}/java/util/MissingFormatArgumentException.html#getFormatSpecifier%2D%2D[getFormatSpecifier]()++ (link:{java9-javadoc}/java/util/MissingFormatArgumentException.html#getFormatSpecifier%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/MissingFormatWidthException.asciidoc b/docs/painless/painless-api-reference/MissingFormatWidthException.asciidoc deleted file mode 100644 index 155637b48dd2e..0000000000000 --- a/docs/painless/painless-api-reference/MissingFormatWidthException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MissingFormatWidthException]]++MissingFormatWidthException++:: -* ++[[painless-api-reference-MissingFormatWidthException-MissingFormatWidthException-1]]link:{java8-javadoc}/java/util/MissingFormatWidthException.html#MissingFormatWidthException%2Djava.lang.String%2D[MissingFormatWidthException](<>)++ (link:{java9-javadoc}/java/util/MissingFormatWidthException.html#MissingFormatWidthException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-MissingFormatWidthException-getFormatSpecifier-0]]<> link:{java8-javadoc}/java/util/MissingFormatWidthException.html#getFormatSpecifier%2D%2D[getFormatSpecifier]()++ (link:{java9-javadoc}/java/util/MissingFormatWidthException.html#getFormatSpecifier%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/MissingResourceException.asciidoc b/docs/painless/painless-api-reference/MissingResourceException.asciidoc deleted file mode 100644 index a415708a728a6..0000000000000 --- a/docs/painless/painless-api-reference/MissingResourceException.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MissingResourceException]]++MissingResourceException++:: -* ++[[painless-api-reference-MissingResourceException-MissingResourceException-3]]link:{java8-javadoc}/java/util/MissingResourceException.html#MissingResourceException%2Djava.lang.String%2Djava.lang.String%2Djava.lang.String%2D[MissingResourceException](<>, <>, <>)++ (link:{java9-javadoc}/java/util/MissingResourceException.html#MissingResourceException%2Djava.lang.String%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-MissingResourceException-getClassName-0]]<> link:{java8-javadoc}/java/util/MissingResourceException.html#getClassName%2D%2D[getClassName]()++ (link:{java9-javadoc}/java/util/MissingResourceException.html#getClassName%2D%2D[java 9]) -* ++[[painless-api-reference-MissingResourceException-getKey-0]]<> link:{java8-javadoc}/java/util/MissingResourceException.html#getKey%2D%2D[getKey]()++ (link:{java9-javadoc}/java/util/MissingResourceException.html#getKey%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Month.asciidoc b/docs/painless/painless-api-reference/Month.asciidoc deleted file mode 100644 index f4eaa9cc12944..0000000000000 --- a/docs/painless/painless-api-reference/Month.asciidoc +++ /dev/null @@ -1,32 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Month]]++Month++:: -** [[painless-api-reference-Month-APRIL]]static <> link:{java8-javadoc}/java/time/Month.html#APRIL[APRIL] (link:{java9-javadoc}/java/time/Month.html#APRIL[java 9]) -** [[painless-api-reference-Month-AUGUST]]static <> link:{java8-javadoc}/java/time/Month.html#AUGUST[AUGUST] (link:{java9-javadoc}/java/time/Month.html#AUGUST[java 9]) -** [[painless-api-reference-Month-DECEMBER]]static <> link:{java8-javadoc}/java/time/Month.html#DECEMBER[DECEMBER] (link:{java9-javadoc}/java/time/Month.html#DECEMBER[java 9]) -** [[painless-api-reference-Month-FEBRUARY]]static <> link:{java8-javadoc}/java/time/Month.html#FEBRUARY[FEBRUARY] (link:{java9-javadoc}/java/time/Month.html#FEBRUARY[java 9]) -** [[painless-api-reference-Month-JANUARY]]static <> link:{java8-javadoc}/java/time/Month.html#JANUARY[JANUARY] (link:{java9-javadoc}/java/time/Month.html#JANUARY[java 9]) -** [[painless-api-reference-Month-JULY]]static <> link:{java8-javadoc}/java/time/Month.html#JULY[JULY] (link:{java9-javadoc}/java/time/Month.html#JULY[java 9]) -** [[painless-api-reference-Month-JUNE]]static <> link:{java8-javadoc}/java/time/Month.html#JUNE[JUNE] (link:{java9-javadoc}/java/time/Month.html#JUNE[java 9]) -** [[painless-api-reference-Month-MARCH]]static <> link:{java8-javadoc}/java/time/Month.html#MARCH[MARCH] (link:{java9-javadoc}/java/time/Month.html#MARCH[java 9]) -** [[painless-api-reference-Month-MAY]]static <> link:{java8-javadoc}/java/time/Month.html#MAY[MAY] (link:{java9-javadoc}/java/time/Month.html#MAY[java 9]) -** [[painless-api-reference-Month-NOVEMBER]]static <> link:{java8-javadoc}/java/time/Month.html#NOVEMBER[NOVEMBER] (link:{java9-javadoc}/java/time/Month.html#NOVEMBER[java 9]) -** [[painless-api-reference-Month-OCTOBER]]static <> link:{java8-javadoc}/java/time/Month.html#OCTOBER[OCTOBER] (link:{java9-javadoc}/java/time/Month.html#OCTOBER[java 9]) -** [[painless-api-reference-Month-SEPTEMBER]]static <> link:{java8-javadoc}/java/time/Month.html#SEPTEMBER[SEPTEMBER] (link:{java9-javadoc}/java/time/Month.html#SEPTEMBER[java 9]) -* ++[[painless-api-reference-Month-from-1]]static <> link:{java8-javadoc}/java/time/Month.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/Month.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Month-of-1]]static <> link:{java8-javadoc}/java/time/Month.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/Month.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-Month-valueOf-1]]static <> link:{java8-javadoc}/java/time/Month.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/Month.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Month-values-0]]static <>[] link:{java8-javadoc}/java/time/Month.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/Month.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-Month-firstDayOfYear-1]]int link:{java8-javadoc}/java/time/Month.html#firstDayOfYear%2Dboolean%2D[firstDayOfYear](boolean)++ (link:{java9-javadoc}/java/time/Month.html#firstDayOfYear%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Month-firstMonthOfQuarter-0]]<> link:{java8-javadoc}/java/time/Month.html#firstMonthOfQuarter%2D%2D[firstMonthOfQuarter]()++ (link:{java9-javadoc}/java/time/Month.html#firstMonthOfQuarter%2D%2D[java 9]) -* ++[[painless-api-reference-Month-getDisplayName-2]]<> link:{java8-javadoc}/java/time/Month.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[getDisplayName](<>, <>)++ (link:{java9-javadoc}/java/time/Month.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-Month-getValue-0]]int link:{java8-javadoc}/java/time/Month.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/Month.html#getValue%2D%2D[java 9]) -* ++[[painless-api-reference-Month-length-1]]int link:{java8-javadoc}/java/time/Month.html#length%2Dboolean%2D[length](boolean)++ (link:{java9-javadoc}/java/time/Month.html#length%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-Month-maxLength-0]]int link:{java8-javadoc}/java/time/Month.html#maxLength%2D%2D[maxLength]()++ (link:{java9-javadoc}/java/time/Month.html#maxLength%2D%2D[java 9]) -* ++[[painless-api-reference-Month-minLength-0]]int link:{java8-javadoc}/java/time/Month.html#minLength%2D%2D[minLength]()++ (link:{java9-javadoc}/java/time/Month.html#minLength%2D%2D[java 9]) -* ++[[painless-api-reference-Month-minus-1]]<> link:{java8-javadoc}/java/time/Month.html#minus%2Dlong%2D[minus](long)++ (link:{java9-javadoc}/java/time/Month.html#minus%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Month-plus-1]]<> link:{java8-javadoc}/java/time/Month.html#plus%2Dlong%2D[plus](long)++ (link:{java9-javadoc}/java/time/Month.html#plus%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/MonthDay.asciidoc b/docs/painless/painless-api-reference/MonthDay.asciidoc deleted file mode 100644 index 4bb7c675d70da..0000000000000 --- a/docs/painless/painless-api-reference/MonthDay.asciidoc +++ /dev/null @@ -1,23 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-MonthDay]]++MonthDay++:: -* ++[[painless-api-reference-MonthDay-from-1]]static <> link:{java8-javadoc}/java/time/MonthDay.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-MonthDay-of-2]]static <> link:{java8-javadoc}/java/time/MonthDay.html#of%2Dint%2Dint%2D[of](int, int)++ (link:{java9-javadoc}/java/time/MonthDay.html#of%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-MonthDay-parse-1]]static <> link:{java8-javadoc}/java/time/MonthDay.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-MonthDay-parse-2]]static <> link:{java8-javadoc}/java/time/MonthDay.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/MonthDay.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-MonthDay-atYear-1]]<> link:{java8-javadoc}/java/time/MonthDay.html#atYear%2Dint%2D[atYear](int)++ (link:{java9-javadoc}/java/time/MonthDay.html#atYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-MonthDay-compareTo-1]]int link:{java8-javadoc}/java/time/MonthDay.html#compareTo%2Djava.time.MonthDay%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#compareTo%2Djava.time.MonthDay%2D[java 9]) -* ++[[painless-api-reference-MonthDay-format-1]]<> link:{java8-javadoc}/java/time/MonthDay.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-MonthDay-getDayOfMonth-0]]int link:{java8-javadoc}/java/time/MonthDay.html#getDayOfMonth%2D%2D[getDayOfMonth]()++ (link:{java9-javadoc}/java/time/MonthDay.html#getDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-MonthDay-getMonth-0]]<> link:{java8-javadoc}/java/time/MonthDay.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/MonthDay.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-MonthDay-getMonthValue-0]]int link:{java8-javadoc}/java/time/MonthDay.html#getMonthValue%2D%2D[getMonthValue]()++ (link:{java9-javadoc}/java/time/MonthDay.html#getMonthValue%2D%2D[java 9]) -* ++[[painless-api-reference-MonthDay-isAfter-1]]boolean link:{java8-javadoc}/java/time/MonthDay.html#isAfter%2Djava.time.MonthDay%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#isAfter%2Djava.time.MonthDay%2D[java 9]) -* ++[[painless-api-reference-MonthDay-isBefore-1]]boolean link:{java8-javadoc}/java/time/MonthDay.html#isBefore%2Djava.time.MonthDay%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#isBefore%2Djava.time.MonthDay%2D[java 9]) -* ++[[painless-api-reference-MonthDay-isValidYear-1]]boolean link:{java8-javadoc}/java/time/MonthDay.html#isValidYear%2Dint%2D[isValidYear](int)++ (link:{java9-javadoc}/java/time/MonthDay.html#isValidYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-MonthDay-with-1]]<> link:{java8-javadoc}/java/time/MonthDay.html#with%2Djava.time.Month%2D[with](<>)++ (link:{java9-javadoc}/java/time/MonthDay.html#with%2Djava.time.Month%2D[java 9]) -* ++[[painless-api-reference-MonthDay-withDayOfMonth-1]]<> link:{java8-javadoc}/java/time/MonthDay.html#withDayOfMonth%2Dint%2D[withDayOfMonth](int)++ (link:{java9-javadoc}/java/time/MonthDay.html#withDayOfMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-MonthDay-withMonth-1]]<> link:{java8-javadoc}/java/time/MonthDay.html#withMonth%2Dint%2D[withMonth](int)++ (link:{java9-javadoc}/java/time/MonthDay.html#withMonth%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NavigableMap.asciidoc b/docs/painless/painless-api-reference/NavigableMap.asciidoc deleted file mode 100644 index 507ab8acd61f3..0000000000000 --- a/docs/painless/painless-api-reference/NavigableMap.asciidoc +++ /dev/null @@ -1,24 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NavigableMap]]++NavigableMap++:: -* ++[[painless-api-reference-NavigableMap-ceilingEntry-1]]<> link:{java8-javadoc}/java/util/NavigableMap.html#ceilingEntry%2Djava.lang.Object%2D[ceilingEntry](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#ceilingEntry%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-ceilingKey-1]]def link:{java8-javadoc}/java/util/NavigableMap.html#ceilingKey%2Djava.lang.Object%2D[ceilingKey](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#ceilingKey%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-descendingKeySet-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#descendingKeySet%2D%2D[descendingKeySet]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#descendingKeySet%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-descendingMap-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#descendingMap%2D%2D[descendingMap]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#descendingMap%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-firstEntry-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#firstEntry%2D%2D[firstEntry]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#firstEntry%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-floorEntry-1]]<> link:{java8-javadoc}/java/util/NavigableMap.html#floorEntry%2Djava.lang.Object%2D[floorEntry](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#floorEntry%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-floorKey-1]]def link:{java8-javadoc}/java/util/NavigableMap.html#floorKey%2Djava.lang.Object%2D[floorKey](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#floorKey%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-headMap-2]]<> link:{java8-javadoc}/java/util/NavigableMap.html#headMap%2Djava.lang.Object%2Dboolean%2D[headMap](def, boolean)++ (link:{java9-javadoc}/java/util/NavigableMap.html#headMap%2Djava.lang.Object%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-higherEntry-1]]<> link:{java8-javadoc}/java/util/NavigableMap.html#higherEntry%2Djava.lang.Object%2D[higherEntry](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#higherEntry%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-higherKey-1]]def link:{java8-javadoc}/java/util/NavigableMap.html#higherKey%2Djava.lang.Object%2D[higherKey](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#higherKey%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-lastEntry-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#lastEntry%2D%2D[lastEntry]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#lastEntry%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-lowerEntry-1]]<> link:{java8-javadoc}/java/util/NavigableMap.html#lowerEntry%2Djava.lang.Object%2D[lowerEntry](def)++ (link:{java9-javadoc}/java/util/NavigableMap.html#lowerEntry%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-navigableKeySet-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#navigableKeySet%2D%2D[navigableKeySet]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#navigableKeySet%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-pollFirstEntry-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#pollFirstEntry%2D%2D[pollFirstEntry]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#pollFirstEntry%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-pollLastEntry-0]]<> link:{java8-javadoc}/java/util/NavigableMap.html#pollLastEntry%2D%2D[pollLastEntry]()++ (link:{java9-javadoc}/java/util/NavigableMap.html#pollLastEntry%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-subMap-4]]<> link:{java8-javadoc}/java/util/NavigableMap.html#subMap%2Djava.lang.Object%2Dboolean%2Djava.lang.Object%2Dboolean%2D[subMap](def, boolean, def, boolean)++ (link:{java9-javadoc}/java/util/NavigableMap.html#subMap%2Djava.lang.Object%2Dboolean%2Djava.lang.Object%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-NavigableMap-tailMap-2]]<> link:{java8-javadoc}/java/util/NavigableMap.html#tailMap%2Djava.lang.Object%2Dboolean%2D[tailMap](def, boolean)++ (link:{java9-javadoc}/java/util/NavigableMap.html#tailMap%2Djava.lang.Object%2Dboolean%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NavigableSet.asciidoc b/docs/painless/painless-api-reference/NavigableSet.asciidoc deleted file mode 100644 index 15fff25f44d5b..0000000000000 --- a/docs/painless/painless-api-reference/NavigableSet.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NavigableSet]]++NavigableSet++:: -* ++[[painless-api-reference-NavigableSet-ceiling-1]]def link:{java8-javadoc}/java/util/NavigableSet.html#ceiling%2Djava.lang.Object%2D[ceiling](def)++ (link:{java9-javadoc}/java/util/NavigableSet.html#ceiling%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-descendingIterator-0]]<> link:{java8-javadoc}/java/util/NavigableSet.html#descendingIterator%2D%2D[descendingIterator]()++ (link:{java9-javadoc}/java/util/NavigableSet.html#descendingIterator%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-descendingSet-0]]<> link:{java8-javadoc}/java/util/NavigableSet.html#descendingSet%2D%2D[descendingSet]()++ (link:{java9-javadoc}/java/util/NavigableSet.html#descendingSet%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-floor-1]]def link:{java8-javadoc}/java/util/NavigableSet.html#floor%2Djava.lang.Object%2D[floor](def)++ (link:{java9-javadoc}/java/util/NavigableSet.html#floor%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-headSet-2]]<> link:{java8-javadoc}/java/util/NavigableSet.html#headSet%2Djava.lang.Object%2Dboolean%2D[headSet](def, boolean)++ (link:{java9-javadoc}/java/util/NavigableSet.html#headSet%2Djava.lang.Object%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-higher-1]]def link:{java8-javadoc}/java/util/NavigableSet.html#higher%2Djava.lang.Object%2D[higher](def)++ (link:{java9-javadoc}/java/util/NavigableSet.html#higher%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-lower-1]]def link:{java8-javadoc}/java/util/NavigableSet.html#lower%2Djava.lang.Object%2D[lower](def)++ (link:{java9-javadoc}/java/util/NavigableSet.html#lower%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-pollFirst-0]]def link:{java8-javadoc}/java/util/NavigableSet.html#pollFirst%2D%2D[pollFirst]()++ (link:{java9-javadoc}/java/util/NavigableSet.html#pollFirst%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-pollLast-0]]def link:{java8-javadoc}/java/util/NavigableSet.html#pollLast%2D%2D[pollLast]()++ (link:{java9-javadoc}/java/util/NavigableSet.html#pollLast%2D%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-subSet-4]]<> link:{java8-javadoc}/java/util/NavigableSet.html#subSet%2Djava.lang.Object%2Dboolean%2Djava.lang.Object%2Dboolean%2D[subSet](def, boolean, def, boolean)++ (link:{java9-javadoc}/java/util/NavigableSet.html#subSet%2Djava.lang.Object%2Dboolean%2Djava.lang.Object%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-NavigableSet-tailSet-2]]<> link:{java8-javadoc}/java/util/NavigableSet.html#tailSet%2Djava.lang.Object%2Dboolean%2D[tailSet](def, boolean)++ (link:{java9-javadoc}/java/util/NavigableSet.html#tailSet%2Djava.lang.Object%2Dboolean%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NegativeArraySizeException.asciidoc b/docs/painless/painless-api-reference/NegativeArraySizeException.asciidoc deleted file mode 100644 index 78ee0006eca71..0000000000000 --- a/docs/painless/painless-api-reference/NegativeArraySizeException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NegativeArraySizeException]]++NegativeArraySizeException++:: -* ++[[painless-api-reference-NegativeArraySizeException-NegativeArraySizeException-0]]link:{java8-javadoc}/java/lang/NegativeArraySizeException.html#NegativeArraySizeException%2D%2D[NegativeArraySizeException]()++ (link:{java9-javadoc}/java/lang/NegativeArraySizeException.html#NegativeArraySizeException%2D%2D[java 9]) -* ++[[painless-api-reference-NegativeArraySizeException-NegativeArraySizeException-1]]link:{java8-javadoc}/java/lang/NegativeArraySizeException.html#NegativeArraySizeException%2Djava.lang.String%2D[NegativeArraySizeException](<>)++ (link:{java9-javadoc}/java/lang/NegativeArraySizeException.html#NegativeArraySizeException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NoSuchElementException.asciidoc b/docs/painless/painless-api-reference/NoSuchElementException.asciidoc deleted file mode 100644 index 7a28bdfb5dd24..0000000000000 --- a/docs/painless/painless-api-reference/NoSuchElementException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NoSuchElementException]]++NoSuchElementException++:: -* ++[[painless-api-reference-NoSuchElementException-NoSuchElementException-0]]link:{java8-javadoc}/java/util/NoSuchElementException.html#NoSuchElementException%2D%2D[NoSuchElementException]()++ (link:{java9-javadoc}/java/util/NoSuchElementException.html#NoSuchElementException%2D%2D[java 9]) -* ++[[painless-api-reference-NoSuchElementException-NoSuchElementException-1]]link:{java8-javadoc}/java/util/NoSuchElementException.html#NoSuchElementException%2Djava.lang.String%2D[NoSuchElementException](<>)++ (link:{java9-javadoc}/java/util/NoSuchElementException.html#NoSuchElementException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NoSuchFieldException.asciidoc b/docs/painless/painless-api-reference/NoSuchFieldException.asciidoc deleted file mode 100644 index 6f3e12d16d7d0..0000000000000 --- a/docs/painless/painless-api-reference/NoSuchFieldException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NoSuchFieldException]]++NoSuchFieldException++:: -* ++[[painless-api-reference-NoSuchFieldException-NoSuchFieldException-0]]link:{java8-javadoc}/java/lang/NoSuchFieldException.html#NoSuchFieldException%2D%2D[NoSuchFieldException]()++ (link:{java9-javadoc}/java/lang/NoSuchFieldException.html#NoSuchFieldException%2D%2D[java 9]) -* ++[[painless-api-reference-NoSuchFieldException-NoSuchFieldException-1]]link:{java8-javadoc}/java/lang/NoSuchFieldException.html#NoSuchFieldException%2Djava.lang.String%2D[NoSuchFieldException](<>)++ (link:{java9-javadoc}/java/lang/NoSuchFieldException.html#NoSuchFieldException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NoSuchMethodException.asciidoc b/docs/painless/painless-api-reference/NoSuchMethodException.asciidoc deleted file mode 100644 index 0fec067e9d281..0000000000000 --- a/docs/painless/painless-api-reference/NoSuchMethodException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NoSuchMethodException]]++NoSuchMethodException++:: -* ++[[painless-api-reference-NoSuchMethodException-NoSuchMethodException-0]]link:{java8-javadoc}/java/lang/NoSuchMethodException.html#NoSuchMethodException%2D%2D[NoSuchMethodException]()++ (link:{java9-javadoc}/java/lang/NoSuchMethodException.html#NoSuchMethodException%2D%2D[java 9]) -* ++[[painless-api-reference-NoSuchMethodException-NoSuchMethodException-1]]link:{java8-javadoc}/java/lang/NoSuchMethodException.html#NoSuchMethodException%2Djava.lang.String%2D[NoSuchMethodException](<>)++ (link:{java9-javadoc}/java/lang/NoSuchMethodException.html#NoSuchMethodException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Normalizer.Form.asciidoc b/docs/painless/painless-api-reference/Normalizer.Form.asciidoc deleted file mode 100644 index 5255435473004..0000000000000 --- a/docs/painless/painless-api-reference/Normalizer.Form.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Normalizer-Form]]++Normalizer.Form++:: -** [[painless-api-reference-Normalizer-Form-NFC]]static <> link:{java8-javadoc}/java/text/Normalizer.Form.html#NFC[NFC] (link:{java9-javadoc}/java/text/Normalizer.Form.html#NFC[java 9]) -** [[painless-api-reference-Normalizer-Form-NFD]]static <> link:{java8-javadoc}/java/text/Normalizer.Form.html#NFD[NFD] (link:{java9-javadoc}/java/text/Normalizer.Form.html#NFD[java 9]) -** [[painless-api-reference-Normalizer-Form-NFKC]]static <> link:{java8-javadoc}/java/text/Normalizer.Form.html#NFKC[NFKC] (link:{java9-javadoc}/java/text/Normalizer.Form.html#NFKC[java 9]) -** [[painless-api-reference-Normalizer-Form-NFKD]]static <> link:{java8-javadoc}/java/text/Normalizer.Form.html#NFKD[NFKD] (link:{java9-javadoc}/java/text/Normalizer.Form.html#NFKD[java 9]) -* ++[[painless-api-reference-Normalizer-Form-valueOf-1]]static <> link:{java8-javadoc}/java/text/Normalizer.Form.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/text/Normalizer.Form.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Normalizer-Form-values-0]]static <>[] link:{java8-javadoc}/java/text/Normalizer.Form.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/text/Normalizer.Form.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Normalizer.asciidoc b/docs/painless/painless-api-reference/Normalizer.asciidoc deleted file mode 100644 index ecfe03c7fb4bd..0000000000000 --- a/docs/painless/painless-api-reference/Normalizer.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Normalizer]]++Normalizer++:: -* ++[[painless-api-reference-Normalizer-isNormalized-2]]static boolean link:{java8-javadoc}/java/text/Normalizer.html#isNormalized%2Djava.lang.CharSequence%2Djava.text.Normalizer$Form%2D[isNormalized](<>, <>)++ (link:{java9-javadoc}/java/text/Normalizer.html#isNormalized%2Djava.lang.CharSequence%2Djava.text.Normalizer$Form%2D[java 9]) -* ++[[painless-api-reference-Normalizer-normalize-2]]static <> link:{java8-javadoc}/java/text/Normalizer.html#normalize%2Djava.lang.CharSequence%2Djava.text.Normalizer$Form%2D[normalize](<>, <>)++ (link:{java9-javadoc}/java/text/Normalizer.html#normalize%2Djava.lang.CharSequence%2Djava.text.Normalizer$Form%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/NullPointerException.asciidoc b/docs/painless/painless-api-reference/NullPointerException.asciidoc deleted file mode 100644 index 72795c7399060..0000000000000 --- a/docs/painless/painless-api-reference/NullPointerException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NullPointerException]]++NullPointerException++:: -* ++[[painless-api-reference-NullPointerException-NullPointerException-0]]link:{java8-javadoc}/java/lang/NullPointerException.html#NullPointerException%2D%2D[NullPointerException]()++ (link:{java9-javadoc}/java/lang/NullPointerException.html#NullPointerException%2D%2D[java 9]) -* ++[[painless-api-reference-NullPointerException-NullPointerException-1]]link:{java8-javadoc}/java/lang/NullPointerException.html#NullPointerException%2Djava.lang.String%2D[NullPointerException](<>)++ (link:{java9-javadoc}/java/lang/NullPointerException.html#NullPointerException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Number.asciidoc b/docs/painless/painless-api-reference/Number.asciidoc deleted file mode 100644 index a8c588f33704e..0000000000000 --- a/docs/painless/painless-api-reference/Number.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Number]]++Number++:: -* ++[[painless-api-reference-Number-byteValue-0]]byte link:{java8-javadoc}/java/lang/Number.html#byteValue%2D%2D[byteValue]()++ (link:{java9-javadoc}/java/lang/Number.html#byteValue%2D%2D[java 9]) -* ++[[painless-api-reference-Number-doubleValue-0]]double link:{java8-javadoc}/java/lang/Number.html#doubleValue%2D%2D[doubleValue]()++ (link:{java9-javadoc}/java/lang/Number.html#doubleValue%2D%2D[java 9]) -* ++[[painless-api-reference-Number-floatValue-0]]float link:{java8-javadoc}/java/lang/Number.html#floatValue%2D%2D[floatValue]()++ (link:{java9-javadoc}/java/lang/Number.html#floatValue%2D%2D[java 9]) -* ++[[painless-api-reference-Number-intValue-0]]int link:{java8-javadoc}/java/lang/Number.html#intValue%2D%2D[intValue]()++ (link:{java9-javadoc}/java/lang/Number.html#intValue%2D%2D[java 9]) -* ++[[painless-api-reference-Number-longValue-0]]long link:{java8-javadoc}/java/lang/Number.html#longValue%2D%2D[longValue]()++ (link:{java9-javadoc}/java/lang/Number.html#longValue%2D%2D[java 9]) -* ++[[painless-api-reference-Number-shortValue-0]]short link:{java8-javadoc}/java/lang/Number.html#shortValue%2D%2D[shortValue]()++ (link:{java9-javadoc}/java/lang/Number.html#shortValue%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/NumberFormat.Field.asciidoc b/docs/painless/painless-api-reference/NumberFormat.Field.asciidoc deleted file mode 100644 index 279b0dbaaf5b7..0000000000000 --- a/docs/painless/painless-api-reference/NumberFormat.Field.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NumberFormat-Field]]++NumberFormat.Field++:: -** [[painless-api-reference-NumberFormat-Field-CURRENCY]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#CURRENCY[CURRENCY] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#CURRENCY[java 9]) -** [[painless-api-reference-NumberFormat-Field-DECIMAL_SEPARATOR]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#DECIMAL_SEPARATOR[DECIMAL_SEPARATOR] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#DECIMAL_SEPARATOR[java 9]) -** [[painless-api-reference-NumberFormat-Field-EXPONENT]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#EXPONENT[EXPONENT] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#EXPONENT[java 9]) -** [[painless-api-reference-NumberFormat-Field-EXPONENT_SIGN]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#EXPONENT_SIGN[EXPONENT_SIGN] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#EXPONENT_SIGN[java 9]) -** [[painless-api-reference-NumberFormat-Field-EXPONENT_SYMBOL]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#EXPONENT_SYMBOL[EXPONENT_SYMBOL] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#EXPONENT_SYMBOL[java 9]) -** [[painless-api-reference-NumberFormat-Field-FRACTION]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#FRACTION[FRACTION] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#FRACTION[java 9]) -** [[painless-api-reference-NumberFormat-Field-GROUPING_SEPARATOR]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#GROUPING_SEPARATOR[GROUPING_SEPARATOR] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#GROUPING_SEPARATOR[java 9]) -** [[painless-api-reference-NumberFormat-Field-INTEGER]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#INTEGER[INTEGER] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#INTEGER[java 9]) -** [[painless-api-reference-NumberFormat-Field-PERCENT]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#PERCENT[PERCENT] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#PERCENT[java 9]) -** [[painless-api-reference-NumberFormat-Field-PERMILLE]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#PERMILLE[PERMILLE] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#PERMILLE[java 9]) -** [[painless-api-reference-NumberFormat-Field-SIGN]]static <> link:{java8-javadoc}/java/text/NumberFormat.Field.html#SIGN[SIGN] (link:{java9-javadoc}/java/text/NumberFormat.Field.html#SIGN[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/NumberFormat.asciidoc b/docs/painless/painless-api-reference/NumberFormat.asciidoc deleted file mode 100644 index ddca8ccd6826f..0000000000000 --- a/docs/painless/painless-api-reference/NumberFormat.asciidoc +++ /dev/null @@ -1,38 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NumberFormat]]++NumberFormat++:: -** [[painless-api-reference-NumberFormat-FRACTION_FIELD]]static int link:{java8-javadoc}/java/text/NumberFormat.html#FRACTION_FIELD[FRACTION_FIELD] (link:{java9-javadoc}/java/text/NumberFormat.html#FRACTION_FIELD[java 9]) -** [[painless-api-reference-NumberFormat-INTEGER_FIELD]]static int link:{java8-javadoc}/java/text/NumberFormat.html#INTEGER_FIELD[INTEGER_FIELD] (link:{java9-javadoc}/java/text/NumberFormat.html#INTEGER_FIELD[java 9]) -* ++[[painless-api-reference-NumberFormat-getAvailableLocales-0]]static <>[] link:{java8-javadoc}/java/text/NumberFormat.html#getAvailableLocales%2D%2D[getAvailableLocales]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getAvailableLocales%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getCurrencyInstance-0]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getCurrencyInstance%2D%2D[getCurrencyInstance]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getCurrencyInstance%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getCurrencyInstance-1]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getCurrencyInstance%2Djava.util.Locale%2D[getCurrencyInstance](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#getCurrencyInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getInstance-0]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getInstance%2D%2D[getInstance]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getInstance%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getInstance-1]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getInstance%2Djava.util.Locale%2D[getInstance](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#getInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getIntegerInstance-0]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getIntegerInstance%2D%2D[getIntegerInstance]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getIntegerInstance%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getIntegerInstance-1]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getIntegerInstance%2Djava.util.Locale%2D[getIntegerInstance](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#getIntegerInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getNumberInstance-0]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getNumberInstance%2D%2D[getNumberInstance]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getNumberInstance%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getNumberInstance-1]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getNumberInstance%2Djava.util.Locale%2D[getNumberInstance](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#getNumberInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getPercentInstance-0]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getPercentInstance%2D%2D[getPercentInstance]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getPercentInstance%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getPercentInstance-1]]static <> link:{java8-javadoc}/java/text/NumberFormat.html#getPercentInstance%2Djava.util.Locale%2D[getPercentInstance](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#getPercentInstance%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getCurrency-0]]<> link:{java8-javadoc}/java/text/NumberFormat.html#getCurrency%2D%2D[getCurrency]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getCurrency%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getMaximumFractionDigits-0]]int link:{java8-javadoc}/java/text/NumberFormat.html#getMaximumFractionDigits%2D%2D[getMaximumFractionDigits]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getMaximumFractionDigits%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getMaximumIntegerDigits-0]]int link:{java8-javadoc}/java/text/NumberFormat.html#getMaximumIntegerDigits%2D%2D[getMaximumIntegerDigits]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getMaximumIntegerDigits%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getMinimumFractionDigits-0]]int link:{java8-javadoc}/java/text/NumberFormat.html#getMinimumFractionDigits%2D%2D[getMinimumFractionDigits]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getMinimumFractionDigits%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getMinimumIntegerDigits-0]]int link:{java8-javadoc}/java/text/NumberFormat.html#getMinimumIntegerDigits%2D%2D[getMinimumIntegerDigits]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getMinimumIntegerDigits%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-getRoundingMode-0]]<> link:{java8-javadoc}/java/text/NumberFormat.html#getRoundingMode%2D%2D[getRoundingMode]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#getRoundingMode%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-isGroupingUsed-0]]boolean link:{java8-javadoc}/java/text/NumberFormat.html#isGroupingUsed%2D%2D[isGroupingUsed]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#isGroupingUsed%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-isParseIntegerOnly-0]]boolean link:{java8-javadoc}/java/text/NumberFormat.html#isParseIntegerOnly%2D%2D[isParseIntegerOnly]()++ (link:{java9-javadoc}/java/text/NumberFormat.html#isParseIntegerOnly%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-parse-1]]<> link:{java8-javadoc}/java/text/NumberFormat.html#parse%2Djava.lang.String%2D[parse](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#parse%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-parse-2]]<> link:{java8-javadoc}/java/text/NumberFormat.html#parse%2Djava.lang.String%2Djava.text.ParsePosition%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#parse%2Djava.lang.String%2Djava.text.ParsePosition%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setCurrency-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setCurrency%2Djava.util.Currency%2D[setCurrency](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setCurrency%2Djava.util.Currency%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setGroupingUsed-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setGroupingUsed%2Dboolean%2D[setGroupingUsed](boolean)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setGroupingUsed%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setMaximumFractionDigits-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setMaximumFractionDigits%2Dint%2D[setMaximumFractionDigits](int)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setMaximumFractionDigits%2Dint%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setMaximumIntegerDigits-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setMaximumIntegerDigits%2Dint%2D[setMaximumIntegerDigits](int)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setMaximumIntegerDigits%2Dint%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setMinimumFractionDigits-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setMinimumFractionDigits%2Dint%2D[setMinimumFractionDigits](int)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setMinimumFractionDigits%2Dint%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setMinimumIntegerDigits-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setMinimumIntegerDigits%2Dint%2D[setMinimumIntegerDigits](int)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setMinimumIntegerDigits%2Dint%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setParseIntegerOnly-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setParseIntegerOnly%2Dboolean%2D[setParseIntegerOnly](boolean)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setParseIntegerOnly%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-NumberFormat-setRoundingMode-1]]void link:{java8-javadoc}/java/text/NumberFormat.html#setRoundingMode%2Djava.math.RoundingMode%2D[setRoundingMode](<>)++ (link:{java9-javadoc}/java/text/NumberFormat.html#setRoundingMode%2Djava.math.RoundingMode%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/NumberFormatException.asciidoc b/docs/painless/painless-api-reference/NumberFormatException.asciidoc deleted file mode 100644 index 4066147650c33..0000000000000 --- a/docs/painless/painless-api-reference/NumberFormatException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-NumberFormatException]]++NumberFormatException++:: -* ++[[painless-api-reference-NumberFormatException-NumberFormatException-0]]link:{java8-javadoc}/java/lang/NumberFormatException.html#NumberFormatException%2D%2D[NumberFormatException]()++ (link:{java9-javadoc}/java/lang/NumberFormatException.html#NumberFormatException%2D%2D[java 9]) -* ++[[painless-api-reference-NumberFormatException-NumberFormatException-1]]link:{java8-javadoc}/java/lang/NumberFormatException.html#NumberFormatException%2Djava.lang.String%2D[NumberFormatException](<>)++ (link:{java9-javadoc}/java/lang/NumberFormatException.html#NumberFormatException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ObjDoubleConsumer.asciidoc b/docs/painless/painless-api-reference/ObjDoubleConsumer.asciidoc deleted file mode 100644 index 2fc41f9ce0fb5..0000000000000 --- a/docs/painless/painless-api-reference/ObjDoubleConsumer.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ObjDoubleConsumer]]++ObjDoubleConsumer++:: -* ++[[painless-api-reference-ObjDoubleConsumer-accept-2]]void link:{java8-javadoc}/java/util/function/ObjDoubleConsumer.html#accept%2Djava.lang.Object%2Ddouble%2D[accept](def, double)++ (link:{java9-javadoc}/java/util/function/ObjDoubleConsumer.html#accept%2Djava.lang.Object%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ObjIntConsumer.asciidoc b/docs/painless/painless-api-reference/ObjIntConsumer.asciidoc deleted file mode 100644 index 8779b333df92c..0000000000000 --- a/docs/painless/painless-api-reference/ObjIntConsumer.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ObjIntConsumer]]++ObjIntConsumer++:: -* ++[[painless-api-reference-ObjIntConsumer-accept-2]]void link:{java8-javadoc}/java/util/function/ObjIntConsumer.html#accept%2Djava.lang.Object%2Dint%2D[accept](def, int)++ (link:{java9-javadoc}/java/util/function/ObjIntConsumer.html#accept%2Djava.lang.Object%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ObjLongConsumer.asciidoc b/docs/painless/painless-api-reference/ObjLongConsumer.asciidoc deleted file mode 100644 index cdb0ed6546c4f..0000000000000 --- a/docs/painless/painless-api-reference/ObjLongConsumer.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ObjLongConsumer]]++ObjLongConsumer++:: -* ++[[painless-api-reference-ObjLongConsumer-accept-2]]void link:{java8-javadoc}/java/util/function/ObjLongConsumer.html#accept%2Djava.lang.Object%2Dlong%2D[accept](def, long)++ (link:{java9-javadoc}/java/util/function/ObjLongConsumer.html#accept%2Djava.lang.Object%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Object.asciidoc b/docs/painless/painless-api-reference/Object.asciidoc deleted file mode 100644 index 99c4ae5cffdb6..0000000000000 --- a/docs/painless/painless-api-reference/Object.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Object]]++Object++:: -* ++[[painless-api-reference-Object-equals-1]]boolean link:{java8-javadoc}/java/lang/Object.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/lang/Object.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Object-hashCode-0]]int link:{java8-javadoc}/java/lang/Object.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/lang/Object.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-Object-toString-0]]<> link:{java8-javadoc}/java/lang/Object.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/lang/Object.html#toString%2D%2D[java 9]) diff --git a/docs/painless/painless-api-reference/Objects.asciidoc b/docs/painless/painless-api-reference/Objects.asciidoc deleted file mode 100644 index 9fdbad2e5b71d..0000000000000 --- a/docs/painless/painless-api-reference/Objects.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Objects]]++Objects++:: -* ++[[painless-api-reference-Objects-compare-3]]static int link:{java8-javadoc}/java/util/Objects.html#compare%2Djava.lang.Object%2Djava.lang.Object%2Djava.util.Comparator%2D[compare](def, def, <>)++ (link:{java9-javadoc}/java/util/Objects.html#compare%2Djava.lang.Object%2Djava.lang.Object%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Objects-deepEquals-2]]static boolean link:{java8-javadoc}/java/util/Objects.html#deepEquals%2Djava.lang.Object%2Djava.lang.Object%2D[deepEquals](<>, <>)++ (link:{java9-javadoc}/java/util/Objects.html#deepEquals%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-equals-2]]static boolean link:{java8-javadoc}/java/util/Objects.html#equals%2Djava.lang.Object%2Djava.lang.Object%2D[equals](<>, <>)++ (link:{java9-javadoc}/java/util/Objects.html#equals%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-hash-1]]static int link:{java8-javadoc}/java/util/Objects.html#hash%2Djava.lang.Object:A%2D[hash](<>[])++ (link:{java9-javadoc}/java/util/Objects.html#hash%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Objects-hashCode-1]]static int link:{java8-javadoc}/java/util/Objects.html#hashCode%2Djava.lang.Object%2D[hashCode](<>)++ (link:{java9-javadoc}/java/util/Objects.html#hashCode%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-isNull-1]]static boolean link:{java8-javadoc}/java/util/Objects.html#isNull%2Djava.lang.Object%2D[isNull](<>)++ (link:{java9-javadoc}/java/util/Objects.html#isNull%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-nonNull-1]]static boolean link:{java8-javadoc}/java/util/Objects.html#nonNull%2Djava.lang.Object%2D[nonNull](<>)++ (link:{java9-javadoc}/java/util/Objects.html#nonNull%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-requireNonNull-1]]static def link:{java8-javadoc}/java/util/Objects.html#requireNonNull%2Djava.lang.Object%2D[requireNonNull](def)++ (link:{java9-javadoc}/java/util/Objects.html#requireNonNull%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-requireNonNull-2]]static def link:{java8-javadoc}/java/util/Objects.html#requireNonNull%2Djava.lang.Object%2Djava.lang.String%2D[requireNonNull](def, <>)++ (link:{java9-javadoc}/java/util/Objects.html#requireNonNull%2Djava.lang.Object%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Objects-toString-1]]static <> link:{java8-javadoc}/java/util/Objects.html#toString%2Djava.lang.Object%2D[toString](<>)++ (link:{java9-javadoc}/java/util/Objects.html#toString%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Objects-toString-2]]static <> link:{java8-javadoc}/java/util/Objects.html#toString%2Djava.lang.Object%2Djava.lang.String%2D[toString](<>, <>)++ (link:{java9-javadoc}/java/util/Objects.html#toString%2Djava.lang.Object%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Observable.asciidoc b/docs/painless/painless-api-reference/Observable.asciidoc deleted file mode 100644 index 65d8fae45ae36..0000000000000 --- a/docs/painless/painless-api-reference/Observable.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Observable]]++Observable++:: -* ++[[painless-api-reference-Observable-Observable-0]]link:{java8-javadoc}/java/util/Observable.html#Observable%2D%2D[Observable]()++ (link:{java9-javadoc}/java/util/Observable.html#Observable%2D%2D[java 9]) -* ++[[painless-api-reference-Observable-addObserver-1]]void link:{java8-javadoc}/java/util/Observable.html#addObserver%2Djava.util.Observer%2D[addObserver](<>)++ (link:{java9-javadoc}/java/util/Observable.html#addObserver%2Djava.util.Observer%2D[java 9]) -* ++[[painless-api-reference-Observable-countObservers-0]]int link:{java8-javadoc}/java/util/Observable.html#countObservers%2D%2D[countObservers]()++ (link:{java9-javadoc}/java/util/Observable.html#countObservers%2D%2D[java 9]) -* ++[[painless-api-reference-Observable-deleteObserver-1]]void link:{java8-javadoc}/java/util/Observable.html#deleteObserver%2Djava.util.Observer%2D[deleteObserver](<>)++ (link:{java9-javadoc}/java/util/Observable.html#deleteObserver%2Djava.util.Observer%2D[java 9]) -* ++[[painless-api-reference-Observable-deleteObservers-0]]void link:{java8-javadoc}/java/util/Observable.html#deleteObservers%2D%2D[deleteObservers]()++ (link:{java9-javadoc}/java/util/Observable.html#deleteObservers%2D%2D[java 9]) -* ++[[painless-api-reference-Observable-hasChanged-0]]boolean link:{java8-javadoc}/java/util/Observable.html#hasChanged%2D%2D[hasChanged]()++ (link:{java9-javadoc}/java/util/Observable.html#hasChanged%2D%2D[java 9]) -* ++[[painless-api-reference-Observable-notifyObservers-0]]void link:{java8-javadoc}/java/util/Observable.html#notifyObservers%2D%2D[notifyObservers]()++ (link:{java9-javadoc}/java/util/Observable.html#notifyObservers%2D%2D[java 9]) -* ++[[painless-api-reference-Observable-notifyObservers-1]]void link:{java8-javadoc}/java/util/Observable.html#notifyObservers%2Djava.lang.Object%2D[notifyObservers](<>)++ (link:{java9-javadoc}/java/util/Observable.html#notifyObservers%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Observer.asciidoc b/docs/painless/painless-api-reference/Observer.asciidoc deleted file mode 100644 index 9277663dd6232..0000000000000 --- a/docs/painless/painless-api-reference/Observer.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Observer]]++Observer++:: -* ++[[painless-api-reference-Observer-update-2]]void link:{java8-javadoc}/java/util/Observer.html#update%2Djava.util.Observable%2Djava.lang.Object%2D[update](<>, <>)++ (link:{java9-javadoc}/java/util/Observer.html#update%2Djava.util.Observable%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/OffsetDateTime.asciidoc b/docs/painless/painless-api-reference/OffsetDateTime.asciidoc deleted file mode 100644 index d476c181c55f0..0000000000000 --- a/docs/painless/painless-api-reference/OffsetDateTime.asciidoc +++ /dev/null @@ -1,75 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-OffsetDateTime]]++OffsetDateTime++:: -** [[painless-api-reference-OffsetDateTime-MAX]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#MAX[MAX] (link:{java9-javadoc}/java/time/OffsetDateTime.html#MAX[java 9]) -** [[painless-api-reference-OffsetDateTime-MIN]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#MIN[MIN] (link:{java9-javadoc}/java/time/OffsetDateTime.html#MIN[java 9]) -* ++[[painless-api-reference-OffsetDateTime-from-1]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-of-2]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#of%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2D[of](<>, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#of%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-of-3]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#of%2Djava.time.LocalDate%2Djava.time.LocalTime%2Djava.time.ZoneOffset%2D[of](<>, <>, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#of%2Djava.time.LocalDate%2Djava.time.LocalTime%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-of-8]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Djava.time.ZoneOffset%2D[of](int, int, int, int, int, int, int, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-ofInstant-2]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[ofInstant](<>, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-parse-1]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-parse-2]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-timeLineOrder-0]]static <> link:{java8-javadoc}/java/time/OffsetDateTime.html#timeLineOrder%2D%2D[timeLineOrder]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#timeLineOrder%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-atZoneSameInstant-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#atZoneSameInstant%2Djava.time.ZoneId%2D[atZoneSameInstant](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#atZoneSameInstant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-atZoneSimilarLocal-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#atZoneSimilarLocal%2Djava.time.ZoneId%2D[atZoneSimilarLocal](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#atZoneSimilarLocal%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-compareTo-1]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#compareTo%2Djava.time.OffsetDateTime%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#compareTo%2Djava.time.OffsetDateTime%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-format-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getDayOfMonth-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getDayOfMonth%2D%2D[getDayOfMonth]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getDayOfWeek-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#getDayOfWeek%2D%2D[getDayOfWeek]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getDayOfYear-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getDayOfYear%2D%2D[getDayOfYear]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getDayOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getHour-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getHour%2D%2D[getHour]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getHour%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getMinute-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getMinute%2D%2D[getMinute]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getMinute%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getMonth-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getMonthValue-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getMonthValue%2D%2D[getMonthValue]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getMonthValue%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getNano-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getOffset-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#getOffset%2D%2D[getOffset]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getOffset%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getSecond-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getSecond%2D%2D[getSecond]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getSecond%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-getYear-0]]int link:{java8-javadoc}/java/time/OffsetDateTime.html#getYear%2D%2D[getYear]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#getYear%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-isAfter-1]]boolean link:{java8-javadoc}/java/time/OffsetDateTime.html#isAfter%2Djava.time.OffsetDateTime%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#isAfter%2Djava.time.OffsetDateTime%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-isBefore-1]]boolean link:{java8-javadoc}/java/time/OffsetDateTime.html#isBefore%2Djava.time.OffsetDateTime%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#isBefore%2Djava.time.OffsetDateTime%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-isEqual-1]]boolean link:{java8-javadoc}/java/time/OffsetDateTime.html#isEqual%2Djava.time.OffsetDateTime%2D[isEqual](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#isEqual%2Djava.time.OffsetDateTime%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minus-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minus-2]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusDays-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusDays%2Dlong%2D[minusDays](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusHours-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusHours%2Dlong%2D[minusHours](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusMinutes-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusMinutes%2Dlong%2D[minusMinutes](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusMonths-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusMonths%2Dlong%2D[minusMonths](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusNanos-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusSeconds-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusWeeks-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusWeeks%2Dlong%2D[minusWeeks](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-minusYears-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plus-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plus-2]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusDays-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusDays%2Dlong%2D[plusDays](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusHours-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusHours%2Dlong%2D[plusHours](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusMinutes-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusMinutes%2Dlong%2D[plusMinutes](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusMonths-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusMonths%2Dlong%2D[plusMonths](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusNanos-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusSeconds-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusWeeks-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusWeeks%2Dlong%2D[plusWeeks](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-plusYears-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toEpochSecond-0]]long link:{java8-javadoc}/java/time/OffsetDateTime.html#toEpochSecond%2D%2D[toEpochSecond]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toEpochSecond%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toInstant-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#toInstant%2D%2D[toInstant]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toInstant%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toLocalDate-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#toLocalDate%2D%2D[toLocalDate]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toLocalDate%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toLocalDateTime-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#toLocalDateTime%2D%2D[toLocalDateTime]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toLocalDateTime%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toLocalTime-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#toLocalTime%2D%2D[toLocalTime]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toLocalTime%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toOffsetTime-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#toOffsetTime%2D%2D[toOffsetTime]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toOffsetTime%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-toZonedDateTime-0]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#toZonedDateTime%2D%2D[toZonedDateTime]()++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#toZonedDateTime%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-truncatedTo-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[truncatedTo](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-with-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-with-2]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withDayOfMonth-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withDayOfMonth%2Dint%2D[withDayOfMonth](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withDayOfMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withDayOfYear-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withDayOfYear%2Dint%2D[withDayOfYear](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withDayOfYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withHour-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withHour%2Dint%2D[withHour](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withHour%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withMinute-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withMinute%2Dint%2D[withMinute](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withMinute%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withMonth-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withMonth%2Dint%2D[withMonth](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withNano-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withNano%2Dint%2D[withNano](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withNano%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withOffsetSameInstant-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withOffsetSameInstant%2Djava.time.ZoneOffset%2D[withOffsetSameInstant](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withOffsetSameInstant%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withOffsetSameLocal-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withOffsetSameLocal%2Djava.time.ZoneOffset%2D[withOffsetSameLocal](<>)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withOffsetSameLocal%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withSecond-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withSecond%2Dint%2D[withSecond](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withSecond%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetDateTime-withYear-1]]<> link:{java8-javadoc}/java/time/OffsetDateTime.html#withYear%2Dint%2D[withYear](int)++ (link:{java9-javadoc}/java/time/OffsetDateTime.html#withYear%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/OffsetTime.asciidoc b/docs/painless/painless-api-reference/OffsetTime.asciidoc deleted file mode 100644 index b23e9bb9577d0..0000000000000 --- a/docs/painless/painless-api-reference/OffsetTime.asciidoc +++ /dev/null @@ -1,47 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-OffsetTime]]++OffsetTime++:: -** [[painless-api-reference-OffsetTime-MAX]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#MAX[MAX] (link:{java9-javadoc}/java/time/OffsetTime.html#MAX[java 9]) -** [[painless-api-reference-OffsetTime-MIN]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#MIN[MIN] (link:{java9-javadoc}/java/time/OffsetTime.html#MIN[java 9]) -* ++[[painless-api-reference-OffsetTime-from-1]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-of-2]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#of%2Djava.time.LocalTime%2Djava.time.ZoneOffset%2D[of](<>, <>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#of%2Djava.time.LocalTime%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-of-5]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#of%2Dint%2Dint%2Dint%2Dint%2Djava.time.ZoneOffset%2D[of](int, int, int, int, <>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#of%2Dint%2Dint%2Dint%2Dint%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-ofInstant-2]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[ofInstant](<>, <>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-parse-1]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-parse-2]]static <> link:{java8-javadoc}/java/time/OffsetTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-compareTo-1]]int link:{java8-javadoc}/java/time/OffsetTime.html#compareTo%2Djava.time.OffsetTime%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#compareTo%2Djava.time.OffsetTime%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-format-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-getHour-0]]int link:{java8-javadoc}/java/time/OffsetTime.html#getHour%2D%2D[getHour]()++ (link:{java9-javadoc}/java/time/OffsetTime.html#getHour%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-getMinute-0]]int link:{java8-javadoc}/java/time/OffsetTime.html#getMinute%2D%2D[getMinute]()++ (link:{java9-javadoc}/java/time/OffsetTime.html#getMinute%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-getNano-0]]int link:{java8-javadoc}/java/time/OffsetTime.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/OffsetTime.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-getOffset-0]]<> link:{java8-javadoc}/java/time/OffsetTime.html#getOffset%2D%2D[getOffset]()++ (link:{java9-javadoc}/java/time/OffsetTime.html#getOffset%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-getSecond-0]]int link:{java8-javadoc}/java/time/OffsetTime.html#getSecond%2D%2D[getSecond]()++ (link:{java9-javadoc}/java/time/OffsetTime.html#getSecond%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-isAfter-1]]boolean link:{java8-javadoc}/java/time/OffsetTime.html#isAfter%2Djava.time.OffsetTime%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#isAfter%2Djava.time.OffsetTime%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-isBefore-1]]boolean link:{java8-javadoc}/java/time/OffsetTime.html#isBefore%2Djava.time.OffsetTime%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#isBefore%2Djava.time.OffsetTime%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-isEqual-1]]boolean link:{java8-javadoc}/java/time/OffsetTime.html#isEqual%2Djava.time.OffsetTime%2D[isEqual](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#isEqual%2Djava.time.OffsetTime%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-minus-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-minus-2]]<> link:{java8-javadoc}/java/time/OffsetTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-minusHours-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#minusHours%2Dlong%2D[minusHours](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#minusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-minusMinutes-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#minusMinutes%2Dlong%2D[minusMinutes](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#minusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-minusNanos-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-minusSeconds-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-plus-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-plus-2]]<> link:{java8-javadoc}/java/time/OffsetTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-plusHours-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#plusHours%2Dlong%2D[plusHours](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#plusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-plusMinutes-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#plusMinutes%2Dlong%2D[plusMinutes](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#plusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-plusNanos-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-plusSeconds-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-toLocalTime-0]]<> link:{java8-javadoc}/java/time/OffsetTime.html#toLocalTime%2D%2D[toLocalTime]()++ (link:{java9-javadoc}/java/time/OffsetTime.html#toLocalTime%2D%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-truncatedTo-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[truncatedTo](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-with-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-with-2]]<> link:{java8-javadoc}/java/time/OffsetTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/OffsetTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-withHour-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#withHour%2Dint%2D[withHour](int)++ (link:{java9-javadoc}/java/time/OffsetTime.html#withHour%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-withMinute-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#withMinute%2Dint%2D[withMinute](int)++ (link:{java9-javadoc}/java/time/OffsetTime.html#withMinute%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-withNano-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#withNano%2Dint%2D[withNano](int)++ (link:{java9-javadoc}/java/time/OffsetTime.html#withNano%2Dint%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-withOffsetSameInstant-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#withOffsetSameInstant%2Djava.time.ZoneOffset%2D[withOffsetSameInstant](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#withOffsetSameInstant%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-withOffsetSameLocal-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#withOffsetSameLocal%2Djava.time.ZoneOffset%2D[withOffsetSameLocal](<>)++ (link:{java9-javadoc}/java/time/OffsetTime.html#withOffsetSameLocal%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-OffsetTime-withSecond-1]]<> link:{java8-javadoc}/java/time/OffsetTime.html#withSecond%2Dint%2D[withSecond](int)++ (link:{java9-javadoc}/java/time/OffsetTime.html#withSecond%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Optional.asciidoc b/docs/painless/painless-api-reference/Optional.asciidoc deleted file mode 100644 index a67616f006fea..0000000000000 --- a/docs/painless/painless-api-reference/Optional.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Optional]]++Optional++:: -* ++[[painless-api-reference-Optional-empty-0]]static <> link:{java8-javadoc}/java/util/Optional.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/Optional.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-Optional-of-1]]static <> link:{java8-javadoc}/java/util/Optional.html#of%2Djava.lang.Object%2D[of](def)++ (link:{java9-javadoc}/java/util/Optional.html#of%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Optional-ofNullable-1]]static <> link:{java8-javadoc}/java/util/Optional.html#ofNullable%2Djava.lang.Object%2D[ofNullable](def)++ (link:{java9-javadoc}/java/util/Optional.html#ofNullable%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Optional-filter-1]]<> link:{java8-javadoc}/java/util/Optional.html#filter%2Djava.util.function.Predicate%2D[filter](<>)++ (link:{java9-javadoc}/java/util/Optional.html#filter%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Optional-flatMap-1]]<> link:{java8-javadoc}/java/util/Optional.html#flatMap%2Djava.util.function.Function%2D[flatMap](<>)++ (link:{java9-javadoc}/java/util/Optional.html#flatMap%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Optional-get-0]]def link:{java8-javadoc}/java/util/Optional.html#get%2D%2D[get]()++ (link:{java9-javadoc}/java/util/Optional.html#get%2D%2D[java 9]) -* ++[[painless-api-reference-Optional-ifPresent-1]]void link:{java8-javadoc}/java/util/Optional.html#ifPresent%2Djava.util.function.Consumer%2D[ifPresent](<>)++ (link:{java9-javadoc}/java/util/Optional.html#ifPresent%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Optional-isPresent-0]]boolean link:{java8-javadoc}/java/util/Optional.html#isPresent%2D%2D[isPresent]()++ (link:{java9-javadoc}/java/util/Optional.html#isPresent%2D%2D[java 9]) -* ++[[painless-api-reference-Optional-map-1]]<> link:{java8-javadoc}/java/util/Optional.html#map%2Djava.util.function.Function%2D[map](<>)++ (link:{java9-javadoc}/java/util/Optional.html#map%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Optional-orElse-1]]def link:{java8-javadoc}/java/util/Optional.html#orElse%2Djava.lang.Object%2D[orElse](def)++ (link:{java9-javadoc}/java/util/Optional.html#orElse%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Optional-orElseGet-1]]def link:{java8-javadoc}/java/util/Optional.html#orElseGet%2Djava.util.function.Supplier%2D[orElseGet](<>)++ (link:{java9-javadoc}/java/util/Optional.html#orElseGet%2Djava.util.function.Supplier%2D[java 9]) -* ++[[painless-api-reference-Optional-orElseThrow-1]]def link:{java8-javadoc}/java/util/Optional.html#orElseThrow%2Djava.util.function.Supplier%2D[orElseThrow](<>)++ (link:{java9-javadoc}/java/util/Optional.html#orElseThrow%2Djava.util.function.Supplier%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/OptionalDouble.asciidoc b/docs/painless/painless-api-reference/OptionalDouble.asciidoc deleted file mode 100644 index 56ac37deaa336..0000000000000 --- a/docs/painless/painless-api-reference/OptionalDouble.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-OptionalDouble]]++OptionalDouble++:: -* ++[[painless-api-reference-OptionalDouble-empty-0]]static <> link:{java8-javadoc}/java/util/OptionalDouble.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/OptionalDouble.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-of-1]]static <> link:{java8-javadoc}/java/util/OptionalDouble.html#of%2Ddouble%2D[of](double)++ (link:{java9-javadoc}/java/util/OptionalDouble.html#of%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-getAsDouble-0]]double link:{java8-javadoc}/java/util/OptionalDouble.html#getAsDouble%2D%2D[getAsDouble]()++ (link:{java9-javadoc}/java/util/OptionalDouble.html#getAsDouble%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-ifPresent-1]]void link:{java8-javadoc}/java/util/OptionalDouble.html#ifPresent%2Djava.util.function.DoubleConsumer%2D[ifPresent](<>)++ (link:{java9-javadoc}/java/util/OptionalDouble.html#ifPresent%2Djava.util.function.DoubleConsumer%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-isPresent-0]]boolean link:{java8-javadoc}/java/util/OptionalDouble.html#isPresent%2D%2D[isPresent]()++ (link:{java9-javadoc}/java/util/OptionalDouble.html#isPresent%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-orElse-1]]double link:{java8-javadoc}/java/util/OptionalDouble.html#orElse%2Ddouble%2D[orElse](double)++ (link:{java9-javadoc}/java/util/OptionalDouble.html#orElse%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-orElseGet-1]]double link:{java8-javadoc}/java/util/OptionalDouble.html#orElseGet%2Djava.util.function.DoubleSupplier%2D[orElseGet](<>)++ (link:{java9-javadoc}/java/util/OptionalDouble.html#orElseGet%2Djava.util.function.DoubleSupplier%2D[java 9]) -* ++[[painless-api-reference-OptionalDouble-orElseThrow-1]]double link:{java8-javadoc}/java/util/OptionalDouble.html#orElseThrow%2Djava.util.function.Supplier%2D[orElseThrow](<>)++ (link:{java9-javadoc}/java/util/OptionalDouble.html#orElseThrow%2Djava.util.function.Supplier%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/OptionalInt.asciidoc b/docs/painless/painless-api-reference/OptionalInt.asciidoc deleted file mode 100644 index 69f0059b85f9c..0000000000000 --- a/docs/painless/painless-api-reference/OptionalInt.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-OptionalInt]]++OptionalInt++:: -* ++[[painless-api-reference-OptionalInt-empty-0]]static <> link:{java8-javadoc}/java/util/OptionalInt.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/OptionalInt.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-of-1]]static <> link:{java8-javadoc}/java/util/OptionalInt.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/util/OptionalInt.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-getAsInt-0]]int link:{java8-javadoc}/java/util/OptionalInt.html#getAsInt%2D%2D[getAsInt]()++ (link:{java9-javadoc}/java/util/OptionalInt.html#getAsInt%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-ifPresent-1]]void link:{java8-javadoc}/java/util/OptionalInt.html#ifPresent%2Djava.util.function.IntConsumer%2D[ifPresent](<>)++ (link:{java9-javadoc}/java/util/OptionalInt.html#ifPresent%2Djava.util.function.IntConsumer%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-isPresent-0]]boolean link:{java8-javadoc}/java/util/OptionalInt.html#isPresent%2D%2D[isPresent]()++ (link:{java9-javadoc}/java/util/OptionalInt.html#isPresent%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-orElse-1]]int link:{java8-javadoc}/java/util/OptionalInt.html#orElse%2Dint%2D[orElse](int)++ (link:{java9-javadoc}/java/util/OptionalInt.html#orElse%2Dint%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-orElseGet-1]]int link:{java8-javadoc}/java/util/OptionalInt.html#orElseGet%2Djava.util.function.IntSupplier%2D[orElseGet](<>)++ (link:{java9-javadoc}/java/util/OptionalInt.html#orElseGet%2Djava.util.function.IntSupplier%2D[java 9]) -* ++[[painless-api-reference-OptionalInt-orElseThrow-1]]int link:{java8-javadoc}/java/util/OptionalInt.html#orElseThrow%2Djava.util.function.Supplier%2D[orElseThrow](<>)++ (link:{java9-javadoc}/java/util/OptionalInt.html#orElseThrow%2Djava.util.function.Supplier%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/OptionalLong.asciidoc b/docs/painless/painless-api-reference/OptionalLong.asciidoc deleted file mode 100644 index a4b135e55115a..0000000000000 --- a/docs/painless/painless-api-reference/OptionalLong.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-OptionalLong]]++OptionalLong++:: -* ++[[painless-api-reference-OptionalLong-empty-0]]static <> link:{java8-javadoc}/java/util/OptionalLong.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/OptionalLong.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-of-1]]static <> link:{java8-javadoc}/java/util/OptionalLong.html#of%2Dlong%2D[of](long)++ (link:{java9-javadoc}/java/util/OptionalLong.html#of%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-getAsLong-0]]long link:{java8-javadoc}/java/util/OptionalLong.html#getAsLong%2D%2D[getAsLong]()++ (link:{java9-javadoc}/java/util/OptionalLong.html#getAsLong%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-ifPresent-1]]void link:{java8-javadoc}/java/util/OptionalLong.html#ifPresent%2Djava.util.function.LongConsumer%2D[ifPresent](<>)++ (link:{java9-javadoc}/java/util/OptionalLong.html#ifPresent%2Djava.util.function.LongConsumer%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-isPresent-0]]boolean link:{java8-javadoc}/java/util/OptionalLong.html#isPresent%2D%2D[isPresent]()++ (link:{java9-javadoc}/java/util/OptionalLong.html#isPresent%2D%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-orElse-1]]long link:{java8-javadoc}/java/util/OptionalLong.html#orElse%2Dlong%2D[orElse](long)++ (link:{java9-javadoc}/java/util/OptionalLong.html#orElse%2Dlong%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-orElseGet-1]]long link:{java8-javadoc}/java/util/OptionalLong.html#orElseGet%2Djava.util.function.LongSupplier%2D[orElseGet](<>)++ (link:{java9-javadoc}/java/util/OptionalLong.html#orElseGet%2Djava.util.function.LongSupplier%2D[java 9]) -* ++[[painless-api-reference-OptionalLong-orElseThrow-1]]long link:{java8-javadoc}/java/util/OptionalLong.html#orElseThrow%2Djava.util.function.Supplier%2D[orElseThrow](<>)++ (link:{java9-javadoc}/java/util/OptionalLong.html#orElseThrow%2Djava.util.function.Supplier%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ParseException.asciidoc b/docs/painless/painless-api-reference/ParseException.asciidoc deleted file mode 100644 index 086bcd0b7aaed..0000000000000 --- a/docs/painless/painless-api-reference/ParseException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ParseException]]++ParseException++:: -* ++[[painless-api-reference-ParseException-ParseException-2]]link:{java8-javadoc}/java/text/ParseException.html#ParseException%2Djava.lang.String%2Dint%2D[ParseException](<>, int)++ (link:{java9-javadoc}/java/text/ParseException.html#ParseException%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-ParseException-getErrorOffset-0]]int link:{java8-javadoc}/java/text/ParseException.html#getErrorOffset%2D%2D[getErrorOffset]()++ (link:{java9-javadoc}/java/text/ParseException.html#getErrorOffset%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ParsePosition.asciidoc b/docs/painless/painless-api-reference/ParsePosition.asciidoc deleted file mode 100644 index 88fe6369657bd..0000000000000 --- a/docs/painless/painless-api-reference/ParsePosition.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ParsePosition]]++ParsePosition++:: -* ++[[painless-api-reference-ParsePosition-ParsePosition-1]]link:{java8-javadoc}/java/text/ParsePosition.html#ParsePosition%2Dint%2D[ParsePosition](int)++ (link:{java9-javadoc}/java/text/ParsePosition.html#ParsePosition%2Dint%2D[java 9]) -* ++[[painless-api-reference-ParsePosition-getErrorIndex-0]]int link:{java8-javadoc}/java/text/ParsePosition.html#getErrorIndex%2D%2D[getErrorIndex]()++ (link:{java9-javadoc}/java/text/ParsePosition.html#getErrorIndex%2D%2D[java 9]) -* ++[[painless-api-reference-ParsePosition-getIndex-0]]int link:{java8-javadoc}/java/text/ParsePosition.html#getIndex%2D%2D[getIndex]()++ (link:{java9-javadoc}/java/text/ParsePosition.html#getIndex%2D%2D[java 9]) -* ++[[painless-api-reference-ParsePosition-setErrorIndex-1]]void link:{java8-javadoc}/java/text/ParsePosition.html#setErrorIndex%2Dint%2D[setErrorIndex](int)++ (link:{java9-javadoc}/java/text/ParsePosition.html#setErrorIndex%2Dint%2D[java 9]) -* ++[[painless-api-reference-ParsePosition-setIndex-1]]void link:{java8-javadoc}/java/text/ParsePosition.html#setIndex%2Dint%2D[setIndex](int)++ (link:{java9-javadoc}/java/text/ParsePosition.html#setIndex%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Pattern.asciidoc b/docs/painless/painless-api-reference/Pattern.asciidoc deleted file mode 100644 index ad7b3603036b1..0000000000000 --- a/docs/painless/painless-api-reference/Pattern.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Pattern]]++Pattern++:: -* ++[[painless-api-reference-Pattern-quote-1]]static <> link:{java8-javadoc}/java/util/regex/Pattern.html#quote%2Djava.lang.String%2D[quote](<>)++ (link:{java9-javadoc}/java/util/regex/Pattern.html#quote%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Pattern-asPredicate-0]]<> link:{java8-javadoc}/java/util/regex/Pattern.html#asPredicate%2D%2D[asPredicate]()++ (link:{java9-javadoc}/java/util/regex/Pattern.html#asPredicate%2D%2D[java 9]) -* ++[[painless-api-reference-Pattern-flags-0]]int link:{java8-javadoc}/java/util/regex/Pattern.html#flags%2D%2D[flags]()++ (link:{java9-javadoc}/java/util/regex/Pattern.html#flags%2D%2D[java 9]) -* ++[[painless-api-reference-Pattern-matcher-1]]<> link:{java8-javadoc}/java/util/regex/Pattern.html#matcher%2Djava.lang.CharSequence%2D[matcher](<>)++ (link:{java9-javadoc}/java/util/regex/Pattern.html#matcher%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Pattern-pattern-0]]<> link:{java8-javadoc}/java/util/regex/Pattern.html#pattern%2D%2D[pattern]()++ (link:{java9-javadoc}/java/util/regex/Pattern.html#pattern%2D%2D[java 9]) -* ++[[painless-api-reference-Pattern-split-1]]<>[] link:{java8-javadoc}/java/util/regex/Pattern.html#split%2Djava.lang.CharSequence%2D[split](<>)++ (link:{java9-javadoc}/java/util/regex/Pattern.html#split%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Pattern-split-2]]<>[] link:{java8-javadoc}/java/util/regex/Pattern.html#split%2Djava.lang.CharSequence%2Dint%2D[split](<>, int)++ (link:{java9-javadoc}/java/util/regex/Pattern.html#split%2Djava.lang.CharSequence%2Dint%2D[java 9]) -* ++[[painless-api-reference-Pattern-splitAsStream-1]]<> link:{java8-javadoc}/java/util/regex/Pattern.html#splitAsStream%2Djava.lang.CharSequence%2D[splitAsStream](<>)++ (link:{java9-javadoc}/java/util/regex/Pattern.html#splitAsStream%2Djava.lang.CharSequence%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Period.asciidoc b/docs/painless/painless-api-reference/Period.asciidoc deleted file mode 100644 index bb20dddae60a8..0000000000000 --- a/docs/painless/painless-api-reference/Period.asciidoc +++ /dev/null @@ -1,35 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Period]]++Period++:: -** [[painless-api-reference-Period-ZERO]]static <> link:{java8-javadoc}/java/time/Period.html#ZERO[ZERO] (link:{java9-javadoc}/java/time/Period.html#ZERO[java 9]) -* ++[[painless-api-reference-Period-between-2]]static <> link:{java8-javadoc}/java/time/Period.html#between%2Djava.time.LocalDate%2Djava.time.LocalDate%2D[between](<>, <>)++ (link:{java9-javadoc}/java/time/Period.html#between%2Djava.time.LocalDate%2Djava.time.LocalDate%2D[java 9]) -* ++[[painless-api-reference-Period-from-1]]static <> link:{java8-javadoc}/java/time/Period.html#from%2Djava.time.temporal.TemporalAmount%2D[from](<>)++ (link:{java9-javadoc}/java/time/Period.html#from%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Period-of-3]]static <> link:{java8-javadoc}/java/time/Period.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/Period.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-ofDays-1]]static <> link:{java8-javadoc}/java/time/Period.html#ofDays%2Dint%2D[ofDays](int)++ (link:{java9-javadoc}/java/time/Period.html#ofDays%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-ofMonths-1]]static <> link:{java8-javadoc}/java/time/Period.html#ofMonths%2Dint%2D[ofMonths](int)++ (link:{java9-javadoc}/java/time/Period.html#ofMonths%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-ofWeeks-1]]static <> link:{java8-javadoc}/java/time/Period.html#ofWeeks%2Dint%2D[ofWeeks](int)++ (link:{java9-javadoc}/java/time/Period.html#ofWeeks%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-ofYears-1]]static <> link:{java8-javadoc}/java/time/Period.html#ofYears%2Dint%2D[ofYears](int)++ (link:{java9-javadoc}/java/time/Period.html#ofYears%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-parse-1]]static <> link:{java8-javadoc}/java/time/Period.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/Period.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Period-getChronology-0]]<> link:{java8-javadoc}/java/time/Period.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/Period.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-Period-getDays-0]]int link:{java8-javadoc}/java/time/Period.html#getDays%2D%2D[getDays]()++ (link:{java9-javadoc}/java/time/Period.html#getDays%2D%2D[java 9]) -* ++[[painless-api-reference-Period-getMonths-0]]int link:{java8-javadoc}/java/time/Period.html#getMonths%2D%2D[getMonths]()++ (link:{java9-javadoc}/java/time/Period.html#getMonths%2D%2D[java 9]) -* ++[[painless-api-reference-Period-getYears-0]]int link:{java8-javadoc}/java/time/Period.html#getYears%2D%2D[getYears]()++ (link:{java9-javadoc}/java/time/Period.html#getYears%2D%2D[java 9]) -* ++[[painless-api-reference-Period-minus-1]]<> link:{java8-javadoc}/java/time/Period.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/Period.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Period-minusDays-1]]<> link:{java8-javadoc}/java/time/Period.html#minusDays%2Dlong%2D[minusDays](long)++ (link:{java9-javadoc}/java/time/Period.html#minusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Period-minusMonths-1]]<> link:{java8-javadoc}/java/time/Period.html#minusMonths%2Dlong%2D[minusMonths](long)++ (link:{java9-javadoc}/java/time/Period.html#minusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Period-minusYears-1]]<> link:{java8-javadoc}/java/time/Period.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/Period.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Period-multipliedBy-1]]<> link:{java8-javadoc}/java/time/Period.html#multipliedBy%2Dint%2D[multipliedBy](int)++ (link:{java9-javadoc}/java/time/Period.html#multipliedBy%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-negated-0]]<> link:{java8-javadoc}/java/time/Period.html#negated%2D%2D[negated]()++ (link:{java9-javadoc}/java/time/Period.html#negated%2D%2D[java 9]) -* ++[[painless-api-reference-Period-normalized-0]]<> link:{java8-javadoc}/java/time/Period.html#normalized%2D%2D[normalized]()++ (link:{java9-javadoc}/java/time/Period.html#normalized%2D%2D[java 9]) -* ++[[painless-api-reference-Period-plus-1]]<> link:{java8-javadoc}/java/time/Period.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/Period.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Period-plusDays-1]]<> link:{java8-javadoc}/java/time/Period.html#plusDays%2Dlong%2D[plusDays](long)++ (link:{java9-javadoc}/java/time/Period.html#plusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Period-plusMonths-1]]<> link:{java8-javadoc}/java/time/Period.html#plusMonths%2Dlong%2D[plusMonths](long)++ (link:{java9-javadoc}/java/time/Period.html#plusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Period-plusYears-1]]<> link:{java8-javadoc}/java/time/Period.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/Period.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Period-toTotalMonths-0]]long link:{java8-javadoc}/java/time/Period.html#toTotalMonths%2D%2D[toTotalMonths]()++ (link:{java9-javadoc}/java/time/Period.html#toTotalMonths%2D%2D[java 9]) -* ++[[painless-api-reference-Period-withDays-1]]<> link:{java8-javadoc}/java/time/Period.html#withDays%2Dint%2D[withDays](int)++ (link:{java9-javadoc}/java/time/Period.html#withDays%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-withMonths-1]]<> link:{java8-javadoc}/java/time/Period.html#withMonths%2Dint%2D[withMonths](int)++ (link:{java9-javadoc}/java/time/Period.html#withMonths%2Dint%2D[java 9]) -* ++[[painless-api-reference-Period-withYears-1]]<> link:{java8-javadoc}/java/time/Period.html#withYears%2Dint%2D[withYears](int)++ (link:{java9-javadoc}/java/time/Period.html#withYears%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Predicate.asciidoc b/docs/painless/painless-api-reference/Predicate.asciidoc deleted file mode 100644 index 6eef93ea27c02..0000000000000 --- a/docs/painless/painless-api-reference/Predicate.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Predicate]]++Predicate++:: -* ++[[painless-api-reference-Predicate-isEqual-1]]static <> link:{java8-javadoc}/java/util/function/Predicate.html#isEqual%2Djava.lang.Object%2D[isEqual](def)++ (link:{java9-javadoc}/java/util/function/Predicate.html#isEqual%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Predicate-and-1]]<> link:{java8-javadoc}/java/util/function/Predicate.html#and%2Djava.util.function.Predicate%2D[and](<>)++ (link:{java9-javadoc}/java/util/function/Predicate.html#and%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Predicate-negate-0]]<> link:{java8-javadoc}/java/util/function/Predicate.html#negate%2D%2D[negate]()++ (link:{java9-javadoc}/java/util/function/Predicate.html#negate%2D%2D[java 9]) -* ++[[painless-api-reference-Predicate-or-1]]<> link:{java8-javadoc}/java/util/function/Predicate.html#or%2Djava.util.function.Predicate%2D[or](<>)++ (link:{java9-javadoc}/java/util/function/Predicate.html#or%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Predicate-test-1]]boolean link:{java8-javadoc}/java/util/function/Predicate.html#test%2Djava.lang.Object%2D[test](def)++ (link:{java9-javadoc}/java/util/function/Predicate.html#test%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/PrimitiveIterator.OfDouble.asciidoc b/docs/painless/painless-api-reference/PrimitiveIterator.OfDouble.asciidoc deleted file mode 100644 index 678c1265f04c9..0000000000000 --- a/docs/painless/painless-api-reference/PrimitiveIterator.OfDouble.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-PrimitiveIterator-OfDouble]]++PrimitiveIterator.OfDouble++:: -* ++[[painless-api-reference-PrimitiveIterator-OfDouble-next-0]]<> link:{java8-javadoc}/java/util/PrimitiveIterator.OfDouble.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/util/PrimitiveIterator.OfDouble.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-PrimitiveIterator-OfDouble-nextDouble-0]]double link:{java8-javadoc}/java/util/PrimitiveIterator.OfDouble.html#nextDouble%2D%2D[nextDouble]()++ (link:{java9-javadoc}/java/util/PrimitiveIterator.OfDouble.html#nextDouble%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/PrimitiveIterator.OfInt.asciidoc b/docs/painless/painless-api-reference/PrimitiveIterator.OfInt.asciidoc deleted file mode 100644 index 0f7c1f31f0366..0000000000000 --- a/docs/painless/painless-api-reference/PrimitiveIterator.OfInt.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-PrimitiveIterator-OfInt]]++PrimitiveIterator.OfInt++:: -* ++[[painless-api-reference-PrimitiveIterator-OfInt-next-0]]<> link:{java8-javadoc}/java/util/PrimitiveIterator.OfInt.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/util/PrimitiveIterator.OfInt.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-PrimitiveIterator-OfInt-nextInt-0]]int link:{java8-javadoc}/java/util/PrimitiveIterator.OfInt.html#nextInt%2D%2D[nextInt]()++ (link:{java9-javadoc}/java/util/PrimitiveIterator.OfInt.html#nextInt%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/PrimitiveIterator.OfLong.asciidoc b/docs/painless/painless-api-reference/PrimitiveIterator.OfLong.asciidoc deleted file mode 100644 index ce5b3f5b3a72a..0000000000000 --- a/docs/painless/painless-api-reference/PrimitiveIterator.OfLong.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-PrimitiveIterator-OfLong]]++PrimitiveIterator.OfLong++:: -* ++[[painless-api-reference-PrimitiveIterator-OfLong-next-0]]<> link:{java8-javadoc}/java/util/PrimitiveIterator.OfLong.html#next%2D%2D[next]()++ (link:{java9-javadoc}/java/util/PrimitiveIterator.OfLong.html#next%2D%2D[java 9]) -* ++[[painless-api-reference-PrimitiveIterator-OfLong-nextLong-0]]long link:{java8-javadoc}/java/util/PrimitiveIterator.OfLong.html#nextLong%2D%2D[nextLong]()++ (link:{java9-javadoc}/java/util/PrimitiveIterator.OfLong.html#nextLong%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/PrimitiveIterator.asciidoc b/docs/painless/painless-api-reference/PrimitiveIterator.asciidoc deleted file mode 100644 index 42742f79d3d40..0000000000000 --- a/docs/painless/painless-api-reference/PrimitiveIterator.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-PrimitiveIterator]]++PrimitiveIterator++:: -* ++[[painless-api-reference-PrimitiveIterator-forEachRemaining-1]]void link:{java8-javadoc}/java/util/PrimitiveIterator.html#forEachRemaining%2Djava.lang.Object%2D[forEachRemaining](def)++ (link:{java9-javadoc}/java/util/PrimitiveIterator.html#forEachRemaining%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/PriorityQueue.asciidoc b/docs/painless/painless-api-reference/PriorityQueue.asciidoc deleted file mode 100644 index 88334cd8b9c63..0000000000000 --- a/docs/painless/painless-api-reference/PriorityQueue.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-PriorityQueue]]++PriorityQueue++:: -* ++[[painless-api-reference-PriorityQueue-PriorityQueue-0]]link:{java8-javadoc}/java/util/PriorityQueue.html#PriorityQueue%2D%2D[PriorityQueue]()++ (link:{java9-javadoc}/java/util/PriorityQueue.html#PriorityQueue%2D%2D[java 9]) -* ++[[painless-api-reference-PriorityQueue-PriorityQueue-1]]link:{java8-javadoc}/java/util/PriorityQueue.html#PriorityQueue%2Djava.util.Comparator%2D[PriorityQueue](<>)++ (link:{java9-javadoc}/java/util/PriorityQueue.html#PriorityQueue%2Djava.util.Comparator%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Queue.asciidoc b/docs/painless/painless-api-reference/Queue.asciidoc deleted file mode 100644 index 7d323457591ff..0000000000000 --- a/docs/painless/painless-api-reference/Queue.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Queue]]++Queue++:: -* ++[[painless-api-reference-Queue-element-0]]def link:{java8-javadoc}/java/util/Queue.html#element%2D%2D[element]()++ (link:{java9-javadoc}/java/util/Queue.html#element%2D%2D[java 9]) -* ++[[painless-api-reference-Queue-offer-1]]boolean link:{java8-javadoc}/java/util/Queue.html#offer%2Djava.lang.Object%2D[offer](def)++ (link:{java9-javadoc}/java/util/Queue.html#offer%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Queue-peek-0]]def link:{java8-javadoc}/java/util/Queue.html#peek%2D%2D[peek]()++ (link:{java9-javadoc}/java/util/Queue.html#peek%2D%2D[java 9]) -* ++[[painless-api-reference-Queue-poll-0]]def link:{java8-javadoc}/java/util/Queue.html#poll%2D%2D[poll]()++ (link:{java9-javadoc}/java/util/Queue.html#poll%2D%2D[java 9]) -* ++[[painless-api-reference-Queue-remove-0]]def link:{java8-javadoc}/java/util/Queue.html#remove%2D%2D[remove]()++ (link:{java9-javadoc}/java/util/Queue.html#remove%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Random.asciidoc b/docs/painless/painless-api-reference/Random.asciidoc deleted file mode 100644 index 4288c330850da..0000000000000 --- a/docs/painless/painless-api-reference/Random.asciidoc +++ /dev/null @@ -1,24 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Random]]++Random++:: -* ++[[painless-api-reference-Random-Random-0]]link:{java8-javadoc}/java/util/Random.html#Random%2D%2D[Random]()++ (link:{java9-javadoc}/java/util/Random.html#Random%2D%2D[java 9]) -* ++[[painless-api-reference-Random-Random-1]]link:{java8-javadoc}/java/util/Random.html#Random%2Dlong%2D[Random](long)++ (link:{java9-javadoc}/java/util/Random.html#Random%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Random-doubles-1]]<> link:{java8-javadoc}/java/util/Random.html#doubles%2Dlong%2D[doubles](long)++ (link:{java9-javadoc}/java/util/Random.html#doubles%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Random-doubles-3]]<> link:{java8-javadoc}/java/util/Random.html#doubles%2Dlong%2Ddouble%2Ddouble%2D[doubles](long, double, double)++ (link:{java9-javadoc}/java/util/Random.html#doubles%2Dlong%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-Random-ints-1]]<> link:{java8-javadoc}/java/util/Random.html#ints%2Dlong%2D[ints](long)++ (link:{java9-javadoc}/java/util/Random.html#ints%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Random-ints-3]]<> link:{java8-javadoc}/java/util/Random.html#ints%2Dlong%2Dint%2Dint%2D[ints](long, int, int)++ (link:{java9-javadoc}/java/util/Random.html#ints%2Dlong%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-Random-longs-1]]<> link:{java8-javadoc}/java/util/Random.html#longs%2Dlong%2D[longs](long)++ (link:{java9-javadoc}/java/util/Random.html#longs%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Random-longs-3]]<> link:{java8-javadoc}/java/util/Random.html#longs%2Dlong%2Dlong%2Dlong%2D[longs](long, long, long)++ (link:{java9-javadoc}/java/util/Random.html#longs%2Dlong%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Random-nextBoolean-0]]boolean link:{java8-javadoc}/java/util/Random.html#nextBoolean%2D%2D[nextBoolean]()++ (link:{java9-javadoc}/java/util/Random.html#nextBoolean%2D%2D[java 9]) -* ++[[painless-api-reference-Random-nextBytes-1]]void link:{java8-javadoc}/java/util/Random.html#nextBytes%2Dbyte:A%2D[nextBytes](byte[])++ (link:{java9-javadoc}/java/util/Random.html#nextBytes%2Dbyte:A%2D[java 9]) -* ++[[painless-api-reference-Random-nextDouble-0]]double link:{java8-javadoc}/java/util/Random.html#nextDouble%2D%2D[nextDouble]()++ (link:{java9-javadoc}/java/util/Random.html#nextDouble%2D%2D[java 9]) -* ++[[painless-api-reference-Random-nextFloat-0]]float link:{java8-javadoc}/java/util/Random.html#nextFloat%2D%2D[nextFloat]()++ (link:{java9-javadoc}/java/util/Random.html#nextFloat%2D%2D[java 9]) -* ++[[painless-api-reference-Random-nextGaussian-0]]double link:{java8-javadoc}/java/util/Random.html#nextGaussian%2D%2D[nextGaussian]()++ (link:{java9-javadoc}/java/util/Random.html#nextGaussian%2D%2D[java 9]) -* ++[[painless-api-reference-Random-nextInt-0]]int link:{java8-javadoc}/java/util/Random.html#nextInt%2D%2D[nextInt]()++ (link:{java9-javadoc}/java/util/Random.html#nextInt%2D%2D[java 9]) -* ++[[painless-api-reference-Random-nextInt-1]]int link:{java8-javadoc}/java/util/Random.html#nextInt%2Dint%2D[nextInt](int)++ (link:{java9-javadoc}/java/util/Random.html#nextInt%2Dint%2D[java 9]) -* ++[[painless-api-reference-Random-nextLong-0]]long link:{java8-javadoc}/java/util/Random.html#nextLong%2D%2D[nextLong]()++ (link:{java9-javadoc}/java/util/Random.html#nextLong%2D%2D[java 9]) -* ++[[painless-api-reference-Random-setSeed-1]]void link:{java8-javadoc}/java/util/Random.html#setSeed%2Dlong%2D[setSeed](long)++ (link:{java9-javadoc}/java/util/Random.html#setSeed%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/RandomAccess.asciidoc b/docs/painless/painless-api-reference/RandomAccess.asciidoc deleted file mode 100644 index b7afb205f0c6c..0000000000000 --- a/docs/painless/painless-api-reference/RandomAccess.asciidoc +++ /dev/null @@ -1,7 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-RandomAccess]]++RandomAccess++:: -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ReflectiveOperationException.asciidoc b/docs/painless/painless-api-reference/ReflectiveOperationException.asciidoc deleted file mode 100644 index eee134d373a93..0000000000000 --- a/docs/painless/painless-api-reference/ReflectiveOperationException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ReflectiveOperationException]]++ReflectiveOperationException++:: -* ++[[painless-api-reference-ReflectiveOperationException-ReflectiveOperationException-0]]link:{java8-javadoc}/java/lang/ReflectiveOperationException.html#ReflectiveOperationException%2D%2D[ReflectiveOperationException]()++ (link:{java9-javadoc}/java/lang/ReflectiveOperationException.html#ReflectiveOperationException%2D%2D[java 9]) -* ++[[painless-api-reference-ReflectiveOperationException-ReflectiveOperationException-1]]link:{java8-javadoc}/java/lang/ReflectiveOperationException.html#ReflectiveOperationException%2Djava.lang.String%2D[ReflectiveOperationException](<>)++ (link:{java9-javadoc}/java/lang/ReflectiveOperationException.html#ReflectiveOperationException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ResolverStyle.asciidoc b/docs/painless/painless-api-reference/ResolverStyle.asciidoc deleted file mode 100644 index 00576a2d93a0e..0000000000000 --- a/docs/painless/painless-api-reference/ResolverStyle.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ResolverStyle]]++ResolverStyle++:: -** [[painless-api-reference-ResolverStyle-LENIENT]]static <> link:{java8-javadoc}/java/time/format/ResolverStyle.html#LENIENT[LENIENT] (link:{java9-javadoc}/java/time/format/ResolverStyle.html#LENIENT[java 9]) -** [[painless-api-reference-ResolverStyle-SMART]]static <> link:{java8-javadoc}/java/time/format/ResolverStyle.html#SMART[SMART] (link:{java9-javadoc}/java/time/format/ResolverStyle.html#SMART[java 9]) -** [[painless-api-reference-ResolverStyle-STRICT]]static <> link:{java8-javadoc}/java/time/format/ResolverStyle.html#STRICT[STRICT] (link:{java9-javadoc}/java/time/format/ResolverStyle.html#STRICT[java 9]) -* ++[[painless-api-reference-ResolverStyle-valueOf-1]]static <> link:{java8-javadoc}/java/time/format/ResolverStyle.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/format/ResolverStyle.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ResolverStyle-values-0]]static <>[] link:{java8-javadoc}/java/time/format/ResolverStyle.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/format/ResolverStyle.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/RoundingMode.asciidoc b/docs/painless/painless-api-reference/RoundingMode.asciidoc deleted file mode 100644 index e0b4d579835e9..0000000000000 --- a/docs/painless/painless-api-reference/RoundingMode.asciidoc +++ /dev/null @@ -1,17 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-RoundingMode]]++RoundingMode++:: -** [[painless-api-reference-RoundingMode-CEILING]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#CEILING[CEILING] (link:{java9-javadoc}/java/math/RoundingMode.html#CEILING[java 9]) -** [[painless-api-reference-RoundingMode-DOWN]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#DOWN[DOWN] (link:{java9-javadoc}/java/math/RoundingMode.html#DOWN[java 9]) -** [[painless-api-reference-RoundingMode-FLOOR]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#FLOOR[FLOOR] (link:{java9-javadoc}/java/math/RoundingMode.html#FLOOR[java 9]) -** [[painless-api-reference-RoundingMode-HALF_DOWN]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#HALF_DOWN[HALF_DOWN] (link:{java9-javadoc}/java/math/RoundingMode.html#HALF_DOWN[java 9]) -** [[painless-api-reference-RoundingMode-HALF_EVEN]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#HALF_EVEN[HALF_EVEN] (link:{java9-javadoc}/java/math/RoundingMode.html#HALF_EVEN[java 9]) -** [[painless-api-reference-RoundingMode-HALF_UP]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#HALF_UP[HALF_UP] (link:{java9-javadoc}/java/math/RoundingMode.html#HALF_UP[java 9]) -** [[painless-api-reference-RoundingMode-UNNECESSARY]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#UNNECESSARY[UNNECESSARY] (link:{java9-javadoc}/java/math/RoundingMode.html#UNNECESSARY[java 9]) -** [[painless-api-reference-RoundingMode-UP]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#UP[UP] (link:{java9-javadoc}/java/math/RoundingMode.html#UP[java 9]) -* ++[[painless-api-reference-RoundingMode-valueOf-1]]static <> link:{java8-javadoc}/java/math/RoundingMode.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/math/RoundingMode.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-RoundingMode-values-0]]static <>[] link:{java8-javadoc}/java/math/RoundingMode.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/math/RoundingMode.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/RuleBasedCollator.asciidoc b/docs/painless/painless-api-reference/RuleBasedCollator.asciidoc deleted file mode 100644 index 5581768e2f300..0000000000000 --- a/docs/painless/painless-api-reference/RuleBasedCollator.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-RuleBasedCollator]]++RuleBasedCollator++:: -* ++[[painless-api-reference-RuleBasedCollator-RuleBasedCollator-1]]link:{java8-javadoc}/java/text/RuleBasedCollator.html#RuleBasedCollator%2Djava.lang.String%2D[RuleBasedCollator](<>)++ (link:{java9-javadoc}/java/text/RuleBasedCollator.html#RuleBasedCollator%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-RuleBasedCollator-getCollationElementIterator-1]]<> link:{java8-javadoc}/java/text/RuleBasedCollator.html#getCollationElementIterator%2Djava.lang.String%2D[getCollationElementIterator](<>)++ (link:{java9-javadoc}/java/text/RuleBasedCollator.html#getCollationElementIterator%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-RuleBasedCollator-getRules-0]]<> link:{java8-javadoc}/java/text/RuleBasedCollator.html#getRules%2D%2D[getRules]()++ (link:{java9-javadoc}/java/text/RuleBasedCollator.html#getRules%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/RuntimeException.asciidoc b/docs/painless/painless-api-reference/RuntimeException.asciidoc deleted file mode 100644 index fc8523169e33d..0000000000000 --- a/docs/painless/painless-api-reference/RuntimeException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-RuntimeException]]++RuntimeException++:: -* ++[[painless-api-reference-RuntimeException-RuntimeException-0]]link:{java8-javadoc}/java/lang/RuntimeException.html#RuntimeException%2D%2D[RuntimeException]()++ (link:{java9-javadoc}/java/lang/RuntimeException.html#RuntimeException%2D%2D[java 9]) -* ++[[painless-api-reference-RuntimeException-RuntimeException-1]]link:{java8-javadoc}/java/lang/RuntimeException.html#RuntimeException%2Djava.lang.String%2D[RuntimeException](<>)++ (link:{java9-javadoc}/java/lang/RuntimeException.html#RuntimeException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/SecurityException.asciidoc b/docs/painless/painless-api-reference/SecurityException.asciidoc deleted file mode 100644 index ec6ea157e5587..0000000000000 --- a/docs/painless/painless-api-reference/SecurityException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-SecurityException]]++SecurityException++:: -* ++[[painless-api-reference-SecurityException-SecurityException-0]]link:{java8-javadoc}/java/lang/SecurityException.html#SecurityException%2D%2D[SecurityException]()++ (link:{java9-javadoc}/java/lang/SecurityException.html#SecurityException%2D%2D[java 9]) -* ++[[painless-api-reference-SecurityException-SecurityException-1]]link:{java8-javadoc}/java/lang/SecurityException.html#SecurityException%2Djava.lang.String%2D[SecurityException](<>)++ (link:{java9-javadoc}/java/lang/SecurityException.html#SecurityException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Set.asciidoc b/docs/painless/painless-api-reference/Set.asciidoc deleted file mode 100644 index 23d8a352f231f..0000000000000 --- a/docs/painless/painless-api-reference/Set.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Set]]++Set++:: -* ++[[painless-api-reference-Set-equals-1]]boolean link:{java8-javadoc}/java/util/Set.html#equals%2Djava.lang.Object%2D[equals](<>)++ (link:{java9-javadoc}/java/util/Set.html#equals%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Set-hashCode-0]]int link:{java8-javadoc}/java/util/Set.html#hashCode%2D%2D[hashCode]()++ (link:{java9-javadoc}/java/util/Set.html#hashCode%2D%2D[java 9]) -* ++[[painless-api-reference-Set-remove-1]]boolean link:{java8-javadoc}/java/util/Set.html#remove%2Djava.lang.Object%2D[remove](def)++ (link:{java9-javadoc}/java/util/Set.html#remove%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Short.asciidoc b/docs/painless/painless-api-reference/Short.asciidoc deleted file mode 100644 index 12f0336e8fd40..0000000000000 --- a/docs/painless/painless-api-reference/Short.asciidoc +++ /dev/null @@ -1,23 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Short]]++Short++:: -** [[painless-api-reference-Short-BYTES]]static int link:{java8-javadoc}/java/lang/Short.html#BYTES[BYTES] (link:{java9-javadoc}/java/lang/Short.html#BYTES[java 9]) -** [[painless-api-reference-Short-MAX_VALUE]]static short link:{java8-javadoc}/java/lang/Short.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/lang/Short.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Short-MIN_VALUE]]static short link:{java8-javadoc}/java/lang/Short.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/lang/Short.html#MIN_VALUE[java 9]) -** [[painless-api-reference-Short-SIZE]]static int link:{java8-javadoc}/java/lang/Short.html#SIZE[SIZE] (link:{java9-javadoc}/java/lang/Short.html#SIZE[java 9]) -* ++[[painless-api-reference-Short-compare-2]]static int link:{java8-javadoc}/java/lang/Short.html#compare%2Dshort%2Dshort%2D[compare](short, short)++ (link:{java9-javadoc}/java/lang/Short.html#compare%2Dshort%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-decode-1]]static <> link:{java8-javadoc}/java/lang/Short.html#decode%2Djava.lang.String%2D[decode](<>)++ (link:{java9-javadoc}/java/lang/Short.html#decode%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Short-hashCode-1]]static int link:{java8-javadoc}/java/lang/Short.html#hashCode%2Dshort%2D[hashCode](short)++ (link:{java9-javadoc}/java/lang/Short.html#hashCode%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-parseShort-1]]static short link:{java8-javadoc}/java/lang/Short.html#parseShort%2Djava.lang.String%2D[parseShort](<>)++ (link:{java9-javadoc}/java/lang/Short.html#parseShort%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-Short-parseShort-2]]static short link:{java8-javadoc}/java/lang/Short.html#parseShort%2Djava.lang.String%2Dint%2D[parseShort](<>, int)++ (link:{java9-javadoc}/java/lang/Short.html#parseShort%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Short-reverseBytes-1]]static short link:{java8-javadoc}/java/lang/Short.html#reverseBytes%2Dshort%2D[reverseBytes](short)++ (link:{java9-javadoc}/java/lang/Short.html#reverseBytes%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-toString-1]]static <> link:{java8-javadoc}/java/lang/Short.html#toString%2Dshort%2D[toString](short)++ (link:{java9-javadoc}/java/lang/Short.html#toString%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-toUnsignedInt-1]]static int link:{java8-javadoc}/java/lang/Short.html#toUnsignedInt%2Dshort%2D[toUnsignedInt](short)++ (link:{java9-javadoc}/java/lang/Short.html#toUnsignedInt%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-toUnsignedLong-1]]static long link:{java8-javadoc}/java/lang/Short.html#toUnsignedLong%2Dshort%2D[toUnsignedLong](short)++ (link:{java9-javadoc}/java/lang/Short.html#toUnsignedLong%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-valueOf-1]]static <> link:{java8-javadoc}/java/lang/Short.html#valueOf%2Dshort%2D[valueOf](short)++ (link:{java9-javadoc}/java/lang/Short.html#valueOf%2Dshort%2D[java 9]) -* ++[[painless-api-reference-Short-valueOf-2]]static <> link:{java8-javadoc}/java/lang/Short.html#valueOf%2Djava.lang.String%2Dint%2D[valueOf](<>, int)++ (link:{java9-javadoc}/java/lang/Short.html#valueOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-Short-compareTo-1]]int link:{java8-javadoc}/java/lang/Short.html#compareTo%2Djava.lang.Short%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/Short.html#compareTo%2Djava.lang.Short%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/SignStyle.asciidoc b/docs/painless/painless-api-reference/SignStyle.asciidoc deleted file mode 100644 index a42ce94ebfa33..0000000000000 --- a/docs/painless/painless-api-reference/SignStyle.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-SignStyle]]++SignStyle++:: -** [[painless-api-reference-SignStyle-ALWAYS]]static <> link:{java8-javadoc}/java/time/format/SignStyle.html#ALWAYS[ALWAYS] (link:{java9-javadoc}/java/time/format/SignStyle.html#ALWAYS[java 9]) -** [[painless-api-reference-SignStyle-EXCEEDS_PAD]]static <> link:{java8-javadoc}/java/time/format/SignStyle.html#EXCEEDS_PAD[EXCEEDS_PAD] (link:{java9-javadoc}/java/time/format/SignStyle.html#EXCEEDS_PAD[java 9]) -** [[painless-api-reference-SignStyle-NEVER]]static <> link:{java8-javadoc}/java/time/format/SignStyle.html#NEVER[NEVER] (link:{java9-javadoc}/java/time/format/SignStyle.html#NEVER[java 9]) -** [[painless-api-reference-SignStyle-NORMAL]]static <> link:{java8-javadoc}/java/time/format/SignStyle.html#NORMAL[NORMAL] (link:{java9-javadoc}/java/time/format/SignStyle.html#NORMAL[java 9]) -** [[painless-api-reference-SignStyle-NOT_NEGATIVE]]static <> link:{java8-javadoc}/java/time/format/SignStyle.html#NOT_NEGATIVE[NOT_NEGATIVE] (link:{java9-javadoc}/java/time/format/SignStyle.html#NOT_NEGATIVE[java 9]) -* ++[[painless-api-reference-SignStyle-valueOf-1]]static <> link:{java8-javadoc}/java/time/format/SignStyle.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/format/SignStyle.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-SignStyle-values-0]]static <>[] link:{java8-javadoc}/java/time/format/SignStyle.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/format/SignStyle.html#values%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/SimpleDateFormat.asciidoc b/docs/painless/painless-api-reference/SimpleDateFormat.asciidoc deleted file mode 100644 index b7a8ffc191c45..0000000000000 --- a/docs/painless/painless-api-reference/SimpleDateFormat.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-SimpleDateFormat]]++SimpleDateFormat++:: -* ++[[painless-api-reference-SimpleDateFormat-SimpleDateFormat-0]]link:{java8-javadoc}/java/text/SimpleDateFormat.html#SimpleDateFormat%2D%2D[SimpleDateFormat]()++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#SimpleDateFormat%2D%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-SimpleDateFormat-1]]link:{java8-javadoc}/java/text/SimpleDateFormat.html#SimpleDateFormat%2Djava.lang.String%2D[SimpleDateFormat](<>)++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#SimpleDateFormat%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-SimpleDateFormat-2]]link:{java8-javadoc}/java/text/SimpleDateFormat.html#SimpleDateFormat%2Djava.lang.String%2Djava.util.Locale%2D[SimpleDateFormat](<>, <>)++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#SimpleDateFormat%2Djava.lang.String%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-applyLocalizedPattern-1]]void link:{java8-javadoc}/java/text/SimpleDateFormat.html#applyLocalizedPattern%2Djava.lang.String%2D[applyLocalizedPattern](<>)++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#applyLocalizedPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-applyPattern-1]]void link:{java8-javadoc}/java/text/SimpleDateFormat.html#applyPattern%2Djava.lang.String%2D[applyPattern](<>)++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#applyPattern%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-get2DigitYearStart-0]]<> link:{java8-javadoc}/java/text/SimpleDateFormat.html#get2DigitYearStart%2D%2D[get2DigitYearStart]()++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#get2DigitYearStart%2D%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-getDateFormatSymbols-0]]<> link:{java8-javadoc}/java/text/SimpleDateFormat.html#getDateFormatSymbols%2D%2D[getDateFormatSymbols]()++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#getDateFormatSymbols%2D%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-set2DigitYearStart-1]]void link:{java8-javadoc}/java/text/SimpleDateFormat.html#set2DigitYearStart%2Djava.util.Date%2D[set2DigitYearStart](<>)++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#set2DigitYearStart%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-setDateFormatSymbols-1]]void link:{java8-javadoc}/java/text/SimpleDateFormat.html#setDateFormatSymbols%2Djava.text.DateFormatSymbols%2D[setDateFormatSymbols](<>)++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#setDateFormatSymbols%2Djava.text.DateFormatSymbols%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-toLocalizedPattern-0]]<> link:{java8-javadoc}/java/text/SimpleDateFormat.html#toLocalizedPattern%2D%2D[toLocalizedPattern]()++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#toLocalizedPattern%2D%2D[java 9]) -* ++[[painless-api-reference-SimpleDateFormat-toPattern-0]]<> link:{java8-javadoc}/java/text/SimpleDateFormat.html#toPattern%2D%2D[toPattern]()++ (link:{java9-javadoc}/java/text/SimpleDateFormat.html#toPattern%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/SimpleTimeZone.asciidoc b/docs/painless/painless-api-reference/SimpleTimeZone.asciidoc deleted file mode 100644 index eff23a40723a3..0000000000000 --- a/docs/painless/painless-api-reference/SimpleTimeZone.asciidoc +++ /dev/null @@ -1,23 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-SimpleTimeZone]]++SimpleTimeZone++:: -** [[painless-api-reference-SimpleTimeZone-STANDARD_TIME]]static int link:{java8-javadoc}/java/util/SimpleTimeZone.html#STANDARD_TIME[STANDARD_TIME] (link:{java9-javadoc}/java/util/SimpleTimeZone.html#STANDARD_TIME[java 9]) -** [[painless-api-reference-SimpleTimeZone-UTC_TIME]]static int link:{java8-javadoc}/java/util/SimpleTimeZone.html#UTC_TIME[UTC_TIME] (link:{java9-javadoc}/java/util/SimpleTimeZone.html#UTC_TIME[java 9]) -** [[painless-api-reference-SimpleTimeZone-WALL_TIME]]static int link:{java8-javadoc}/java/util/SimpleTimeZone.html#WALL_TIME[WALL_TIME] (link:{java9-javadoc}/java/util/SimpleTimeZone.html#WALL_TIME[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-SimpleTimeZone-2]]link:{java8-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2D[SimpleTimeZone](int, <>)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-SimpleTimeZone-10]]link:{java8-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[SimpleTimeZone](int, <>, int, int, int, int, int, int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-SimpleTimeZone-11]]link:{java8-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[SimpleTimeZone](int, <>, int, int, int, int, int, int, int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-SimpleTimeZone-13]]link:{java8-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[SimpleTimeZone](int, <>, int, int, int, int, int, int, int, int, int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#SimpleTimeZone%2Dint%2Djava.lang.String%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-getDSTSavings-0]]int link:{java8-javadoc}/java/util/SimpleTimeZone.html#getDSTSavings%2D%2D[getDSTSavings]()++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#getDSTSavings%2D%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setDSTSavings-1]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setDSTSavings%2Dint%2D[setDSTSavings](int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setDSTSavings%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setEndRule-3]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setEndRule%2Dint%2Dint%2Dint%2D[setEndRule](int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setEndRule%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setEndRule-4]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setEndRule%2Dint%2Dint%2Dint%2Dint%2D[setEndRule](int, int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setEndRule%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setEndRule-5]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setEndRule%2Dint%2Dint%2Dint%2Dint%2Dboolean%2D[setEndRule](int, int, int, int, boolean)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setEndRule%2Dint%2Dint%2Dint%2Dint%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setStartRule-3]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setStartRule%2Dint%2Dint%2Dint%2D[setStartRule](int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setStartRule%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setStartRule-4]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setStartRule%2Dint%2Dint%2Dint%2Dint%2D[setStartRule](int, int, int, int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setStartRule%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setStartRule-5]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setStartRule%2Dint%2Dint%2Dint%2Dint%2Dboolean%2D[setStartRule](int, int, int, int, boolean)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setStartRule%2Dint%2Dint%2Dint%2Dint%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-SimpleTimeZone-setStartYear-1]]void link:{java8-javadoc}/java/util/SimpleTimeZone.html#setStartYear%2Dint%2D[setStartYear](int)++ (link:{java9-javadoc}/java/util/SimpleTimeZone.html#setStartYear%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/SortedMap.asciidoc b/docs/painless/painless-api-reference/SortedMap.asciidoc deleted file mode 100644 index d4e31e6c8fa37..0000000000000 --- a/docs/painless/painless-api-reference/SortedMap.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-SortedMap]]++SortedMap++:: -* ++[[painless-api-reference-SortedMap-comparator-0]]<> link:{java8-javadoc}/java/util/SortedMap.html#comparator%2D%2D[comparator]()++ (link:{java9-javadoc}/java/util/SortedMap.html#comparator%2D%2D[java 9]) -* ++[[painless-api-reference-SortedMap-firstKey-0]]def link:{java8-javadoc}/java/util/SortedMap.html#firstKey%2D%2D[firstKey]()++ (link:{java9-javadoc}/java/util/SortedMap.html#firstKey%2D%2D[java 9]) -* ++[[painless-api-reference-SortedMap-headMap-1]]<> link:{java8-javadoc}/java/util/SortedMap.html#headMap%2Djava.lang.Object%2D[headMap](def)++ (link:{java9-javadoc}/java/util/SortedMap.html#headMap%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-SortedMap-lastKey-0]]def link:{java8-javadoc}/java/util/SortedMap.html#lastKey%2D%2D[lastKey]()++ (link:{java9-javadoc}/java/util/SortedMap.html#lastKey%2D%2D[java 9]) -* ++[[painless-api-reference-SortedMap-subMap-2]]<> link:{java8-javadoc}/java/util/SortedMap.html#subMap%2Djava.lang.Object%2Djava.lang.Object%2D[subMap](def, def)++ (link:{java9-javadoc}/java/util/SortedMap.html#subMap%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-SortedMap-tailMap-1]]<> link:{java8-javadoc}/java/util/SortedMap.html#tailMap%2Djava.lang.Object%2D[tailMap](def)++ (link:{java9-javadoc}/java/util/SortedMap.html#tailMap%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/SortedSet.asciidoc b/docs/painless/painless-api-reference/SortedSet.asciidoc deleted file mode 100644 index 2a7160aa2ba44..0000000000000 --- a/docs/painless/painless-api-reference/SortedSet.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-SortedSet]]++SortedSet++:: -* ++[[painless-api-reference-SortedSet-comparator-0]]<> link:{java8-javadoc}/java/util/SortedSet.html#comparator%2D%2D[comparator]()++ (link:{java9-javadoc}/java/util/SortedSet.html#comparator%2D%2D[java 9]) -* ++[[painless-api-reference-SortedSet-first-0]]def link:{java8-javadoc}/java/util/SortedSet.html#first%2D%2D[first]()++ (link:{java9-javadoc}/java/util/SortedSet.html#first%2D%2D[java 9]) -* ++[[painless-api-reference-SortedSet-headSet-1]]<> link:{java8-javadoc}/java/util/SortedSet.html#headSet%2Djava.lang.Object%2D[headSet](def)++ (link:{java9-javadoc}/java/util/SortedSet.html#headSet%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-SortedSet-last-0]]def link:{java8-javadoc}/java/util/SortedSet.html#last%2D%2D[last]()++ (link:{java9-javadoc}/java/util/SortedSet.html#last%2D%2D[java 9]) -* ++[[painless-api-reference-SortedSet-subSet-2]]<> link:{java8-javadoc}/java/util/SortedSet.html#subSet%2Djava.lang.Object%2Djava.lang.Object%2D[subSet](def, def)++ (link:{java9-javadoc}/java/util/SortedSet.html#subSet%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-SortedSet-tailSet-1]]<> link:{java8-javadoc}/java/util/SortedSet.html#tailSet%2Djava.lang.Object%2D[tailSet](def)++ (link:{java9-javadoc}/java/util/SortedSet.html#tailSet%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Spliterator.OfDouble.asciidoc b/docs/painless/painless-api-reference/Spliterator.OfDouble.asciidoc deleted file mode 100644 index 4722a8e6e0005..0000000000000 --- a/docs/painless/painless-api-reference/Spliterator.OfDouble.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Spliterator-OfDouble]]++Spliterator.OfDouble++:: -* ++[[painless-api-reference-Spliterator-OfDouble-trySplit-0]]<> link:{java8-javadoc}/java/util/Spliterator.OfDouble.html#trySplit%2D%2D[trySplit]()++ (link:{java9-javadoc}/java/util/Spliterator.OfDouble.html#trySplit%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Spliterator.OfInt.asciidoc b/docs/painless/painless-api-reference/Spliterator.OfInt.asciidoc deleted file mode 100644 index e70221d6bf1c7..0000000000000 --- a/docs/painless/painless-api-reference/Spliterator.OfInt.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Spliterator-OfInt]]++Spliterator.OfInt++:: -* ++[[painless-api-reference-Spliterator-OfInt-trySplit-0]]<> link:{java8-javadoc}/java/util/Spliterator.OfInt.html#trySplit%2D%2D[trySplit]()++ (link:{java9-javadoc}/java/util/Spliterator.OfInt.html#trySplit%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Spliterator.OfLong.asciidoc b/docs/painless/painless-api-reference/Spliterator.OfLong.asciidoc deleted file mode 100644 index 8f709add01e17..0000000000000 --- a/docs/painless/painless-api-reference/Spliterator.OfLong.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Spliterator-OfLong]]++Spliterator.OfLong++:: -* ++[[painless-api-reference-Spliterator-OfLong-trySplit-0]]<> link:{java8-javadoc}/java/util/Spliterator.OfLong.html#trySplit%2D%2D[trySplit]()++ (link:{java9-javadoc}/java/util/Spliterator.OfLong.html#trySplit%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Spliterator.OfPrimitive.asciidoc b/docs/painless/painless-api-reference/Spliterator.OfPrimitive.asciidoc deleted file mode 100644 index 297c213b44f50..0000000000000 --- a/docs/painless/painless-api-reference/Spliterator.OfPrimitive.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Spliterator-OfPrimitive]]++Spliterator.OfPrimitive++:: -* ++[[painless-api-reference-Spliterator-OfPrimitive-forEachRemaining-1]]void link:{java8-javadoc}/java/util/Spliterator.OfPrimitive.html#forEachRemaining%2Djava.lang.Object%2D[forEachRemaining](def)++ (link:{java9-javadoc}/java/util/Spliterator.OfPrimitive.html#forEachRemaining%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Spliterator-OfPrimitive-tryAdvance-1]]boolean link:{java8-javadoc}/java/util/Spliterator.OfPrimitive.html#tryAdvance%2Djava.lang.Object%2D[tryAdvance](def)++ (link:{java9-javadoc}/java/util/Spliterator.OfPrimitive.html#tryAdvance%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Spliterator-OfPrimitive-trySplit-0]]<> link:{java8-javadoc}/java/util/Spliterator.OfPrimitive.html#trySplit%2D%2D[trySplit]()++ (link:{java9-javadoc}/java/util/Spliterator.OfPrimitive.html#trySplit%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Spliterator.asciidoc b/docs/painless/painless-api-reference/Spliterator.asciidoc deleted file mode 100644 index fcf919409858d..0000000000000 --- a/docs/painless/painless-api-reference/Spliterator.asciidoc +++ /dev/null @@ -1,23 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Spliterator]]++Spliterator++:: -** [[painless-api-reference-Spliterator-CONCURRENT]]static int link:{java8-javadoc}/java/util/Spliterator.html#CONCURRENT[CONCURRENT] (link:{java9-javadoc}/java/util/Spliterator.html#CONCURRENT[java 9]) -** [[painless-api-reference-Spliterator-DISTINCT]]static int link:{java8-javadoc}/java/util/Spliterator.html#DISTINCT[DISTINCT] (link:{java9-javadoc}/java/util/Spliterator.html#DISTINCT[java 9]) -** [[painless-api-reference-Spliterator-IMMUTABLE]]static int link:{java8-javadoc}/java/util/Spliterator.html#IMMUTABLE[IMMUTABLE] (link:{java9-javadoc}/java/util/Spliterator.html#IMMUTABLE[java 9]) -** [[painless-api-reference-Spliterator-NONNULL]]static int link:{java8-javadoc}/java/util/Spliterator.html#NONNULL[NONNULL] (link:{java9-javadoc}/java/util/Spliterator.html#NONNULL[java 9]) -** [[painless-api-reference-Spliterator-ORDERED]]static int link:{java8-javadoc}/java/util/Spliterator.html#ORDERED[ORDERED] (link:{java9-javadoc}/java/util/Spliterator.html#ORDERED[java 9]) -** [[painless-api-reference-Spliterator-SIZED]]static int link:{java8-javadoc}/java/util/Spliterator.html#SIZED[SIZED] (link:{java9-javadoc}/java/util/Spliterator.html#SIZED[java 9]) -** [[painless-api-reference-Spliterator-SORTED]]static int link:{java8-javadoc}/java/util/Spliterator.html#SORTED[SORTED] (link:{java9-javadoc}/java/util/Spliterator.html#SORTED[java 9]) -** [[painless-api-reference-Spliterator-SUBSIZED]]static int link:{java8-javadoc}/java/util/Spliterator.html#SUBSIZED[SUBSIZED] (link:{java9-javadoc}/java/util/Spliterator.html#SUBSIZED[java 9]) -* ++[[painless-api-reference-Spliterator-characteristics-0]]int link:{java8-javadoc}/java/util/Spliterator.html#characteristics%2D%2D[characteristics]()++ (link:{java9-javadoc}/java/util/Spliterator.html#characteristics%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterator-estimateSize-0]]long link:{java8-javadoc}/java/util/Spliterator.html#estimateSize%2D%2D[estimateSize]()++ (link:{java9-javadoc}/java/util/Spliterator.html#estimateSize%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterator-forEachRemaining-1]]void link:{java8-javadoc}/java/util/Spliterator.html#forEachRemaining%2Djava.util.function.Consumer%2D[forEachRemaining](<>)++ (link:{java9-javadoc}/java/util/Spliterator.html#forEachRemaining%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Spliterator-getComparator-0]]<> link:{java8-javadoc}/java/util/Spliterator.html#getComparator%2D%2D[getComparator]()++ (link:{java9-javadoc}/java/util/Spliterator.html#getComparator%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterator-getExactSizeIfKnown-0]]long link:{java8-javadoc}/java/util/Spliterator.html#getExactSizeIfKnown%2D%2D[getExactSizeIfKnown]()++ (link:{java9-javadoc}/java/util/Spliterator.html#getExactSizeIfKnown%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterator-hasCharacteristics-1]]boolean link:{java8-javadoc}/java/util/Spliterator.html#hasCharacteristics%2Dint%2D[hasCharacteristics](int)++ (link:{java9-javadoc}/java/util/Spliterator.html#hasCharacteristics%2Dint%2D[java 9]) -* ++[[painless-api-reference-Spliterator-tryAdvance-1]]boolean link:{java8-javadoc}/java/util/Spliterator.html#tryAdvance%2Djava.util.function.Consumer%2D[tryAdvance](<>)++ (link:{java9-javadoc}/java/util/Spliterator.html#tryAdvance%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Spliterator-trySplit-0]]<> link:{java8-javadoc}/java/util/Spliterator.html#trySplit%2D%2D[trySplit]()++ (link:{java9-javadoc}/java/util/Spliterator.html#trySplit%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Spliterators.asciidoc b/docs/painless/painless-api-reference/Spliterators.asciidoc deleted file mode 100644 index 6d8202a567411..0000000000000 --- a/docs/painless/painless-api-reference/Spliterators.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Spliterators]]++Spliterators++:: -* ++[[painless-api-reference-Spliterators-emptyDoubleSpliterator-0]]static <> link:{java8-javadoc}/java/util/Spliterators.html#emptyDoubleSpliterator%2D%2D[emptyDoubleSpliterator]()++ (link:{java9-javadoc}/java/util/Spliterators.html#emptyDoubleSpliterator%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterators-emptyIntSpliterator-0]]static <> link:{java8-javadoc}/java/util/Spliterators.html#emptyIntSpliterator%2D%2D[emptyIntSpliterator]()++ (link:{java9-javadoc}/java/util/Spliterators.html#emptyIntSpliterator%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterators-emptyLongSpliterator-0]]static <> link:{java8-javadoc}/java/util/Spliterators.html#emptyLongSpliterator%2D%2D[emptyLongSpliterator]()++ (link:{java9-javadoc}/java/util/Spliterators.html#emptyLongSpliterator%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterators-emptySpliterator-0]]static <> link:{java8-javadoc}/java/util/Spliterators.html#emptySpliterator%2D%2D[emptySpliterator]()++ (link:{java9-javadoc}/java/util/Spliterators.html#emptySpliterator%2D%2D[java 9]) -* ++[[painless-api-reference-Spliterators-iterator-1]]static <> link:{java8-javadoc}/java/util/Spliterators.html#iterator%2Djava.util.Spliterator%2D[iterator](<>)++ (link:{java9-javadoc}/java/util/Spliterators.html#iterator%2Djava.util.Spliterator%2D[java 9]) -* ++[[painless-api-reference-Spliterators-spliterator-2]]static <> link:{java8-javadoc}/java/util/Spliterators.html#spliterator%2Djava.util.Collection%2Dint%2D[spliterator](<>, int)++ (link:{java9-javadoc}/java/util/Spliterators.html#spliterator%2Djava.util.Collection%2Dint%2D[java 9]) -* ++[[painless-api-reference-Spliterators-spliterator-3]]static <> link:{java8-javadoc}/java/util/Spliterators.html#spliterator%2Djava.util.Iterator%2Dlong%2Dint%2D[spliterator](<>, long, int)++ (link:{java9-javadoc}/java/util/Spliterators.html#spliterator%2Djava.util.Iterator%2Dlong%2Dint%2D[java 9]) -* ++[[painless-api-reference-Spliterators-spliteratorUnknownSize-2]]static <> link:{java8-javadoc}/java/util/Spliterators.html#spliteratorUnknownSize%2Djava.util.Iterator%2Dint%2D[spliteratorUnknownSize](<>, int)++ (link:{java9-javadoc}/java/util/Spliterators.html#spliteratorUnknownSize%2Djava.util.Iterator%2Dint%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Stack.asciidoc b/docs/painless/painless-api-reference/Stack.asciidoc deleted file mode 100644 index ad1520d9d885d..0000000000000 --- a/docs/painless/painless-api-reference/Stack.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Stack]]++Stack++:: -* ++[[painless-api-reference-Stack-Stack-0]]link:{java8-javadoc}/java/util/Stack.html#Stack%2D%2D[Stack]()++ (link:{java9-javadoc}/java/util/Stack.html#Stack%2D%2D[java 9]) -* ++[[painless-api-reference-Stack-empty-0]]boolean link:{java8-javadoc}/java/util/Stack.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/Stack.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-Stack-peek-0]]def link:{java8-javadoc}/java/util/Stack.html#peek%2D%2D[peek]()++ (link:{java9-javadoc}/java/util/Stack.html#peek%2D%2D[java 9]) -* ++[[painless-api-reference-Stack-pop-0]]def link:{java8-javadoc}/java/util/Stack.html#pop%2D%2D[pop]()++ (link:{java9-javadoc}/java/util/Stack.html#pop%2D%2D[java 9]) -* ++[[painless-api-reference-Stack-push-1]]def link:{java8-javadoc}/java/util/Stack.html#push%2Djava.lang.Object%2D[push](def)++ (link:{java9-javadoc}/java/util/Stack.html#push%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Stack-search-1]]int link:{java8-javadoc}/java/util/Stack.html#search%2Djava.lang.Object%2D[search](def)++ (link:{java9-javadoc}/java/util/Stack.html#search%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StackTraceElement.asciidoc b/docs/painless/painless-api-reference/StackTraceElement.asciidoc deleted file mode 100644 index 046e56ef1f016..0000000000000 --- a/docs/painless/painless-api-reference/StackTraceElement.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StackTraceElement]]++StackTraceElement++:: -* ++[[painless-api-reference-StackTraceElement-StackTraceElement-4]]link:{java8-javadoc}/java/lang/StackTraceElement.html#StackTraceElement%2Djava.lang.String%2Djava.lang.String%2Djava.lang.String%2Dint%2D[StackTraceElement](<>, <>, <>, int)++ (link:{java9-javadoc}/java/lang/StackTraceElement.html#StackTraceElement%2Djava.lang.String%2Djava.lang.String%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-StackTraceElement-getClassName-0]]<> link:{java8-javadoc}/java/lang/StackTraceElement.html#getClassName%2D%2D[getClassName]()++ (link:{java9-javadoc}/java/lang/StackTraceElement.html#getClassName%2D%2D[java 9]) -* ++[[painless-api-reference-StackTraceElement-getFileName-0]]<> link:{java8-javadoc}/java/lang/StackTraceElement.html#getFileName%2D%2D[getFileName]()++ (link:{java9-javadoc}/java/lang/StackTraceElement.html#getFileName%2D%2D[java 9]) -* ++[[painless-api-reference-StackTraceElement-getLineNumber-0]]int link:{java8-javadoc}/java/lang/StackTraceElement.html#getLineNumber%2D%2D[getLineNumber]()++ (link:{java9-javadoc}/java/lang/StackTraceElement.html#getLineNumber%2D%2D[java 9]) -* ++[[painless-api-reference-StackTraceElement-getMethodName-0]]<> link:{java8-javadoc}/java/lang/StackTraceElement.html#getMethodName%2D%2D[getMethodName]()++ (link:{java9-javadoc}/java/lang/StackTraceElement.html#getMethodName%2D%2D[java 9]) -* ++[[painless-api-reference-StackTraceElement-isNativeMethod-0]]boolean link:{java8-javadoc}/java/lang/StackTraceElement.html#isNativeMethod%2D%2D[isNativeMethod]()++ (link:{java9-javadoc}/java/lang/StackTraceElement.html#isNativeMethod%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Stream.Builder.asciidoc b/docs/painless/painless-api-reference/Stream.Builder.asciidoc deleted file mode 100644 index 42d91e01a2703..0000000000000 --- a/docs/painless/painless-api-reference/Stream.Builder.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Stream-Builder]]++Stream.Builder++:: -* ++[[painless-api-reference-Stream-Builder-add-1]]<> link:{java8-javadoc}/java/util/stream/Stream.Builder.html#add%2Djava.lang.Object%2D[add](def)++ (link:{java9-javadoc}/java/util/stream/Stream.Builder.html#add%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Stream-Builder-build-0]]<> link:{java8-javadoc}/java/util/stream/Stream.Builder.html#build%2D%2D[build]()++ (link:{java9-javadoc}/java/util/stream/Stream.Builder.html#build%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Stream.asciidoc b/docs/painless/painless-api-reference/Stream.asciidoc deleted file mode 100644 index 0713ad1ce9d90..0000000000000 --- a/docs/painless/painless-api-reference/Stream.asciidoc +++ /dev/null @@ -1,43 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Stream]]++Stream++:: -* ++[[painless-api-reference-Stream-builder-0]]static <> link:{java8-javadoc}/java/util/stream/Stream.html#builder%2D%2D[builder]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#builder%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-concat-2]]static <> link:{java8-javadoc}/java/util/stream/Stream.html#concat%2Djava.util.stream.Stream%2Djava.util.stream.Stream%2D[concat](<>, <>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#concat%2Djava.util.stream.Stream%2Djava.util.stream.Stream%2D[java 9]) -* ++[[painless-api-reference-Stream-empty-0]]static <> link:{java8-javadoc}/java/util/stream/Stream.html#empty%2D%2D[empty]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#empty%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-of-1]]static <> link:{java8-javadoc}/java/util/stream/Stream.html#of%2Djava.lang.Object:A%2D[of](def[])++ (link:{java9-javadoc}/java/util/stream/Stream.html#of%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Stream-allMatch-1]]boolean link:{java8-javadoc}/java/util/stream/Stream.html#allMatch%2Djava.util.function.Predicate%2D[allMatch](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#allMatch%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Stream-anyMatch-1]]boolean link:{java8-javadoc}/java/util/stream/Stream.html#anyMatch%2Djava.util.function.Predicate%2D[anyMatch](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#anyMatch%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Stream-collect-1]]def link:{java8-javadoc}/java/util/stream/Stream.html#collect%2Djava.util.stream.Collector%2D[collect](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#collect%2Djava.util.stream.Collector%2D[java 9]) -* ++[[painless-api-reference-Stream-collect-3]]def link:{java8-javadoc}/java/util/stream/Stream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.BiConsumer%2Djava.util.function.BiConsumer%2D[collect](<>, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#collect%2Djava.util.function.Supplier%2Djava.util.function.BiConsumer%2Djava.util.function.BiConsumer%2D[java 9]) -* ++[[painless-api-reference-Stream-count-0]]long link:{java8-javadoc}/java/util/stream/Stream.html#count%2D%2D[count]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#count%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-distinct-0]]<> link:{java8-javadoc}/java/util/stream/Stream.html#distinct%2D%2D[distinct]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#distinct%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-filter-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#filter%2Djava.util.function.Predicate%2D[filter](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#filter%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Stream-findAny-0]]<> link:{java8-javadoc}/java/util/stream/Stream.html#findAny%2D%2D[findAny]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#findAny%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-findFirst-0]]<> link:{java8-javadoc}/java/util/stream/Stream.html#findFirst%2D%2D[findFirst]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#findFirst%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-flatMap-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#flatMap%2Djava.util.function.Function%2D[flatMap](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#flatMap%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Stream-flatMapToDouble-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#flatMapToDouble%2Djava.util.function.Function%2D[flatMapToDouble](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#flatMapToDouble%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Stream-flatMapToInt-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#flatMapToInt%2Djava.util.function.Function%2D[flatMapToInt](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#flatMapToInt%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Stream-flatMapToLong-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#flatMapToLong%2Djava.util.function.Function%2D[flatMapToLong](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#flatMapToLong%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Stream-forEach-1]]void link:{java8-javadoc}/java/util/stream/Stream.html#forEach%2Djava.util.function.Consumer%2D[forEach](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#forEach%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Stream-forEachOrdered-1]]void link:{java8-javadoc}/java/util/stream/Stream.html#forEachOrdered%2Djava.util.function.Consumer%2D[forEachOrdered](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#forEachOrdered%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Stream-limit-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#limit%2Dlong%2D[limit](long)++ (link:{java9-javadoc}/java/util/stream/Stream.html#limit%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Stream-map-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#map%2Djava.util.function.Function%2D[map](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#map%2Djava.util.function.Function%2D[java 9]) -* ++[[painless-api-reference-Stream-mapToDouble-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#mapToDouble%2Djava.util.function.ToDoubleFunction%2D[mapToDouble](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#mapToDouble%2Djava.util.function.ToDoubleFunction%2D[java 9]) -* ++[[painless-api-reference-Stream-mapToInt-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#mapToInt%2Djava.util.function.ToIntFunction%2D[mapToInt](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#mapToInt%2Djava.util.function.ToIntFunction%2D[java 9]) -* ++[[painless-api-reference-Stream-mapToLong-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#mapToLong%2Djava.util.function.ToLongFunction%2D[mapToLong](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#mapToLong%2Djava.util.function.ToLongFunction%2D[java 9]) -* ++[[painless-api-reference-Stream-max-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#max%2Djava.util.Comparator%2D[max](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#max%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Stream-min-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#min%2Djava.util.Comparator%2D[min](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#min%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Stream-noneMatch-1]]boolean link:{java8-javadoc}/java/util/stream/Stream.html#noneMatch%2Djava.util.function.Predicate%2D[noneMatch](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#noneMatch%2Djava.util.function.Predicate%2D[java 9]) -* ++[[painless-api-reference-Stream-peek-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#peek%2Djava.util.function.Consumer%2D[peek](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#peek%2Djava.util.function.Consumer%2D[java 9]) -* ++[[painless-api-reference-Stream-reduce-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#reduce%2Djava.util.function.BinaryOperator%2D[reduce](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#reduce%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Stream-reduce-2]]def link:{java8-javadoc}/java/util/stream/Stream.html#reduce%2Djava.lang.Object%2Djava.util.function.BinaryOperator%2D[reduce](def, <>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#reduce%2Djava.lang.Object%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Stream-reduce-3]]def link:{java8-javadoc}/java/util/stream/Stream.html#reduce%2Djava.lang.Object%2Djava.util.function.BiFunction%2Djava.util.function.BinaryOperator%2D[reduce](def, <>, <>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#reduce%2Djava.lang.Object%2Djava.util.function.BiFunction%2Djava.util.function.BinaryOperator%2D[java 9]) -* ++[[painless-api-reference-Stream-skip-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#skip%2Dlong%2D[skip](long)++ (link:{java9-javadoc}/java/util/stream/Stream.html#skip%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Stream-sorted-0]]<> link:{java8-javadoc}/java/util/stream/Stream.html#sorted%2D%2D[sorted]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#sorted%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-sorted-1]]<> link:{java8-javadoc}/java/util/stream/Stream.html#sorted%2Djava.util.Comparator%2D[sorted](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#sorted%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-Stream-toArray-0]]def[] link:{java8-javadoc}/java/util/stream/Stream.html#toArray%2D%2D[toArray]()++ (link:{java9-javadoc}/java/util/stream/Stream.html#toArray%2D%2D[java 9]) -* ++[[painless-api-reference-Stream-toArray-1]]def[] link:{java8-javadoc}/java/util/stream/Stream.html#toArray%2Djava.util.function.IntFunction%2D[toArray](<>)++ (link:{java9-javadoc}/java/util/stream/Stream.html#toArray%2Djava.util.function.IntFunction%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StrictMath.asciidoc b/docs/painless/painless-api-reference/StrictMath.asciidoc deleted file mode 100644 index 17346c33ba3f9..0000000000000 --- a/docs/painless/painless-api-reference/StrictMath.asciidoc +++ /dev/null @@ -1,46 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StrictMath]]++StrictMath++:: -** [[painless-api-reference-StrictMath-E]]static double link:{java8-javadoc}/java/lang/StrictMath.html#E[E] (link:{java9-javadoc}/java/lang/StrictMath.html#E[java 9]) -** [[painless-api-reference-StrictMath-PI]]static double link:{java8-javadoc}/java/lang/StrictMath.html#PI[PI] (link:{java9-javadoc}/java/lang/StrictMath.html#PI[java 9]) -* ++[[painless-api-reference-StrictMath-IEEEremainder-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#IEEEremainder%2Ddouble%2Ddouble%2D[IEEEremainder](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#IEEEremainder%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-abs-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#abs%2Ddouble%2D[abs](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#abs%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-acos-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#acos%2Ddouble%2D[acos](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#acos%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-asin-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#asin%2Ddouble%2D[asin](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#asin%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-atan-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#atan%2Ddouble%2D[atan](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#atan%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-atan2-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#atan2%2Ddouble%2Ddouble%2D[atan2](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#atan2%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-cbrt-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#cbrt%2Ddouble%2D[cbrt](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#cbrt%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-ceil-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#ceil%2Ddouble%2D[ceil](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#ceil%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-copySign-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#copySign%2Ddouble%2Ddouble%2D[copySign](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#copySign%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-cos-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#cos%2Ddouble%2D[cos](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#cos%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-cosh-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#cosh%2Ddouble%2D[cosh](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#cosh%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-exp-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#exp%2Ddouble%2D[exp](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#exp%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-expm1-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#expm1%2Ddouble%2D[expm1](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#expm1%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-floor-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#floor%2Ddouble%2D[floor](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#floor%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-hypot-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#hypot%2Ddouble%2Ddouble%2D[hypot](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#hypot%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-log-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#log%2Ddouble%2D[log](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#log%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-log10-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#log10%2Ddouble%2D[log10](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#log10%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-log1p-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#log1p%2Ddouble%2D[log1p](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#log1p%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-max-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#max%2Ddouble%2Ddouble%2D[max](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#max%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-min-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#min%2Ddouble%2Ddouble%2D[min](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#min%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-nextAfter-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#nextAfter%2Ddouble%2Ddouble%2D[nextAfter](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#nextAfter%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-nextDown-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#nextDown%2Ddouble%2D[nextDown](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#nextDown%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-nextUp-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#nextUp%2Ddouble%2D[nextUp](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#nextUp%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-pow-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#pow%2Ddouble%2Ddouble%2D[pow](double, double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#pow%2Ddouble%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-random-0]]static double link:{java8-javadoc}/java/lang/StrictMath.html#random%2D%2D[random]()++ (link:{java9-javadoc}/java/lang/StrictMath.html#random%2D%2D[java 9]) -* ++[[painless-api-reference-StrictMath-rint-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#rint%2Ddouble%2D[rint](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#rint%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-round-1]]static long link:{java8-javadoc}/java/lang/StrictMath.html#round%2Ddouble%2D[round](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#round%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-scalb-2]]static double link:{java8-javadoc}/java/lang/StrictMath.html#scalb%2Ddouble%2Dint%2D[scalb](double, int)++ (link:{java9-javadoc}/java/lang/StrictMath.html#scalb%2Ddouble%2Dint%2D[java 9]) -* ++[[painless-api-reference-StrictMath-signum-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#signum%2Ddouble%2D[signum](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#signum%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-sin-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#sin%2Ddouble%2D[sin](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#sin%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-sinh-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#sinh%2Ddouble%2D[sinh](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#sinh%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-sqrt-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#sqrt%2Ddouble%2D[sqrt](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#sqrt%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-tan-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#tan%2Ddouble%2D[tan](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#tan%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-tanh-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#tanh%2Ddouble%2D[tanh](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#tanh%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-toDegrees-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#toDegrees%2Ddouble%2D[toDegrees](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#toDegrees%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-toRadians-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#toRadians%2Ddouble%2D[toRadians](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#toRadians%2Ddouble%2D[java 9]) -* ++[[painless-api-reference-StrictMath-ulp-1]]static double link:{java8-javadoc}/java/lang/StrictMath.html#ulp%2Ddouble%2D[ulp](double)++ (link:{java9-javadoc}/java/lang/StrictMath.html#ulp%2Ddouble%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/String.asciidoc b/docs/painless/painless-api-reference/String.asciidoc deleted file mode 100644 index a40e1811183d5..0000000000000 --- a/docs/painless/painless-api-reference/String.asciidoc +++ /dev/null @@ -1,46 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-String]]++String++:: -* ++[[painless-api-reference-String-copyValueOf-1]]static <> link:{java8-javadoc}/java/lang/String.html#copyValueOf%2Dchar:A%2D[copyValueOf](char[])++ (link:{java9-javadoc}/java/lang/String.html#copyValueOf%2Dchar:A%2D[java 9]) -* ++[[painless-api-reference-String-copyValueOf-3]]static <> link:{java8-javadoc}/java/lang/String.html#copyValueOf%2Dchar:A%2Dint%2Dint%2D[copyValueOf](char[], int, int)++ (link:{java9-javadoc}/java/lang/String.html#copyValueOf%2Dchar:A%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-format-2]]static <> link:{java8-javadoc}/java/lang/String.html#format%2Djava.lang.String%2Djava.lang.Object:A%2D[format](<>, def[])++ (link:{java9-javadoc}/java/lang/String.html#format%2Djava.lang.String%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-String-format-3]]static <> link:{java8-javadoc}/java/lang/String.html#format%2Djava.util.Locale%2Djava.lang.String%2Djava.lang.Object:A%2D[format](<>, <>, def[])++ (link:{java9-javadoc}/java/lang/String.html#format%2Djava.util.Locale%2Djava.lang.String%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-String-join-2]]static <> link:{java8-javadoc}/java/lang/String.html#join%2Djava.lang.CharSequence%2Djava.lang.Iterable%2D[join](<>, <>)++ (link:{java9-javadoc}/java/lang/String.html#join%2Djava.lang.CharSequence%2Djava.lang.Iterable%2D[java 9]) -* ++[[painless-api-reference-String-valueOf-1]]static <> link:{java8-javadoc}/java/lang/String.html#valueOf%2Djava.lang.Object%2D[valueOf](def)++ (link:{java9-javadoc}/java/lang/String.html#valueOf%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-String-String-0]]link:{java8-javadoc}/java/lang/String.html#String%2D%2D[String]()++ (link:{java9-javadoc}/java/lang/String.html#String%2D%2D[java 9]) -* ++[[painless-api-reference-String-codePointAt-1]]int link:{java8-javadoc}/java/lang/String.html#codePointAt%2Dint%2D[codePointAt](int)++ (link:{java9-javadoc}/java/lang/String.html#codePointAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-codePointBefore-1]]int link:{java8-javadoc}/java/lang/String.html#codePointBefore%2Dint%2D[codePointBefore](int)++ (link:{java9-javadoc}/java/lang/String.html#codePointBefore%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-codePointCount-2]]int link:{java8-javadoc}/java/lang/String.html#codePointCount%2Dint%2Dint%2D[codePointCount](int, int)++ (link:{java9-javadoc}/java/lang/String.html#codePointCount%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-compareTo-1]]int link:{java8-javadoc}/java/lang/String.html#compareTo%2Djava.lang.String%2D[compareTo](<>)++ (link:{java9-javadoc}/java/lang/String.html#compareTo%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-compareToIgnoreCase-1]]int link:{java8-javadoc}/java/lang/String.html#compareToIgnoreCase%2Djava.lang.String%2D[compareToIgnoreCase](<>)++ (link:{java9-javadoc}/java/lang/String.html#compareToIgnoreCase%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-concat-1]]<> link:{java8-javadoc}/java/lang/String.html#concat%2Djava.lang.String%2D[concat](<>)++ (link:{java9-javadoc}/java/lang/String.html#concat%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-contains-1]]boolean link:{java8-javadoc}/java/lang/String.html#contains%2Djava.lang.CharSequence%2D[contains](<>)++ (link:{java9-javadoc}/java/lang/String.html#contains%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-String-contentEquals-1]]boolean link:{java8-javadoc}/java/lang/String.html#contentEquals%2Djava.lang.CharSequence%2D[contentEquals](<>)++ (link:{java9-javadoc}/java/lang/String.html#contentEquals%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-String-decodeBase64-0]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#decodeBase64%2Djava.lang.String%2D[decodeBase64]()++ -* ++[[painless-api-reference-String-encodeBase64-0]]<> link:{painless-javadoc}/org/elasticsearch/painless/api/Augmentation.html#encodeBase64%2Djava.lang.String%2D[encodeBase64]()++ -* ++[[painless-api-reference-String-endsWith-1]]boolean link:{java8-javadoc}/java/lang/String.html#endsWith%2Djava.lang.String%2D[endsWith](<>)++ (link:{java9-javadoc}/java/lang/String.html#endsWith%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-equalsIgnoreCase-1]]boolean link:{java8-javadoc}/java/lang/String.html#equalsIgnoreCase%2Djava.lang.String%2D[equalsIgnoreCase](<>)++ (link:{java9-javadoc}/java/lang/String.html#equalsIgnoreCase%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-getChars-4]]void link:{java8-javadoc}/java/lang/String.html#getChars%2Dint%2Dint%2Dchar:A%2Dint%2D[getChars](int, int, char[], int)++ (link:{java9-javadoc}/java/lang/String.html#getChars%2Dint%2Dint%2Dchar:A%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-indexOf-1]]int link:{java8-javadoc}/java/lang/String.html#indexOf%2Djava.lang.String%2D[indexOf](<>)++ (link:{java9-javadoc}/java/lang/String.html#indexOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-indexOf-2]]int link:{java8-javadoc}/java/lang/String.html#indexOf%2Djava.lang.String%2Dint%2D[indexOf](<>, int)++ (link:{java9-javadoc}/java/lang/String.html#indexOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-isEmpty-0]]boolean link:{java8-javadoc}/java/lang/String.html#isEmpty%2D%2D[isEmpty]()++ (link:{java9-javadoc}/java/lang/String.html#isEmpty%2D%2D[java 9]) -* ++[[painless-api-reference-String-lastIndexOf-1]]int link:{java8-javadoc}/java/lang/String.html#lastIndexOf%2Djava.lang.String%2D[lastIndexOf](<>)++ (link:{java9-javadoc}/java/lang/String.html#lastIndexOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-lastIndexOf-2]]int link:{java8-javadoc}/java/lang/String.html#lastIndexOf%2Djava.lang.String%2Dint%2D[lastIndexOf](<>, int)++ (link:{java9-javadoc}/java/lang/String.html#lastIndexOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-offsetByCodePoints-2]]int link:{java8-javadoc}/java/lang/String.html#offsetByCodePoints%2Dint%2Dint%2D[offsetByCodePoints](int, int)++ (link:{java9-javadoc}/java/lang/String.html#offsetByCodePoints%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-regionMatches-4]]boolean link:{java8-javadoc}/java/lang/String.html#regionMatches%2Dint%2Djava.lang.String%2Dint%2Dint%2D[regionMatches](int, <>, int, int)++ (link:{java9-javadoc}/java/lang/String.html#regionMatches%2Dint%2Djava.lang.String%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-regionMatches-5]]boolean link:{java8-javadoc}/java/lang/String.html#regionMatches%2Dboolean%2Dint%2Djava.lang.String%2Dint%2Dint%2D[regionMatches](boolean, int, <>, int, int)++ (link:{java9-javadoc}/java/lang/String.html#regionMatches%2Dboolean%2Dint%2Djava.lang.String%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-replace-2]]<> link:{java8-javadoc}/java/lang/String.html#replace%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2D[replace](<>, <>)++ (link:{java9-javadoc}/java/lang/String.html#replace%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-String-startsWith-1]]boolean link:{java8-javadoc}/java/lang/String.html#startsWith%2Djava.lang.String%2D[startsWith](<>)++ (link:{java9-javadoc}/java/lang/String.html#startsWith%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-String-startsWith-2]]boolean link:{java8-javadoc}/java/lang/String.html#startsWith%2Djava.lang.String%2Dint%2D[startsWith](<>, int)++ (link:{java9-javadoc}/java/lang/String.html#startsWith%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-substring-1]]<> link:{java8-javadoc}/java/lang/String.html#substring%2Dint%2D[substring](int)++ (link:{java9-javadoc}/java/lang/String.html#substring%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-substring-2]]<> link:{java8-javadoc}/java/lang/String.html#substring%2Dint%2Dint%2D[substring](int, int)++ (link:{java9-javadoc}/java/lang/String.html#substring%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-String-toCharArray-0]]char[] link:{java8-javadoc}/java/lang/String.html#toCharArray%2D%2D[toCharArray]()++ (link:{java9-javadoc}/java/lang/String.html#toCharArray%2D%2D[java 9]) -* ++[[painless-api-reference-String-toLowerCase-0]]<> link:{java8-javadoc}/java/lang/String.html#toLowerCase%2D%2D[toLowerCase]()++ (link:{java9-javadoc}/java/lang/String.html#toLowerCase%2D%2D[java 9]) -* ++[[painless-api-reference-String-toLowerCase-1]]<> link:{java8-javadoc}/java/lang/String.html#toLowerCase%2Djava.util.Locale%2D[toLowerCase](<>)++ (link:{java9-javadoc}/java/lang/String.html#toLowerCase%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-String-toUpperCase-0]]<> link:{java8-javadoc}/java/lang/String.html#toUpperCase%2D%2D[toUpperCase]()++ (link:{java9-javadoc}/java/lang/String.html#toUpperCase%2D%2D[java 9]) -* ++[[painless-api-reference-String-toUpperCase-1]]<> link:{java8-javadoc}/java/lang/String.html#toUpperCase%2Djava.util.Locale%2D[toUpperCase](<>)++ (link:{java9-javadoc}/java/lang/String.html#toUpperCase%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-String-trim-0]]<> link:{java8-javadoc}/java/lang/String.html#trim%2D%2D[trim]()++ (link:{java9-javadoc}/java/lang/String.html#trim%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StringBuffer.asciidoc b/docs/painless/painless-api-reference/StringBuffer.asciidoc deleted file mode 100644 index 5993832a983da..0000000000000 --- a/docs/painless/painless-api-reference/StringBuffer.asciidoc +++ /dev/null @@ -1,31 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StringBuffer]]++StringBuffer++:: -* ++[[painless-api-reference-StringBuffer-StringBuffer-0]]link:{java8-javadoc}/java/lang/StringBuffer.html#StringBuffer%2D%2D[StringBuffer]()++ (link:{java9-javadoc}/java/lang/StringBuffer.html#StringBuffer%2D%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-StringBuffer-1]]link:{java8-javadoc}/java/lang/StringBuffer.html#StringBuffer%2Djava.lang.CharSequence%2D[StringBuffer](<>)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#StringBuffer%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-append-1]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#append%2Djava.lang.Object%2D[append](def)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#append%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-append-3]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#append%2Djava.lang.CharSequence%2Dint%2Dint%2D[append](<>, int, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#append%2Djava.lang.CharSequence%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-appendCodePoint-1]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#appendCodePoint%2Dint%2D[appendCodePoint](int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#appendCodePoint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-capacity-0]]int link:{java8-javadoc}/java/lang/StringBuffer.html#capacity%2D%2D[capacity]()++ (link:{java9-javadoc}/java/lang/StringBuffer.html#capacity%2D%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-codePointAt-1]]int link:{java8-javadoc}/java/lang/StringBuffer.html#codePointAt%2Dint%2D[codePointAt](int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#codePointAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-codePointBefore-1]]int link:{java8-javadoc}/java/lang/StringBuffer.html#codePointBefore%2Dint%2D[codePointBefore](int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#codePointBefore%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-codePointCount-2]]int link:{java8-javadoc}/java/lang/StringBuffer.html#codePointCount%2Dint%2Dint%2D[codePointCount](int, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#codePointCount%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-delete-2]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#delete%2Dint%2Dint%2D[delete](int, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#delete%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-deleteCharAt-1]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#deleteCharAt%2Dint%2D[deleteCharAt](int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#deleteCharAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-getChars-4]]void link:{java8-javadoc}/java/lang/StringBuffer.html#getChars%2Dint%2Dint%2Dchar:A%2Dint%2D[getChars](int, int, char[], int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#getChars%2Dint%2Dint%2Dchar:A%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-indexOf-1]]int link:{java8-javadoc}/java/lang/StringBuffer.html#indexOf%2Djava.lang.String%2D[indexOf](<>)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#indexOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-indexOf-2]]int link:{java8-javadoc}/java/lang/StringBuffer.html#indexOf%2Djava.lang.String%2Dint%2D[indexOf](<>, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#indexOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-insert-2]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#insert%2Dint%2Djava.lang.Object%2D[insert](int, def)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#insert%2Dint%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-lastIndexOf-1]]int link:{java8-javadoc}/java/lang/StringBuffer.html#lastIndexOf%2Djava.lang.String%2D[lastIndexOf](<>)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#lastIndexOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-lastIndexOf-2]]int link:{java8-javadoc}/java/lang/StringBuffer.html#lastIndexOf%2Djava.lang.String%2Dint%2D[lastIndexOf](<>, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#lastIndexOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-offsetByCodePoints-2]]int link:{java8-javadoc}/java/lang/StringBuffer.html#offsetByCodePoints%2Dint%2Dint%2D[offsetByCodePoints](int, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#offsetByCodePoints%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-replace-3]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#replace%2Dint%2Dint%2Djava.lang.String%2D[replace](int, int, <>)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#replace%2Dint%2Dint%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-reverse-0]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#reverse%2D%2D[reverse]()++ (link:{java9-javadoc}/java/lang/StringBuffer.html#reverse%2D%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-setCharAt-2]]void link:{java8-javadoc}/java/lang/StringBuffer.html#setCharAt%2Dint%2Dchar%2D[setCharAt](int, char)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#setCharAt%2Dint%2Dchar%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-setLength-1]]void link:{java8-javadoc}/java/lang/StringBuffer.html#setLength%2Dint%2D[setLength](int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#setLength%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-substring-1]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#substring%2Dint%2D[substring](int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#substring%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuffer-substring-2]]<> link:{java8-javadoc}/java/lang/StringBuffer.html#substring%2Dint%2Dint%2D[substring](int, int)++ (link:{java9-javadoc}/java/lang/StringBuffer.html#substring%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StringBuilder.asciidoc b/docs/painless/painless-api-reference/StringBuilder.asciidoc deleted file mode 100644 index fd9c0c097cc03..0000000000000 --- a/docs/painless/painless-api-reference/StringBuilder.asciidoc +++ /dev/null @@ -1,31 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StringBuilder]]++StringBuilder++:: -* ++[[painless-api-reference-StringBuilder-StringBuilder-0]]link:{java8-javadoc}/java/lang/StringBuilder.html#StringBuilder%2D%2D[StringBuilder]()++ (link:{java9-javadoc}/java/lang/StringBuilder.html#StringBuilder%2D%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-StringBuilder-1]]link:{java8-javadoc}/java/lang/StringBuilder.html#StringBuilder%2Djava.lang.CharSequence%2D[StringBuilder](<>)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#StringBuilder%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-append-1]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#append%2Djava.lang.Object%2D[append](def)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#append%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-append-3]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#append%2Djava.lang.CharSequence%2Dint%2Dint%2D[append](<>, int, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#append%2Djava.lang.CharSequence%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-appendCodePoint-1]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#appendCodePoint%2Dint%2D[appendCodePoint](int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#appendCodePoint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-capacity-0]]int link:{java8-javadoc}/java/lang/StringBuilder.html#capacity%2D%2D[capacity]()++ (link:{java9-javadoc}/java/lang/StringBuilder.html#capacity%2D%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-codePointAt-1]]int link:{java8-javadoc}/java/lang/StringBuilder.html#codePointAt%2Dint%2D[codePointAt](int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#codePointAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-codePointBefore-1]]int link:{java8-javadoc}/java/lang/StringBuilder.html#codePointBefore%2Dint%2D[codePointBefore](int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#codePointBefore%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-codePointCount-2]]int link:{java8-javadoc}/java/lang/StringBuilder.html#codePointCount%2Dint%2Dint%2D[codePointCount](int, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#codePointCount%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-delete-2]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#delete%2Dint%2Dint%2D[delete](int, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#delete%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-deleteCharAt-1]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#deleteCharAt%2Dint%2D[deleteCharAt](int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#deleteCharAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-getChars-4]]void link:{java8-javadoc}/java/lang/StringBuilder.html#getChars%2Dint%2Dint%2Dchar:A%2Dint%2D[getChars](int, int, char[], int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#getChars%2Dint%2Dint%2Dchar:A%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-indexOf-1]]int link:{java8-javadoc}/java/lang/StringBuilder.html#indexOf%2Djava.lang.String%2D[indexOf](<>)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#indexOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-indexOf-2]]int link:{java8-javadoc}/java/lang/StringBuilder.html#indexOf%2Djava.lang.String%2Dint%2D[indexOf](<>, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#indexOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-insert-2]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#insert%2Dint%2Djava.lang.Object%2D[insert](int, def)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#insert%2Dint%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-lastIndexOf-1]]int link:{java8-javadoc}/java/lang/StringBuilder.html#lastIndexOf%2Djava.lang.String%2D[lastIndexOf](<>)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#lastIndexOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-lastIndexOf-2]]int link:{java8-javadoc}/java/lang/StringBuilder.html#lastIndexOf%2Djava.lang.String%2Dint%2D[lastIndexOf](<>, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#lastIndexOf%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-offsetByCodePoints-2]]int link:{java8-javadoc}/java/lang/StringBuilder.html#offsetByCodePoints%2Dint%2Dint%2D[offsetByCodePoints](int, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#offsetByCodePoints%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-replace-3]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#replace%2Dint%2Dint%2Djava.lang.String%2D[replace](int, int, <>)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#replace%2Dint%2Dint%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-reverse-0]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#reverse%2D%2D[reverse]()++ (link:{java9-javadoc}/java/lang/StringBuilder.html#reverse%2D%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-setCharAt-2]]void link:{java8-javadoc}/java/lang/StringBuilder.html#setCharAt%2Dint%2Dchar%2D[setCharAt](int, char)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#setCharAt%2Dint%2Dchar%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-setLength-1]]void link:{java8-javadoc}/java/lang/StringBuilder.html#setLength%2Dint%2D[setLength](int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#setLength%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-substring-1]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#substring%2Dint%2D[substring](int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#substring%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringBuilder-substring-2]]<> link:{java8-javadoc}/java/lang/StringBuilder.html#substring%2Dint%2Dint%2D[substring](int, int)++ (link:{java9-javadoc}/java/lang/StringBuilder.html#substring%2Dint%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StringCharacterIterator.asciidoc b/docs/painless/painless-api-reference/StringCharacterIterator.asciidoc deleted file mode 100644 index 8f65f2b6c5e53..0000000000000 --- a/docs/painless/painless-api-reference/StringCharacterIterator.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StringCharacterIterator]]++StringCharacterIterator++:: -* ++[[painless-api-reference-StringCharacterIterator-StringCharacterIterator-1]]link:{java8-javadoc}/java/text/StringCharacterIterator.html#StringCharacterIterator%2Djava.lang.String%2D[StringCharacterIterator](<>)++ (link:{java9-javadoc}/java/text/StringCharacterIterator.html#StringCharacterIterator%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringCharacterIterator-StringCharacterIterator-2]]link:{java8-javadoc}/java/text/StringCharacterIterator.html#StringCharacterIterator%2Djava.lang.String%2Dint%2D[StringCharacterIterator](<>, int)++ (link:{java9-javadoc}/java/text/StringCharacterIterator.html#StringCharacterIterator%2Djava.lang.String%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringCharacterIterator-StringCharacterIterator-4]]link:{java8-javadoc}/java/text/StringCharacterIterator.html#StringCharacterIterator%2Djava.lang.String%2Dint%2Dint%2Dint%2D[StringCharacterIterator](<>, int, int, int)++ (link:{java9-javadoc}/java/text/StringCharacterIterator.html#StringCharacterIterator%2Djava.lang.String%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-StringCharacterIterator-setText-1]]void link:{java8-javadoc}/java/text/StringCharacterIterator.html#setText%2Djava.lang.String%2D[setText](<>)++ (link:{java9-javadoc}/java/text/StringCharacterIterator.html#setText%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StringIndexOutOfBoundsException.asciidoc b/docs/painless/painless-api-reference/StringIndexOutOfBoundsException.asciidoc deleted file mode 100644 index a29cae3f857f0..0000000000000 --- a/docs/painless/painless-api-reference/StringIndexOutOfBoundsException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StringIndexOutOfBoundsException]]++StringIndexOutOfBoundsException++:: -* ++[[painless-api-reference-StringIndexOutOfBoundsException-StringIndexOutOfBoundsException-0]]link:{java8-javadoc}/java/lang/StringIndexOutOfBoundsException.html#StringIndexOutOfBoundsException%2D%2D[StringIndexOutOfBoundsException]()++ (link:{java9-javadoc}/java/lang/StringIndexOutOfBoundsException.html#StringIndexOutOfBoundsException%2D%2D[java 9]) -* ++[[painless-api-reference-StringIndexOutOfBoundsException-StringIndexOutOfBoundsException-1]]link:{java8-javadoc}/java/lang/StringIndexOutOfBoundsException.html#StringIndexOutOfBoundsException%2Djava.lang.String%2D[StringIndexOutOfBoundsException](<>)++ (link:{java9-javadoc}/java/lang/StringIndexOutOfBoundsException.html#StringIndexOutOfBoundsException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/StringJoiner.asciidoc b/docs/painless/painless-api-reference/StringJoiner.asciidoc deleted file mode 100644 index 7bd9baa4460ac..0000000000000 --- a/docs/painless/painless-api-reference/StringJoiner.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StringJoiner]]++StringJoiner++:: -* ++[[painless-api-reference-StringJoiner-StringJoiner-1]]link:{java8-javadoc}/java/util/StringJoiner.html#StringJoiner%2Djava.lang.CharSequence%2D[StringJoiner](<>)++ (link:{java9-javadoc}/java/util/StringJoiner.html#StringJoiner%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-StringJoiner-StringJoiner-3]]link:{java8-javadoc}/java/util/StringJoiner.html#StringJoiner%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2D[StringJoiner](<>, <>, <>)++ (link:{java9-javadoc}/java/util/StringJoiner.html#StringJoiner%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-StringJoiner-add-1]]<> link:{java8-javadoc}/java/util/StringJoiner.html#add%2Djava.lang.CharSequence%2D[add](<>)++ (link:{java9-javadoc}/java/util/StringJoiner.html#add%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-StringJoiner-length-0]]int link:{java8-javadoc}/java/util/StringJoiner.html#length%2D%2D[length]()++ (link:{java9-javadoc}/java/util/StringJoiner.html#length%2D%2D[java 9]) -* ++[[painless-api-reference-StringJoiner-merge-1]]<> link:{java8-javadoc}/java/util/StringJoiner.html#merge%2Djava.util.StringJoiner%2D[merge](<>)++ (link:{java9-javadoc}/java/util/StringJoiner.html#merge%2Djava.util.StringJoiner%2D[java 9]) -* ++[[painless-api-reference-StringJoiner-setEmptyValue-1]]<> link:{java8-javadoc}/java/util/StringJoiner.html#setEmptyValue%2Djava.lang.CharSequence%2D[setEmptyValue](<>)++ (link:{java9-javadoc}/java/util/StringJoiner.html#setEmptyValue%2Djava.lang.CharSequence%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/StringTokenizer.asciidoc b/docs/painless/painless-api-reference/StringTokenizer.asciidoc deleted file mode 100644 index 9462cecc33b4e..0000000000000 --- a/docs/painless/painless-api-reference/StringTokenizer.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-StringTokenizer]]++StringTokenizer++:: -* ++[[painless-api-reference-StringTokenizer-StringTokenizer-1]]link:{java8-javadoc}/java/util/StringTokenizer.html#StringTokenizer%2Djava.lang.String%2D[StringTokenizer](<>)++ (link:{java9-javadoc}/java/util/StringTokenizer.html#StringTokenizer%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringTokenizer-StringTokenizer-2]]link:{java8-javadoc}/java/util/StringTokenizer.html#StringTokenizer%2Djava.lang.String%2Djava.lang.String%2D[StringTokenizer](<>, <>)++ (link:{java9-javadoc}/java/util/StringTokenizer.html#StringTokenizer%2Djava.lang.String%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-StringTokenizer-StringTokenizer-3]]link:{java8-javadoc}/java/util/StringTokenizer.html#StringTokenizer%2Djava.lang.String%2Djava.lang.String%2Dboolean%2D[StringTokenizer](<>, <>, boolean)++ (link:{java9-javadoc}/java/util/StringTokenizer.html#StringTokenizer%2Djava.lang.String%2Djava.lang.String%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-StringTokenizer-countTokens-0]]int link:{java8-javadoc}/java/util/StringTokenizer.html#countTokens%2D%2D[countTokens]()++ (link:{java9-javadoc}/java/util/StringTokenizer.html#countTokens%2D%2D[java 9]) -* ++[[painless-api-reference-StringTokenizer-hasMoreTokens-0]]boolean link:{java8-javadoc}/java/util/StringTokenizer.html#hasMoreTokens%2D%2D[hasMoreTokens]()++ (link:{java9-javadoc}/java/util/StringTokenizer.html#hasMoreTokens%2D%2D[java 9]) -* ++[[painless-api-reference-StringTokenizer-nextToken-0]]<> link:{java8-javadoc}/java/util/StringTokenizer.html#nextToken%2D%2D[nextToken]()++ (link:{java9-javadoc}/java/util/StringTokenizer.html#nextToken%2D%2D[java 9]) -* ++[[painless-api-reference-StringTokenizer-nextToken-1]]<> link:{java8-javadoc}/java/util/StringTokenizer.html#nextToken%2Djava.lang.String%2D[nextToken](<>)++ (link:{java9-javadoc}/java/util/StringTokenizer.html#nextToken%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/Supplier.asciidoc b/docs/painless/painless-api-reference/Supplier.asciidoc deleted file mode 100644 index ba3cd9da951ec..0000000000000 --- a/docs/painless/painless-api-reference/Supplier.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Supplier]]++Supplier++:: -* ++[[painless-api-reference-Supplier-get-0]]def link:{java8-javadoc}/java/util/function/Supplier.html#get%2D%2D[get]()++ (link:{java9-javadoc}/java/util/function/Supplier.html#get%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/System.asciidoc b/docs/painless/painless-api-reference/System.asciidoc deleted file mode 100644 index a791d5dc15966..0000000000000 --- a/docs/painless/painless-api-reference/System.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-System]]++System++:: -* ++[[painless-api-reference-System-arraycopy-5]]static void link:{java8-javadoc}/java/lang/System.html#arraycopy%2Djava.lang.Object%2Dint%2Djava.lang.Object%2Dint%2Dint%2D[arraycopy](<>, int, <>, int, int)++ (link:{java9-javadoc}/java/lang/System.html#arraycopy%2Djava.lang.Object%2Dint%2Djava.lang.Object%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-System-currentTimeMillis-0]]static long link:{java8-javadoc}/java/lang/System.html#currentTimeMillis%2D%2D[currentTimeMillis]()++ (link:{java9-javadoc}/java/lang/System.html#currentTimeMillis%2D%2D[java 9]) -* ++[[painless-api-reference-System-nanoTime-0]]static long link:{java8-javadoc}/java/lang/System.html#nanoTime%2D%2D[nanoTime]()++ (link:{java9-javadoc}/java/lang/System.html#nanoTime%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Temporal.asciidoc b/docs/painless/painless-api-reference/Temporal.asciidoc deleted file mode 100644 index ce693cfde51fa..0000000000000 --- a/docs/painless/painless-api-reference/Temporal.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Temporal]]++Temporal++:: -* ++[[painless-api-reference-Temporal-minus-1]]<> link:{java8-javadoc}/java/time/temporal/Temporal.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Temporal-minus-2]]<> link:{java8-javadoc}/java/time/temporal/Temporal.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Temporal-plus-1]]<> link:{java8-javadoc}/java/time/temporal/Temporal.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Temporal-plus-2]]<> link:{java8-javadoc}/java/time/temporal/Temporal.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Temporal-until-2]]long link:{java8-javadoc}/java/time/temporal/Temporal.html#until%2Djava.time.temporal.Temporal%2Djava.time.temporal.TemporalUnit%2D[until](<>, <>)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#until%2Djava.time.temporal.Temporal%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Temporal-with-1]]<> link:{java8-javadoc}/java/time/temporal/Temporal.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-Temporal-with-2]]<> link:{java8-javadoc}/java/time/temporal/Temporal.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/temporal/Temporal.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalAccessor.asciidoc b/docs/painless/painless-api-reference/TemporalAccessor.asciidoc deleted file mode 100644 index 7583b2180d940..0000000000000 --- a/docs/painless/painless-api-reference/TemporalAccessor.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalAccessor]]++TemporalAccessor++:: -* ++[[painless-api-reference-TemporalAccessor-get-1]]int link:{java8-javadoc}/java/time/temporal/TemporalAccessor.html#get%2Djava.time.temporal.TemporalField%2D[get](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAccessor.html#get%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-TemporalAccessor-getLong-1]]long link:{java8-javadoc}/java/time/temporal/TemporalAccessor.html#getLong%2Djava.time.temporal.TemporalField%2D[getLong](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAccessor.html#getLong%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-TemporalAccessor-isSupported-1]]boolean link:{java8-javadoc}/java/time/temporal/TemporalAccessor.html#isSupported%2Djava.time.temporal.TemporalField%2D[isSupported](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAccessor.html#isSupported%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-TemporalAccessor-query-1]]def link:{java8-javadoc}/java/time/temporal/TemporalAccessor.html#query%2Djava.time.temporal.TemporalQuery%2D[query](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAccessor.html#query%2Djava.time.temporal.TemporalQuery%2D[java 9]) -* ++[[painless-api-reference-TemporalAccessor-range-1]]<> link:{java8-javadoc}/java/time/temporal/TemporalAccessor.html#range%2Djava.time.temporal.TemporalField%2D[range](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAccessor.html#range%2Djava.time.temporal.TemporalField%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalAdjuster.asciidoc b/docs/painless/painless-api-reference/TemporalAdjuster.asciidoc deleted file mode 100644 index b80f0d7afbfae..0000000000000 --- a/docs/painless/painless-api-reference/TemporalAdjuster.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalAdjuster]]++TemporalAdjuster++:: -* ++[[painless-api-reference-TemporalAdjuster-adjustInto-1]]<> link:{java8-javadoc}/java/time/temporal/TemporalAdjuster.html#adjustInto%2Djava.time.temporal.Temporal%2D[adjustInto](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjuster.html#adjustInto%2Djava.time.temporal.Temporal%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalAdjusters.asciidoc b/docs/painless/painless-api-reference/TemporalAdjusters.asciidoc deleted file mode 100644 index 7299ae9256abc..0000000000000 --- a/docs/painless/painless-api-reference/TemporalAdjusters.asciidoc +++ /dev/null @@ -1,21 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalAdjusters]]++TemporalAdjusters++:: -* ++[[painless-api-reference-TemporalAdjusters-dayOfWeekInMonth-2]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#dayOfWeekInMonth%2Dint%2Djava.time.DayOfWeek%2D[dayOfWeekInMonth](int, <>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#dayOfWeekInMonth%2Dint%2Djava.time.DayOfWeek%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-firstDayOfMonth-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfMonth%2D%2D[firstDayOfMonth]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-firstDayOfNextMonth-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfNextMonth%2D%2D[firstDayOfNextMonth]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfNextMonth%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-firstDayOfNextYear-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfNextYear%2D%2D[firstDayOfNextYear]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfNextYear%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-firstDayOfYear-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfYear%2D%2D[firstDayOfYear]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#firstDayOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-firstInMonth-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#firstInMonth%2Djava.time.DayOfWeek%2D[firstInMonth](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#firstInMonth%2Djava.time.DayOfWeek%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-lastDayOfMonth-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#lastDayOfMonth%2D%2D[lastDayOfMonth]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#lastDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-lastDayOfYear-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#lastDayOfYear%2D%2D[lastDayOfYear]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#lastDayOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-lastInMonth-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#lastInMonth%2Djava.time.DayOfWeek%2D[lastInMonth](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#lastInMonth%2Djava.time.DayOfWeek%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-next-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#next%2Djava.time.DayOfWeek%2D[next](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#next%2Djava.time.DayOfWeek%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-nextOrSame-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#nextOrSame%2Djava.time.DayOfWeek%2D[nextOrSame](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#nextOrSame%2Djava.time.DayOfWeek%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-ofDateAdjuster-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#ofDateAdjuster%2Djava.util.function.UnaryOperator%2D[ofDateAdjuster](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#ofDateAdjuster%2Djava.util.function.UnaryOperator%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-previous-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#previous%2Djava.time.DayOfWeek%2D[previous](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#previous%2Djava.time.DayOfWeek%2D[java 9]) -* ++[[painless-api-reference-TemporalAdjusters-previousOrSame-1]]static <> link:{java8-javadoc}/java/time/temporal/TemporalAdjusters.html#previousOrSame%2Djava.time.DayOfWeek%2D[previousOrSame](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAdjusters.html#previousOrSame%2Djava.time.DayOfWeek%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalAmount.asciidoc b/docs/painless/painless-api-reference/TemporalAmount.asciidoc deleted file mode 100644 index e27954b4db3e4..0000000000000 --- a/docs/painless/painless-api-reference/TemporalAmount.asciidoc +++ /dev/null @@ -1,11 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalAmount]]++TemporalAmount++:: -* ++[[painless-api-reference-TemporalAmount-addTo-1]]<> link:{java8-javadoc}/java/time/temporal/TemporalAmount.html#addTo%2Djava.time.temporal.Temporal%2D[addTo](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAmount.html#addTo%2Djava.time.temporal.Temporal%2D[java 9]) -* ++[[painless-api-reference-TemporalAmount-get-1]]long link:{java8-javadoc}/java/time/temporal/TemporalAmount.html#get%2Djava.time.temporal.TemporalUnit%2D[get](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAmount.html#get%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-TemporalAmount-getUnits-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalAmount.html#getUnits%2D%2D[getUnits]()++ (link:{java9-javadoc}/java/time/temporal/TemporalAmount.html#getUnits%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalAmount-subtractFrom-1]]<> link:{java8-javadoc}/java/time/temporal/TemporalAmount.html#subtractFrom%2Djava.time.temporal.Temporal%2D[subtractFrom](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalAmount.html#subtractFrom%2Djava.time.temporal.Temporal%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalField.asciidoc b/docs/painless/painless-api-reference/TemporalField.asciidoc deleted file mode 100644 index dd38544d46e4c..0000000000000 --- a/docs/painless/painless-api-reference/TemporalField.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalField]]++TemporalField++:: -* ++[[painless-api-reference-TemporalField-adjustInto-2]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#adjustInto%2Djava.time.temporal.Temporal%2Dlong%2D[adjustInto](<>, long)++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#adjustInto%2Djava.time.temporal.Temporal%2Dlong%2D[java 9]) -* ++[[painless-api-reference-TemporalField-getBaseUnit-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#getBaseUnit%2D%2D[getBaseUnit]()++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#getBaseUnit%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalField-getDisplayName-1]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#getDisplayName%2Djava.util.Locale%2D[getDisplayName](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#getDisplayName%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-TemporalField-getFrom-1]]long link:{java8-javadoc}/java/time/temporal/TemporalField.html#getFrom%2Djava.time.temporal.TemporalAccessor%2D[getFrom](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#getFrom%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-TemporalField-getRangeUnit-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#getRangeUnit%2D%2D[getRangeUnit]()++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#getRangeUnit%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalField-isDateBased-0]]boolean link:{java8-javadoc}/java/time/temporal/TemporalField.html#isDateBased%2D%2D[isDateBased]()++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#isDateBased%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalField-isSupportedBy-1]]boolean link:{java8-javadoc}/java/time/temporal/TemporalField.html#isSupportedBy%2Djava.time.temporal.TemporalAccessor%2D[isSupportedBy](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#isSupportedBy%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-TemporalField-isTimeBased-0]]boolean link:{java8-javadoc}/java/time/temporal/TemporalField.html#isTimeBased%2D%2D[isTimeBased]()++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#isTimeBased%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalField-range-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#range%2D%2D[range]()++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#range%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalField-rangeRefinedBy-1]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#rangeRefinedBy%2Djava.time.temporal.TemporalAccessor%2D[rangeRefinedBy](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#rangeRefinedBy%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-TemporalField-resolve-3]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#resolve%2Djava.util.Map%2Djava.time.temporal.TemporalAccessor%2Djava.time.format.ResolverStyle%2D[resolve](<>, <>, <>)++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#resolve%2Djava.util.Map%2Djava.time.temporal.TemporalAccessor%2Djava.time.format.ResolverStyle%2D[java 9]) -* ++[[painless-api-reference-TemporalField-toString-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalField.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/temporal/TemporalField.html#toString%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalQueries.asciidoc b/docs/painless/painless-api-reference/TemporalQueries.asciidoc deleted file mode 100644 index bd01ba6c2e0ed..0000000000000 --- a/docs/painless/painless-api-reference/TemporalQueries.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalQueries]]++TemporalQueries++:: -* ++[[painless-api-reference-TemporalQueries-chronology-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#chronology%2D%2D[chronology]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#chronology%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalQueries-localDate-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#localDate%2D%2D[localDate]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#localDate%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalQueries-localTime-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#localTime%2D%2D[localTime]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#localTime%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalQueries-offset-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#offset%2D%2D[offset]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#offset%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalQueries-precision-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#precision%2D%2D[precision]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#precision%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalQueries-zone-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#zone%2D%2D[zone]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#zone%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalQueries-zoneId-0]]static <> link:{java8-javadoc}/java/time/temporal/TemporalQueries.html#zoneId%2D%2D[zoneId]()++ (link:{java9-javadoc}/java/time/temporal/TemporalQueries.html#zoneId%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalQuery.asciidoc b/docs/painless/painless-api-reference/TemporalQuery.asciidoc deleted file mode 100644 index ca09eff368de3..0000000000000 --- a/docs/painless/painless-api-reference/TemporalQuery.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalQuery]]++TemporalQuery++:: -* ++[[painless-api-reference-TemporalQuery-queryFrom-1]]def link:{java8-javadoc}/java/time/temporal/TemporalQuery.html#queryFrom%2Djava.time.temporal.TemporalAccessor%2D[queryFrom](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalQuery.html#queryFrom%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TemporalUnit.asciidoc b/docs/painless/painless-api-reference/TemporalUnit.asciidoc deleted file mode 100644 index cc940d838efc2..0000000000000 --- a/docs/painless/painless-api-reference/TemporalUnit.asciidoc +++ /dev/null @@ -1,15 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TemporalUnit]]++TemporalUnit++:: -* ++[[painless-api-reference-TemporalUnit-addTo-2]]<> link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#addTo%2Djava.time.temporal.Temporal%2Dlong%2D[addTo](<>, long)++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#addTo%2Djava.time.temporal.Temporal%2Dlong%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-between-2]]long link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#between%2Djava.time.temporal.Temporal%2Djava.time.temporal.Temporal%2D[between](<>, <>)++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#between%2Djava.time.temporal.Temporal%2Djava.time.temporal.Temporal%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-getDuration-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#getDuration%2D%2D[getDuration]()++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#getDuration%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-isDateBased-0]]boolean link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#isDateBased%2D%2D[isDateBased]()++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#isDateBased%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-isDurationEstimated-0]]boolean link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#isDurationEstimated%2D%2D[isDurationEstimated]()++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#isDurationEstimated%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-isSupportedBy-1]]boolean link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#isSupportedBy%2Djava.time.temporal.Temporal%2D[isSupportedBy](<>)++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#isSupportedBy%2Djava.time.temporal.Temporal%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-isTimeBased-0]]boolean link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#isTimeBased%2D%2D[isTimeBased]()++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#isTimeBased%2D%2D[java 9]) -* ++[[painless-api-reference-TemporalUnit-toString-0]]<> link:{java8-javadoc}/java/time/temporal/TemporalUnit.html#toString%2D%2D[toString]()++ (link:{java9-javadoc}/java/time/temporal/TemporalUnit.html#toString%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TextStyle.asciidoc b/docs/painless/painless-api-reference/TextStyle.asciidoc deleted file mode 100644 index c7b122f934592..0000000000000 --- a/docs/painless/painless-api-reference/TextStyle.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TextStyle]]++TextStyle++:: -** [[painless-api-reference-TextStyle-FULL]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#FULL[FULL] (link:{java9-javadoc}/java/time/format/TextStyle.html#FULL[java 9]) -** [[painless-api-reference-TextStyle-FULL_STANDALONE]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#FULL_STANDALONE[FULL_STANDALONE] (link:{java9-javadoc}/java/time/format/TextStyle.html#FULL_STANDALONE[java 9]) -** [[painless-api-reference-TextStyle-NARROW]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#NARROW[NARROW] (link:{java9-javadoc}/java/time/format/TextStyle.html#NARROW[java 9]) -** [[painless-api-reference-TextStyle-NARROW_STANDALONE]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#NARROW_STANDALONE[NARROW_STANDALONE] (link:{java9-javadoc}/java/time/format/TextStyle.html#NARROW_STANDALONE[java 9]) -** [[painless-api-reference-TextStyle-SHORT]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#SHORT[SHORT] (link:{java9-javadoc}/java/time/format/TextStyle.html#SHORT[java 9]) -** [[painless-api-reference-TextStyle-SHORT_STANDALONE]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#SHORT_STANDALONE[SHORT_STANDALONE] (link:{java9-javadoc}/java/time/format/TextStyle.html#SHORT_STANDALONE[java 9]) -* ++[[painless-api-reference-TextStyle-valueOf-1]]static <> link:{java8-javadoc}/java/time/format/TextStyle.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/format/TextStyle.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-TextStyle-values-0]]static <>[] link:{java8-javadoc}/java/time/format/TextStyle.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/format/TextStyle.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-TextStyle-asNormal-0]]<> link:{java8-javadoc}/java/time/format/TextStyle.html#asNormal%2D%2D[asNormal]()++ (link:{java9-javadoc}/java/time/format/TextStyle.html#asNormal%2D%2D[java 9]) -* ++[[painless-api-reference-TextStyle-asStandalone-0]]<> link:{java8-javadoc}/java/time/format/TextStyle.html#asStandalone%2D%2D[asStandalone]()++ (link:{java9-javadoc}/java/time/format/TextStyle.html#asStandalone%2D%2D[java 9]) -* ++[[painless-api-reference-TextStyle-isStandalone-0]]boolean link:{java8-javadoc}/java/time/format/TextStyle.html#isStandalone%2D%2D[isStandalone]()++ (link:{java9-javadoc}/java/time/format/TextStyle.html#isStandalone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ThaiBuddhistChronology.asciidoc b/docs/painless/painless-api-reference/ThaiBuddhistChronology.asciidoc deleted file mode 100644 index 86c721fda7ada..0000000000000 --- a/docs/painless/painless-api-reference/ThaiBuddhistChronology.asciidoc +++ /dev/null @@ -1,16 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ThaiBuddhistChronology]]++ThaiBuddhistChronology++:: -** [[painless-api-reference-ThaiBuddhistChronology-INSTANCE]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#INSTANCE[INSTANCE] (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#INSTANCE[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-date-1]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[date](<>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#date%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-date-3]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#date%2Dint%2Dint%2Dint%2D[date](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#date%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-date-4]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[date](<>, int, int, int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#date%2Djava.time.chrono.Era%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-dateEpochDay-1]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#dateEpochDay%2Dlong%2D[dateEpochDay](long)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#dateEpochDay%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-dateYearDay-2]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#dateYearDay%2Dint%2Dint%2D[dateYearDay](int, int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#dateYearDay%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-dateYearDay-3]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[dateYearDay](<>, int, int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#dateYearDay%2Djava.time.chrono.Era%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-eraOf-1]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#eraOf%2Dint%2D[eraOf](int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#eraOf%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistChronology-resolveDate-2]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[resolveDate](<>, <>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistChronology.html#resolveDate%2Djava.util.Map%2Djava.time.format.ResolverStyle%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ThaiBuddhistDate.asciidoc b/docs/painless/painless-api-reference/ThaiBuddhistDate.asciidoc deleted file mode 100644 index 94d620182dff4..0000000000000 --- a/docs/painless/painless-api-reference/ThaiBuddhistDate.asciidoc +++ /dev/null @@ -1,17 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ThaiBuddhistDate]]++ThaiBuddhistDate++:: -* ++[[painless-api-reference-ThaiBuddhistDate-from-1]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-of-3]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#of%2Dint%2Dint%2Dint%2D[of](int, int, int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#of%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-getChronology-0]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#getChronology%2D%2D[getChronology]()++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#getChronology%2D%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-getEra-0]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#getEra%2D%2D[getEra]()++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#getEra%2D%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-minus-1]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-minus-2]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-plus-1]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-plus-2]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-with-1]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistDate-with-2]]<> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistDate.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ThaiBuddhistEra.asciidoc b/docs/painless/painless-api-reference/ThaiBuddhistEra.asciidoc deleted file mode 100644 index b6c070503a9ee..0000000000000 --- a/docs/painless/painless-api-reference/ThaiBuddhistEra.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ThaiBuddhistEra]]++ThaiBuddhistEra++:: -** [[painless-api-reference-ThaiBuddhistEra-BE]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistEra.html#BE[BE] (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistEra.html#BE[java 9]) -** [[painless-api-reference-ThaiBuddhistEra-BEFORE_BE]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistEra.html#BEFORE_BE[BEFORE_BE] (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistEra.html#BEFORE_BE[java 9]) -* ++[[painless-api-reference-ThaiBuddhistEra-of-1]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistEra.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistEra.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistEra-valueOf-1]]static <> link:{java8-javadoc}/java/time/chrono/ThaiBuddhistEra.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistEra.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistEra-values-0]]static <>[] link:{java8-javadoc}/java/time/chrono/ThaiBuddhistEra.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistEra.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-ThaiBuddhistEra-getValue-0]]int link:{java8-javadoc}/java/time/chrono/ThaiBuddhistEra.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/chrono/ThaiBuddhistEra.html#getValue%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/TimeZone.asciidoc b/docs/painless/painless-api-reference/TimeZone.asciidoc deleted file mode 100644 index 5618e388a4429..0000000000000 --- a/docs/painless/painless-api-reference/TimeZone.asciidoc +++ /dev/null @@ -1,29 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TimeZone]]++TimeZone++:: -** [[painless-api-reference-TimeZone-LONG]]static int link:{java8-javadoc}/java/util/TimeZone.html#LONG[LONG] (link:{java9-javadoc}/java/util/TimeZone.html#LONG[java 9]) -** [[painless-api-reference-TimeZone-SHORT]]static int link:{java8-javadoc}/java/util/TimeZone.html#SHORT[SHORT] (link:{java9-javadoc}/java/util/TimeZone.html#SHORT[java 9]) -* ++[[painless-api-reference-TimeZone-getAvailableIDs-0]]static <>[] link:{java8-javadoc}/java/util/TimeZone.html#getAvailableIDs%2D%2D[getAvailableIDs]()++ (link:{java9-javadoc}/java/util/TimeZone.html#getAvailableIDs%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getAvailableIDs-1]]static <>[] link:{java8-javadoc}/java/util/TimeZone.html#getAvailableIDs%2Dint%2D[getAvailableIDs](int)++ (link:{java9-javadoc}/java/util/TimeZone.html#getAvailableIDs%2Dint%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getDefault-0]]static <> link:{java8-javadoc}/java/util/TimeZone.html#getDefault%2D%2D[getDefault]()++ (link:{java9-javadoc}/java/util/TimeZone.html#getDefault%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getTimeZone-1]]static <> link:{java8-javadoc}/java/util/TimeZone.html#getTimeZone%2Djava.lang.String%2D[getTimeZone](<>)++ (link:{java9-javadoc}/java/util/TimeZone.html#getTimeZone%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-TimeZone-clone-0]]def link:{java8-javadoc}/java/util/TimeZone.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/TimeZone.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getDSTSavings-0]]int link:{java8-javadoc}/java/util/TimeZone.html#getDSTSavings%2D%2D[getDSTSavings]()++ (link:{java9-javadoc}/java/util/TimeZone.html#getDSTSavings%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getDisplayName-0]]<> link:{java8-javadoc}/java/util/TimeZone.html#getDisplayName%2D%2D[getDisplayName]()++ (link:{java9-javadoc}/java/util/TimeZone.html#getDisplayName%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getDisplayName-1]]<> link:{java8-javadoc}/java/util/TimeZone.html#getDisplayName%2Djava.util.Locale%2D[getDisplayName](<>)++ (link:{java9-javadoc}/java/util/TimeZone.html#getDisplayName%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getDisplayName-2]]<> link:{java8-javadoc}/java/util/TimeZone.html#getDisplayName%2Dboolean%2Dint%2D[getDisplayName](boolean, int)++ (link:{java9-javadoc}/java/util/TimeZone.html#getDisplayName%2Dboolean%2Dint%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getDisplayName-3]]<> link:{java8-javadoc}/java/util/TimeZone.html#getDisplayName%2Dboolean%2Dint%2Djava.util.Locale%2D[getDisplayName](boolean, int, <>)++ (link:{java9-javadoc}/java/util/TimeZone.html#getDisplayName%2Dboolean%2Dint%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getID-0]]<> link:{java8-javadoc}/java/util/TimeZone.html#getID%2D%2D[getID]()++ (link:{java9-javadoc}/java/util/TimeZone.html#getID%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getOffset-1]]int link:{java8-javadoc}/java/util/TimeZone.html#getOffset%2Dlong%2D[getOffset](long)++ (link:{java9-javadoc}/java/util/TimeZone.html#getOffset%2Dlong%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getOffset-6]]int link:{java8-javadoc}/java/util/TimeZone.html#getOffset%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[getOffset](int, int, int, int, int, int)++ (link:{java9-javadoc}/java/util/TimeZone.html#getOffset%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-TimeZone-getRawOffset-0]]int link:{java8-javadoc}/java/util/TimeZone.html#getRawOffset%2D%2D[getRawOffset]()++ (link:{java9-javadoc}/java/util/TimeZone.html#getRawOffset%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-hasSameRules-1]]boolean link:{java8-javadoc}/java/util/TimeZone.html#hasSameRules%2Djava.util.TimeZone%2D[hasSameRules](<>)++ (link:{java9-javadoc}/java/util/TimeZone.html#hasSameRules%2Djava.util.TimeZone%2D[java 9]) -* ++[[painless-api-reference-TimeZone-inDaylightTime-1]]boolean link:{java8-javadoc}/java/util/TimeZone.html#inDaylightTime%2Djava.util.Date%2D[inDaylightTime](<>)++ (link:{java9-javadoc}/java/util/TimeZone.html#inDaylightTime%2Djava.util.Date%2D[java 9]) -* ++[[painless-api-reference-TimeZone-observesDaylightTime-0]]boolean link:{java8-javadoc}/java/util/TimeZone.html#observesDaylightTime%2D%2D[observesDaylightTime]()++ (link:{java9-javadoc}/java/util/TimeZone.html#observesDaylightTime%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-setRawOffset-1]]void link:{java8-javadoc}/java/util/TimeZone.html#setRawOffset%2Dint%2D[setRawOffset](int)++ (link:{java9-javadoc}/java/util/TimeZone.html#setRawOffset%2Dint%2D[java 9]) -* ++[[painless-api-reference-TimeZone-toZoneId-0]]<> link:{java8-javadoc}/java/util/TimeZone.html#toZoneId%2D%2D[toZoneId]()++ (link:{java9-javadoc}/java/util/TimeZone.html#toZoneId%2D%2D[java 9]) -* ++[[painless-api-reference-TimeZone-useDaylightTime-0]]boolean link:{java8-javadoc}/java/util/TimeZone.html#useDaylightTime%2D%2D[useDaylightTime]()++ (link:{java9-javadoc}/java/util/TimeZone.html#useDaylightTime%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ToDoubleBiFunction.asciidoc b/docs/painless/painless-api-reference/ToDoubleBiFunction.asciidoc deleted file mode 100644 index 8ef0873fb5e15..0000000000000 --- a/docs/painless/painless-api-reference/ToDoubleBiFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ToDoubleBiFunction]]++ToDoubleBiFunction++:: -* ++[[painless-api-reference-ToDoubleBiFunction-applyAsDouble-2]]double link:{java8-javadoc}/java/util/function/ToDoubleBiFunction.html#applyAsDouble%2Djava.lang.Object%2Djava.lang.Object%2D[applyAsDouble](def, def)++ (link:{java9-javadoc}/java/util/function/ToDoubleBiFunction.html#applyAsDouble%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ToDoubleFunction.asciidoc b/docs/painless/painless-api-reference/ToDoubleFunction.asciidoc deleted file mode 100644 index b0d66be9dc09e..0000000000000 --- a/docs/painless/painless-api-reference/ToDoubleFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ToDoubleFunction]]++ToDoubleFunction++:: -* ++[[painless-api-reference-ToDoubleFunction-applyAsDouble-1]]double link:{java8-javadoc}/java/util/function/ToDoubleFunction.html#applyAsDouble%2Djava.lang.Object%2D[applyAsDouble](def)++ (link:{java9-javadoc}/java/util/function/ToDoubleFunction.html#applyAsDouble%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ToIntBiFunction.asciidoc b/docs/painless/painless-api-reference/ToIntBiFunction.asciidoc deleted file mode 100644 index 42cc396a1a989..0000000000000 --- a/docs/painless/painless-api-reference/ToIntBiFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ToIntBiFunction]]++ToIntBiFunction++:: -* ++[[painless-api-reference-ToIntBiFunction-applyAsInt-2]]int link:{java8-javadoc}/java/util/function/ToIntBiFunction.html#applyAsInt%2Djava.lang.Object%2Djava.lang.Object%2D[applyAsInt](def, def)++ (link:{java9-javadoc}/java/util/function/ToIntBiFunction.html#applyAsInt%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ToIntFunction.asciidoc b/docs/painless/painless-api-reference/ToIntFunction.asciidoc deleted file mode 100644 index c471de6b77dc4..0000000000000 --- a/docs/painless/painless-api-reference/ToIntFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ToIntFunction]]++ToIntFunction++:: -* ++[[painless-api-reference-ToIntFunction-applyAsInt-1]]int link:{java8-javadoc}/java/util/function/ToIntFunction.html#applyAsInt%2Djava.lang.Object%2D[applyAsInt](def)++ (link:{java9-javadoc}/java/util/function/ToIntFunction.html#applyAsInt%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ToLongBiFunction.asciidoc b/docs/painless/painless-api-reference/ToLongBiFunction.asciidoc deleted file mode 100644 index ffda1353210a8..0000000000000 --- a/docs/painless/painless-api-reference/ToLongBiFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ToLongBiFunction]]++ToLongBiFunction++:: -* ++[[painless-api-reference-ToLongBiFunction-applyAsLong-2]]long link:{java8-javadoc}/java/util/function/ToLongBiFunction.html#applyAsLong%2Djava.lang.Object%2Djava.lang.Object%2D[applyAsLong](def, def)++ (link:{java9-javadoc}/java/util/function/ToLongBiFunction.html#applyAsLong%2Djava.lang.Object%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ToLongFunction.asciidoc b/docs/painless/painless-api-reference/ToLongFunction.asciidoc deleted file mode 100644 index 785c414f8c906..0000000000000 --- a/docs/painless/painless-api-reference/ToLongFunction.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ToLongFunction]]++ToLongFunction++:: -* ++[[painless-api-reference-ToLongFunction-applyAsLong-1]]long link:{java8-javadoc}/java/util/function/ToLongFunction.html#applyAsLong%2Djava.lang.Object%2D[applyAsLong](def)++ (link:{java9-javadoc}/java/util/function/ToLongFunction.html#applyAsLong%2Djava.lang.Object%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/TooManyListenersException.asciidoc b/docs/painless/painless-api-reference/TooManyListenersException.asciidoc deleted file mode 100644 index 5a2d3d57b314d..0000000000000 --- a/docs/painless/painless-api-reference/TooManyListenersException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TooManyListenersException]]++TooManyListenersException++:: -* ++[[painless-api-reference-TooManyListenersException-TooManyListenersException-0]]link:{java8-javadoc}/java/util/TooManyListenersException.html#TooManyListenersException%2D%2D[TooManyListenersException]()++ (link:{java9-javadoc}/java/util/TooManyListenersException.html#TooManyListenersException%2D%2D[java 9]) -* ++[[painless-api-reference-TooManyListenersException-TooManyListenersException-1]]link:{java8-javadoc}/java/util/TooManyListenersException.html#TooManyListenersException%2Djava.lang.String%2D[TooManyListenersException](<>)++ (link:{java9-javadoc}/java/util/TooManyListenersException.html#TooManyListenersException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/TreeMap.asciidoc b/docs/painless/painless-api-reference/TreeMap.asciidoc deleted file mode 100644 index f91bae69162f6..0000000000000 --- a/docs/painless/painless-api-reference/TreeMap.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TreeMap]]++TreeMap++:: -* ++[[painless-api-reference-TreeMap-TreeMap-0]]link:{java8-javadoc}/java/util/TreeMap.html#TreeMap%2D%2D[TreeMap]()++ (link:{java9-javadoc}/java/util/TreeMap.html#TreeMap%2D%2D[java 9]) -* ++[[painless-api-reference-TreeMap-TreeMap-1]]link:{java8-javadoc}/java/util/TreeMap.html#TreeMap%2Djava.util.Comparator%2D[TreeMap](<>)++ (link:{java9-javadoc}/java/util/TreeMap.html#TreeMap%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-TreeMap-clone-0]]def link:{java8-javadoc}/java/util/TreeMap.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/TreeMap.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/TreeSet.asciidoc b/docs/painless/painless-api-reference/TreeSet.asciidoc deleted file mode 100644 index 84e8a339fe3bf..0000000000000 --- a/docs/painless/painless-api-reference/TreeSet.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TreeSet]]++TreeSet++:: -* ++[[painless-api-reference-TreeSet-TreeSet-0]]link:{java8-javadoc}/java/util/TreeSet.html#TreeSet%2D%2D[TreeSet]()++ (link:{java9-javadoc}/java/util/TreeSet.html#TreeSet%2D%2D[java 9]) -* ++[[painless-api-reference-TreeSet-TreeSet-1]]link:{java8-javadoc}/java/util/TreeSet.html#TreeSet%2Djava.util.Comparator%2D[TreeSet](<>)++ (link:{java9-javadoc}/java/util/TreeSet.html#TreeSet%2Djava.util.Comparator%2D[java 9]) -* ++[[painless-api-reference-TreeSet-clone-0]]def link:{java8-javadoc}/java/util/TreeSet.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/TreeSet.html#clone%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/TypeNotPresentException.asciidoc b/docs/painless/painless-api-reference/TypeNotPresentException.asciidoc deleted file mode 100644 index cecb588d83b17..0000000000000 --- a/docs/painless/painless-api-reference/TypeNotPresentException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-TypeNotPresentException]]++TypeNotPresentException++:: -* ++[[painless-api-reference-TypeNotPresentException-typeName-0]]<> link:{java8-javadoc}/java/lang/TypeNotPresentException.html#typeName%2D%2D[typeName]()++ (link:{java9-javadoc}/java/lang/TypeNotPresentException.html#typeName%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/UUID.asciidoc b/docs/painless/painless-api-reference/UUID.asciidoc deleted file mode 100644 index a47620f9bc1d6..0000000000000 --- a/docs/painless/painless-api-reference/UUID.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-UUID]]++UUID++:: -* ++[[painless-api-reference-UUID-fromString-1]]static <> link:{java8-javadoc}/java/util/UUID.html#fromString%2Djava.lang.String%2D[fromString](<>)++ (link:{java9-javadoc}/java/util/UUID.html#fromString%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-UUID-nameUUIDFromBytes-1]]static <> link:{java8-javadoc}/java/util/UUID.html#nameUUIDFromBytes%2Dbyte:A%2D[nameUUIDFromBytes](byte[])++ (link:{java9-javadoc}/java/util/UUID.html#nameUUIDFromBytes%2Dbyte:A%2D[java 9]) -* ++[[painless-api-reference-UUID-UUID-2]]link:{java8-javadoc}/java/util/UUID.html#UUID%2Dlong%2Dlong%2D[UUID](long, long)++ (link:{java9-javadoc}/java/util/UUID.html#UUID%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-UUID-clockSequence-0]]int link:{java8-javadoc}/java/util/UUID.html#clockSequence%2D%2D[clockSequence]()++ (link:{java9-javadoc}/java/util/UUID.html#clockSequence%2D%2D[java 9]) -* ++[[painless-api-reference-UUID-compareTo-1]]int link:{java8-javadoc}/java/util/UUID.html#compareTo%2Djava.util.UUID%2D[compareTo](<>)++ (link:{java9-javadoc}/java/util/UUID.html#compareTo%2Djava.util.UUID%2D[java 9]) -* ++[[painless-api-reference-UUID-getLeastSignificantBits-0]]long link:{java8-javadoc}/java/util/UUID.html#getLeastSignificantBits%2D%2D[getLeastSignificantBits]()++ (link:{java9-javadoc}/java/util/UUID.html#getLeastSignificantBits%2D%2D[java 9]) -* ++[[painless-api-reference-UUID-getMostSignificantBits-0]]long link:{java8-javadoc}/java/util/UUID.html#getMostSignificantBits%2D%2D[getMostSignificantBits]()++ (link:{java9-javadoc}/java/util/UUID.html#getMostSignificantBits%2D%2D[java 9]) -* ++[[painless-api-reference-UUID-node-0]]long link:{java8-javadoc}/java/util/UUID.html#node%2D%2D[node]()++ (link:{java9-javadoc}/java/util/UUID.html#node%2D%2D[java 9]) -* ++[[painless-api-reference-UUID-timestamp-0]]long link:{java8-javadoc}/java/util/UUID.html#timestamp%2D%2D[timestamp]()++ (link:{java9-javadoc}/java/util/UUID.html#timestamp%2D%2D[java 9]) -* ++[[painless-api-reference-UUID-variant-0]]int link:{java8-javadoc}/java/util/UUID.html#variant%2D%2D[variant]()++ (link:{java9-javadoc}/java/util/UUID.html#variant%2D%2D[java 9]) -* ++[[painless-api-reference-UUID-version-0]]int link:{java8-javadoc}/java/util/UUID.html#version%2D%2D[version]()++ (link:{java9-javadoc}/java/util/UUID.html#version%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/UnaryOperator.asciidoc b/docs/painless/painless-api-reference/UnaryOperator.asciidoc deleted file mode 100644 index fb7c6ed70d25d..0000000000000 --- a/docs/painless/painless-api-reference/UnaryOperator.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-UnaryOperator]]++UnaryOperator++:: -* ++[[painless-api-reference-UnaryOperator-identity-0]]static <> link:{java8-javadoc}/java/util/function/UnaryOperator.html#identity%2D%2D[identity]()++ (link:{java9-javadoc}/java/util/function/UnaryOperator.html#identity%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/UnknownFormatConversionException.asciidoc b/docs/painless/painless-api-reference/UnknownFormatConversionException.asciidoc deleted file mode 100644 index 95dd348f422c1..0000000000000 --- a/docs/painless/painless-api-reference/UnknownFormatConversionException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-UnknownFormatConversionException]]++UnknownFormatConversionException++:: -* ++[[painless-api-reference-UnknownFormatConversionException-UnknownFormatConversionException-1]]link:{java8-javadoc}/java/util/UnknownFormatConversionException.html#UnknownFormatConversionException%2Djava.lang.String%2D[UnknownFormatConversionException](<>)++ (link:{java9-javadoc}/java/util/UnknownFormatConversionException.html#UnknownFormatConversionException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-UnknownFormatConversionException-getConversion-0]]<> link:{java8-javadoc}/java/util/UnknownFormatConversionException.html#getConversion%2D%2D[getConversion]()++ (link:{java9-javadoc}/java/util/UnknownFormatConversionException.html#getConversion%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/UnknownFormatFlagsException.asciidoc b/docs/painless/painless-api-reference/UnknownFormatFlagsException.asciidoc deleted file mode 100644 index f14d986c05b38..0000000000000 --- a/docs/painless/painless-api-reference/UnknownFormatFlagsException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-UnknownFormatFlagsException]]++UnknownFormatFlagsException++:: -* ++[[painless-api-reference-UnknownFormatFlagsException-UnknownFormatFlagsException-1]]link:{java8-javadoc}/java/util/UnknownFormatFlagsException.html#UnknownFormatFlagsException%2Djava.lang.String%2D[UnknownFormatFlagsException](<>)++ (link:{java9-javadoc}/java/util/UnknownFormatFlagsException.html#UnknownFormatFlagsException%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-UnknownFormatFlagsException-getFlags-0]]<> link:{java8-javadoc}/java/util/UnknownFormatFlagsException.html#getFlags%2D%2D[getFlags]()++ (link:{java9-javadoc}/java/util/UnknownFormatFlagsException.html#getFlags%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/UnsupportedOperationException.asciidoc b/docs/painless/painless-api-reference/UnsupportedOperationException.asciidoc deleted file mode 100644 index ad9cbdb1478bf..0000000000000 --- a/docs/painless/painless-api-reference/UnsupportedOperationException.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-UnsupportedOperationException]]++UnsupportedOperationException++:: -* ++[[painless-api-reference-UnsupportedOperationException-UnsupportedOperationException-0]]link:{java8-javadoc}/java/lang/UnsupportedOperationException.html#UnsupportedOperationException%2D%2D[UnsupportedOperationException]()++ (link:{java9-javadoc}/java/lang/UnsupportedOperationException.html#UnsupportedOperationException%2D%2D[java 9]) -* ++[[painless-api-reference-UnsupportedOperationException-UnsupportedOperationException-1]]link:{java8-javadoc}/java/lang/UnsupportedOperationException.html#UnsupportedOperationException%2Djava.lang.String%2D[UnsupportedOperationException](<>)++ (link:{java9-javadoc}/java/lang/UnsupportedOperationException.html#UnsupportedOperationException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/UnsupportedTemporalTypeException.asciidoc b/docs/painless/painless-api-reference/UnsupportedTemporalTypeException.asciidoc deleted file mode 100644 index 96e4c1d840220..0000000000000 --- a/docs/painless/painless-api-reference/UnsupportedTemporalTypeException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-UnsupportedTemporalTypeException]]++UnsupportedTemporalTypeException++:: -* ++[[painless-api-reference-UnsupportedTemporalTypeException-UnsupportedTemporalTypeException-1]]link:{java8-javadoc}/java/time/temporal/UnsupportedTemporalTypeException.html#UnsupportedTemporalTypeException%2Djava.lang.String%2D[UnsupportedTemporalTypeException](<>)++ (link:{java9-javadoc}/java/time/temporal/UnsupportedTemporalTypeException.html#UnsupportedTemporalTypeException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ValueRange.asciidoc b/docs/painless/painless-api-reference/ValueRange.asciidoc deleted file mode 100644 index 409d155f9546a..0000000000000 --- a/docs/painless/painless-api-reference/ValueRange.asciidoc +++ /dev/null @@ -1,20 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ValueRange]]++ValueRange++:: -* ++[[painless-api-reference-ValueRange-of-2]]static <> link:{java8-javadoc}/java/time/temporal/ValueRange.html#of%2Dlong%2Dlong%2D[of](long, long)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#of%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ValueRange-of-3]]static <> link:{java8-javadoc}/java/time/temporal/ValueRange.html#of%2Dlong%2Dlong%2Dlong%2D[of](long, long, long)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#of%2Dlong%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ValueRange-of-4]]static <> link:{java8-javadoc}/java/time/temporal/ValueRange.html#of%2Dlong%2Dlong%2Dlong%2Dlong%2D[of](long, long, long, long)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#of%2Dlong%2Dlong%2Dlong%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ValueRange-checkValidIntValue-2]]int link:{java8-javadoc}/java/time/temporal/ValueRange.html#checkValidIntValue%2Dlong%2Djava.time.temporal.TemporalField%2D[checkValidIntValue](long, <>)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#checkValidIntValue%2Dlong%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-ValueRange-checkValidValue-2]]long link:{java8-javadoc}/java/time/temporal/ValueRange.html#checkValidValue%2Dlong%2Djava.time.temporal.TemporalField%2D[checkValidValue](long, <>)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#checkValidValue%2Dlong%2Djava.time.temporal.TemporalField%2D[java 9]) -* ++[[painless-api-reference-ValueRange-getLargestMinimum-0]]long link:{java8-javadoc}/java/time/temporal/ValueRange.html#getLargestMinimum%2D%2D[getLargestMinimum]()++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#getLargestMinimum%2D%2D[java 9]) -* ++[[painless-api-reference-ValueRange-getMaximum-0]]long link:{java8-javadoc}/java/time/temporal/ValueRange.html#getMaximum%2D%2D[getMaximum]()++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#getMaximum%2D%2D[java 9]) -* ++[[painless-api-reference-ValueRange-getMinimum-0]]long link:{java8-javadoc}/java/time/temporal/ValueRange.html#getMinimum%2D%2D[getMinimum]()++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#getMinimum%2D%2D[java 9]) -* ++[[painless-api-reference-ValueRange-getSmallestMaximum-0]]long link:{java8-javadoc}/java/time/temporal/ValueRange.html#getSmallestMaximum%2D%2D[getSmallestMaximum]()++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#getSmallestMaximum%2D%2D[java 9]) -* ++[[painless-api-reference-ValueRange-isFixed-0]]boolean link:{java8-javadoc}/java/time/temporal/ValueRange.html#isFixed%2D%2D[isFixed]()++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#isFixed%2D%2D[java 9]) -* ++[[painless-api-reference-ValueRange-isIntValue-0]]boolean link:{java8-javadoc}/java/time/temporal/ValueRange.html#isIntValue%2D%2D[isIntValue]()++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#isIntValue%2D%2D[java 9]) -* ++[[painless-api-reference-ValueRange-isValidIntValue-1]]boolean link:{java8-javadoc}/java/time/temporal/ValueRange.html#isValidIntValue%2Dlong%2D[isValidIntValue](long)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#isValidIntValue%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ValueRange-isValidValue-1]]boolean link:{java8-javadoc}/java/time/temporal/ValueRange.html#isValidValue%2Dlong%2D[isValidValue](long)++ (link:{java9-javadoc}/java/time/temporal/ValueRange.html#isValidValue%2Dlong%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Vector.asciidoc b/docs/painless/painless-api-reference/Vector.asciidoc deleted file mode 100644 index b1d9ed88b6f9a..0000000000000 --- a/docs/painless/painless-api-reference/Vector.asciidoc +++ /dev/null @@ -1,22 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Vector]]++Vector++:: -* ++[[painless-api-reference-Vector-Vector-0]]link:{java8-javadoc}/java/util/Vector.html#Vector%2D%2D[Vector]()++ (link:{java9-javadoc}/java/util/Vector.html#Vector%2D%2D[java 9]) -* ++[[painless-api-reference-Vector-Vector-1]]link:{java8-javadoc}/java/util/Vector.html#Vector%2Djava.util.Collection%2D[Vector](<>)++ (link:{java9-javadoc}/java/util/Vector.html#Vector%2Djava.util.Collection%2D[java 9]) -* ++[[painless-api-reference-Vector-addElement-1]]void link:{java8-javadoc}/java/util/Vector.html#addElement%2Djava.lang.Object%2D[addElement](def)++ (link:{java9-javadoc}/java/util/Vector.html#addElement%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Vector-clone-0]]def link:{java8-javadoc}/java/util/Vector.html#clone%2D%2D[clone]()++ (link:{java9-javadoc}/java/util/Vector.html#clone%2D%2D[java 9]) -* ++[[painless-api-reference-Vector-copyInto-1]]void link:{java8-javadoc}/java/util/Vector.html#copyInto%2Djava.lang.Object:A%2D[copyInto](<>[])++ (link:{java9-javadoc}/java/util/Vector.html#copyInto%2Djava.lang.Object:A%2D[java 9]) -* ++[[painless-api-reference-Vector-elementAt-1]]def link:{java8-javadoc}/java/util/Vector.html#elementAt%2Dint%2D[elementAt](int)++ (link:{java9-javadoc}/java/util/Vector.html#elementAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-Vector-elements-0]]<> link:{java8-javadoc}/java/util/Vector.html#elements%2D%2D[elements]()++ (link:{java9-javadoc}/java/util/Vector.html#elements%2D%2D[java 9]) -* ++[[painless-api-reference-Vector-firstElement-0]]def link:{java8-javadoc}/java/util/Vector.html#firstElement%2D%2D[firstElement]()++ (link:{java9-javadoc}/java/util/Vector.html#firstElement%2D%2D[java 9]) -* ++[[painless-api-reference-Vector-insertElementAt-2]]void link:{java8-javadoc}/java/util/Vector.html#insertElementAt%2Djava.lang.Object%2Dint%2D[insertElementAt](def, int)++ (link:{java9-javadoc}/java/util/Vector.html#insertElementAt%2Djava.lang.Object%2Dint%2D[java 9]) -* ++[[painless-api-reference-Vector-lastElement-0]]def link:{java8-javadoc}/java/util/Vector.html#lastElement%2D%2D[lastElement]()++ (link:{java9-javadoc}/java/util/Vector.html#lastElement%2D%2D[java 9]) -* ++[[painless-api-reference-Vector-lastIndexOf-2]]int link:{java8-javadoc}/java/util/Vector.html#lastIndexOf%2Djava.lang.Object%2Dint%2D[lastIndexOf](def, int)++ (link:{java9-javadoc}/java/util/Vector.html#lastIndexOf%2Djava.lang.Object%2Dint%2D[java 9]) -* ++[[painless-api-reference-Vector-removeAllElements-0]]void link:{java8-javadoc}/java/util/Vector.html#removeAllElements%2D%2D[removeAllElements]()++ (link:{java9-javadoc}/java/util/Vector.html#removeAllElements%2D%2D[java 9]) -* ++[[painless-api-reference-Vector-removeElement-1]]boolean link:{java8-javadoc}/java/util/Vector.html#removeElement%2Djava.lang.Object%2D[removeElement](def)++ (link:{java9-javadoc}/java/util/Vector.html#removeElement%2Djava.lang.Object%2D[java 9]) -* ++[[painless-api-reference-Vector-removeElementAt-1]]void link:{java8-javadoc}/java/util/Vector.html#removeElementAt%2Dint%2D[removeElementAt](int)++ (link:{java9-javadoc}/java/util/Vector.html#removeElementAt%2Dint%2D[java 9]) -* ++[[painless-api-reference-Vector-setElementAt-2]]void link:{java8-javadoc}/java/util/Vector.html#setElementAt%2Djava.lang.Object%2Dint%2D[setElementAt](def, int)++ (link:{java9-javadoc}/java/util/Vector.html#setElementAt%2Djava.lang.Object%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/WeekFields.asciidoc b/docs/painless/painless-api-reference/WeekFields.asciidoc deleted file mode 100644 index 330b2e98d2e7a..0000000000000 --- a/docs/painless/painless-api-reference/WeekFields.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-WeekFields]]++WeekFields++:: -** [[painless-api-reference-WeekFields-ISO]]static <> link:{java8-javadoc}/java/time/temporal/WeekFields.html#ISO[ISO] (link:{java9-javadoc}/java/time/temporal/WeekFields.html#ISO[java 9]) -** [[painless-api-reference-WeekFields-SUNDAY_START]]static <> link:{java8-javadoc}/java/time/temporal/WeekFields.html#SUNDAY_START[SUNDAY_START] (link:{java9-javadoc}/java/time/temporal/WeekFields.html#SUNDAY_START[java 9]) -** [[painless-api-reference-WeekFields-WEEK_BASED_YEARS]]static <> link:{java8-javadoc}/java/time/temporal/WeekFields.html#WEEK_BASED_YEARS[WEEK_BASED_YEARS] (link:{java9-javadoc}/java/time/temporal/WeekFields.html#WEEK_BASED_YEARS[java 9]) -* ++[[painless-api-reference-WeekFields-of-1]]static <> link:{java8-javadoc}/java/time/temporal/WeekFields.html#of%2Djava.util.Locale%2D[of](<>)++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#of%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-WeekFields-of-2]]static <> link:{java8-javadoc}/java/time/temporal/WeekFields.html#of%2Djava.time.DayOfWeek%2Dint%2D[of](<>, int)++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#of%2Djava.time.DayOfWeek%2Dint%2D[java 9]) -* ++[[painless-api-reference-WeekFields-dayOfWeek-0]]<> link:{java8-javadoc}/java/time/temporal/WeekFields.html#dayOfWeek%2D%2D[dayOfWeek]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#dayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-WeekFields-getFirstDayOfWeek-0]]<> link:{java8-javadoc}/java/time/temporal/WeekFields.html#getFirstDayOfWeek%2D%2D[getFirstDayOfWeek]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#getFirstDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-WeekFields-getMinimalDaysInFirstWeek-0]]int link:{java8-javadoc}/java/time/temporal/WeekFields.html#getMinimalDaysInFirstWeek%2D%2D[getMinimalDaysInFirstWeek]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#getMinimalDaysInFirstWeek%2D%2D[java 9]) -* ++[[painless-api-reference-WeekFields-weekBasedYear-0]]<> link:{java8-javadoc}/java/time/temporal/WeekFields.html#weekBasedYear%2D%2D[weekBasedYear]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#weekBasedYear%2D%2D[java 9]) -* ++[[painless-api-reference-WeekFields-weekOfMonth-0]]<> link:{java8-javadoc}/java/time/temporal/WeekFields.html#weekOfMonth%2D%2D[weekOfMonth]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#weekOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-WeekFields-weekOfWeekBasedYear-0]]<> link:{java8-javadoc}/java/time/temporal/WeekFields.html#weekOfWeekBasedYear%2D%2D[weekOfWeekBasedYear]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#weekOfWeekBasedYear%2D%2D[java 9]) -* ++[[painless-api-reference-WeekFields-weekOfYear-0]]<> link:{java8-javadoc}/java/time/temporal/WeekFields.html#weekOfYear%2D%2D[weekOfYear]()++ (link:{java9-javadoc}/java/time/temporal/WeekFields.html#weekOfYear%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/Year.asciidoc b/docs/painless/painless-api-reference/Year.asciidoc deleted file mode 100644 index e3800991039d6..0000000000000 --- a/docs/painless/painless-api-reference/Year.asciidoc +++ /dev/null @@ -1,32 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-Year]]++Year++:: -** [[painless-api-reference-Year-MAX_VALUE]]static int link:{java8-javadoc}/java/time/Year.html#MAX_VALUE[MAX_VALUE] (link:{java9-javadoc}/java/time/Year.html#MAX_VALUE[java 9]) -** [[painless-api-reference-Year-MIN_VALUE]]static int link:{java8-javadoc}/java/time/Year.html#MIN_VALUE[MIN_VALUE] (link:{java9-javadoc}/java/time/Year.html#MIN_VALUE[java 9]) -* ++[[painless-api-reference-Year-from-1]]static <> link:{java8-javadoc}/java/time/Year.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/Year.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-Year-isLeap-1]]static boolean link:{java8-javadoc}/java/time/Year.html#isLeap%2Dlong%2D[isLeap](long)++ (link:{java9-javadoc}/java/time/Year.html#isLeap%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Year-of-1]]static <> link:{java8-javadoc}/java/time/Year.html#of%2Dint%2D[of](int)++ (link:{java9-javadoc}/java/time/Year.html#of%2Dint%2D[java 9]) -* ++[[painless-api-reference-Year-parse-1]]static <> link:{java8-javadoc}/java/time/Year.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/Year.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-Year-parse-2]]static <> link:{java8-javadoc}/java/time/Year.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/Year.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-Year-atDay-1]]<> link:{java8-javadoc}/java/time/Year.html#atDay%2Dint%2D[atDay](int)++ (link:{java9-javadoc}/java/time/Year.html#atDay%2Dint%2D[java 9]) -* ++[[painless-api-reference-Year-atMonth-1]]<> link:{java8-javadoc}/java/time/Year.html#atMonth%2Dint%2D[atMonth](int)++ (link:{java9-javadoc}/java/time/Year.html#atMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-Year-atMonthDay-1]]<> link:{java8-javadoc}/java/time/Year.html#atMonthDay%2Djava.time.MonthDay%2D[atMonthDay](<>)++ (link:{java9-javadoc}/java/time/Year.html#atMonthDay%2Djava.time.MonthDay%2D[java 9]) -* ++[[painless-api-reference-Year-compareTo-1]]int link:{java8-javadoc}/java/time/Year.html#compareTo%2Djava.time.Year%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/Year.html#compareTo%2Djava.time.Year%2D[java 9]) -* ++[[painless-api-reference-Year-format-1]]<> link:{java8-javadoc}/java/time/Year.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/Year.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-Year-getValue-0]]int link:{java8-javadoc}/java/time/Year.html#getValue%2D%2D[getValue]()++ (link:{java9-javadoc}/java/time/Year.html#getValue%2D%2D[java 9]) -* ++[[painless-api-reference-Year-isAfter-1]]boolean link:{java8-javadoc}/java/time/Year.html#isAfter%2Djava.time.Year%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/Year.html#isAfter%2Djava.time.Year%2D[java 9]) -* ++[[painless-api-reference-Year-isLeap-0]]boolean link:{java8-javadoc}/java/time/Year.html#isLeap%2D%2D[isLeap]()++ (link:{java9-javadoc}/java/time/Year.html#isLeap%2D%2D[java 9]) -* ++[[painless-api-reference-Year-isValidMonthDay-1]]boolean link:{java8-javadoc}/java/time/Year.html#isValidMonthDay%2Djava.time.MonthDay%2D[isValidMonthDay](<>)++ (link:{java9-javadoc}/java/time/Year.html#isValidMonthDay%2Djava.time.MonthDay%2D[java 9]) -* ++[[painless-api-reference-Year-length-0]]int link:{java8-javadoc}/java/time/Year.html#length%2D%2D[length]()++ (link:{java9-javadoc}/java/time/Year.html#length%2D%2D[java 9]) -* ++[[painless-api-reference-Year-minus-1]]<> link:{java8-javadoc}/java/time/Year.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/Year.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Year-minus-2]]<> link:{java8-javadoc}/java/time/Year.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/Year.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Year-minusYears-1]]<> link:{java8-javadoc}/java/time/Year.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/Year.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Year-plus-1]]<> link:{java8-javadoc}/java/time/Year.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/Year.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-Year-plus-2]]<> link:{java8-javadoc}/java/time/Year.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/Year.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-Year-plusYears-1]]<> link:{java8-javadoc}/java/time/Year.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/Year.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-Year-with-1]]<> link:{java8-javadoc}/java/time/Year.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/Year.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-Year-with-2]]<> link:{java8-javadoc}/java/time/Year.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/Year.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/YearMonth.asciidoc b/docs/painless/painless-api-reference/YearMonth.asciidoc deleted file mode 100644 index 5573b16c0640c..0000000000000 --- a/docs/painless/painless-api-reference/YearMonth.asciidoc +++ /dev/null @@ -1,36 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-YearMonth]]++YearMonth++:: -* ++[[painless-api-reference-YearMonth-from-1]]static <> link:{java8-javadoc}/java/time/YearMonth.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-YearMonth-of-2]]static <> link:{java8-javadoc}/java/time/YearMonth.html#of%2Dint%2Dint%2D[of](int, int)++ (link:{java9-javadoc}/java/time/YearMonth.html#of%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-YearMonth-parse-1]]static <> link:{java8-javadoc}/java/time/YearMonth.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-YearMonth-parse-2]]static <> link:{java8-javadoc}/java/time/YearMonth.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/YearMonth.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-YearMonth-atDay-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#atDay%2Dint%2D[atDay](int)++ (link:{java9-javadoc}/java/time/YearMonth.html#atDay%2Dint%2D[java 9]) -* ++[[painless-api-reference-YearMonth-atEndOfMonth-0]]<> link:{java8-javadoc}/java/time/YearMonth.html#atEndOfMonth%2D%2D[atEndOfMonth]()++ (link:{java9-javadoc}/java/time/YearMonth.html#atEndOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-compareTo-1]]int link:{java8-javadoc}/java/time/YearMonth.html#compareTo%2Djava.time.YearMonth%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#compareTo%2Djava.time.YearMonth%2D[java 9]) -* ++[[painless-api-reference-YearMonth-format-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#format%2Djava.time.format.DateTimeFormatter%2D[format](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#format%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-YearMonth-getMonth-0]]<> link:{java8-javadoc}/java/time/YearMonth.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/YearMonth.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-getMonthValue-0]]int link:{java8-javadoc}/java/time/YearMonth.html#getMonthValue%2D%2D[getMonthValue]()++ (link:{java9-javadoc}/java/time/YearMonth.html#getMonthValue%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-getYear-0]]int link:{java8-javadoc}/java/time/YearMonth.html#getYear%2D%2D[getYear]()++ (link:{java9-javadoc}/java/time/YearMonth.html#getYear%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-isAfter-1]]boolean link:{java8-javadoc}/java/time/YearMonth.html#isAfter%2Djava.time.YearMonth%2D[isAfter](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#isAfter%2Djava.time.YearMonth%2D[java 9]) -* ++[[painless-api-reference-YearMonth-isBefore-1]]boolean link:{java8-javadoc}/java/time/YearMonth.html#isBefore%2Djava.time.YearMonth%2D[isBefore](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#isBefore%2Djava.time.YearMonth%2D[java 9]) -* ++[[painless-api-reference-YearMonth-isLeapYear-0]]boolean link:{java8-javadoc}/java/time/YearMonth.html#isLeapYear%2D%2D[isLeapYear]()++ (link:{java9-javadoc}/java/time/YearMonth.html#isLeapYear%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-isValidDay-1]]boolean link:{java8-javadoc}/java/time/YearMonth.html#isValidDay%2Dint%2D[isValidDay](int)++ (link:{java9-javadoc}/java/time/YearMonth.html#isValidDay%2Dint%2D[java 9]) -* ++[[painless-api-reference-YearMonth-lengthOfMonth-0]]int link:{java8-javadoc}/java/time/YearMonth.html#lengthOfMonth%2D%2D[lengthOfMonth]()++ (link:{java9-javadoc}/java/time/YearMonth.html#lengthOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-lengthOfYear-0]]int link:{java8-javadoc}/java/time/YearMonth.html#lengthOfYear%2D%2D[lengthOfYear]()++ (link:{java9-javadoc}/java/time/YearMonth.html#lengthOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-YearMonth-minus-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-YearMonth-minus-2]]<> link:{java8-javadoc}/java/time/YearMonth.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/YearMonth.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-YearMonth-minusMonths-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#minusMonths%2Dlong%2D[minusMonths](long)++ (link:{java9-javadoc}/java/time/YearMonth.html#minusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-YearMonth-minusYears-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/YearMonth.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-YearMonth-plus-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-YearMonth-plus-2]]<> link:{java8-javadoc}/java/time/YearMonth.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/YearMonth.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-YearMonth-plusMonths-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#plusMonths%2Dlong%2D[plusMonths](long)++ (link:{java9-javadoc}/java/time/YearMonth.html#plusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-YearMonth-plusYears-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/YearMonth.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-YearMonth-with-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/YearMonth.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-YearMonth-with-2]]<> link:{java8-javadoc}/java/time/YearMonth.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/YearMonth.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-YearMonth-withMonth-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#withMonth%2Dint%2D[withMonth](int)++ (link:{java9-javadoc}/java/time/YearMonth.html#withMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-YearMonth-withYear-1]]<> link:{java8-javadoc}/java/time/YearMonth.html#withYear%2Dint%2D[withYear](int)++ (link:{java9-javadoc}/java/time/YearMonth.html#withYear%2Dint%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneId.asciidoc b/docs/painless/painless-api-reference/ZoneId.asciidoc deleted file mode 100644 index fe31c9df00bb1..0000000000000 --- a/docs/painless/painless-api-reference/ZoneId.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneId]]++ZoneId++:: -** [[painless-api-reference-ZoneId-SHORT_IDS]]static <> link:{java8-javadoc}/java/time/ZoneId.html#SHORT_IDS[SHORT_IDS] (link:{java9-javadoc}/java/time/ZoneId.html#SHORT_IDS[java 9]) -* ++[[painless-api-reference-ZoneId-from-1]]static <> link:{java8-javadoc}/java/time/ZoneId.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/ZoneId.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ZoneId-getAvailableZoneIds-0]]static <> link:{java8-javadoc}/java/time/ZoneId.html#getAvailableZoneIds%2D%2D[getAvailableZoneIds]()++ (link:{java9-javadoc}/java/time/ZoneId.html#getAvailableZoneIds%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneId-of-1]]static <> link:{java8-javadoc}/java/time/ZoneId.html#of%2Djava.lang.String%2D[of](<>)++ (link:{java9-javadoc}/java/time/ZoneId.html#of%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ZoneId-of-2]]static <> link:{java8-javadoc}/java/time/ZoneId.html#of%2Djava.lang.String%2Djava.util.Map%2D[of](<>, <>)++ (link:{java9-javadoc}/java/time/ZoneId.html#of%2Djava.lang.String%2Djava.util.Map%2D[java 9]) -* ++[[painless-api-reference-ZoneId-ofOffset-2]]static <> link:{java8-javadoc}/java/time/ZoneId.html#ofOffset%2Djava.lang.String%2Djava.time.ZoneOffset%2D[ofOffset](<>, <>)++ (link:{java9-javadoc}/java/time/ZoneId.html#ofOffset%2Djava.lang.String%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZoneId-systemDefault-0]]static <> link:{java8-javadoc}/java/time/ZoneId.html#systemDefault%2D%2D[systemDefault]()++ (link:{java9-javadoc}/java/time/ZoneId.html#systemDefault%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneId-getDisplayName-2]]<> link:{java8-javadoc}/java/time/ZoneId.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[getDisplayName](<>, <>)++ (link:{java9-javadoc}/java/time/ZoneId.html#getDisplayName%2Djava.time.format.TextStyle%2Djava.util.Locale%2D[java 9]) -* ++[[painless-api-reference-ZoneId-getId-0]]<> link:{java8-javadoc}/java/time/ZoneId.html#getId%2D%2D[getId]()++ (link:{java9-javadoc}/java/time/ZoneId.html#getId%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneId-getRules-0]]<> link:{java8-javadoc}/java/time/ZoneId.html#getRules%2D%2D[getRules]()++ (link:{java9-javadoc}/java/time/ZoneId.html#getRules%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneId-normalized-0]]<> link:{java8-javadoc}/java/time/ZoneId.html#normalized%2D%2D[normalized]()++ (link:{java9-javadoc}/java/time/ZoneId.html#normalized%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneOffset.asciidoc b/docs/painless/painless-api-reference/ZoneOffset.asciidoc deleted file mode 100644 index 84e8530f5938b..0000000000000 --- a/docs/painless/painless-api-reference/ZoneOffset.asciidoc +++ /dev/null @@ -1,17 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneOffset]]++ZoneOffset++:: -** [[painless-api-reference-ZoneOffset-MAX]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#MAX[MAX] (link:{java9-javadoc}/java/time/ZoneOffset.html#MAX[java 9]) -** [[painless-api-reference-ZoneOffset-MIN]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#MIN[MIN] (link:{java9-javadoc}/java/time/ZoneOffset.html#MIN[java 9]) -** [[painless-api-reference-ZoneOffset-UTC]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#UTC[UTC] (link:{java9-javadoc}/java/time/ZoneOffset.html#UTC[java 9]) -* ++[[painless-api-reference-ZoneOffset-from-1]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/ZoneOffset.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ZoneOffset-of-1]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#of%2Djava.lang.String%2D[of](<>)++ (link:{java9-javadoc}/java/time/ZoneOffset.html#of%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ZoneOffset-ofHours-1]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#ofHours%2Dint%2D[ofHours](int)++ (link:{java9-javadoc}/java/time/ZoneOffset.html#ofHours%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZoneOffset-ofHoursMinutes-2]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#ofHoursMinutes%2Dint%2Dint%2D[ofHoursMinutes](int, int)++ (link:{java9-javadoc}/java/time/ZoneOffset.html#ofHoursMinutes%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZoneOffset-ofHoursMinutesSeconds-3]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#ofHoursMinutesSeconds%2Dint%2Dint%2Dint%2D[ofHoursMinutesSeconds](int, int, int)++ (link:{java9-javadoc}/java/time/ZoneOffset.html#ofHoursMinutesSeconds%2Dint%2Dint%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZoneOffset-ofTotalSeconds-1]]static <> link:{java8-javadoc}/java/time/ZoneOffset.html#ofTotalSeconds%2Dint%2D[ofTotalSeconds](int)++ (link:{java9-javadoc}/java/time/ZoneOffset.html#ofTotalSeconds%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZoneOffset-getTotalSeconds-0]]int link:{java8-javadoc}/java/time/ZoneOffset.html#getTotalSeconds%2D%2D[getTotalSeconds]()++ (link:{java9-javadoc}/java/time/ZoneOffset.html#getTotalSeconds%2D%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneOffsetTransition.asciidoc b/docs/painless/painless-api-reference/ZoneOffsetTransition.asciidoc deleted file mode 100644 index 43be4f56ee4b3..0000000000000 --- a/docs/painless/painless-api-reference/ZoneOffsetTransition.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneOffsetTransition]]++ZoneOffsetTransition++:: -* ++[[painless-api-reference-ZoneOffsetTransition-of-3]]static <> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#of%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2D[of](<>, <>, <>)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#of%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-compareTo-1]]int link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#compareTo%2Djava.time.zone.ZoneOffsetTransition%2D[compareTo](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#compareTo%2Djava.time.zone.ZoneOffsetTransition%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-getDateTimeAfter-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#getDateTimeAfter%2D%2D[getDateTimeAfter]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#getDateTimeAfter%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-getDateTimeBefore-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#getDateTimeBefore%2D%2D[getDateTimeBefore]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#getDateTimeBefore%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-getDuration-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#getDuration%2D%2D[getDuration]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#getDuration%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-getInstant-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#getInstant%2D%2D[getInstant]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#getInstant%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-getOffsetAfter-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#getOffsetAfter%2D%2D[getOffsetAfter]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#getOffsetAfter%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-getOffsetBefore-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#getOffsetBefore%2D%2D[getOffsetBefore]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#getOffsetBefore%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-isGap-0]]boolean link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#isGap%2D%2D[isGap]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#isGap%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-isOverlap-0]]boolean link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#isOverlap%2D%2D[isOverlap]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#isOverlap%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-isValidOffset-1]]boolean link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#isValidOffset%2Djava.time.ZoneOffset%2D[isValidOffset](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#isValidOffset%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransition-toEpochSecond-0]]long link:{java8-javadoc}/java/time/zone/ZoneOffsetTransition.html#toEpochSecond%2D%2D[toEpochSecond]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransition.html#toEpochSecond%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneOffsetTransitionRule.TimeDefinition.asciidoc b/docs/painless/painless-api-reference/ZoneOffsetTransitionRule.TimeDefinition.asciidoc deleted file mode 100644 index 045f2a0188948..0000000000000 --- a/docs/painless/painless-api-reference/ZoneOffsetTransitionRule.TimeDefinition.asciidoc +++ /dev/null @@ -1,13 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition]]++ZoneOffsetTransitionRule.TimeDefinition++:: -** [[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition-STANDARD]]static <> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#STANDARD[STANDARD] (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#STANDARD[java 9]) -** [[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition-UTC]]static <> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#UTC[UTC] (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#UTC[java 9]) -** [[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition-WALL]]static <> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#WALL[WALL] (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#WALL[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition-valueOf-1]]static <> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#valueOf%2Djava.lang.String%2D[valueOf](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#valueOf%2Djava.lang.String%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition-values-0]]static <>[] link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#values%2D%2D[values]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#values%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-TimeDefinition-createDateTime-3]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#createDateTime%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2D[createDateTime](<>, <>, <>)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.TimeDefinition.html#createDateTime%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneOffsetTransitionRule.asciidoc b/docs/painless/painless-api-reference/ZoneOffsetTransitionRule.asciidoc deleted file mode 100644 index 8e84bff01722b..0000000000000 --- a/docs/painless/painless-api-reference/ZoneOffsetTransitionRule.asciidoc +++ /dev/null @@ -1,18 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneOffsetTransitionRule]]++ZoneOffsetTransitionRule++:: -* ++[[painless-api-reference-ZoneOffsetTransitionRule-of-9]]static <> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#of%2Djava.time.Month%2Dint%2Djava.time.DayOfWeek%2Djava.time.LocalTime%2Dboolean%2Djava.time.zone.ZoneOffsetTransitionRule$TimeDefinition%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2D[of](<>, int, <>, <>, boolean, <>, <>, <>, <>)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#of%2Djava.time.Month%2Dint%2Djava.time.DayOfWeek%2Djava.time.LocalTime%2Dboolean%2Djava.time.zone.ZoneOffsetTransitionRule$TimeDefinition%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-createTransition-1]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#createTransition%2Dint%2D[createTransition](int)++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#createTransition%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getDayOfMonthIndicator-0]]int link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getDayOfMonthIndicator%2D%2D[getDayOfMonthIndicator]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getDayOfMonthIndicator%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getDayOfWeek-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getDayOfWeek%2D%2D[getDayOfWeek]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getLocalTime-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getLocalTime%2D%2D[getLocalTime]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getLocalTime%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getMonth-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getOffsetAfter-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getOffsetAfter%2D%2D[getOffsetAfter]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getOffsetAfter%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getOffsetBefore-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getOffsetBefore%2D%2D[getOffsetBefore]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getOffsetBefore%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getStandardOffset-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getStandardOffset%2D%2D[getStandardOffset]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getStandardOffset%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-getTimeDefinition-0]]<> link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getTimeDefinition%2D%2D[getTimeDefinition]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#getTimeDefinition%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneOffsetTransitionRule-isMidnightEndOfDay-0]]boolean link:{java8-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#isMidnightEndOfDay%2D%2D[isMidnightEndOfDay]()++ (link:{java9-javadoc}/java/time/zone/ZoneOffsetTransitionRule.html#isMidnightEndOfDay%2D%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneRules.asciidoc b/docs/painless/painless-api-reference/ZoneRules.asciidoc deleted file mode 100644 index 6c3104a7e02ab..0000000000000 --- a/docs/painless/painless-api-reference/ZoneRules.asciidoc +++ /dev/null @@ -1,21 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneRules]]++ZoneRules++:: -* ++[[painless-api-reference-ZoneRules-of-1]]static <> link:{java8-javadoc}/java/time/zone/ZoneRules.html#of%2Djava.time.ZoneOffset%2D[of](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#of%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-of-5]]static <> link:{java8-javadoc}/java/time/zone/ZoneRules.html#of%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2Djava.util.List%2Djava.util.List%2Djava.util.List%2D[of](<>, <>, <>, <>, <>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#of%2Djava.time.ZoneOffset%2Djava.time.ZoneOffset%2Djava.util.List%2Djava.util.List%2Djava.util.List%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getDaylightSavings-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getDaylightSavings%2Djava.time.Instant%2D[getDaylightSavings](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getDaylightSavings%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getOffset-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getOffset%2Djava.time.Instant%2D[getOffset](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getOffset%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getStandardOffset-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getStandardOffset%2Djava.time.Instant%2D[getStandardOffset](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getStandardOffset%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getTransition-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getTransition%2Djava.time.LocalDateTime%2D[getTransition](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getTransition%2Djava.time.LocalDateTime%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getTransitionRules-0]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getTransitionRules%2D%2D[getTransitionRules]()++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getTransitionRules%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getTransitions-0]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getTransitions%2D%2D[getTransitions]()++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getTransitions%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-getValidOffsets-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#getValidOffsets%2Djava.time.LocalDateTime%2D[getValidOffsets](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#getValidOffsets%2Djava.time.LocalDateTime%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-isDaylightSavings-1]]boolean link:{java8-javadoc}/java/time/zone/ZoneRules.html#isDaylightSavings%2Djava.time.Instant%2D[isDaylightSavings](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#isDaylightSavings%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-isFixedOffset-0]]boolean link:{java8-javadoc}/java/time/zone/ZoneRules.html#isFixedOffset%2D%2D[isFixedOffset]()++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#isFixedOffset%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-isValidOffset-2]]boolean link:{java8-javadoc}/java/time/zone/ZoneRules.html#isValidOffset%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2D[isValidOffset](<>, <>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#isValidOffset%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-nextTransition-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#nextTransition%2Djava.time.Instant%2D[nextTransition](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#nextTransition%2Djava.time.Instant%2D[java 9]) -* ++[[painless-api-reference-ZoneRules-previousTransition-1]]<> link:{java8-javadoc}/java/time/zone/ZoneRules.html#previousTransition%2Djava.time.Instant%2D[previousTransition](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRules.html#previousTransition%2Djava.time.Instant%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneRulesException.asciidoc b/docs/painless/painless-api-reference/ZoneRulesException.asciidoc deleted file mode 100644 index 8bb3ef0cc98b0..0000000000000 --- a/docs/painless/painless-api-reference/ZoneRulesException.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneRulesException]]++ZoneRulesException++:: -* ++[[painless-api-reference-ZoneRulesException-ZoneRulesException-1]]link:{java8-javadoc}/java/time/zone/ZoneRulesException.html#ZoneRulesException%2Djava.lang.String%2D[ZoneRulesException](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRulesException.html#ZoneRulesException%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/ZoneRulesProvider.asciidoc b/docs/painless/painless-api-reference/ZoneRulesProvider.asciidoc deleted file mode 100644 index be33f691f370e..0000000000000 --- a/docs/painless/painless-api-reference/ZoneRulesProvider.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZoneRulesProvider]]++ZoneRulesProvider++:: -* ++[[painless-api-reference-ZoneRulesProvider-getAvailableZoneIds-0]]static <> link:{java8-javadoc}/java/time/zone/ZoneRulesProvider.html#getAvailableZoneIds%2D%2D[getAvailableZoneIds]()++ (link:{java9-javadoc}/java/time/zone/ZoneRulesProvider.html#getAvailableZoneIds%2D%2D[java 9]) -* ++[[painless-api-reference-ZoneRulesProvider-getRules-2]]static <> link:{java8-javadoc}/java/time/zone/ZoneRulesProvider.html#getRules%2Djava.lang.String%2Dboolean%2D[getRules](<>, boolean)++ (link:{java9-javadoc}/java/time/zone/ZoneRulesProvider.html#getRules%2Djava.lang.String%2Dboolean%2D[java 9]) -* ++[[painless-api-reference-ZoneRulesProvider-getVersions-1]]static <> link:{java8-javadoc}/java/time/zone/ZoneRulesProvider.html#getVersions%2Djava.lang.String%2D[getVersions](<>)++ (link:{java9-javadoc}/java/time/zone/ZoneRulesProvider.html#getVersions%2Djava.lang.String%2D[java 9]) -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/ZonedDateTime.asciidoc b/docs/painless/painless-api-reference/ZonedDateTime.asciidoc deleted file mode 100644 index e8ba7501c6c20..0000000000000 --- a/docs/painless/painless-api-reference/ZonedDateTime.asciidoc +++ /dev/null @@ -1,66 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-ZonedDateTime]]++ZonedDateTime++:: -* ++[[painless-api-reference-ZonedDateTime-from-1]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[from](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#from%2Djava.time.temporal.TemporalAccessor%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-of-2]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#of%2Djava.time.LocalDateTime%2Djava.time.ZoneId%2D[of](<>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#of%2Djava.time.LocalDateTime%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-of-3]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#of%2Djava.time.LocalDate%2Djava.time.LocalTime%2Djava.time.ZoneId%2D[of](<>, <>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#of%2Djava.time.LocalDate%2Djava.time.LocalTime%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-of-8]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Djava.time.ZoneId%2D[of](int, int, int, int, int, int, int, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#of%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Dint%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-ofInstant-2]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[ofInstant](<>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#ofInstant%2Djava.time.Instant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-ofInstant-3]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#ofInstant%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneId%2D[ofInstant](<>, <>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#ofInstant%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-ofLocal-3]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#ofLocal%2Djava.time.LocalDateTime%2Djava.time.ZoneId%2Djava.time.ZoneOffset%2D[ofLocal](<>, <>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#ofLocal%2Djava.time.LocalDateTime%2Djava.time.ZoneId%2Djava.time.ZoneOffset%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-ofStrict-3]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#ofStrict%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneId%2D[ofStrict](<>, <>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#ofStrict%2Djava.time.LocalDateTime%2Djava.time.ZoneOffset%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-parse-1]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#parse%2Djava.lang.CharSequence%2D[parse](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#parse%2Djava.lang.CharSequence%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-parse-2]]static <> link:{java8-javadoc}/java/time/ZonedDateTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[parse](<>, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#parse%2Djava.lang.CharSequence%2Djava.time.format.DateTimeFormatter%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getDayOfMonth-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getDayOfMonth%2D%2D[getDayOfMonth]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getDayOfMonth%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getDayOfWeek-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#getDayOfWeek%2D%2D[getDayOfWeek]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getDayOfWeek%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getDayOfYear-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getDayOfYear%2D%2D[getDayOfYear]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getDayOfYear%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getHour-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getHour%2D%2D[getHour]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getHour%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getMinute-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getMinute%2D%2D[getMinute]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getMinute%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getMonth-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#getMonth%2D%2D[getMonth]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getMonth%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getMonthValue-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getMonthValue%2D%2D[getMonthValue]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getMonthValue%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getNano-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getNano%2D%2D[getNano]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getNano%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getSecond-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getSecond%2D%2D[getSecond]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getSecond%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-getYear-0]]int link:{java8-javadoc}/java/time/ZonedDateTime.html#getYear%2D%2D[getYear]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#getYear%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minus-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[minus](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minus-2]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[minus](long, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusDays-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusDays%2Dlong%2D[minusDays](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusHours-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusHours%2Dlong%2D[minusHours](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusMinutes-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusMinutes%2Dlong%2D[minusMinutes](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusMonths-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusMonths%2Dlong%2D[minusMonths](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusNanos-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusNanos%2Dlong%2D[minusNanos](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusSeconds-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusSeconds%2Dlong%2D[minusSeconds](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusWeeks-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusWeeks%2Dlong%2D[minusWeeks](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-minusYears-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#minusYears%2Dlong%2D[minusYears](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#minusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plus-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[plus](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plus%2Djava.time.temporal.TemporalAmount%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plus-2]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[plus](long, <>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plus%2Dlong%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusDays-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusDays%2Dlong%2D[plusDays](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusDays%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusHours-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusHours%2Dlong%2D[plusHours](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusHours%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusMinutes-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusMinutes%2Dlong%2D[plusMinutes](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusMinutes%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusMonths-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusMonths%2Dlong%2D[plusMonths](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusMonths%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusNanos-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusNanos%2Dlong%2D[plusNanos](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusNanos%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusSeconds-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusSeconds%2Dlong%2D[plusSeconds](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusSeconds%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusWeeks-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusWeeks%2Dlong%2D[plusWeeks](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusWeeks%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-plusYears-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#plusYears%2Dlong%2D[plusYears](long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#plusYears%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-toLocalDate-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#toLocalDate%2D%2D[toLocalDate]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#toLocalDate%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-toLocalDateTime-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#toLocalDateTime%2D%2D[toLocalDateTime]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#toLocalDateTime%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-toOffsetDateTime-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#toOffsetDateTime%2D%2D[toOffsetDateTime]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#toOffsetDateTime%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-truncatedTo-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[truncatedTo](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#truncatedTo%2Djava.time.temporal.TemporalUnit%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-with-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[with](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#with%2Djava.time.temporal.TemporalAdjuster%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-with-2]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[with](<>, long)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#with%2Djava.time.temporal.TemporalField%2Dlong%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withDayOfMonth-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withDayOfMonth%2Dint%2D[withDayOfMonth](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withDayOfMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withDayOfYear-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withDayOfYear%2Dint%2D[withDayOfYear](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withDayOfYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withEarlierOffsetAtOverlap-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withEarlierOffsetAtOverlap%2D%2D[withEarlierOffsetAtOverlap]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withEarlierOffsetAtOverlap%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withFixedOffsetZone-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withFixedOffsetZone%2D%2D[withFixedOffsetZone]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withFixedOffsetZone%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withHour-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withHour%2Dint%2D[withHour](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withHour%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withLaterOffsetAtOverlap-0]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withLaterOffsetAtOverlap%2D%2D[withLaterOffsetAtOverlap]()++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withLaterOffsetAtOverlap%2D%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withMinute-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withMinute%2Dint%2D[withMinute](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withMinute%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withMonth-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withMonth%2Dint%2D[withMonth](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withMonth%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withNano-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withNano%2Dint%2D[withNano](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withNano%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withSecond-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withSecond%2Dint%2D[withSecond](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withSecond%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withYear-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withYear%2Dint%2D[withYear](int)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withYear%2Dint%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withZoneSameInstant-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withZoneSameInstant%2Djava.time.ZoneId%2D[withZoneSameInstant](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withZoneSameInstant%2Djava.time.ZoneId%2D[java 9]) -* ++[[painless-api-reference-ZonedDateTime-withZoneSameLocal-1]]<> link:{java8-javadoc}/java/time/ZonedDateTime.html#withZoneSameLocal%2Djava.time.ZoneId%2D[withZoneSameLocal](<>)++ (link:{java9-javadoc}/java/time/ZonedDateTime.html#withZoneSameLocal%2Djava.time.ZoneId%2D[java 9]) -* Inherits methods from ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/index.asciidoc b/docs/painless/painless-api-reference/index.asciidoc index d5deb8900caec..88130f7fdfc02 100644 --- a/docs/painless/painless-api-reference/index.asciidoc +++ b/docs/painless/painless-api-reference/index.asciidoc @@ -1,344 +1,38 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// +// This file is auto-generated. Do not edit. -include::AbstractChronology.asciidoc[] -include::AbstractCollection.asciidoc[] -include::AbstractList.asciidoc[] -include::AbstractMap.asciidoc[] -include::AbstractMap.SimpleEntry.asciidoc[] -include::AbstractMap.SimpleImmutableEntry.asciidoc[] -include::AbstractQueue.asciidoc[] -include::AbstractSequentialList.asciidoc[] -include::AbstractSet.asciidoc[] -include::Annotation.asciidoc[] -include::Appendable.asciidoc[] -include::ArithmeticException.asciidoc[] -include::ArrayDeque.asciidoc[] -include::ArrayIndexOutOfBoundsException.asciidoc[] -include::ArrayList.asciidoc[] -include::ArrayStoreException.asciidoc[] -include::Arrays.asciidoc[] -include::AttributedCharacterIterator.asciidoc[] -include::AttributedCharacterIterator.Attribute.asciidoc[] -include::AttributedString.asciidoc[] -include::Base64.asciidoc[] -include::Base64.Decoder.asciidoc[] -include::Base64.Encoder.asciidoc[] -include::BaseStream.asciidoc[] -include::BiConsumer.asciidoc[] -include::BiFunction.asciidoc[] -include::BiPredicate.asciidoc[] -include::Bidi.asciidoc[] -include::BigDecimal.asciidoc[] -include::BigInteger.asciidoc[] -include::BinaryOperator.asciidoc[] -include::BitSet.asciidoc[] -include::Boolean.asciidoc[] -include::BooleanSupplier.asciidoc[] -include::BreakIterator.asciidoc[] -include::Byte.asciidoc[] -include::BytesRef.asciidoc[] -include::Calendar.asciidoc[] -include::Calendar.Builder.asciidoc[] -include::CharSequence.asciidoc[] -include::Character.asciidoc[] -include::Character.Subset.asciidoc[] -include::Character.UnicodeBlock.asciidoc[] -include::Character.UnicodeScript.asciidoc[] -include::CharacterIterator.asciidoc[] -include::ChoiceFormat.asciidoc[] -include::ChronoField.asciidoc[] -include::ChronoLocalDate.asciidoc[] -include::ChronoLocalDateTime.asciidoc[] -include::ChronoPeriod.asciidoc[] -include::ChronoUnit.asciidoc[] -include::ChronoZonedDateTime.asciidoc[] -include::Chronology.asciidoc[] -include::ClassCastException.asciidoc[] -include::ClassNotFoundException.asciidoc[] -include::Clock.asciidoc[] -include::CloneNotSupportedException.asciidoc[] -include::CollationElementIterator.asciidoc[] -include::CollationKey.asciidoc[] -include::Collator.asciidoc[] -include::Collection.asciidoc[] -include::Collections.asciidoc[] -include::Collector.asciidoc[] -include::Collector.Characteristics.asciidoc[] -include::Collectors.asciidoc[] -include::Comparable.asciidoc[] -include::Comparator.asciidoc[] -include::ConcurrentModificationException.asciidoc[] -include::Consumer.asciidoc[] -include::Currency.asciidoc[] -include::Date.asciidoc[] -include::DateFormat.asciidoc[] -include::DateFormat.Field.asciidoc[] -include::DateFormatSymbols.asciidoc[] -include::DateTimeException.asciidoc[] -include::DateTimeFormatter.asciidoc[] -include::DateTimeFormatterBuilder.asciidoc[] -include::DateTimeParseException.asciidoc[] -include::DayOfWeek.asciidoc[] -include::Debug.asciidoc[] -include::DecimalFormat.asciidoc[] -include::DecimalFormatSymbols.asciidoc[] -include::DecimalStyle.asciidoc[] -include::Deque.asciidoc[] -include::Dictionary.asciidoc[] -include::Double.asciidoc[] -include::DoubleBinaryOperator.asciidoc[] -include::DoubleConsumer.asciidoc[] -include::DoubleFunction.asciidoc[] -include::DoublePredicate.asciidoc[] -include::DoubleStream.asciidoc[] -include::DoubleStream.Builder.asciidoc[] -include::DoubleSummaryStatistics.asciidoc[] -include::DoubleSupplier.asciidoc[] -include::DoubleToIntFunction.asciidoc[] -include::DoubleToLongFunction.asciidoc[] -include::DoubleUnaryOperator.asciidoc[] -include::DuplicateFormatFlagsException.asciidoc[] -include::Duration.asciidoc[] -include::EmptyStackException.asciidoc[] -include::Enum.asciidoc[] -include::EnumConstantNotPresentException.asciidoc[] -include::Enumeration.asciidoc[] -include::Era.asciidoc[] -include::EventListener.asciidoc[] -include::EventListenerProxy.asciidoc[] -include::EventObject.asciidoc[] -include::Exception.asciidoc[] -include::FieldPosition.asciidoc[] -include::Float.asciidoc[] -include::Format.asciidoc[] -include::Format.Field.asciidoc[] -include::FormatFlagsConversionMismatchException.asciidoc[] -include::FormatStyle.asciidoc[] -include::Formattable.asciidoc[] -include::FormattableFlags.asciidoc[] -include::Formatter.asciidoc[] -include::Formatter.BigDecimalLayoutForm.asciidoc[] -include::FormatterClosedException.asciidoc[] -include::Function.asciidoc[] -include::GregorianCalendar.asciidoc[] -include::HashMap.asciidoc[] -include::HashSet.asciidoc[] -include::Hashtable.asciidoc[] -include::HijrahChronology.asciidoc[] -include::HijrahDate.asciidoc[] -include::HijrahEra.asciidoc[] -include::IdentityHashMap.asciidoc[] -include::IllegalAccessException.asciidoc[] -include::IllegalArgumentException.asciidoc[] -include::IllegalFormatCodePointException.asciidoc[] -include::IllegalFormatConversionException.asciidoc[] -include::IllegalFormatException.asciidoc[] -include::IllegalFormatFlagsException.asciidoc[] -include::IllegalFormatPrecisionException.asciidoc[] -include::IllegalFormatWidthException.asciidoc[] -include::IllegalMonitorStateException.asciidoc[] -include::IllegalStateException.asciidoc[] -include::IllegalThreadStateException.asciidoc[] -include::IllformedLocaleException.asciidoc[] -include::IndexOutOfBoundsException.asciidoc[] -include::InputMismatchException.asciidoc[] -include::Instant.asciidoc[] -include::InstantiationException.asciidoc[] -include::IntBinaryOperator.asciidoc[] -include::IntConsumer.asciidoc[] -include::IntFunction.asciidoc[] -include::IntPredicate.asciidoc[] -include::IntStream.asciidoc[] -include::IntStream.Builder.asciidoc[] -include::IntSummaryStatistics.asciidoc[] -include::IntSupplier.asciidoc[] -include::IntToDoubleFunction.asciidoc[] -include::IntToLongFunction.asciidoc[] -include::IntUnaryOperator.asciidoc[] -include::Integer.asciidoc[] -include::InterruptedException.asciidoc[] -include::IsoChronology.asciidoc[] -include::IsoEra.asciidoc[] -include::IsoFields.asciidoc[] -include::Iterable.asciidoc[] -include::Iterator.asciidoc[] -include::JapaneseChronology.asciidoc[] -include::JapaneseDate.asciidoc[] -include::JapaneseEra.asciidoc[] -include::JulianFields.asciidoc[] -include::LinkedHashMap.asciidoc[] -include::LinkedHashSet.asciidoc[] -include::LinkedList.asciidoc[] -include::List.asciidoc[] -include::ListIterator.asciidoc[] -include::LocalDate.asciidoc[] -include::LocalDateTime.asciidoc[] -include::LocalTime.asciidoc[] -include::Locale.asciidoc[] -include::Locale.Builder.asciidoc[] -include::Locale.Category.asciidoc[] -include::Locale.FilteringMode.asciidoc[] -include::Locale.LanguageRange.asciidoc[] -include::Long.asciidoc[] -include::LongBinaryOperator.asciidoc[] -include::LongConsumer.asciidoc[] -include::LongFunction.asciidoc[] -include::LongPredicate.asciidoc[] -include::LongStream.asciidoc[] -include::LongStream.Builder.asciidoc[] -include::LongSummaryStatistics.asciidoc[] -include::LongSupplier.asciidoc[] -include::LongToDoubleFunction.asciidoc[] -include::LongToIntFunction.asciidoc[] -include::LongUnaryOperator.asciidoc[] -include::Map.asciidoc[] -include::Map.Entry.asciidoc[] -include::Matcher.asciidoc[] -include::Math.asciidoc[] -include::MathContext.asciidoc[] -include::MessageFormat.asciidoc[] -include::MessageFormat.Field.asciidoc[] -include::MinguoChronology.asciidoc[] -include::MinguoDate.asciidoc[] -include::MinguoEra.asciidoc[] -include::MissingFormatArgumentException.asciidoc[] -include::MissingFormatWidthException.asciidoc[] -include::MissingResourceException.asciidoc[] -include::Month.asciidoc[] -include::MonthDay.asciidoc[] -include::NavigableMap.asciidoc[] -include::NavigableSet.asciidoc[] -include::NegativeArraySizeException.asciidoc[] -include::NoSuchElementException.asciidoc[] -include::NoSuchFieldException.asciidoc[] -include::NoSuchMethodException.asciidoc[] -include::Normalizer.asciidoc[] -include::Normalizer.Form.asciidoc[] -include::NullPointerException.asciidoc[] -include::Number.asciidoc[] -include::NumberFormat.asciidoc[] -include::NumberFormat.Field.asciidoc[] -include::NumberFormatException.asciidoc[] -include::ObjDoubleConsumer.asciidoc[] -include::ObjIntConsumer.asciidoc[] -include::ObjLongConsumer.asciidoc[] -include::Object.asciidoc[] -include::Objects.asciidoc[] -include::Observable.asciidoc[] -include::Observer.asciidoc[] -include::OffsetDateTime.asciidoc[] -include::OffsetTime.asciidoc[] -include::Optional.asciidoc[] -include::OptionalDouble.asciidoc[] -include::OptionalInt.asciidoc[] -include::OptionalLong.asciidoc[] -include::ParseException.asciidoc[] -include::ParsePosition.asciidoc[] -include::Pattern.asciidoc[] -include::Period.asciidoc[] -include::Predicate.asciidoc[] -include::PrimitiveIterator.asciidoc[] -include::PrimitiveIterator.OfDouble.asciidoc[] -include::PrimitiveIterator.OfInt.asciidoc[] -include::PrimitiveIterator.OfLong.asciidoc[] -include::PriorityQueue.asciidoc[] -include::Queue.asciidoc[] -include::Random.asciidoc[] -include::RandomAccess.asciidoc[] -include::ReflectiveOperationException.asciidoc[] -include::ResolverStyle.asciidoc[] -include::RoundingMode.asciidoc[] -include::RuleBasedCollator.asciidoc[] -include::RuntimeException.asciidoc[] -include::SecurityException.asciidoc[] -include::Set.asciidoc[] -include::Short.asciidoc[] -include::SignStyle.asciidoc[] -include::SimpleDateFormat.asciidoc[] -include::SimpleTimeZone.asciidoc[] -include::SortedMap.asciidoc[] -include::SortedSet.asciidoc[] -include::Spliterator.asciidoc[] -include::Spliterator.OfDouble.asciidoc[] -include::Spliterator.OfInt.asciidoc[] -include::Spliterator.OfLong.asciidoc[] -include::Spliterator.OfPrimitive.asciidoc[] -include::Spliterators.asciidoc[] -include::Stack.asciidoc[] -include::StackTraceElement.asciidoc[] -include::Stream.asciidoc[] -include::Stream.Builder.asciidoc[] -include::StrictMath.asciidoc[] -include::String.asciidoc[] -include::StringBuffer.asciidoc[] -include::StringBuilder.asciidoc[] -include::StringCharacterIterator.asciidoc[] -include::StringIndexOutOfBoundsException.asciidoc[] -include::StringJoiner.asciidoc[] -include::StringTokenizer.asciidoc[] -include::Supplier.asciidoc[] -include::System.asciidoc[] -include::Temporal.asciidoc[] -include::TemporalAccessor.asciidoc[] -include::TemporalAdjuster.asciidoc[] -include::TemporalAdjusters.asciidoc[] -include::TemporalAmount.asciidoc[] -include::TemporalField.asciidoc[] -include::TemporalQueries.asciidoc[] -include::TemporalQuery.asciidoc[] -include::TemporalUnit.asciidoc[] -include::TextStyle.asciidoc[] -include::ThaiBuddhistChronology.asciidoc[] -include::ThaiBuddhistDate.asciidoc[] -include::ThaiBuddhistEra.asciidoc[] -include::TimeZone.asciidoc[] -include::ToDoubleBiFunction.asciidoc[] -include::ToDoubleFunction.asciidoc[] -include::ToIntBiFunction.asciidoc[] -include::ToIntFunction.asciidoc[] -include::ToLongBiFunction.asciidoc[] -include::ToLongFunction.asciidoc[] -include::TooManyListenersException.asciidoc[] -include::TreeMap.asciidoc[] -include::TreeSet.asciidoc[] -include::TypeNotPresentException.asciidoc[] -include::UUID.asciidoc[] -include::UnaryOperator.asciidoc[] -include::UnknownFormatConversionException.asciidoc[] -include::UnknownFormatFlagsException.asciidoc[] -include::UnsupportedOperationException.asciidoc[] -include::UnsupportedTemporalTypeException.asciidoc[] -include::ValueRange.asciidoc[] -include::Vector.asciidoc[] -include::WeekFields.asciidoc[] -include::Year.asciidoc[] -include::YearMonth.asciidoc[] -include::ZoneId.asciidoc[] -include::ZoneOffset.asciidoc[] -include::ZoneOffsetTransition.asciidoc[] -include::ZoneOffsetTransitionRule.asciidoc[] -include::ZoneOffsetTransitionRule.TimeDefinition.asciidoc[] -include::ZoneRules.asciidoc[] -include::ZoneRulesException.asciidoc[] -include::ZoneRulesProvider.asciidoc[] -include::ZonedDateTime.asciidoc[] -include::org.elasticsearch.common.geo.GeoPoint.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.Booleans.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.BytesRefs.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.Dates.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.Doubles.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.Longs.asciidoc[] -include::org.elasticsearch.index.fielddata.ScriptDocValues.Strings.asciidoc[] -include::org.elasticsearch.index.mapper.IpFieldMapper.IpFieldType.IpScriptDocValues.asciidoc[] -include::org.elasticsearch.index.similarity.ScriptedSimilarity.Doc.asciidoc[] -include::org.elasticsearch.index.similarity.ScriptedSimilarity.Field.asciidoc[] -include::org.elasticsearch.index.similarity.ScriptedSimilarity.Query.asciidoc[] -include::org.elasticsearch.index.similarity.ScriptedSimilarity.Term.asciidoc[] -include::org.elasticsearch.painless.FeatureTest.asciidoc[] -include::org.elasticsearch.search.lookup.FieldLookup.asciidoc[] -include::org.joda.time.ReadableDateTime.asciidoc[] -include::org.joda.time.ReadableInstant.asciidoc[] +[cols="<3,^3,^3"] +|==== +|Aggregation Selector | <> | +|Aggs | <> | +|Aggs Combine | <> | +|Aggs Init | <> | +|Aggs Map | <> | +|Aggs Reduce | <> | +|Analysis | <> | <> +|Bucket Aggregation | <> | +|Field | <> | +|Filter | <> | +|Ingest | <> | <> +|Interval | <> | +|Moving Function | <> | <> +|Number Sort | <> | +|Painless Test | <> | +|Processor Conditional | <> | +|Score | <> | <> +|Script Heuristic | <> | +|Similarity | <> | +|Similarity Weight | <> | +|String Sort | <> | +|Template | <> | +|Terms Set | <> | +|Update | <> | +|Watcher Condition | <> | +|Watcher Transform | <> | +|Xpack Template | <> | +|==== + +include::painless-api-reference-shared/index.asciidoc[] +include::painless-api-reference-analysis/index.asciidoc[] +include::painless-api-reference-ingest/index.asciidoc[] +include::painless-api-reference-moving-function/index.asciidoc[] +include::painless-api-reference-score/index.asciidoc[] diff --git a/docs/painless/painless-api-reference/org.elasticsearch.common.geo.GeoPoint.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.common.geo.GeoPoint.asciidoc deleted file mode 100644 index 2a593d0ec6314..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.common.geo.GeoPoint.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-common-geo-GeoPoint]]++org.elasticsearch.common.geo.GeoPoint++:: -* ++[[painless-api-reference-org-elasticsearch-common-geo-GeoPoint-getLat-0]]double link:{elasticsearch-javadoc}/org/elasticsearch/common/geo/GeoPoint.html#getLat%2D%2D[getLat]()++ -* ++[[painless-api-reference-org-elasticsearch-common-geo-GeoPoint-getLon-0]]double link:{elasticsearch-javadoc}/org/elasticsearch/common/geo/GeoPoint.html#getLon%2D%2D[getLon]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Booleans.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Booleans.asciidoc deleted file mode 100644 index b3e77c6fbc5cb..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Booleans.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Booleans]]++org.elasticsearch.index.fielddata.ScriptDocValues.Booleans++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Booleans-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Booleans.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Booleans-getValue-0]]boolean link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Booleans.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Booleans-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Booleans.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.BytesRefs.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.BytesRefs.asciidoc deleted file mode 100644 index 3b61905bd0ebf..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.BytesRefs.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-BytesRefs]]++org.elasticsearch.index.fielddata.ScriptDocValues.BytesRefs++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-BytesRefs-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.BytesRefs.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-BytesRefs-getValue-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.BytesRefs.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-BytesRefs-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.BytesRefs.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Dates.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Dates.asciidoc deleted file mode 100644 index 061f16b7352a9..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Dates.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Dates]]++org.elasticsearch.index.fielddata.ScriptDocValues.Dates++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Dates-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Dates.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Dates-getDate-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Dates.html#getDate%2D%2D[getDate]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Dates-getDates-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Dates.html#getDates%2D%2D[getDates]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Dates-getValue-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Dates.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Dates-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Dates.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Doubles.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Doubles.asciidoc deleted file mode 100644 index 53c43a9963514..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Doubles.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Doubles]]++org.elasticsearch.index.fielddata.ScriptDocValues.Doubles++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Doubles-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Doubles.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Doubles-getValue-0]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Doubles.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Doubles-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Doubles.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints.asciidoc deleted file mode 100644 index a345cd712cf0d..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints.asciidoc +++ /dev/null @@ -1,20 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints]]++org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-arcDistance-2]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#arcDistance%2Ddouble%2Ddouble%2D[arcDistance](double, double)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-arcDistanceWithDefault-3]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#arcDistanceWithDefault%2Ddouble%2Ddouble%2Ddouble%2D[arcDistanceWithDefault](double, double, double)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-geohashDistance-1]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#geohashDistance%2Djava.lang.String%2D[geohashDistance](<>)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-geohashDistanceWithDefault-2]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#geohashDistanceWithDefault%2Djava.lang.String%2Ddouble%2D[geohashDistanceWithDefault](<>, double)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-getLat-0]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#getLat%2D%2D[getLat]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-getLats-0]]double[] link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#getLats%2D%2D[getLats]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-getLon-0]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#getLon%2D%2D[getLon]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-getLons-0]]double[] link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#getLons%2D%2D[getLons]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-getValue-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#getValues%2D%2D[getValues]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-planeDistance-2]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#planeDistance%2Ddouble%2Ddouble%2D[planeDistance](double, double)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-GeoPoints-planeDistanceWithDefault-3]]double link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.GeoPoints.html#planeDistanceWithDefault%2Ddouble%2Ddouble%2Ddouble%2D[planeDistanceWithDefault](double, double, double)++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Longs.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Longs.asciidoc deleted file mode 100644 index f8a55344f35ee..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Longs.asciidoc +++ /dev/null @@ -1,12 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Longs]]++org.elasticsearch.index.fielddata.ScriptDocValues.Longs++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Longs-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Longs.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Longs-getDate-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Longs.html#getDate%2D%2D[getDate]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Longs-getDates-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Longs.html#getDates%2D%2D[getDates]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Longs-getValue-0]]long link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Longs.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Longs-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Longs.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Strings.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Strings.asciidoc deleted file mode 100644 index 3284662cb9008..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.fielddata.ScriptDocValues.Strings.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Strings]]++org.elasticsearch.index.fielddata.ScriptDocValues.Strings++:: -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Strings-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Strings.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Strings-getValue-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Strings.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-fielddata-ScriptDocValues-Strings-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/fielddata/ScriptDocValues.Strings.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.mapper.IpFieldMapper.IpFieldType.IpScriptDocValues.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.mapper.IpFieldMapper.IpFieldType.IpScriptDocValues.asciidoc deleted file mode 100644 index 872f355f308c8..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.mapper.IpFieldMapper.IpFieldType.IpScriptDocValues.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-mapper-IpFieldMapper-IpFieldType-IpScriptDocValues]]++org.elasticsearch.index.mapper.IpFieldMapper.IpFieldType.IpScriptDocValues++:: -* ++[[painless-api-reference-org-elasticsearch-index-mapper-IpFieldMapper-IpFieldType-IpScriptDocValues-get-1]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/mapper/IpFieldMapper.IpFieldType.IpScriptDocValues.html#get%2Dint%2D[get](int)++ -* ++[[painless-api-reference-org-elasticsearch-index-mapper-IpFieldMapper-IpFieldType-IpScriptDocValues-getValue-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/mapper/IpFieldMapper.IpFieldType.IpScriptDocValues.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-index-mapper-IpFieldMapper-IpFieldType-IpScriptDocValues-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/index/mapper/IpFieldMapper.IpFieldType.IpScriptDocValues.html#getValues%2D%2D[getValues]()++ -* Inherits methods from ++<>++, ++<>++, ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Doc.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Doc.asciidoc deleted file mode 100644 index dcddbf40368b8..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Doc.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Doc]]++org.elasticsearch.index.similarity.ScriptedSimilarity.Doc++:: -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Doc-getFreq-0]]float link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Doc.html#getFreq%2D%2D[getFreq]()++ -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Doc-getLength-0]]int link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Doc.html#getLength%2D%2D[getLength]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Field.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Field.asciidoc deleted file mode 100644 index 339bb5d619493..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Field.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Field]]++org.elasticsearch.index.similarity.ScriptedSimilarity.Field++:: -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Field-getDocCount-0]]long link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Field.html#getDocCount%2D%2D[getDocCount]()++ -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Field-getSumDocFreq-0]]long link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Field.html#getSumDocFreq%2D%2D[getSumDocFreq]()++ -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Field-getSumTotalTermFreq-0]]long link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Field.html#getSumTotalTermFreq%2D%2D[getSumTotalTermFreq]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Query.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Query.asciidoc deleted file mode 100644 index b15a8476cdabf..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Query.asciidoc +++ /dev/null @@ -1,8 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Query]]++org.elasticsearch.index.similarity.ScriptedSimilarity.Query++:: -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Query-getBoost-0]]float link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Query.html#getBoost%2D%2D[getBoost]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Term.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Term.asciidoc deleted file mode 100644 index 42ed2538f972e..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.index.similarity.ScriptedSimilarity.Term.asciidoc +++ /dev/null @@ -1,9 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Term]]++org.elasticsearch.index.similarity.ScriptedSimilarity.Term++:: -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Term-getDocFreq-0]]long link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Term.html#getDocFreq%2D%2D[getDocFreq]()++ -* ++[[painless-api-reference-org-elasticsearch-index-similarity-ScriptedSimilarity-Term-getTotalTermFreq-0]]long link:{elasticsearch-javadoc}/org/elasticsearch/index/similarity/ScriptedSimilarity.Term.html#getTotalTermFreq%2D%2D[getTotalTermFreq]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.painless.FeatureTest.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.painless.FeatureTest.asciidoc deleted file mode 100644 index 6687f0dd651bd..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.painless.FeatureTest.asciidoc +++ /dev/null @@ -1,19 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-painless-FeatureTest]]++org.elasticsearch.painless.FeatureTest++:: -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-overloadedStatic-0]]static boolean link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#overloadedStatic%2D%2D[overloadedStatic]()++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-overloadedStatic-1]]static boolean link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#overloadedStatic%2Dboolean%2D[overloadedStatic](boolean)++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-org.elasticsearch.painless.FeatureTest-0]]link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#org.elasticsearch.painless.FeatureTest%2D%2D[org.elasticsearch.painless.FeatureTest]()++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-org.elasticsearch.painless.FeatureTest-2]]link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#org.elasticsearch.painless.FeatureTest%2Dint%2Dint%2D[org.elasticsearch.painless.FeatureTest](int, int)++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-addToTotal-1]]int link:{painless-javadoc}/org/elasticsearch/painless/FeatureTestAugmentation.html#addToTotal%2Dorg.elasticsearch.painless.FeatureTest%2Dint%2D[addToTotal](int)++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-getTotal-0]]int link:{painless-javadoc}/org/elasticsearch/painless/FeatureTestAugmentation.html#getTotal%2Dorg.elasticsearch.painless.FeatureTest%2D[getTotal]()++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-getX-0]]int link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#getX%2D%2D[getX]()++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-getY-0]]int link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#getY%2D%2D[getY]()++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-listInput-1]]void link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#listInput%2Djava.util.List%2D[listInput](<>)++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-setX-1]]void link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#setX%2Dint%2D[setX](int)++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-setY-1]]void link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#setY%2Dint%2D[setY](int)++ -* ++[[painless-api-reference-org-elasticsearch-painless-FeatureTest-twoFunctionsOfX-2]]<> link:{painless-javadoc}/org/elasticsearch/painless/FeatureTest.html#twoFunctionsOfX%2Djava.util.function.Function%2Djava.util.function.Function%2D[twoFunctionsOfX](<>, <>)++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.elasticsearch.search.lookup.FieldLookup.asciidoc b/docs/painless/painless-api-reference/org.elasticsearch.search.lookup.FieldLookup.asciidoc deleted file mode 100644 index a9735010ed6f6..0000000000000 --- a/docs/painless/painless-api-reference/org.elasticsearch.search.lookup.FieldLookup.asciidoc +++ /dev/null @@ -1,10 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-elasticsearch-search-lookup-FieldLookup]]++org.elasticsearch.search.lookup.FieldLookup++:: -* ++[[painless-api-reference-org-elasticsearch-search-lookup-FieldLookup-getValue-0]]def link:{elasticsearch-javadoc}/org/elasticsearch/search/lookup/FieldLookup.html#getValue%2D%2D[getValue]()++ -* ++[[painless-api-reference-org-elasticsearch-search-lookup-FieldLookup-getValues-0]]<> link:{elasticsearch-javadoc}/org/elasticsearch/search/lookup/FieldLookup.html#getValues%2D%2D[getValues]()++ -* ++[[painless-api-reference-org-elasticsearch-search-lookup-FieldLookup-isEmpty-0]]boolean link:{elasticsearch-javadoc}/org/elasticsearch/search/lookup/FieldLookup.html#isEmpty%2D%2D[isEmpty]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/org.joda.time.ReadableDateTime.asciidoc b/docs/painless/painless-api-reference/org.joda.time.ReadableDateTime.asciidoc deleted file mode 100644 index a7a0a17c1adb8..0000000000000 --- a/docs/painless/painless-api-reference/org.joda.time.ReadableDateTime.asciidoc +++ /dev/null @@ -1,27 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-joda-time-ReadableDateTime]]++org.joda.time.ReadableDateTime++:: -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getCenturyOfEra-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getCenturyOfEra%2D%2D[getCenturyOfEra]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getDayOfMonth-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getDayOfMonth%2D%2D[getDayOfMonth]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getDayOfWeek-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getDayOfWeek%2D%2D[getDayOfWeek]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getDayOfYear-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getDayOfYear%2D%2D[getDayOfYear]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getEra-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getEra%2D%2D[getEra]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getHourOfDay-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getHourOfDay%2D%2D[getHourOfDay]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getMillisOfDay-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getMillisOfDay%2D%2D[getMillisOfDay]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getMillisOfSecond-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getMillisOfSecond%2D%2D[getMillisOfSecond]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getMinuteOfDay-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getMinuteOfDay%2D%2D[getMinuteOfDay]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getMinuteOfHour-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getMinuteOfHour%2D%2D[getMinuteOfHour]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getMonthOfYear-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getMonthOfYear%2D%2D[getMonthOfYear]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getSecondOfDay-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getSecondOfDay%2D%2D[getSecondOfDay]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getSecondOfMinute-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getSecondOfMinute%2D%2D[getSecondOfMinute]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getWeekOfWeekyear-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getWeekOfWeekyear%2D%2D[getWeekOfWeekyear]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getWeekyear-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getWeekyear%2D%2D[getWeekyear]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getYear-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getYear%2D%2D[getYear]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getYearOfCentury-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getYearOfCentury%2D%2D[getYearOfCentury]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-getYearOfEra-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#getYearOfEra%2D%2D[getYearOfEra]()++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-toString-1]]<> link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#toString%2Djava.lang.String%2D[toString](<>)++ -* ++[[painless-api-reference-org-joda-time-ReadableDateTime-toString-2]]<> link:{joda-time-javadoc}/org/joda/time/ReadableDateTime.html#toString%2Djava.lang.String%2Djava.util.Locale%2D[toString](<>, <>)++ -* Inherits methods from ++<>++, ++<>++ diff --git a/docs/painless/painless-api-reference/org.joda.time.ReadableInstant.asciidoc b/docs/painless/painless-api-reference/org.joda.time.ReadableInstant.asciidoc deleted file mode 100644 index 96d1f61a81f55..0000000000000 --- a/docs/painless/painless-api-reference/org.joda.time.ReadableInstant.asciidoc +++ /dev/null @@ -1,14 +0,0 @@ -//// -Automatically generated by PainlessDocGenerator. Do not edit. -Rebuild by running `gradle generatePainlessApi`. -//// - -[[painless-api-reference-org-joda-time-ReadableInstant]]++org.joda.time.ReadableInstant++:: -* ++[[painless-api-reference-org-joda-time-ReadableInstant-equals-1]]boolean link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#equals%2Djava.lang.Object%2D[equals](<>)++ -* ++[[painless-api-reference-org-joda-time-ReadableInstant-getMillis-0]]long link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#getMillis%2D%2D[getMillis]()++ -* ++[[painless-api-reference-org-joda-time-ReadableInstant-hashCode-0]]int link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#hashCode%2D%2D[hashCode]()++ -* ++[[painless-api-reference-org-joda-time-ReadableInstant-isAfter-1]]boolean link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#isAfter%2Dorg.joda.time.ReadableInstant%2D[isAfter](<>)++ -* ++[[painless-api-reference-org-joda-time-ReadableInstant-isBefore-1]]boolean link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#isBefore%2Dorg.joda.time.ReadableInstant%2D[isBefore](<>)++ -* ++[[painless-api-reference-org-joda-time-ReadableInstant-isEqual-1]]boolean link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#isEqual%2Dorg.joda.time.ReadableInstant%2D[isEqual](<>)++ -* ++[[painless-api-reference-org-joda-time-ReadableInstant-toString-0]]<> link:{joda-time-javadoc}/org/joda/time/ReadableInstant.html#toString%2D%2D[toString]()++ -* Inherits methods from ++<>++ diff --git a/docs/painless/painless-api-reference/painless-api-reference-analysis/index.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-analysis/index.asciidoc new file mode 100644 index 0000000000000..8dc729b31ea1f --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-analysis/index.asciidoc @@ -0,0 +1,16 @@ +// This file is auto-generated. Do not edit. + +[[painless-api-reference-analysis]] +=== Analysis API + +The following specialized API is available in the Analysis context. + +* See the <> for further API available in all contexts. + +==== org.elasticsearch.analysis.common +<> + +* <> + +include::packages.asciidoc[] + diff --git a/docs/painless/painless-api-reference/painless-api-reference-analysis/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-analysis/packages.asciidoc new file mode 100644 index 0000000000000..106f9272df4a8 --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-analysis/packages.asciidoc @@ -0,0 +1,22 @@ +// This file is auto-generated. Do not edit. + + +[role="exclude",id="painless-api-reference-analysis-org-elasticsearch-analysis-common"] +=== Analysis API for package org.elasticsearch.analysis.common +See the <> for a high-level overview of all packages. + +[[painless-api-reference-analysis-AnalysisPredicateScript-Token]] +==== AnalysisPredicateScript.Token +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int getEndOffset() +* int getPosition() +* int getPositionIncrement() +* int getPositionLength() +* int getStartOffset() +* CharSequence getTerm() +* String getType() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean isKeyword() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + diff --git a/docs/painless/painless-api-reference/painless-api-reference-ingest/index.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-ingest/index.asciidoc new file mode 100644 index 0000000000000..e4067c24dcea0 --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-ingest/index.asciidoc @@ -0,0 +1,16 @@ +// This file is auto-generated. Do not edit. + +[[painless-api-reference-ingest]] +=== Ingest API + +The following specialized API is available in the Ingest context. + +* See the <> for further API available in all contexts. + +==== org.elasticsearch.ingest.common +<> + +* <> + +include::packages.asciidoc[] + diff --git a/docs/painless/painless-api-reference/painless-api-reference-ingest/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-ingest/packages.asciidoc new file mode 100644 index 0000000000000..b6a48ee7d5d2e --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-ingest/packages.asciidoc @@ -0,0 +1,20 @@ +// This file is auto-generated. Do not edit. + + +[role="exclude",id="painless-api-reference-ingest-org-elasticsearch-ingest-common"] +=== Ingest API for package org.elasticsearch.ingest.common +See the <> for a high-level overview of all packages. + +[[painless-api-reference-ingest-Processors]] +==== Processors +* static long bytes(String) +* static Object json(Object) +* static void json(Map, String) +* static String lowercase(String) +* static String uppercase(String) +* static String urlDecode(String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + diff --git a/docs/painless/painless-api-reference/painless-api-reference-moving-function/index.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-moving-function/index.asciidoc new file mode 100644 index 0000000000000..9d37e81a94fc7 --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-moving-function/index.asciidoc @@ -0,0 +1,16 @@ +// This file is auto-generated. Do not edit. + +[[painless-api-reference-moving-function]] +=== Moving Function API + +The following specialized API is available in the Moving Function context. + +* See the <> for further API available in all contexts. + +==== org.elasticsearch.search.aggregations.pipeline +<> + +* <> + +include::packages.asciidoc[] + diff --git a/docs/painless/painless-api-reference/painless-api-reference-moving-function/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-moving-function/packages.asciidoc new file mode 100644 index 0000000000000..824aa23f7ebfe --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-moving-function/packages.asciidoc @@ -0,0 +1,23 @@ +// This file is auto-generated. Do not edit. + + +[role="exclude",id="painless-api-reference-moving-function-org-elasticsearch-search-aggregations-pipeline"] +=== Moving Function API for package org.elasticsearch.search.aggregations.pipeline +See the <> for a high-level overview of all packages. + +[[painless-api-reference-moving-function-MovingFunctions]] +==== MovingFunctions +* static double ewma(double[], double) +* static double holt(double[], double, double) +* static double holtWinters(double[], double, double, double, int, boolean) +* static double linearWeightedAvg(double[]) +* static double max(double[]) +* static double min(double[]) +* static double stdDev(double[], double) +* static double sum(double[]) +* static double unweightedAvg(double[]) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + diff --git a/docs/painless/painless-api-reference/painless-api-reference-score/index.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-score/index.asciidoc new file mode 100644 index 0000000000000..fe9e0e1d23505 --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-score/index.asciidoc @@ -0,0 +1,18 @@ +// This file is auto-generated. Do not edit. + +[[painless-api-reference-score]] +=== Score API + +The following specialized API is available in the Score context. + +* See the <> for further API available in all contexts. + +==== org.elasticsearch.index.query +<> + +* <> +* <> +* <> + +include::packages.asciidoc[] + diff --git a/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc new file mode 100644 index 0000000000000..287f7a223ca5e --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-score/packages.asciidoc @@ -0,0 +1,169 @@ +// This file is auto-generated. Do not edit. + + +[role="exclude",id="painless-api-reference-score-org-elasticsearch-index-query"] +=== Score API for package org.elasticsearch.index.query +See the <> for a high-level overview of all packages. + +[[painless-api-reference-score-VectorScriptDocValues]] +==== VectorScriptDocValues +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-score-VectorScriptDocValues-DenseVectorScriptDocValues]] +==== VectorScriptDocValues.DenseVectorScriptDocValues +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-score-VectorScriptDocValues-SparseVectorScriptDocValues]] +==== VectorScriptDocValues.SparseVectorScriptDocValues +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + diff --git a/docs/painless/painless-api-reference/painless-api-reference-shared/index.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-shared/index.asciidoc new file mode 100644 index 0000000000000..c349602a7b580 --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-shared/index.asciidoc @@ -0,0 +1,432 @@ +// This file is auto-generated. Do not edit. + +[[painless-api-reference-shared]] +=== Shared API + +The following API is available in all contexts. + +==== java.lang +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.math +<> + +* <> +* <> +* <> +* <> + +==== java.text +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.time +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.time.chrono +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.time.format +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.time.temporal +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.time.zone +<> + +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.util +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.util.function +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== java.util.regex +<> + +* <> +* <> + +==== java.util.stream +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== org.apache.lucene.util +<> + +* <> + +==== org.elasticsearch.common.geo +<> + +* <> + +==== org.elasticsearch.index.fielddata +<> + +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +==== org.elasticsearch.index.mapper +<> + +* <> + +==== org.elasticsearch.index.query +<> + +* <> + +==== org.elasticsearch.index.similarity +<> + +* <> +* <> +* <> +* <> + +==== org.elasticsearch.painless.api +<> + +* <> + +==== org.elasticsearch.script +<> + +* <> + +==== org.elasticsearch.search.lookup +<> + +* <> + +include::packages.asciidoc[] + diff --git a/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc b/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc new file mode 100644 index 0000000000000..ed6e10e7b193c --- /dev/null +++ b/docs/painless/painless-api-reference/painless-api-reference-shared/packages.asciidoc @@ -0,0 +1,8608 @@ +// This file is auto-generated. Do not edit. + + +[role="exclude",id="painless-api-reference-shared-java-lang"] +=== Shared API for package java.lang +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-Appendable]] +==== Appendable +* Appendable {java11-javadoc}/java.base/java/lang/Appendable.html#append(java.lang.CharSequence,int,int)[append](CharSequence, int, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ArithmeticException]] +==== ArithmeticException +* {java11-javadoc}/java.base/java/lang/ArithmeticException.html#()[ArithmeticException]() +* {java11-javadoc}/java.base/java/lang/ArithmeticException.html#(java.lang.String)[ArithmeticException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ArrayIndexOutOfBoundsException]] +==== ArrayIndexOutOfBoundsException +* {java11-javadoc}/java.base/java/lang/ArrayIndexOutOfBoundsException.html#()[ArrayIndexOutOfBoundsException]() +* {java11-javadoc}/java.base/java/lang/ArrayIndexOutOfBoundsException.html#(java.lang.String)[ArrayIndexOutOfBoundsException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ArrayStoreException]] +==== ArrayStoreException +* {java11-javadoc}/java.base/java/lang/ArrayStoreException.html#()[ArrayStoreException]() +* {java11-javadoc}/java.base/java/lang/ArrayStoreException.html#(java.lang.String)[ArrayStoreException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Boolean]] +==== Boolean +* static Boolean {java11-javadoc}/java.base/java/lang/Boolean.html#FALSE[FALSE] +* static Boolean {java11-javadoc}/java.base/java/lang/Boolean.html#TRUE[TRUE] +* static int {java11-javadoc}/java.base/java/lang/Boolean.html#compare(boolean,boolean)[compare](boolean, boolean) +* static int {java11-javadoc}/java.base/java/lang/Boolean.html#hashCode(boolean)[hashCode](boolean) +* static boolean {java11-javadoc}/java.base/java/lang/Boolean.html#logicalAnd(boolean,boolean)[logicalAnd](boolean, boolean) +* static boolean {java11-javadoc}/java.base/java/lang/Boolean.html#logicalOr(boolean,boolean)[logicalOr](boolean, boolean) +* static boolean {java11-javadoc}/java.base/java/lang/Boolean.html#logicalXor(boolean,boolean)[logicalXor](boolean, boolean) +* static boolean {java11-javadoc}/java.base/java/lang/Boolean.html#parseBoolean(java.lang.String)[parseBoolean](String) +* static String {java11-javadoc}/java.base/java/lang/Boolean.html#toString(boolean)[toString](boolean) +* static Boolean {java11-javadoc}/java.base/java/lang/Boolean.html#valueOf(boolean)[valueOf](boolean) +* boolean {java11-javadoc}/java.base/java/lang/Boolean.html#booleanValue()[booleanValue]() +* int {java11-javadoc}/java.base/java/lang/Boolean.html#compareTo(java.lang.Boolean)[compareTo](Boolean) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Byte]] +==== Byte +* static int {java11-javadoc}/java.base/java/lang/Byte.html#BYTES[BYTES] +* static byte {java11-javadoc}/java.base/java/lang/Byte.html#MAX_VALUE[MAX_VALUE] +* static byte {java11-javadoc}/java.base/java/lang/Byte.html#MIN_VALUE[MIN_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Byte.html#SIZE[SIZE] +* static int {java11-javadoc}/java.base/java/lang/Byte.html#compare(byte,byte)[compare](byte, byte) +* static Byte {java11-javadoc}/java.base/java/lang/Byte.html#decode(java.lang.String)[decode](String) +* static int {java11-javadoc}/java.base/java/lang/Byte.html#hashCode(byte)[hashCode](byte) +* static byte {java11-javadoc}/java.base/java/lang/Byte.html#parseByte(java.lang.String)[parseByte](String) +* static byte {java11-javadoc}/java.base/java/lang/Byte.html#parseByte(java.lang.String,int)[parseByte](String, int) +* static String {java11-javadoc}/java.base/java/lang/Byte.html#toString(byte)[toString](byte) +* static int {java11-javadoc}/java.base/java/lang/Byte.html#toUnsignedInt(byte)[toUnsignedInt](byte) +* static long {java11-javadoc}/java.base/java/lang/Byte.html#toUnsignedLong(byte)[toUnsignedLong](byte) +* static Byte {java11-javadoc}/java.base/java/lang/Byte.html#valueOf(byte)[valueOf](byte) +* static Byte {java11-javadoc}/java.base/java/lang/Byte.html#valueOf(java.lang.String,int)[valueOf](String, int) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* int {java11-javadoc}/java.base/java/lang/Byte.html#compareTo(java.lang.Byte)[compareTo](Byte) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-CharSequence]] +==== CharSequence +* char {java11-javadoc}/java.base/java/lang/CharSequence.html#charAt(int)[charAt](int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#chars()[chars]() +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#codePoints()[codePoints]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/CharSequence.html#length()[length]() +* String replaceAll(Pattern, Function) +* String replaceFirst(Pattern, Function) +* CharSequence {java11-javadoc}/java.base/java/lang/CharSequence.html#subSequence(int,int)[subSequence](int, int) +* String {java11-javadoc}/java.base/java/lang/CharSequence.html#toString()[toString]() + + +[[painless-api-reference-shared-Character]] +==== Character +* static int {java11-javadoc}/java.base/java/lang/Character.html#BYTES[BYTES] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#COMBINING_SPACING_MARK[COMBINING_SPACING_MARK] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#CONNECTOR_PUNCTUATION[CONNECTOR_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#CONTROL[CONTROL] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#CURRENCY_SYMBOL[CURRENCY_SYMBOL] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DASH_PUNCTUATION[DASH_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DECIMAL_DIGIT_NUMBER[DECIMAL_DIGIT_NUMBER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_ARABIC_NUMBER[DIRECTIONALITY_ARABIC_NUMBER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_BOUNDARY_NEUTRAL[DIRECTIONALITY_BOUNDARY_NEUTRAL] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_COMMON_NUMBER_SEPARATOR[DIRECTIONALITY_COMMON_NUMBER_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER[DIRECTIONALITY_EUROPEAN_NUMBER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER_SEPARATOR[DIRECTIONALITY_EUROPEAN_NUMBER_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_EUROPEAN_NUMBER_TERMINATOR[DIRECTIONALITY_EUROPEAN_NUMBER_TERMINATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT[DIRECTIONALITY_LEFT_TO_RIGHT] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT_EMBEDDING[DIRECTIONALITY_LEFT_TO_RIGHT_EMBEDDING] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_LEFT_TO_RIGHT_OVERRIDE[DIRECTIONALITY_LEFT_TO_RIGHT_OVERRIDE] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_NONSPACING_MARK[DIRECTIONALITY_NONSPACING_MARK] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_OTHER_NEUTRALS[DIRECTIONALITY_OTHER_NEUTRALS] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_PARAGRAPH_SEPARATOR[DIRECTIONALITY_PARAGRAPH_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_POP_DIRECTIONAL_FORMAT[DIRECTIONALITY_POP_DIRECTIONAL_FORMAT] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT[DIRECTIONALITY_RIGHT_TO_LEFT] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_ARABIC[DIRECTIONALITY_RIGHT_TO_LEFT_ARABIC] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_EMBEDDING[DIRECTIONALITY_RIGHT_TO_LEFT_EMBEDDING] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_RIGHT_TO_LEFT_OVERRIDE[DIRECTIONALITY_RIGHT_TO_LEFT_OVERRIDE] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_SEGMENT_SEPARATOR[DIRECTIONALITY_SEGMENT_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_UNDEFINED[DIRECTIONALITY_UNDEFINED] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#DIRECTIONALITY_WHITESPACE[DIRECTIONALITY_WHITESPACE] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#ENCLOSING_MARK[ENCLOSING_MARK] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#END_PUNCTUATION[END_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#FINAL_QUOTE_PUNCTUATION[FINAL_QUOTE_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#FORMAT[FORMAT] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#INITIAL_QUOTE_PUNCTUATION[INITIAL_QUOTE_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#LETTER_NUMBER[LETTER_NUMBER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#LINE_SEPARATOR[LINE_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#LOWERCASE_LETTER[LOWERCASE_LETTER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#MATH_SYMBOL[MATH_SYMBOL] +* static int {java11-javadoc}/java.base/java/lang/Character.html#MAX_CODE_POINT[MAX_CODE_POINT] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MAX_HIGH_SURROGATE[MAX_HIGH_SURROGATE] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MAX_LOW_SURROGATE[MAX_LOW_SURROGATE] +* static int {java11-javadoc}/java.base/java/lang/Character.html#MAX_RADIX[MAX_RADIX] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MAX_SURROGATE[MAX_SURROGATE] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MAX_VALUE[MAX_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Character.html#MIN_CODE_POINT[MIN_CODE_POINT] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MIN_HIGH_SURROGATE[MIN_HIGH_SURROGATE] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MIN_LOW_SURROGATE[MIN_LOW_SURROGATE] +* static int {java11-javadoc}/java.base/java/lang/Character.html#MIN_RADIX[MIN_RADIX] +* static int {java11-javadoc}/java.base/java/lang/Character.html#MIN_SUPPLEMENTARY_CODE_POINT[MIN_SUPPLEMENTARY_CODE_POINT] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MIN_SURROGATE[MIN_SURROGATE] +* static char {java11-javadoc}/java.base/java/lang/Character.html#MIN_VALUE[MIN_VALUE] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#MODIFIER_LETTER[MODIFIER_LETTER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#MODIFIER_SYMBOL[MODIFIER_SYMBOL] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#NON_SPACING_MARK[NON_SPACING_MARK] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#OTHER_LETTER[OTHER_LETTER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#OTHER_NUMBER[OTHER_NUMBER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#OTHER_PUNCTUATION[OTHER_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#OTHER_SYMBOL[OTHER_SYMBOL] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#PARAGRAPH_SEPARATOR[PARAGRAPH_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#PRIVATE_USE[PRIVATE_USE] +* static int {java11-javadoc}/java.base/java/lang/Character.html#SIZE[SIZE] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#SPACE_SEPARATOR[SPACE_SEPARATOR] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#START_PUNCTUATION[START_PUNCTUATION] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#SURROGATE[SURROGATE] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#TITLECASE_LETTER[TITLECASE_LETTER] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#UNASSIGNED[UNASSIGNED] +* static byte {java11-javadoc}/java.base/java/lang/Character.html#UPPERCASE_LETTER[UPPERCASE_LETTER] +* static int {java11-javadoc}/java.base/java/lang/Character.html#charCount(int)[charCount](int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#codePointAt(java.lang.CharSequence,int)[codePointAt](CharSequence, int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#codePointAt(char%5B%5D,int,int)[codePointAt](char[], int, int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#codePointBefore(java.lang.CharSequence,int)[codePointBefore](CharSequence, int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#codePointBefore(char%5B%5D,int,int)[codePointBefore](char[], int, int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#codePointCount(java.lang.CharSequence,int,int)[codePointCount](CharSequence, int, int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#compare(char,char)[compare](char, char) +* static int {java11-javadoc}/java.base/java/lang/Character.html#digit(int,int)[digit](int, int) +* static char {java11-javadoc}/java.base/java/lang/Character.html#forDigit(int,int)[forDigit](int, int) +* static byte {java11-javadoc}/java.base/java/lang/Character.html#getDirectionality(int)[getDirectionality](int) +* static String {java11-javadoc}/java.base/java/lang/Character.html#getName(int)[getName](int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#getNumericValue(int)[getNumericValue](int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#getType(int)[getType](int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#hashCode(char)[hashCode](char) +* static char {java11-javadoc}/java.base/java/lang/Character.html#highSurrogate(int)[highSurrogate](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isAlphabetic(int)[isAlphabetic](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isBmpCodePoint(int)[isBmpCodePoint](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isDefined(int)[isDefined](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isDigit(int)[isDigit](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isHighSurrogate(char)[isHighSurrogate](char) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isISOControl(int)[isISOControl](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isIdentifierIgnorable(int)[isIdentifierIgnorable](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isIdeographic(int)[isIdeographic](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isJavaIdentifierPart(int)[isJavaIdentifierPart](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isJavaIdentifierStart(int)[isJavaIdentifierStart](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isLetter(int)[isLetter](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isLetterOrDigit(int)[isLetterOrDigit](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isLowerCase(int)[isLowerCase](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isMirrored(int)[isMirrored](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isSpaceChar(int)[isSpaceChar](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isSupplementaryCodePoint(int)[isSupplementaryCodePoint](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isSurrogate(char)[isSurrogate](char) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isSurrogatePair(char,char)[isSurrogatePair](char, char) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isTitleCase(int)[isTitleCase](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isUnicodeIdentifierPart(int)[isUnicodeIdentifierPart](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isUnicodeIdentifierStart(int)[isUnicodeIdentifierStart](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isUpperCase(int)[isUpperCase](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isValidCodePoint(int)[isValidCodePoint](int) +* static boolean {java11-javadoc}/java.base/java/lang/Character.html#isWhitespace(int)[isWhitespace](int) +* static char {java11-javadoc}/java.base/java/lang/Character.html#lowSurrogate(int)[lowSurrogate](int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#offsetByCodePoints(java.lang.CharSequence,int,int)[offsetByCodePoints](CharSequence, int, int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#offsetByCodePoints(char%5B%5D,int,int,int,int)[offsetByCodePoints](char[], int, int, int, int) +* static char {java11-javadoc}/java.base/java/lang/Character.html#reverseBytes(char)[reverseBytes](char) +* static char[] {java11-javadoc}/java.base/java/lang/Character.html#toChars(int)[toChars](int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#toChars(int,char%5B%5D,int)[toChars](int, char[], int) +* static int {java11-javadoc}/java.base/java/lang/Character.html#toCodePoint(char,char)[toCodePoint](char, char) +* static char {java11-javadoc}/java.base/java/lang/Character.html#toLowerCase(char)[toLowerCase](char) +* static String {java11-javadoc}/java.base/java/lang/Character.html#toString(char)[toString](char) +* static char {java11-javadoc}/java.base/java/lang/Character.html#toTitleCase(char)[toTitleCase](char) +* static char {java11-javadoc}/java.base/java/lang/Character.html#toUpperCase(char)[toUpperCase](char) +* static Character {java11-javadoc}/java.base/java/lang/Character.html#valueOf(char)[valueOf](char) +* char {java11-javadoc}/java.base/java/lang/Character.html#charValue()[charValue]() +* int {java11-javadoc}/java.base/java/lang/Character.html#compareTo(java.lang.Character)[compareTo](Character) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Character-Subset]] +==== Character.Subset +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Character-UnicodeBlock]] +==== Character.UnicodeBlock +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#AEGEAN_NUMBERS[AEGEAN_NUMBERS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ALCHEMICAL_SYMBOLS[ALCHEMICAL_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ALPHABETIC_PRESENTATION_FORMS[ALPHABETIC_PRESENTATION_FORMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ANCIENT_GREEK_MUSICAL_NOTATION[ANCIENT_GREEK_MUSICAL_NOTATION] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ANCIENT_GREEK_NUMBERS[ANCIENT_GREEK_NUMBERS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ANCIENT_SYMBOLS[ANCIENT_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARABIC[ARABIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARABIC_EXTENDED_A[ARABIC_EXTENDED_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARABIC_MATHEMATICAL_ALPHABETIC_SYMBOLS[ARABIC_MATHEMATICAL_ALPHABETIC_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARABIC_PRESENTATION_FORMS_A[ARABIC_PRESENTATION_FORMS_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARABIC_PRESENTATION_FORMS_B[ARABIC_PRESENTATION_FORMS_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARABIC_SUPPLEMENT[ARABIC_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARMENIAN[ARMENIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ARROWS[ARROWS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#AVESTAN[AVESTAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BALINESE[BALINESE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BAMUM[BAMUM] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BAMUM_SUPPLEMENT[BAMUM_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BASIC_LATIN[BASIC_LATIN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BATAK[BATAK] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BENGALI[BENGALI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BLOCK_ELEMENTS[BLOCK_ELEMENTS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BOPOMOFO[BOPOMOFO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BOPOMOFO_EXTENDED[BOPOMOFO_EXTENDED] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BOX_DRAWING[BOX_DRAWING] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BRAHMI[BRAHMI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BRAILLE_PATTERNS[BRAILLE_PATTERNS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BUGINESE[BUGINESE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BUHID[BUHID] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#BYZANTINE_MUSICAL_SYMBOLS[BYZANTINE_MUSICAL_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CARIAN[CARIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CHAKMA[CHAKMA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CHAM[CHAM] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CHEROKEE[CHEROKEE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_COMPATIBILITY[CJK_COMPATIBILITY] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_COMPATIBILITY_FORMS[CJK_COMPATIBILITY_FORMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_COMPATIBILITY_IDEOGRAPHS[CJK_COMPATIBILITY_IDEOGRAPHS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_COMPATIBILITY_IDEOGRAPHS_SUPPLEMENT[CJK_COMPATIBILITY_IDEOGRAPHS_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_RADICALS_SUPPLEMENT[CJK_RADICALS_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_STROKES[CJK_STROKES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_SYMBOLS_AND_PUNCTUATION[CJK_SYMBOLS_AND_PUNCTUATION] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS[CJK_UNIFIED_IDEOGRAPHS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_A[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_B[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_C[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_C] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CJK_UNIFIED_IDEOGRAPHS_EXTENSION_D[CJK_UNIFIED_IDEOGRAPHS_EXTENSION_D] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COMBINING_DIACRITICAL_MARKS[COMBINING_DIACRITICAL_MARKS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COMBINING_DIACRITICAL_MARKS_SUPPLEMENT[COMBINING_DIACRITICAL_MARKS_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COMBINING_HALF_MARKS[COMBINING_HALF_MARKS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COMBINING_MARKS_FOR_SYMBOLS[COMBINING_MARKS_FOR_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COMMON_INDIC_NUMBER_FORMS[COMMON_INDIC_NUMBER_FORMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CONTROL_PICTURES[CONTROL_PICTURES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COPTIC[COPTIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#COUNTING_ROD_NUMERALS[COUNTING_ROD_NUMERALS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CUNEIFORM[CUNEIFORM] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CUNEIFORM_NUMBERS_AND_PUNCTUATION[CUNEIFORM_NUMBERS_AND_PUNCTUATION] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CURRENCY_SYMBOLS[CURRENCY_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CYPRIOT_SYLLABARY[CYPRIOT_SYLLABARY] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CYRILLIC[CYRILLIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CYRILLIC_EXTENDED_A[CYRILLIC_EXTENDED_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CYRILLIC_EXTENDED_B[CYRILLIC_EXTENDED_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#CYRILLIC_SUPPLEMENTARY[CYRILLIC_SUPPLEMENTARY] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#DESERET[DESERET] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#DEVANAGARI[DEVANAGARI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#DEVANAGARI_EXTENDED[DEVANAGARI_EXTENDED] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#DINGBATS[DINGBATS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#DOMINO_TILES[DOMINO_TILES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#EGYPTIAN_HIEROGLYPHS[EGYPTIAN_HIEROGLYPHS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#EMOTICONS[EMOTICONS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ENCLOSED_ALPHANUMERICS[ENCLOSED_ALPHANUMERICS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ENCLOSED_ALPHANUMERIC_SUPPLEMENT[ENCLOSED_ALPHANUMERIC_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ENCLOSED_CJK_LETTERS_AND_MONTHS[ENCLOSED_CJK_LETTERS_AND_MONTHS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ENCLOSED_IDEOGRAPHIC_SUPPLEMENT[ENCLOSED_IDEOGRAPHIC_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ETHIOPIC[ETHIOPIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ETHIOPIC_EXTENDED[ETHIOPIC_EXTENDED] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ETHIOPIC_EXTENDED_A[ETHIOPIC_EXTENDED_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ETHIOPIC_SUPPLEMENT[ETHIOPIC_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GENERAL_PUNCTUATION[GENERAL_PUNCTUATION] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GEOMETRIC_SHAPES[GEOMETRIC_SHAPES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GEORGIAN[GEORGIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GEORGIAN_SUPPLEMENT[GEORGIAN_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GLAGOLITIC[GLAGOLITIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GOTHIC[GOTHIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GREEK[GREEK] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GREEK_EXTENDED[GREEK_EXTENDED] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GUJARATI[GUJARATI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#GURMUKHI[GURMUKHI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HALFWIDTH_AND_FULLWIDTH_FORMS[HALFWIDTH_AND_FULLWIDTH_FORMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HANGUL_COMPATIBILITY_JAMO[HANGUL_COMPATIBILITY_JAMO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HANGUL_JAMO[HANGUL_JAMO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HANGUL_JAMO_EXTENDED_A[HANGUL_JAMO_EXTENDED_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HANGUL_JAMO_EXTENDED_B[HANGUL_JAMO_EXTENDED_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HANGUL_SYLLABLES[HANGUL_SYLLABLES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HANUNOO[HANUNOO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HEBREW[HEBREW] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HIGH_PRIVATE_USE_SURROGATES[HIGH_PRIVATE_USE_SURROGATES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HIGH_SURROGATES[HIGH_SURROGATES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#HIRAGANA[HIRAGANA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#IDEOGRAPHIC_DESCRIPTION_CHARACTERS[IDEOGRAPHIC_DESCRIPTION_CHARACTERS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#IMPERIAL_ARAMAIC[IMPERIAL_ARAMAIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#INSCRIPTIONAL_PAHLAVI[INSCRIPTIONAL_PAHLAVI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#INSCRIPTIONAL_PARTHIAN[INSCRIPTIONAL_PARTHIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#IPA_EXTENSIONS[IPA_EXTENSIONS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#JAVANESE[JAVANESE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KAITHI[KAITHI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KANA_SUPPLEMENT[KANA_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KANBUN[KANBUN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KANGXI_RADICALS[KANGXI_RADICALS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KANNADA[KANNADA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KATAKANA[KATAKANA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KATAKANA_PHONETIC_EXTENSIONS[KATAKANA_PHONETIC_EXTENSIONS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KAYAH_LI[KAYAH_LI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KHAROSHTHI[KHAROSHTHI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KHMER[KHMER] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#KHMER_SYMBOLS[KHMER_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LAO[LAO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LATIN_1_SUPPLEMENT[LATIN_1_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LATIN_EXTENDED_A[LATIN_EXTENDED_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LATIN_EXTENDED_ADDITIONAL[LATIN_EXTENDED_ADDITIONAL] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LATIN_EXTENDED_B[LATIN_EXTENDED_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LATIN_EXTENDED_C[LATIN_EXTENDED_C] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LATIN_EXTENDED_D[LATIN_EXTENDED_D] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LEPCHA[LEPCHA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LETTERLIKE_SYMBOLS[LETTERLIKE_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LIMBU[LIMBU] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LINEAR_B_IDEOGRAMS[LINEAR_B_IDEOGRAMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LINEAR_B_SYLLABARY[LINEAR_B_SYLLABARY] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LISU[LISU] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LOW_SURROGATES[LOW_SURROGATES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LYCIAN[LYCIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#LYDIAN[LYDIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MAHJONG_TILES[MAHJONG_TILES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MALAYALAM[MALAYALAM] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MANDAIC[MANDAIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MATHEMATICAL_ALPHANUMERIC_SYMBOLS[MATHEMATICAL_ALPHANUMERIC_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MATHEMATICAL_OPERATORS[MATHEMATICAL_OPERATORS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MEETEI_MAYEK[MEETEI_MAYEK] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MEETEI_MAYEK_EXTENSIONS[MEETEI_MAYEK_EXTENSIONS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MEROITIC_CURSIVE[MEROITIC_CURSIVE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MEROITIC_HIEROGLYPHS[MEROITIC_HIEROGLYPHS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MIAO[MIAO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MISCELLANEOUS_MATHEMATICAL_SYMBOLS_A[MISCELLANEOUS_MATHEMATICAL_SYMBOLS_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MISCELLANEOUS_MATHEMATICAL_SYMBOLS_B[MISCELLANEOUS_MATHEMATICAL_SYMBOLS_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MISCELLANEOUS_SYMBOLS[MISCELLANEOUS_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MISCELLANEOUS_SYMBOLS_AND_ARROWS[MISCELLANEOUS_SYMBOLS_AND_ARROWS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MISCELLANEOUS_SYMBOLS_AND_PICTOGRAPHS[MISCELLANEOUS_SYMBOLS_AND_PICTOGRAPHS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MISCELLANEOUS_TECHNICAL[MISCELLANEOUS_TECHNICAL] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MODIFIER_TONE_LETTERS[MODIFIER_TONE_LETTERS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MONGOLIAN[MONGOLIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MUSICAL_SYMBOLS[MUSICAL_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MYANMAR[MYANMAR] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#MYANMAR_EXTENDED_A[MYANMAR_EXTENDED_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#NEW_TAI_LUE[NEW_TAI_LUE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#NKO[NKO] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#NUMBER_FORMS[NUMBER_FORMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OGHAM[OGHAM] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OLD_ITALIC[OLD_ITALIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OLD_PERSIAN[OLD_PERSIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OLD_SOUTH_ARABIAN[OLD_SOUTH_ARABIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OLD_TURKIC[OLD_TURKIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OL_CHIKI[OL_CHIKI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OPTICAL_CHARACTER_RECOGNITION[OPTICAL_CHARACTER_RECOGNITION] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#ORIYA[ORIYA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#OSMANYA[OSMANYA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PHAGS_PA[PHAGS_PA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PHAISTOS_DISC[PHAISTOS_DISC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PHOENICIAN[PHOENICIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PHONETIC_EXTENSIONS[PHONETIC_EXTENSIONS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PHONETIC_EXTENSIONS_SUPPLEMENT[PHONETIC_EXTENSIONS_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PLAYING_CARDS[PLAYING_CARDS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#PRIVATE_USE_AREA[PRIVATE_USE_AREA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#REJANG[REJANG] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#RUMI_NUMERAL_SYMBOLS[RUMI_NUMERAL_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#RUNIC[RUNIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SAMARITAN[SAMARITAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SAURASHTRA[SAURASHTRA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SHARADA[SHARADA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SHAVIAN[SHAVIAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SINHALA[SINHALA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SMALL_FORM_VARIANTS[SMALL_FORM_VARIANTS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SORA_SOMPENG[SORA_SOMPENG] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SPACING_MODIFIER_LETTERS[SPACING_MODIFIER_LETTERS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SPECIALS[SPECIALS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUNDANESE[SUNDANESE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUNDANESE_SUPPLEMENT[SUNDANESE_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPERSCRIPTS_AND_SUBSCRIPTS[SUPERSCRIPTS_AND_SUBSCRIPTS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPPLEMENTAL_ARROWS_A[SUPPLEMENTAL_ARROWS_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPPLEMENTAL_ARROWS_B[SUPPLEMENTAL_ARROWS_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPPLEMENTAL_MATHEMATICAL_OPERATORS[SUPPLEMENTAL_MATHEMATICAL_OPERATORS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPPLEMENTAL_PUNCTUATION[SUPPLEMENTAL_PUNCTUATION] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPPLEMENTARY_PRIVATE_USE_AREA_A[SUPPLEMENTARY_PRIVATE_USE_AREA_A] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SUPPLEMENTARY_PRIVATE_USE_AREA_B[SUPPLEMENTARY_PRIVATE_USE_AREA_B] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SYLOTI_NAGRI[SYLOTI_NAGRI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#SYRIAC[SYRIAC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAGALOG[TAGALOG] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAGBANWA[TAGBANWA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAGS[TAGS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAI_LE[TAI_LE] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAI_THAM[TAI_THAM] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAI_VIET[TAI_VIET] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAI_XUAN_JING_SYMBOLS[TAI_XUAN_JING_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAKRI[TAKRI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TAMIL[TAMIL] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TELUGU[TELUGU] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#THAANA[THAANA] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#THAI[THAI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TIBETAN[TIBETAN] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TIFINAGH[TIFINAGH] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#TRANSPORT_AND_MAP_SYMBOLS[TRANSPORT_AND_MAP_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#UGARITIC[UGARITIC] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS[UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS_EXTENDED[UNIFIED_CANADIAN_ABORIGINAL_SYLLABICS_EXTENDED] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#VAI[VAI] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#VARIATION_SELECTORS[VARIATION_SELECTORS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#VARIATION_SELECTORS_SUPPLEMENT[VARIATION_SELECTORS_SUPPLEMENT] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#VEDIC_EXTENSIONS[VEDIC_EXTENSIONS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#VERTICAL_FORMS[VERTICAL_FORMS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#YIJING_HEXAGRAM_SYMBOLS[YIJING_HEXAGRAM_SYMBOLS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#YI_RADICALS[YI_RADICALS] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#YI_SYLLABLES[YI_SYLLABLES] +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#forName(java.lang.String)[forName](String) +* static Character.UnicodeBlock {java11-javadoc}/java.base/java/lang/Character$UnicodeBlock.html#of(int)[of](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Character-UnicodeScript]] +==== Character.UnicodeScript +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#ARABIC[ARABIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#ARMENIAN[ARMENIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#AVESTAN[AVESTAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BALINESE[BALINESE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BAMUM[BAMUM] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BATAK[BATAK] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BENGALI[BENGALI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BOPOMOFO[BOPOMOFO] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BRAHMI[BRAHMI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BRAILLE[BRAILLE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BUGINESE[BUGINESE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#BUHID[BUHID] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CANADIAN_ABORIGINAL[CANADIAN_ABORIGINAL] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CARIAN[CARIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CHAKMA[CHAKMA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CHAM[CHAM] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CHEROKEE[CHEROKEE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#COMMON[COMMON] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#COPTIC[COPTIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CUNEIFORM[CUNEIFORM] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CYPRIOT[CYPRIOT] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#CYRILLIC[CYRILLIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#DESERET[DESERET] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#DEVANAGARI[DEVANAGARI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#EGYPTIAN_HIEROGLYPHS[EGYPTIAN_HIEROGLYPHS] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#ETHIOPIC[ETHIOPIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#GEORGIAN[GEORGIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#GLAGOLITIC[GLAGOLITIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#GOTHIC[GOTHIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#GREEK[GREEK] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#GUJARATI[GUJARATI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#GURMUKHI[GURMUKHI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#HAN[HAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#HANGUL[HANGUL] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#HANUNOO[HANUNOO] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#HEBREW[HEBREW] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#HIRAGANA[HIRAGANA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#IMPERIAL_ARAMAIC[IMPERIAL_ARAMAIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#INHERITED[INHERITED] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#INSCRIPTIONAL_PAHLAVI[INSCRIPTIONAL_PAHLAVI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#INSCRIPTIONAL_PARTHIAN[INSCRIPTIONAL_PARTHIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#JAVANESE[JAVANESE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#KAITHI[KAITHI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#KANNADA[KANNADA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#KATAKANA[KATAKANA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#KAYAH_LI[KAYAH_LI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#KHAROSHTHI[KHAROSHTHI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#KHMER[KHMER] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LAO[LAO] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LATIN[LATIN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LEPCHA[LEPCHA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LIMBU[LIMBU] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LINEAR_B[LINEAR_B] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LISU[LISU] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LYCIAN[LYCIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#LYDIAN[LYDIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MALAYALAM[MALAYALAM] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MANDAIC[MANDAIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MEETEI_MAYEK[MEETEI_MAYEK] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MEROITIC_CURSIVE[MEROITIC_CURSIVE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MEROITIC_HIEROGLYPHS[MEROITIC_HIEROGLYPHS] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MIAO[MIAO] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MONGOLIAN[MONGOLIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#MYANMAR[MYANMAR] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#NEW_TAI_LUE[NEW_TAI_LUE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#NKO[NKO] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OGHAM[OGHAM] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OLD_ITALIC[OLD_ITALIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OLD_PERSIAN[OLD_PERSIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OLD_SOUTH_ARABIAN[OLD_SOUTH_ARABIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OLD_TURKIC[OLD_TURKIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OL_CHIKI[OL_CHIKI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#ORIYA[ORIYA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#OSMANYA[OSMANYA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#PHAGS_PA[PHAGS_PA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#PHOENICIAN[PHOENICIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#REJANG[REJANG] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#RUNIC[RUNIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SAMARITAN[SAMARITAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SAURASHTRA[SAURASHTRA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SHARADA[SHARADA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SHAVIAN[SHAVIAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SINHALA[SINHALA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SORA_SOMPENG[SORA_SOMPENG] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SUNDANESE[SUNDANESE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SYLOTI_NAGRI[SYLOTI_NAGRI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#SYRIAC[SYRIAC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAGALOG[TAGALOG] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAGBANWA[TAGBANWA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAI_LE[TAI_LE] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAI_THAM[TAI_THAM] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAI_VIET[TAI_VIET] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAKRI[TAKRI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TAMIL[TAMIL] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TELUGU[TELUGU] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#THAANA[THAANA] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#THAI[THAI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TIBETAN[TIBETAN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#TIFINAGH[TIFINAGH] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#UGARITIC[UGARITIC] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#UNKNOWN[UNKNOWN] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#VAI[VAI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#YI[YI] +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#forName(java.lang.String)[forName](String) +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#of(int)[of](int) +* static Character.UnicodeScript {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#valueOf(java.lang.String)[valueOf](String) +* static Character.UnicodeScript[] {java11-javadoc}/java.base/java/lang/Character$UnicodeScript.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ClassCastException]] +==== ClassCastException +* {java11-javadoc}/java.base/java/lang/ClassCastException.html#()[ClassCastException]() +* {java11-javadoc}/java.base/java/lang/ClassCastException.html#(java.lang.String)[ClassCastException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ClassNotFoundException]] +==== ClassNotFoundException +* {java11-javadoc}/java.base/java/lang/ClassNotFoundException.html#()[ClassNotFoundException]() +* {java11-javadoc}/java.base/java/lang/ClassNotFoundException.html#(java.lang.String)[ClassNotFoundException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-CloneNotSupportedException]] +==== CloneNotSupportedException +* {java11-javadoc}/java.base/java/lang/CloneNotSupportedException.html#()[CloneNotSupportedException]() +* {java11-javadoc}/java.base/java/lang/CloneNotSupportedException.html#(java.lang.String)[CloneNotSupportedException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Comparable]] +==== Comparable +* int {java11-javadoc}/java.base/java/lang/Comparable.html#compareTo(java.lang.Object)[compareTo](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Double]] +==== Double +* static int {java11-javadoc}/java.base/java/lang/Double.html#BYTES[BYTES] +* static int {java11-javadoc}/java.base/java/lang/Double.html#MAX_EXPONENT[MAX_EXPONENT] +* static double {java11-javadoc}/java.base/java/lang/Double.html#MAX_VALUE[MAX_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Double.html#MIN_EXPONENT[MIN_EXPONENT] +* static double {java11-javadoc}/java.base/java/lang/Double.html#MIN_NORMAL[MIN_NORMAL] +* static double {java11-javadoc}/java.base/java/lang/Double.html#MIN_VALUE[MIN_VALUE] +* static double {java11-javadoc}/java.base/java/lang/Double.html#NEGATIVE_INFINITY[NEGATIVE_INFINITY] +* static double {java11-javadoc}/java.base/java/lang/Double.html#NaN[NaN] +* static double {java11-javadoc}/java.base/java/lang/Double.html#POSITIVE_INFINITY[POSITIVE_INFINITY] +* static int {java11-javadoc}/java.base/java/lang/Double.html#SIZE[SIZE] +* static int {java11-javadoc}/java.base/java/lang/Double.html#compare(double,double)[compare](double, double) +* static long {java11-javadoc}/java.base/java/lang/Double.html#doubleToLongBits(double)[doubleToLongBits](double) +* static long {java11-javadoc}/java.base/java/lang/Double.html#doubleToRawLongBits(double)[doubleToRawLongBits](double) +* static int {java11-javadoc}/java.base/java/lang/Double.html#hashCode(double)[hashCode](double) +* static boolean {java11-javadoc}/java.base/java/lang/Double.html#isFinite(double)[isFinite](double) +* static boolean {java11-javadoc}/java.base/java/lang/Double.html#isInfinite(double)[isInfinite](double) +* static boolean {java11-javadoc}/java.base/java/lang/Double.html#isNaN(double)[isNaN](double) +* static double {java11-javadoc}/java.base/java/lang/Double.html#longBitsToDouble(long)[longBitsToDouble](long) +* static double {java11-javadoc}/java.base/java/lang/Double.html#max(double,double)[max](double, double) +* static double {java11-javadoc}/java.base/java/lang/Double.html#min(double,double)[min](double, double) +* static double {java11-javadoc}/java.base/java/lang/Double.html#parseDouble(java.lang.String)[parseDouble](String) +* static double {java11-javadoc}/java.base/java/lang/Double.html#sum(double,double)[sum](double, double) +* static String {java11-javadoc}/java.base/java/lang/Double.html#toHexString(double)[toHexString](double) +* static String {java11-javadoc}/java.base/java/lang/Double.html#toString(double)[toString](double) +* static Double {java11-javadoc}/java.base/java/lang/Double.html#valueOf(double)[valueOf](double) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* int {java11-javadoc}/java.base/java/lang/Double.html#compareTo(java.lang.Double)[compareTo](Double) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* boolean {java11-javadoc}/java.base/java/lang/Double.html#isInfinite()[isInfinite]() +* boolean {java11-javadoc}/java.base/java/lang/Double.html#isNaN()[isNaN]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Enum]] +==== Enum +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-EnumConstantNotPresentException]] +==== EnumConstantNotPresentException +* String {java11-javadoc}/java.base/java/lang/EnumConstantNotPresentException.html#constantName()[constantName]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Exception]] +==== Exception +* {java11-javadoc}/java.base/java/lang/Exception.html#()[Exception]() +* {java11-javadoc}/java.base/java/lang/Exception.html#(java.lang.String)[Exception](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Float]] +==== Float +* static int {java11-javadoc}/java.base/java/lang/Float.html#BYTES[BYTES] +* static int {java11-javadoc}/java.base/java/lang/Float.html#MAX_EXPONENT[MAX_EXPONENT] +* static float {java11-javadoc}/java.base/java/lang/Float.html#MAX_VALUE[MAX_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Float.html#MIN_EXPONENT[MIN_EXPONENT] +* static float {java11-javadoc}/java.base/java/lang/Float.html#MIN_NORMAL[MIN_NORMAL] +* static float {java11-javadoc}/java.base/java/lang/Float.html#MIN_VALUE[MIN_VALUE] +* static float {java11-javadoc}/java.base/java/lang/Float.html#NEGATIVE_INFINITY[NEGATIVE_INFINITY] +* static float {java11-javadoc}/java.base/java/lang/Float.html#NaN[NaN] +* static float {java11-javadoc}/java.base/java/lang/Float.html#POSITIVE_INFINITY[POSITIVE_INFINITY] +* static int {java11-javadoc}/java.base/java/lang/Float.html#SIZE[SIZE] +* static int {java11-javadoc}/java.base/java/lang/Float.html#compare(float,float)[compare](float, float) +* static int {java11-javadoc}/java.base/java/lang/Float.html#floatToIntBits(float)[floatToIntBits](float) +* static int {java11-javadoc}/java.base/java/lang/Float.html#floatToRawIntBits(float)[floatToRawIntBits](float) +* static int {java11-javadoc}/java.base/java/lang/Float.html#hashCode(float)[hashCode](float) +* static float {java11-javadoc}/java.base/java/lang/Float.html#intBitsToFloat(int)[intBitsToFloat](int) +* static boolean {java11-javadoc}/java.base/java/lang/Float.html#isFinite(float)[isFinite](float) +* static boolean {java11-javadoc}/java.base/java/lang/Float.html#isInfinite(float)[isInfinite](float) +* static boolean {java11-javadoc}/java.base/java/lang/Float.html#isNaN(float)[isNaN](float) +* static float {java11-javadoc}/java.base/java/lang/Float.html#max(float,float)[max](float, float) +* static float {java11-javadoc}/java.base/java/lang/Float.html#min(float,float)[min](float, float) +* static float {java11-javadoc}/java.base/java/lang/Float.html#parseFloat(java.lang.String)[parseFloat](String) +* static float {java11-javadoc}/java.base/java/lang/Float.html#sum(float,float)[sum](float, float) +* static String {java11-javadoc}/java.base/java/lang/Float.html#toHexString(float)[toHexString](float) +* static String {java11-javadoc}/java.base/java/lang/Float.html#toString(float)[toString](float) +* static Float {java11-javadoc}/java.base/java/lang/Float.html#valueOf(float)[valueOf](float) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* int {java11-javadoc}/java.base/java/lang/Float.html#compareTo(java.lang.Float)[compareTo](Float) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* boolean {java11-javadoc}/java.base/java/lang/Float.html#isInfinite()[isInfinite]() +* boolean {java11-javadoc}/java.base/java/lang/Float.html#isNaN()[isNaN]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalAccessException]] +==== IllegalAccessException +* {java11-javadoc}/java.base/java/lang/IllegalAccessException.html#()[IllegalAccessException]() +* {java11-javadoc}/java.base/java/lang/IllegalAccessException.html#(java.lang.String)[IllegalAccessException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalArgumentException]] +==== IllegalArgumentException +* {java11-javadoc}/java.base/java/lang/IllegalArgumentException.html#()[IllegalArgumentException]() +* {java11-javadoc}/java.base/java/lang/IllegalArgumentException.html#(java.lang.String)[IllegalArgumentException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalMonitorStateException]] +==== IllegalMonitorStateException +* {java11-javadoc}/java.base/java/lang/IllegalMonitorStateException.html#()[IllegalMonitorStateException]() +* {java11-javadoc}/java.base/java/lang/IllegalMonitorStateException.html#(java.lang.String)[IllegalMonitorStateException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalStateException]] +==== IllegalStateException +* {java11-javadoc}/java.base/java/lang/IllegalStateException.html#()[IllegalStateException]() +* {java11-javadoc}/java.base/java/lang/IllegalStateException.html#(java.lang.String)[IllegalStateException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalThreadStateException]] +==== IllegalThreadStateException +* {java11-javadoc}/java.base/java/lang/IllegalThreadStateException.html#()[IllegalThreadStateException]() +* {java11-javadoc}/java.base/java/lang/IllegalThreadStateException.html#(java.lang.String)[IllegalThreadStateException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IndexOutOfBoundsException]] +==== IndexOutOfBoundsException +* {java11-javadoc}/java.base/java/lang/IndexOutOfBoundsException.html#()[IndexOutOfBoundsException]() +* {java11-javadoc}/java.base/java/lang/IndexOutOfBoundsException.html#(java.lang.String)[IndexOutOfBoundsException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-InstantiationException]] +==== InstantiationException +* {java11-javadoc}/java.base/java/lang/InstantiationException.html#()[InstantiationException]() +* {java11-javadoc}/java.base/java/lang/InstantiationException.html#(java.lang.String)[InstantiationException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Integer]] +==== Integer +* static int {java11-javadoc}/java.base/java/lang/Integer.html#BYTES[BYTES] +* static int {java11-javadoc}/java.base/java/lang/Integer.html#MAX_VALUE[MAX_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Integer.html#MIN_VALUE[MIN_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Integer.html#SIZE[SIZE] +* static int {java11-javadoc}/java.base/java/lang/Integer.html#bitCount(int)[bitCount](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#compare(int,int)[compare](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#compareUnsigned(int,int)[compareUnsigned](int, int) +* static Integer {java11-javadoc}/java.base/java/lang/Integer.html#decode(java.lang.String)[decode](String) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#divideUnsigned(int,int)[divideUnsigned](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#hashCode(int)[hashCode](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#highestOneBit(int)[highestOneBit](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#lowestOneBit(int)[lowestOneBit](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#max(int,int)[max](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#min(int,int)[min](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#numberOfLeadingZeros(int)[numberOfLeadingZeros](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#numberOfTrailingZeros(int)[numberOfTrailingZeros](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#parseInt(java.lang.String)[parseInt](String) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#parseInt(java.lang.String,int)[parseInt](String, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#parseUnsignedInt(java.lang.String)[parseUnsignedInt](String) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#parseUnsignedInt(java.lang.String,int)[parseUnsignedInt](String, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#remainderUnsigned(int,int)[remainderUnsigned](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#reverse(int)[reverse](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#reverseBytes(int)[reverseBytes](int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#rotateLeft(int,int)[rotateLeft](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#rotateRight(int,int)[rotateRight](int, int) +* static int {java11-javadoc}/java.base/java/lang/Integer.html#signum(int)[signum](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toBinaryString(int)[toBinaryString](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toHexString(int)[toHexString](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toOctalString(int)[toOctalString](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toString(int)[toString](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toString(int,int)[toString](int, int) +* static long {java11-javadoc}/java.base/java/lang/Integer.html#toUnsignedLong(int)[toUnsignedLong](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toUnsignedString(int)[toUnsignedString](int) +* static String {java11-javadoc}/java.base/java/lang/Integer.html#toUnsignedString(int,int)[toUnsignedString](int, int) +* static Integer {java11-javadoc}/java.base/java/lang/Integer.html#valueOf(int)[valueOf](int) +* static Integer {java11-javadoc}/java.base/java/lang/Integer.html#valueOf(java.lang.String,int)[valueOf](String, int) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* int {java11-javadoc}/java.base/java/lang/Integer.html#compareTo(java.lang.Integer)[compareTo](Integer) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-InterruptedException]] +==== InterruptedException +* {java11-javadoc}/java.base/java/lang/InterruptedException.html#()[InterruptedException]() +* {java11-javadoc}/java.base/java/lang/InterruptedException.html#(java.lang.String)[InterruptedException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Iterable]] +==== Iterable +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* Spliterator {java11-javadoc}/java.base/java/lang/Iterable.html#spliterator()[spliterator]() +* double sum() +* double sum(ToDoubleFunction) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Long]] +==== Long +* static int {java11-javadoc}/java.base/java/lang/Long.html#BYTES[BYTES] +* static long {java11-javadoc}/java.base/java/lang/Long.html#MAX_VALUE[MAX_VALUE] +* static long {java11-javadoc}/java.base/java/lang/Long.html#MIN_VALUE[MIN_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Long.html#SIZE[SIZE] +* static int {java11-javadoc}/java.base/java/lang/Long.html#bitCount(long)[bitCount](long) +* static int {java11-javadoc}/java.base/java/lang/Long.html#compare(long,long)[compare](long, long) +* static int {java11-javadoc}/java.base/java/lang/Long.html#compareUnsigned(long,long)[compareUnsigned](long, long) +* static Long {java11-javadoc}/java.base/java/lang/Long.html#decode(java.lang.String)[decode](String) +* static long {java11-javadoc}/java.base/java/lang/Long.html#divideUnsigned(long,long)[divideUnsigned](long, long) +* static int {java11-javadoc}/java.base/java/lang/Long.html#hashCode(long)[hashCode](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#highestOneBit(long)[highestOneBit](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#lowestOneBit(long)[lowestOneBit](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#max(long,long)[max](long, long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#min(long,long)[min](long, long) +* static int {java11-javadoc}/java.base/java/lang/Long.html#numberOfLeadingZeros(long)[numberOfLeadingZeros](long) +* static int {java11-javadoc}/java.base/java/lang/Long.html#numberOfTrailingZeros(long)[numberOfTrailingZeros](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#parseLong(java.lang.String)[parseLong](String) +* static long {java11-javadoc}/java.base/java/lang/Long.html#parseLong(java.lang.String,int)[parseLong](String, int) +* static long {java11-javadoc}/java.base/java/lang/Long.html#parseUnsignedLong(java.lang.String)[parseUnsignedLong](String) +* static long {java11-javadoc}/java.base/java/lang/Long.html#parseUnsignedLong(java.lang.String,int)[parseUnsignedLong](String, int) +* static long {java11-javadoc}/java.base/java/lang/Long.html#remainderUnsigned(long,long)[remainderUnsigned](long, long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#reverse(long)[reverse](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#reverseBytes(long)[reverseBytes](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#rotateLeft(long,int)[rotateLeft](long, int) +* static long {java11-javadoc}/java.base/java/lang/Long.html#rotateRight(long,int)[rotateRight](long, int) +* static int {java11-javadoc}/java.base/java/lang/Long.html#signum(long)[signum](long) +* static long {java11-javadoc}/java.base/java/lang/Long.html#sum(long,long)[sum](long, long) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toBinaryString(long)[toBinaryString](long) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toHexString(long)[toHexString](long) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toOctalString(long)[toOctalString](long) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toString(long)[toString](long) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toString(long,int)[toString](long, int) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toUnsignedString(long)[toUnsignedString](long) +* static String {java11-javadoc}/java.base/java/lang/Long.html#toUnsignedString(long,int)[toUnsignedString](long, int) +* static Long {java11-javadoc}/java.base/java/lang/Long.html#valueOf(long)[valueOf](long) +* static Long {java11-javadoc}/java.base/java/lang/Long.html#valueOf(java.lang.String,int)[valueOf](String, int) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* int {java11-javadoc}/java.base/java/lang/Long.html#compareTo(java.lang.Long)[compareTo](Long) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Math]] +==== Math +* static double {java11-javadoc}/java.base/java/lang/Math.html#E[E] +* static double {java11-javadoc}/java.base/java/lang/Math.html#PI[PI] +* static double {java11-javadoc}/java.base/java/lang/Math.html#IEEEremainder(double,double)[IEEEremainder](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#abs(double)[abs](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#acos(double)[acos](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#asin(double)[asin](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#atan(double)[atan](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#atan2(double,double)[atan2](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#cbrt(double)[cbrt](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#ceil(double)[ceil](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#copySign(double,double)[copySign](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#cos(double)[cos](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#cosh(double)[cosh](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#exp(double)[exp](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#expm1(double)[expm1](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#floor(double)[floor](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#hypot(double,double)[hypot](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#log(double)[log](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#log10(double)[log10](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#log1p(double)[log1p](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#max(double,double)[max](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#min(double,double)[min](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#nextAfter(double,double)[nextAfter](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#nextDown(double)[nextDown](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#nextUp(double)[nextUp](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#pow(double,double)[pow](double, double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#random()[random]() +* static double {java11-javadoc}/java.base/java/lang/Math.html#rint(double)[rint](double) +* static long {java11-javadoc}/java.base/java/lang/Math.html#round(double)[round](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#scalb(double,int)[scalb](double, int) +* static double {java11-javadoc}/java.base/java/lang/Math.html#signum(double)[signum](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#sin(double)[sin](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#sinh(double)[sinh](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#sqrt(double)[sqrt](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#tan(double)[tan](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#tanh(double)[tanh](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#toDegrees(double)[toDegrees](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#toRadians(double)[toRadians](double) +* static double {java11-javadoc}/java.base/java/lang/Math.html#ulp(double)[ulp](double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NegativeArraySizeException]] +==== NegativeArraySizeException +* {java11-javadoc}/java.base/java/lang/NegativeArraySizeException.html#()[NegativeArraySizeException]() +* {java11-javadoc}/java.base/java/lang/NegativeArraySizeException.html#(java.lang.String)[NegativeArraySizeException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NoSuchFieldException]] +==== NoSuchFieldException +* {java11-javadoc}/java.base/java/lang/NoSuchFieldException.html#()[NoSuchFieldException]() +* {java11-javadoc}/java.base/java/lang/NoSuchFieldException.html#(java.lang.String)[NoSuchFieldException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NoSuchMethodException]] +==== NoSuchMethodException +* {java11-javadoc}/java.base/java/lang/NoSuchMethodException.html#()[NoSuchMethodException]() +* {java11-javadoc}/java.base/java/lang/NoSuchMethodException.html#(java.lang.String)[NoSuchMethodException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NullPointerException]] +==== NullPointerException +* {java11-javadoc}/java.base/java/lang/NullPointerException.html#()[NullPointerException]() +* {java11-javadoc}/java.base/java/lang/NullPointerException.html#(java.lang.String)[NullPointerException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Number]] +==== Number +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NumberFormatException]] +==== NumberFormatException +* {java11-javadoc}/java.base/java/lang/NumberFormatException.html#()[NumberFormatException]() +* {java11-javadoc}/java.base/java/lang/NumberFormatException.html#(java.lang.String)[NumberFormatException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Object]] +==== Object +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ReflectiveOperationException]] +==== ReflectiveOperationException +* {java11-javadoc}/java.base/java/lang/ReflectiveOperationException.html#()[ReflectiveOperationException]() +* {java11-javadoc}/java.base/java/lang/ReflectiveOperationException.html#(java.lang.String)[ReflectiveOperationException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-RuntimeException]] +==== RuntimeException +* {java11-javadoc}/java.base/java/lang/RuntimeException.html#()[RuntimeException]() +* {java11-javadoc}/java.base/java/lang/RuntimeException.html#(java.lang.String)[RuntimeException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-SecurityException]] +==== SecurityException +* {java11-javadoc}/java.base/java/lang/SecurityException.html#()[SecurityException]() +* {java11-javadoc}/java.base/java/lang/SecurityException.html#(java.lang.String)[SecurityException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Short]] +==== Short +* static int {java11-javadoc}/java.base/java/lang/Short.html#BYTES[BYTES] +* static short {java11-javadoc}/java.base/java/lang/Short.html#MAX_VALUE[MAX_VALUE] +* static short {java11-javadoc}/java.base/java/lang/Short.html#MIN_VALUE[MIN_VALUE] +* static int {java11-javadoc}/java.base/java/lang/Short.html#SIZE[SIZE] +* static int {java11-javadoc}/java.base/java/lang/Short.html#compare(short,short)[compare](short, short) +* static Short {java11-javadoc}/java.base/java/lang/Short.html#decode(java.lang.String)[decode](String) +* static int {java11-javadoc}/java.base/java/lang/Short.html#hashCode(short)[hashCode](short) +* static short {java11-javadoc}/java.base/java/lang/Short.html#parseShort(java.lang.String)[parseShort](String) +* static short {java11-javadoc}/java.base/java/lang/Short.html#parseShort(java.lang.String,int)[parseShort](String, int) +* static short {java11-javadoc}/java.base/java/lang/Short.html#reverseBytes(short)[reverseBytes](short) +* static String {java11-javadoc}/java.base/java/lang/Short.html#toString(short)[toString](short) +* static int {java11-javadoc}/java.base/java/lang/Short.html#toUnsignedInt(short)[toUnsignedInt](short) +* static long {java11-javadoc}/java.base/java/lang/Short.html#toUnsignedLong(short)[toUnsignedLong](short) +* static Short {java11-javadoc}/java.base/java/lang/Short.html#valueOf(short)[valueOf](short) +* static Short {java11-javadoc}/java.base/java/lang/Short.html#valueOf(java.lang.String,int)[valueOf](String, int) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* int {java11-javadoc}/java.base/java/lang/Short.html#compareTo(java.lang.Short)[compareTo](Short) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-StackTraceElement]] +==== StackTraceElement +* {java11-javadoc}/java.base/java/lang/StackTraceElement.html#(java.lang.String,java.lang.String,java.lang.String,int)[StackTraceElement](String, String, String, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/StackTraceElement.html#getClassName()[getClassName]() +* String {java11-javadoc}/java.base/java/lang/StackTraceElement.html#getFileName()[getFileName]() +* int {java11-javadoc}/java.base/java/lang/StackTraceElement.html#getLineNumber()[getLineNumber]() +* String {java11-javadoc}/java.base/java/lang/StackTraceElement.html#getMethodName()[getMethodName]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/lang/StackTraceElement.html#isNativeMethod()[isNativeMethod]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-StrictMath]] +==== StrictMath +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#E[E] +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#PI[PI] +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#IEEEremainder(double,double)[IEEEremainder](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#abs(double)[abs](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#acos(double)[acos](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#asin(double)[asin](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#atan(double)[atan](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#atan2(double,double)[atan2](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#cbrt(double)[cbrt](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#ceil(double)[ceil](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#copySign(double,double)[copySign](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#cos(double)[cos](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#cosh(double)[cosh](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#exp(double)[exp](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#expm1(double)[expm1](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#floor(double)[floor](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#hypot(double,double)[hypot](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#log(double)[log](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#log10(double)[log10](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#log1p(double)[log1p](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#max(double,double)[max](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#min(double,double)[min](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#nextAfter(double,double)[nextAfter](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#nextDown(double)[nextDown](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#nextUp(double)[nextUp](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#pow(double,double)[pow](double, double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#random()[random]() +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#rint(double)[rint](double) +* static long {java11-javadoc}/java.base/java/lang/StrictMath.html#round(double)[round](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#scalb(double,int)[scalb](double, int) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#signum(double)[signum](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#sin(double)[sin](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#sinh(double)[sinh](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#sqrt(double)[sqrt](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#tan(double)[tan](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#tanh(double)[tanh](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#toDegrees(double)[toDegrees](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#toRadians(double)[toRadians](double) +* static double {java11-javadoc}/java.base/java/lang/StrictMath.html#ulp(double)[ulp](double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-String]] +==== String +* static String {java11-javadoc}/java.base/java/lang/String.html#copyValueOf(char%5B%5D)[copyValueOf](char[]) +* static String {java11-javadoc}/java.base/java/lang/String.html#copyValueOf(char%5B%5D,int,int)[copyValueOf](char[], int, int) +* static String {java11-javadoc}/java.base/java/lang/String.html#format(java.lang.String,java.lang.Object%5B%5D)[format](String, def[]) +* static String {java11-javadoc}/java.base/java/lang/String.html#format(java.util.Locale,java.lang.String,java.lang.Object%5B%5D)[format](Locale, String, def[]) +* static String {java11-javadoc}/java.base/java/lang/String.html#join(java.lang.CharSequence,java.lang.Iterable)[join](CharSequence, Iterable) +* static String {java11-javadoc}/java.base/java/lang/String.html#valueOf(java.lang.Object)[valueOf](def) +* {java11-javadoc}/java.base/java/lang/String.html#()[String]() +* char {java11-javadoc}/java.base/java/lang/CharSequence.html#charAt(int)[charAt](int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#chars()[chars]() +* int {java11-javadoc}/java.base/java/lang/String.html#codePointAt(int)[codePointAt](int) +* int {java11-javadoc}/java.base/java/lang/String.html#codePointBefore(int)[codePointBefore](int) +* int {java11-javadoc}/java.base/java/lang/String.html#codePointCount(int,int)[codePointCount](int, int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#codePoints()[codePoints]() +* int {java11-javadoc}/java.base/java/lang/String.html#compareTo(java.lang.String)[compareTo](String) +* int {java11-javadoc}/java.base/java/lang/String.html#compareToIgnoreCase(java.lang.String)[compareToIgnoreCase](String) +* String {java11-javadoc}/java.base/java/lang/String.html#concat(java.lang.String)[concat](String) +* boolean {java11-javadoc}/java.base/java/lang/String.html#contains(java.lang.CharSequence)[contains](CharSequence) +* boolean {java11-javadoc}/java.base/java/lang/String.html#contentEquals(java.lang.CharSequence)[contentEquals](CharSequence) +* String decodeBase64() +* String encodeBase64() +* boolean {java11-javadoc}/java.base/java/lang/String.html#endsWith(java.lang.String)[endsWith](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/lang/String.html#equalsIgnoreCase(java.lang.String)[equalsIgnoreCase](String) +* void {java11-javadoc}/java.base/java/lang/String.html#getChars(int,int,char%5B%5D,int)[getChars](int, int, char[], int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/String.html#indexOf(java.lang.String)[indexOf](String) +* int {java11-javadoc}/java.base/java/lang/String.html#indexOf(java.lang.String,int)[indexOf](String, int) +* boolean {java11-javadoc}/java.base/java/lang/String.html#isEmpty()[isEmpty]() +* int {java11-javadoc}/java.base/java/lang/String.html#lastIndexOf(java.lang.String)[lastIndexOf](String) +* int {java11-javadoc}/java.base/java/lang/String.html#lastIndexOf(java.lang.String,int)[lastIndexOf](String, int) +* int {java11-javadoc}/java.base/java/lang/CharSequence.html#length()[length]() +* int {java11-javadoc}/java.base/java/lang/String.html#offsetByCodePoints(int,int)[offsetByCodePoints](int, int) +* boolean {java11-javadoc}/java.base/java/lang/String.html#regionMatches(int,java.lang.String,int,int)[regionMatches](int, String, int, int) +* boolean {java11-javadoc}/java.base/java/lang/String.html#regionMatches(boolean,int,java.lang.String,int,int)[regionMatches](boolean, int, String, int, int) +* String {java11-javadoc}/java.base/java/lang/String.html#replace(java.lang.CharSequence,java.lang.CharSequence)[replace](CharSequence, CharSequence) +* String replaceAll(Pattern, Function) +* String replaceFirst(Pattern, Function) +* boolean {java11-javadoc}/java.base/java/lang/String.html#startsWith(java.lang.String)[startsWith](String) +* boolean {java11-javadoc}/java.base/java/lang/String.html#startsWith(java.lang.String,int)[startsWith](String, int) +* CharSequence {java11-javadoc}/java.base/java/lang/CharSequence.html#subSequence(int,int)[subSequence](int, int) +* String {java11-javadoc}/java.base/java/lang/String.html#substring(int)[substring](int) +* String {java11-javadoc}/java.base/java/lang/String.html#substring(int,int)[substring](int, int) +* char[] {java11-javadoc}/java.base/java/lang/String.html#toCharArray()[toCharArray]() +* String {java11-javadoc}/java.base/java/lang/String.html#toLowerCase()[toLowerCase]() +* String {java11-javadoc}/java.base/java/lang/String.html#toLowerCase(java.util.Locale)[toLowerCase](Locale) +* String {java11-javadoc}/java.base/java/lang/CharSequence.html#toString()[toString]() +* String {java11-javadoc}/java.base/java/lang/String.html#toUpperCase()[toUpperCase]() +* String {java11-javadoc}/java.base/java/lang/String.html#toUpperCase(java.util.Locale)[toUpperCase](Locale) +* String {java11-javadoc}/java.base/java/lang/String.html#trim()[trim]() + + +[[painless-api-reference-shared-StringBuffer]] +==== StringBuffer +* {java11-javadoc}/java.base/java/lang/StringBuffer.html#()[StringBuffer]() +* {java11-javadoc}/java.base/java/lang/StringBuffer.html#(java.lang.CharSequence)[StringBuffer](CharSequence) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#append(java.lang.Object)[append](def) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#append(java.lang.CharSequence,int,int)[append](CharSequence, int, int) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#appendCodePoint(int)[appendCodePoint](int) +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#capacity()[capacity]() +* char {java11-javadoc}/java.base/java/lang/CharSequence.html#charAt(int)[charAt](int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#chars()[chars]() +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#codePointAt(int)[codePointAt](int) +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#codePointBefore(int)[codePointBefore](int) +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#codePointCount(int,int)[codePointCount](int, int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#codePoints()[codePoints]() +* int {java11-javadoc}/java.base/java/lang/Comparable.html#compareTo(java.lang.Object)[compareTo](def) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#delete(int,int)[delete](int, int) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#deleteCharAt(int)[deleteCharAt](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/lang/StringBuffer.html#getChars(int,int,char%5B%5D,int)[getChars](int, int, char[], int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#indexOf(java.lang.String)[indexOf](String) +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#indexOf(java.lang.String,int)[indexOf](String, int) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#insert(int,java.lang.Object)[insert](int, def) +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#lastIndexOf(java.lang.String)[lastIndexOf](String) +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#lastIndexOf(java.lang.String,int)[lastIndexOf](String, int) +* int {java11-javadoc}/java.base/java/lang/CharSequence.html#length()[length]() +* int {java11-javadoc}/java.base/java/lang/StringBuffer.html#offsetByCodePoints(int,int)[offsetByCodePoints](int, int) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#replace(int,int,java.lang.String)[replace](int, int, String) +* String replaceAll(Pattern, Function) +* String replaceFirst(Pattern, Function) +* StringBuffer {java11-javadoc}/java.base/java/lang/StringBuffer.html#reverse()[reverse]() +* void {java11-javadoc}/java.base/java/lang/StringBuffer.html#setCharAt(int,char)[setCharAt](int, char) +* void {java11-javadoc}/java.base/java/lang/StringBuffer.html#setLength(int)[setLength](int) +* CharSequence {java11-javadoc}/java.base/java/lang/CharSequence.html#subSequence(int,int)[subSequence](int, int) +* String {java11-javadoc}/java.base/java/lang/StringBuffer.html#substring(int)[substring](int) +* String {java11-javadoc}/java.base/java/lang/StringBuffer.html#substring(int,int)[substring](int, int) +* String {java11-javadoc}/java.base/java/lang/CharSequence.html#toString()[toString]() + + +[[painless-api-reference-shared-StringBuilder]] +==== StringBuilder +* {java11-javadoc}/java.base/java/lang/StringBuilder.html#()[StringBuilder]() +* {java11-javadoc}/java.base/java/lang/StringBuilder.html#(java.lang.CharSequence)[StringBuilder](CharSequence) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#append(java.lang.Object)[append](def) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#append(java.lang.CharSequence,int,int)[append](CharSequence, int, int) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#appendCodePoint(int)[appendCodePoint](int) +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#capacity()[capacity]() +* char {java11-javadoc}/java.base/java/lang/CharSequence.html#charAt(int)[charAt](int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#chars()[chars]() +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#codePointAt(int)[codePointAt](int) +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#codePointBefore(int)[codePointBefore](int) +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#codePointCount(int,int)[codePointCount](int, int) +* IntStream {java11-javadoc}/java.base/java/lang/CharSequence.html#codePoints()[codePoints]() +* int {java11-javadoc}/java.base/java/lang/Comparable.html#compareTo(java.lang.Object)[compareTo](def) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#delete(int,int)[delete](int, int) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#deleteCharAt(int)[deleteCharAt](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/lang/StringBuilder.html#getChars(int,int,char%5B%5D,int)[getChars](int, int, char[], int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#indexOf(java.lang.String)[indexOf](String) +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#indexOf(java.lang.String,int)[indexOf](String, int) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#insert(int,java.lang.Object)[insert](int, def) +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#lastIndexOf(java.lang.String)[lastIndexOf](String) +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#lastIndexOf(java.lang.String,int)[lastIndexOf](String, int) +* int {java11-javadoc}/java.base/java/lang/CharSequence.html#length()[length]() +* int {java11-javadoc}/java.base/java/lang/StringBuilder.html#offsetByCodePoints(int,int)[offsetByCodePoints](int, int) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#replace(int,int,java.lang.String)[replace](int, int, String) +* String replaceAll(Pattern, Function) +* String replaceFirst(Pattern, Function) +* StringBuilder {java11-javadoc}/java.base/java/lang/StringBuilder.html#reverse()[reverse]() +* void {java11-javadoc}/java.base/java/lang/StringBuilder.html#setCharAt(int,char)[setCharAt](int, char) +* void {java11-javadoc}/java.base/java/lang/StringBuilder.html#setLength(int)[setLength](int) +* CharSequence {java11-javadoc}/java.base/java/lang/CharSequence.html#subSequence(int,int)[subSequence](int, int) +* String {java11-javadoc}/java.base/java/lang/StringBuilder.html#substring(int)[substring](int) +* String {java11-javadoc}/java.base/java/lang/StringBuilder.html#substring(int,int)[substring](int, int) +* String {java11-javadoc}/java.base/java/lang/CharSequence.html#toString()[toString]() + + +[[painless-api-reference-shared-StringIndexOutOfBoundsException]] +==== StringIndexOutOfBoundsException +* {java11-javadoc}/java.base/java/lang/StringIndexOutOfBoundsException.html#()[StringIndexOutOfBoundsException]() +* {java11-javadoc}/java.base/java/lang/StringIndexOutOfBoundsException.html#(java.lang.String)[StringIndexOutOfBoundsException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-System]] +==== System +* static void {java11-javadoc}/java.base/java/lang/System.html#arraycopy(java.lang.Object,int,java.lang.Object,int,int)[arraycopy](Object, int, Object, int, int) +* static long {java11-javadoc}/java.base/java/lang/System.html#currentTimeMillis()[currentTimeMillis]() +* static long {java11-javadoc}/java.base/java/lang/System.html#nanoTime()[nanoTime]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TypeNotPresentException]] +==== TypeNotPresentException +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* String {java11-javadoc}/java.base/java/lang/TypeNotPresentException.html#typeName()[typeName]() + + +[[painless-api-reference-shared-UnsupportedOperationException]] +==== UnsupportedOperationException +* {java11-javadoc}/java.base/java/lang/UnsupportedOperationException.html#()[UnsupportedOperationException]() +* {java11-javadoc}/java.base/java/lang/UnsupportedOperationException.html#(java.lang.String)[UnsupportedOperationException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Void]] +==== Void +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-math"] +=== Shared API for package java.math +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-BigDecimal]] +==== BigDecimal +* static BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#ONE[ONE] +* static BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#TEN[TEN] +* static BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#ZERO[ZERO] +* static BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#valueOf(double)[valueOf](double) +* {java11-javadoc}/java.base/java/math/BigDecimal.html#(java.lang.String)[BigDecimal](String) +* {java11-javadoc}/java.base/java/math/BigDecimal.html#(java.lang.String,java.math.MathContext)[BigDecimal](String, MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#abs()[abs]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#abs(java.math.MathContext)[abs](MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#add(java.math.BigDecimal)[add](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#add(java.math.BigDecimal,java.math.MathContext)[add](BigDecimal, MathContext) +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* byte {java11-javadoc}/java.base/java/math/BigDecimal.html#byteValueExact()[byteValueExact]() +* int {java11-javadoc}/java.base/java/math/BigDecimal.html#compareTo(java.math.BigDecimal)[compareTo](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#divide(java.math.BigDecimal)[divide](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#divide(java.math.BigDecimal,java.math.MathContext)[divide](BigDecimal, MathContext) +* BigDecimal[] {java11-javadoc}/java.base/java/math/BigDecimal.html#divideAndRemainder(java.math.BigDecimal)[divideAndRemainder](BigDecimal) +* BigDecimal[] {java11-javadoc}/java.base/java/math/BigDecimal.html#divideAndRemainder(java.math.BigDecimal,java.math.MathContext)[divideAndRemainder](BigDecimal, MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#divideToIntegralValue(java.math.BigDecimal)[divideToIntegralValue](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#divideToIntegralValue(java.math.BigDecimal,java.math.MathContext)[divideToIntegralValue](BigDecimal, MathContext) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* int {java11-javadoc}/java.base/java/math/BigDecimal.html#intValueExact()[intValueExact]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* long {java11-javadoc}/java.base/java/math/BigDecimal.html#longValueExact()[longValueExact]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#max(java.math.BigDecimal)[max](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#min(java.math.BigDecimal)[min](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#movePointLeft(int)[movePointLeft](int) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#movePointRight(int)[movePointRight](int) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#multiply(java.math.BigDecimal)[multiply](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#multiply(java.math.BigDecimal,java.math.MathContext)[multiply](BigDecimal, MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#negate()[negate]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#negate(java.math.MathContext)[negate](MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#plus()[plus]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#plus(java.math.MathContext)[plus](MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#pow(int)[pow](int) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#pow(int,java.math.MathContext)[pow](int, MathContext) +* int {java11-javadoc}/java.base/java/math/BigDecimal.html#precision()[precision]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#remainder(java.math.BigDecimal)[remainder](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#remainder(java.math.BigDecimal,java.math.MathContext)[remainder](BigDecimal, MathContext) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#round(java.math.MathContext)[round](MathContext) +* int {java11-javadoc}/java.base/java/math/BigDecimal.html#scale()[scale]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#scaleByPowerOfTen(int)[scaleByPowerOfTen](int) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#setScale(int)[setScale](int) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#setScale(int,java.math.RoundingMode)[setScale](int, RoundingMode) +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* short {java11-javadoc}/java.base/java/math/BigDecimal.html#shortValueExact()[shortValueExact]() +* int {java11-javadoc}/java.base/java/math/BigDecimal.html#signum()[signum]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#stripTrailingZeros()[stripTrailingZeros]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#subtract(java.math.BigDecimal)[subtract](BigDecimal) +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#subtract(java.math.BigDecimal,java.math.MathContext)[subtract](BigDecimal, MathContext) +* BigInteger {java11-javadoc}/java.base/java/math/BigDecimal.html#toBigInteger()[toBigInteger]() +* BigInteger {java11-javadoc}/java.base/java/math/BigDecimal.html#toBigIntegerExact()[toBigIntegerExact]() +* String {java11-javadoc}/java.base/java/math/BigDecimal.html#toEngineeringString()[toEngineeringString]() +* String {java11-javadoc}/java.base/java/math/BigDecimal.html#toPlainString()[toPlainString]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* BigDecimal {java11-javadoc}/java.base/java/math/BigDecimal.html#ulp()[ulp]() + + +[[painless-api-reference-shared-BigInteger]] +==== BigInteger +* static BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#ONE[ONE] +* static BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#TEN[TEN] +* static BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#ZERO[ZERO] +* static BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#valueOf(long)[valueOf](long) +* {java11-javadoc}/java.base/java/math/BigInteger.html#(java.lang.String)[BigInteger](String) +* {java11-javadoc}/java.base/java/math/BigInteger.html#(java.lang.String,int)[BigInteger](String, int) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#abs()[abs]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#add(java.math.BigInteger)[add](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#and(java.math.BigInteger)[and](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#andNot(java.math.BigInteger)[andNot](BigInteger) +* int {java11-javadoc}/java.base/java/math/BigInteger.html#bitCount()[bitCount]() +* int {java11-javadoc}/java.base/java/math/BigInteger.html#bitLength()[bitLength]() +* byte {java11-javadoc}/java.base/java/lang/Number.html#byteValue()[byteValue]() +* byte {java11-javadoc}/java.base/java/math/BigInteger.html#byteValueExact()[byteValueExact]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#clearBit(int)[clearBit](int) +* int {java11-javadoc}/java.base/java/math/BigInteger.html#compareTo(java.math.BigInteger)[compareTo](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#divide(java.math.BigInteger)[divide](BigInteger) +* BigInteger[] {java11-javadoc}/java.base/java/math/BigInteger.html#divideAndRemainder(java.math.BigInteger)[divideAndRemainder](BigInteger) +* double {java11-javadoc}/java.base/java/lang/Number.html#doubleValue()[doubleValue]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#flipBit(int)[flipBit](int) +* float {java11-javadoc}/java.base/java/lang/Number.html#floatValue()[floatValue]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#gcd(java.math.BigInteger)[gcd](BigInteger) +* int {java11-javadoc}/java.base/java/math/BigInteger.html#getLowestSetBit()[getLowestSetBit]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/lang/Number.html#intValue()[intValue]() +* int {java11-javadoc}/java.base/java/math/BigInteger.html#intValueExact()[intValueExact]() +* long {java11-javadoc}/java.base/java/lang/Number.html#longValue()[longValue]() +* long {java11-javadoc}/java.base/java/math/BigInteger.html#longValueExact()[longValueExact]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#max(java.math.BigInteger)[max](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#min(java.math.BigInteger)[min](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#mod(java.math.BigInteger)[mod](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#modInverse(java.math.BigInteger)[modInverse](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#modPow(java.math.BigInteger,java.math.BigInteger)[modPow](BigInteger, BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#multiply(java.math.BigInteger)[multiply](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#negate()[negate]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#not()[not]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#or(java.math.BigInteger)[or](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#pow(int)[pow](int) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#remainder(java.math.BigInteger)[remainder](BigInteger) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#setBit(int)[setBit](int) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#shiftLeft(int)[shiftLeft](int) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#shiftRight(int)[shiftRight](int) +* short {java11-javadoc}/java.base/java/lang/Number.html#shortValue()[shortValue]() +* short {java11-javadoc}/java.base/java/math/BigInteger.html#shortValueExact()[shortValueExact]() +* int {java11-javadoc}/java.base/java/math/BigInteger.html#signum()[signum]() +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#subtract(java.math.BigInteger)[subtract](BigInteger) +* boolean {java11-javadoc}/java.base/java/math/BigInteger.html#testBit(int)[testBit](int) +* byte[] {java11-javadoc}/java.base/java/math/BigInteger.html#toByteArray()[toByteArray]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* String {java11-javadoc}/java.base/java/math/BigInteger.html#toString(int)[toString](int) +* BigInteger {java11-javadoc}/java.base/java/math/BigInteger.html#xor(java.math.BigInteger)[xor](BigInteger) + + +[[painless-api-reference-shared-MathContext]] +==== MathContext +* static MathContext {java11-javadoc}/java.base/java/math/MathContext.html#DECIMAL128[DECIMAL128] +* static MathContext {java11-javadoc}/java.base/java/math/MathContext.html#DECIMAL32[DECIMAL32] +* static MathContext {java11-javadoc}/java.base/java/math/MathContext.html#DECIMAL64[DECIMAL64] +* static MathContext {java11-javadoc}/java.base/java/math/MathContext.html#UNLIMITED[UNLIMITED] +* {java11-javadoc}/java.base/java/math/MathContext.html#(int)[MathContext](int) +* {java11-javadoc}/java.base/java/math/MathContext.html#(int,java.math.RoundingMode)[MathContext](int, RoundingMode) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/math/MathContext.html#getPrecision()[getPrecision]() +* RoundingMode {java11-javadoc}/java.base/java/math/MathContext.html#getRoundingMode()[getRoundingMode]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-RoundingMode]] +==== RoundingMode +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#CEILING[CEILING] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#DOWN[DOWN] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#FLOOR[FLOOR] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#HALF_DOWN[HALF_DOWN] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#HALF_EVEN[HALF_EVEN] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#HALF_UP[HALF_UP] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#UNNECESSARY[UNNECESSARY] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#UP[UP] +* static RoundingMode {java11-javadoc}/java.base/java/math/RoundingMode.html#valueOf(java.lang.String)[valueOf](String) +* static RoundingMode[] {java11-javadoc}/java.base/java/math/RoundingMode.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-text"] +=== Shared API for package java.text +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-Annotation]] +==== Annotation +* {java11-javadoc}/java.base/java/text/Annotation.html#(java.lang.Object)[Annotation](Object) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* def {java11-javadoc}/java.base/java/text/Annotation.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AttributedCharacterIterator]] +==== AttributedCharacterIterator +* def {java11-javadoc}/java.base/java/text/CharacterIterator.html#clone()[clone]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#current()[current]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#first()[first]() +* Set {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getAllAttributeKeys()[getAllAttributeKeys]() +* def {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getAttribute(java.text.AttributedCharacterIterator$Attribute)[getAttribute](AttributedCharacterIterator.Attribute) +* Map {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getAttributes()[getAttributes]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getBeginIndex()[getBeginIndex]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getEndIndex()[getEndIndex]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getIndex()[getIndex]() +* int {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getRunLimit()[getRunLimit]() +* int {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getRunLimit(java.util.Set)[getRunLimit](Set) +* int {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getRunStart()[getRunStart]() +* int {java11-javadoc}/java.base/java/text/AttributedCharacterIterator.html#getRunStart(java.util.Set)[getRunStart](Set) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#last()[last]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#next()[next]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#previous()[previous]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#setIndex(int)[setIndex](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AttributedCharacterIterator-Attribute]] +==== AttributedCharacterIterator.Attribute +* static AttributedCharacterIterator.Attribute {java11-javadoc}/java.base/java/text/AttributedCharacterIterator$Attribute.html#INPUT_METHOD_SEGMENT[INPUT_METHOD_SEGMENT] +* static AttributedCharacterIterator.Attribute {java11-javadoc}/java.base/java/text/AttributedCharacterIterator$Attribute.html#LANGUAGE[LANGUAGE] +* static AttributedCharacterIterator.Attribute {java11-javadoc}/java.base/java/text/AttributedCharacterIterator$Attribute.html#READING[READING] +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AttributedString]] +==== AttributedString +* {java11-javadoc}/java.base/java/text/AttributedString.html#(java.lang.String)[AttributedString](String) +* {java11-javadoc}/java.base/java/text/AttributedString.html#(java.lang.String,java.util.Map)[AttributedString](String, Map) +* void {java11-javadoc}/java.base/java/text/AttributedString.html#addAttribute(java.text.AttributedCharacterIterator$Attribute,java.lang.Object)[addAttribute](AttributedCharacterIterator.Attribute, Object) +* void {java11-javadoc}/java.base/java/text/AttributedString.html#addAttribute(java.text.AttributedCharacterIterator$Attribute,java.lang.Object,int,int)[addAttribute](AttributedCharacterIterator.Attribute, Object, int, int) +* void {java11-javadoc}/java.base/java/text/AttributedString.html#addAttributes(java.util.Map,int,int)[addAttributes](Map, int, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/AttributedString.html#getIterator()[getIterator]() +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/AttributedString.html#getIterator(java.text.AttributedCharacterIterator$Attribute%5B%5D)[getIterator](AttributedCharacterIterator.Attribute[]) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/AttributedString.html#getIterator(java.text.AttributedCharacterIterator$Attribute%5B%5D,int,int)[getIterator](AttributedCharacterIterator.Attribute[], int, int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Bidi]] +==== Bidi +* static int {java11-javadoc}/java.base/java/text/Bidi.html#DIRECTION_DEFAULT_LEFT_TO_RIGHT[DIRECTION_DEFAULT_LEFT_TO_RIGHT] +* static int {java11-javadoc}/java.base/java/text/Bidi.html#DIRECTION_DEFAULT_RIGHT_TO_LEFT[DIRECTION_DEFAULT_RIGHT_TO_LEFT] +* static int {java11-javadoc}/java.base/java/text/Bidi.html#DIRECTION_LEFT_TO_RIGHT[DIRECTION_LEFT_TO_RIGHT] +* static int {java11-javadoc}/java.base/java/text/Bidi.html#DIRECTION_RIGHT_TO_LEFT[DIRECTION_RIGHT_TO_LEFT] +* static void {java11-javadoc}/java.base/java/text/Bidi.html#reorderVisually(byte%5B%5D,int,java.lang.Object%5B%5D,int,int)[reorderVisually](byte[], int, Object[], int, int) +* static boolean {java11-javadoc}/java.base/java/text/Bidi.html#requiresBidi(char%5B%5D,int,int)[requiresBidi](char[], int, int) +* {java11-javadoc}/java.base/java/text/Bidi.html#(java.text.AttributedCharacterIterator)[Bidi](AttributedCharacterIterator) +* {java11-javadoc}/java.base/java/text/Bidi.html#(java.lang.String,int)[Bidi](String, int) +* {java11-javadoc}/java.base/java/text/Bidi.html#(char%5B%5D,int,byte%5B%5D,int,int,int)[Bidi](char[], int, byte[], int, int, int) +* boolean {java11-javadoc}/java.base/java/text/Bidi.html#baseIsLeftToRight()[baseIsLeftToRight]() +* Bidi {java11-javadoc}/java.base/java/text/Bidi.html#createLineBidi(int,int)[createLineBidi](int, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/Bidi.html#getBaseLevel()[getBaseLevel]() +* int {java11-javadoc}/java.base/java/text/Bidi.html#getLength()[getLength]() +* int {java11-javadoc}/java.base/java/text/Bidi.html#getLevelAt(int)[getLevelAt](int) +* int {java11-javadoc}/java.base/java/text/Bidi.html#getRunCount()[getRunCount]() +* int {java11-javadoc}/java.base/java/text/Bidi.html#getRunLevel(int)[getRunLevel](int) +* int {java11-javadoc}/java.base/java/text/Bidi.html#getRunLimit(int)[getRunLimit](int) +* int {java11-javadoc}/java.base/java/text/Bidi.html#getRunStart(int)[getRunStart](int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/Bidi.html#isLeftToRight()[isLeftToRight]() +* boolean {java11-javadoc}/java.base/java/text/Bidi.html#isMixed()[isMixed]() +* boolean {java11-javadoc}/java.base/java/text/Bidi.html#isRightToLeft()[isRightToLeft]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-BreakIterator]] +==== BreakIterator +* static int {java11-javadoc}/java.base/java/text/BreakIterator.html#DONE[DONE] +* static Locale[] {java11-javadoc}/java.base/java/text/BreakIterator.html#getAvailableLocales()[getAvailableLocales]() +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getCharacterInstance()[getCharacterInstance]() +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getCharacterInstance(java.util.Locale)[getCharacterInstance](Locale) +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getLineInstance()[getLineInstance]() +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getLineInstance(java.util.Locale)[getLineInstance](Locale) +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getSentenceInstance()[getSentenceInstance]() +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getSentenceInstance(java.util.Locale)[getSentenceInstance](Locale) +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getWordInstance()[getWordInstance]() +* static BreakIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getWordInstance(java.util.Locale)[getWordInstance](Locale) +* def {java11-javadoc}/java.base/java/text/BreakIterator.html#clone()[clone]() +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#current()[current]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#first()[first]() +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#following(int)[following](int) +* CharacterIterator {java11-javadoc}/java.base/java/text/BreakIterator.html#getText()[getText]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/BreakIterator.html#isBoundary(int)[isBoundary](int) +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#last()[last]() +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#next()[next]() +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#next(int)[next](int) +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#preceding(int)[preceding](int) +* int {java11-javadoc}/java.base/java/text/BreakIterator.html#previous()[previous]() +* void {java11-javadoc}/java.base/java/text/BreakIterator.html#setText(java.lang.String)[setText](String) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-CharacterIterator]] +==== CharacterIterator +* static char {java11-javadoc}/java.base/java/text/CharacterIterator.html#DONE[DONE] +* def {java11-javadoc}/java.base/java/text/CharacterIterator.html#clone()[clone]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#current()[current]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#first()[first]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getBeginIndex()[getBeginIndex]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getEndIndex()[getEndIndex]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getIndex()[getIndex]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#last()[last]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#next()[next]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#previous()[previous]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#setIndex(int)[setIndex](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ChoiceFormat]] +==== ChoiceFormat +* static double {java11-javadoc}/java.base/java/text/ChoiceFormat.html#nextDouble(double)[nextDouble](double) +* static double {java11-javadoc}/java.base/java/text/ChoiceFormat.html#nextDouble(double,boolean)[nextDouble](double, boolean) +* static double {java11-javadoc}/java.base/java/text/ChoiceFormat.html#previousDouble(double)[previousDouble](double) +* {java11-javadoc}/java.base/java/text/ChoiceFormat.html#(java.lang.String)[ChoiceFormat](String) +* {java11-javadoc}/java.base/java/text/ChoiceFormat.html#(double%5B%5D,java.lang.String%5B%5D)[ChoiceFormat](double[], String[]) +* void {java11-javadoc}/java.base/java/text/ChoiceFormat.html#applyPattern(java.lang.String)[applyPattern](String) +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* Currency {java11-javadoc}/java.base/java/text/NumberFormat.html#getCurrency()[getCurrency]() +* def[] {java11-javadoc}/java.base/java/text/ChoiceFormat.html#getFormats()[getFormats]() +* double[] {java11-javadoc}/java.base/java/text/ChoiceFormat.html#getLimits()[getLimits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMaximumFractionDigits()[getMaximumFractionDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMaximumIntegerDigits()[getMaximumIntegerDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMinimumFractionDigits()[getMinimumFractionDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMinimumIntegerDigits()[getMinimumIntegerDigits]() +* RoundingMode {java11-javadoc}/java.base/java/text/NumberFormat.html#getRoundingMode()[getRoundingMode]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/NumberFormat.html#isGroupingUsed()[isGroupingUsed]() +* boolean {java11-javadoc}/java.base/java/text/NumberFormat.html#isParseIntegerOnly()[isParseIntegerOnly]() +* Number {java11-javadoc}/java.base/java/text/NumberFormat.html#parse(java.lang.String)[parse](String) +* Number {java11-javadoc}/java.base/java/text/NumberFormat.html#parse(java.lang.String,java.text.ParsePosition)[parse](String, ParsePosition) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* void {java11-javadoc}/java.base/java/text/ChoiceFormat.html#setChoices(double%5B%5D,java.lang.String%5B%5D)[setChoices](double[], String[]) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setCurrency(java.util.Currency)[setCurrency](Currency) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setGroupingUsed(boolean)[setGroupingUsed](boolean) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMaximumFractionDigits(int)[setMaximumFractionDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMaximumIntegerDigits(int)[setMaximumIntegerDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMinimumFractionDigits(int)[setMinimumFractionDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMinimumIntegerDigits(int)[setMinimumIntegerDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setParseIntegerOnly(boolean)[setParseIntegerOnly](boolean) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setRoundingMode(java.math.RoundingMode)[setRoundingMode](RoundingMode) +* String {java11-javadoc}/java.base/java/text/ChoiceFormat.html#toPattern()[toPattern]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-CollationElementIterator]] +==== CollationElementIterator +* static int {java11-javadoc}/java.base/java/text/CollationElementIterator.html#NULLORDER[NULLORDER] +* static int {java11-javadoc}/java.base/java/text/CollationElementIterator.html#primaryOrder(int)[primaryOrder](int) +* static short {java11-javadoc}/java.base/java/text/CollationElementIterator.html#secondaryOrder(int)[secondaryOrder](int) +* static short {java11-javadoc}/java.base/java/text/CollationElementIterator.html#tertiaryOrder(int)[tertiaryOrder](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/CollationElementIterator.html#getMaxExpansion(int)[getMaxExpansion](int) +* int {java11-javadoc}/java.base/java/text/CollationElementIterator.html#getOffset()[getOffset]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/text/CollationElementIterator.html#next()[next]() +* int {java11-javadoc}/java.base/java/text/CollationElementIterator.html#previous()[previous]() +* void {java11-javadoc}/java.base/java/text/CollationElementIterator.html#reset()[reset]() +* void {java11-javadoc}/java.base/java/text/CollationElementIterator.html#setOffset(int)[setOffset](int) +* void {java11-javadoc}/java.base/java/text/CollationElementIterator.html#setText(java.lang.String)[setText](String) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-CollationKey]] +==== CollationKey +* int {java11-javadoc}/java.base/java/text/CollationKey.html#compareTo(java.text.CollationKey)[compareTo](CollationKey) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/CollationKey.html#getSourceString()[getSourceString]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* byte[] {java11-javadoc}/java.base/java/text/CollationKey.html#toByteArray()[toByteArray]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Collator]] +==== Collator +* static int {java11-javadoc}/java.base/java/text/Collator.html#CANONICAL_DECOMPOSITION[CANONICAL_DECOMPOSITION] +* static int {java11-javadoc}/java.base/java/text/Collator.html#FULL_DECOMPOSITION[FULL_DECOMPOSITION] +* static int {java11-javadoc}/java.base/java/text/Collator.html#IDENTICAL[IDENTICAL] +* static int {java11-javadoc}/java.base/java/text/Collator.html#NO_DECOMPOSITION[NO_DECOMPOSITION] +* static int {java11-javadoc}/java.base/java/text/Collator.html#PRIMARY[PRIMARY] +* static int {java11-javadoc}/java.base/java/text/Collator.html#SECONDARY[SECONDARY] +* static int {java11-javadoc}/java.base/java/text/Collator.html#TERTIARY[TERTIARY] +* static Locale[] {java11-javadoc}/java.base/java/text/Collator.html#getAvailableLocales()[getAvailableLocales]() +* static Collator {java11-javadoc}/java.base/java/text/Collator.html#getInstance()[getInstance]() +* static Collator {java11-javadoc}/java.base/java/text/Collator.html#getInstance(java.util.Locale)[getInstance](Locale) +* def {java11-javadoc}/java.base/java/text/Collator.html#clone()[clone]() +* int {java11-javadoc}/java.base/java/util/Comparator.html#compare(java.lang.Object,java.lang.Object)[compare](def, def) +* boolean {java11-javadoc}/java.base/java/util/Comparator.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/text/Collator.html#equals(java.lang.String,java.lang.String)[equals](String, String) +* CollationKey {java11-javadoc}/java.base/java/text/Collator.html#getCollationKey(java.lang.String)[getCollationKey](String) +* int {java11-javadoc}/java.base/java/text/Collator.html#getDecomposition()[getDecomposition]() +* int {java11-javadoc}/java.base/java/text/Collator.html#getStrength()[getStrength]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#reversed()[reversed]() +* void {java11-javadoc}/java.base/java/text/Collator.html#setDecomposition(int)[setDecomposition](int) +* void {java11-javadoc}/java.base/java/text/Collator.html#setStrength(int)[setStrength](int) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparing(java.util.Comparator)[thenComparing](Comparator) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparing(java.util.function.Function,java.util.Comparator)[thenComparing](Function, Comparator) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingDouble(java.util.function.ToDoubleFunction)[thenComparingDouble](ToDoubleFunction) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingInt(java.util.function.ToIntFunction)[thenComparingInt](ToIntFunction) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingLong(java.util.function.ToLongFunction)[thenComparingLong](ToLongFunction) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DateFormat]] +==== DateFormat +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#AM_PM_FIELD[AM_PM_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#DATE_FIELD[DATE_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#DAY_OF_WEEK_FIELD[DAY_OF_WEEK_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#DAY_OF_WEEK_IN_MONTH_FIELD[DAY_OF_WEEK_IN_MONTH_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#DAY_OF_YEAR_FIELD[DAY_OF_YEAR_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#DEFAULT[DEFAULT] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#ERA_FIELD[ERA_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#FULL[FULL] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#HOUR0_FIELD[HOUR0_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#HOUR1_FIELD[HOUR1_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#HOUR_OF_DAY0_FIELD[HOUR_OF_DAY0_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#HOUR_OF_DAY1_FIELD[HOUR_OF_DAY1_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#LONG[LONG] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#MEDIUM[MEDIUM] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#MILLISECOND_FIELD[MILLISECOND_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#MINUTE_FIELD[MINUTE_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#MONTH_FIELD[MONTH_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#SECOND_FIELD[SECOND_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#SHORT[SHORT] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#TIMEZONE_FIELD[TIMEZONE_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#WEEK_OF_MONTH_FIELD[WEEK_OF_MONTH_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#WEEK_OF_YEAR_FIELD[WEEK_OF_YEAR_FIELD] +* static int {java11-javadoc}/java.base/java/text/DateFormat.html#YEAR_FIELD[YEAR_FIELD] +* static Locale[] {java11-javadoc}/java.base/java/text/DateFormat.html#getAvailableLocales()[getAvailableLocales]() +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getDateInstance()[getDateInstance]() +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getDateInstance(int)[getDateInstance](int) +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getDateInstance(int,java.util.Locale)[getDateInstance](int, Locale) +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getDateTimeInstance()[getDateTimeInstance]() +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getDateTimeInstance(int,int)[getDateTimeInstance](int, int) +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getDateTimeInstance(int,int,java.util.Locale)[getDateTimeInstance](int, int, Locale) +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getInstance()[getInstance]() +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getTimeInstance()[getTimeInstance]() +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getTimeInstance(int)[getTimeInstance](int) +* static DateFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getTimeInstance(int,java.util.Locale)[getTimeInstance](int, Locale) +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* Calendar {java11-javadoc}/java.base/java/text/DateFormat.html#getCalendar()[getCalendar]() +* NumberFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getNumberFormat()[getNumberFormat]() +* TimeZone {java11-javadoc}/java.base/java/text/DateFormat.html#getTimeZone()[getTimeZone]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/DateFormat.html#isLenient()[isLenient]() +* Date {java11-javadoc}/java.base/java/text/DateFormat.html#parse(java.lang.String)[parse](String) +* Date {java11-javadoc}/java.base/java/text/DateFormat.html#parse(java.lang.String,java.text.ParsePosition)[parse](String, ParsePosition) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setCalendar(java.util.Calendar)[setCalendar](Calendar) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setLenient(boolean)[setLenient](boolean) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setNumberFormat(java.text.NumberFormat)[setNumberFormat](NumberFormat) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setTimeZone(java.util.TimeZone)[setTimeZone](TimeZone) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DateFormat-Field]] +==== DateFormat.Field +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#AM_PM[AM_PM] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#DAY_OF_MONTH[DAY_OF_MONTH] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#DAY_OF_WEEK[DAY_OF_WEEK] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#DAY_OF_WEEK_IN_MONTH[DAY_OF_WEEK_IN_MONTH] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#DAY_OF_YEAR[DAY_OF_YEAR] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#ERA[ERA] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#HOUR0[HOUR0] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#HOUR1[HOUR1] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#HOUR_OF_DAY0[HOUR_OF_DAY0] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#HOUR_OF_DAY1[HOUR_OF_DAY1] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#MILLISECOND[MILLISECOND] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#MINUTE[MINUTE] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#MONTH[MONTH] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#SECOND[SECOND] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#TIME_ZONE[TIME_ZONE] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#WEEK_OF_MONTH[WEEK_OF_MONTH] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#WEEK_OF_YEAR[WEEK_OF_YEAR] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#YEAR[YEAR] +* static DateFormat.Field {java11-javadoc}/java.base/java/text/DateFormat$Field.html#ofCalendarField(int)[ofCalendarField](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/DateFormat$Field.html#getCalendarField()[getCalendarField]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DateFormatSymbols]] +==== DateFormatSymbols +* static Locale[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getAvailableLocales()[getAvailableLocales]() +* static DateFormatSymbols {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getInstance()[getInstance]() +* static DateFormatSymbols {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getInstance(java.util.Locale)[getInstance](Locale) +* {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#()[DateFormatSymbols]() +* {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#(java.util.Locale)[DateFormatSymbols](Locale) +* def {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getAmPmStrings()[getAmPmStrings]() +* String[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getEras()[getEras]() +* String {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getLocalPatternChars()[getLocalPatternChars]() +* String[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getMonths()[getMonths]() +* String[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getShortMonths()[getShortMonths]() +* String[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getShortWeekdays()[getShortWeekdays]() +* String[] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getWeekdays()[getWeekdays]() +* String[][] {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#getZoneStrings()[getZoneStrings]() +* int {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setAmPmStrings(java.lang.String%5B%5D)[setAmPmStrings](String[]) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setEras(java.lang.String%5B%5D)[setEras](String[]) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setLocalPatternChars(java.lang.String)[setLocalPatternChars](String) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setMonths(java.lang.String%5B%5D)[setMonths](String[]) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setShortMonths(java.lang.String%5B%5D)[setShortMonths](String[]) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setShortWeekdays(java.lang.String%5B%5D)[setShortWeekdays](String[]) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setWeekdays(java.lang.String%5B%5D)[setWeekdays](String[]) +* void {java11-javadoc}/java.base/java/text/DateFormatSymbols.html#setZoneStrings(java.lang.String%5B%5D%5B%5D)[setZoneStrings](String[][]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DecimalFormat]] +==== DecimalFormat +* {java11-javadoc}/java.base/java/text/DecimalFormat.html#()[DecimalFormat]() +* {java11-javadoc}/java.base/java/text/DecimalFormat.html#(java.lang.String)[DecimalFormat](String) +* {java11-javadoc}/java.base/java/text/DecimalFormat.html#(java.lang.String,java.text.DecimalFormatSymbols)[DecimalFormat](String, DecimalFormatSymbols) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#applyLocalizedPattern(java.lang.String)[applyLocalizedPattern](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#applyPattern(java.lang.String)[applyPattern](String) +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* Currency {java11-javadoc}/java.base/java/text/NumberFormat.html#getCurrency()[getCurrency]() +* DecimalFormatSymbols {java11-javadoc}/java.base/java/text/DecimalFormat.html#getDecimalFormatSymbols()[getDecimalFormatSymbols]() +* int {java11-javadoc}/java.base/java/text/DecimalFormat.html#getGroupingSize()[getGroupingSize]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMaximumFractionDigits()[getMaximumFractionDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMaximumIntegerDigits()[getMaximumIntegerDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMinimumFractionDigits()[getMinimumFractionDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMinimumIntegerDigits()[getMinimumIntegerDigits]() +* int {java11-javadoc}/java.base/java/text/DecimalFormat.html#getMultiplier()[getMultiplier]() +* String {java11-javadoc}/java.base/java/text/DecimalFormat.html#getNegativePrefix()[getNegativePrefix]() +* String {java11-javadoc}/java.base/java/text/DecimalFormat.html#getNegativeSuffix()[getNegativeSuffix]() +* String {java11-javadoc}/java.base/java/text/DecimalFormat.html#getPositivePrefix()[getPositivePrefix]() +* String {java11-javadoc}/java.base/java/text/DecimalFormat.html#getPositiveSuffix()[getPositiveSuffix]() +* RoundingMode {java11-javadoc}/java.base/java/text/NumberFormat.html#getRoundingMode()[getRoundingMode]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/DecimalFormat.html#isDecimalSeparatorAlwaysShown()[isDecimalSeparatorAlwaysShown]() +* boolean {java11-javadoc}/java.base/java/text/NumberFormat.html#isGroupingUsed()[isGroupingUsed]() +* boolean {java11-javadoc}/java.base/java/text/DecimalFormat.html#isParseBigDecimal()[isParseBigDecimal]() +* boolean {java11-javadoc}/java.base/java/text/NumberFormat.html#isParseIntegerOnly()[isParseIntegerOnly]() +* Number {java11-javadoc}/java.base/java/text/NumberFormat.html#parse(java.lang.String)[parse](String) +* Number {java11-javadoc}/java.base/java/text/NumberFormat.html#parse(java.lang.String,java.text.ParsePosition)[parse](String, ParsePosition) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setCurrency(java.util.Currency)[setCurrency](Currency) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setDecimalFormatSymbols(java.text.DecimalFormatSymbols)[setDecimalFormatSymbols](DecimalFormatSymbols) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setDecimalSeparatorAlwaysShown(boolean)[setDecimalSeparatorAlwaysShown](boolean) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setGroupingSize(int)[setGroupingSize](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setGroupingUsed(boolean)[setGroupingUsed](boolean) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMaximumFractionDigits(int)[setMaximumFractionDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMaximumIntegerDigits(int)[setMaximumIntegerDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMinimumFractionDigits(int)[setMinimumFractionDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMinimumIntegerDigits(int)[setMinimumIntegerDigits](int) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setMultiplier(int)[setMultiplier](int) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setNegativePrefix(java.lang.String)[setNegativePrefix](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setNegativeSuffix(java.lang.String)[setNegativeSuffix](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setParseBigDecimal(boolean)[setParseBigDecimal](boolean) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setParseIntegerOnly(boolean)[setParseIntegerOnly](boolean) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setPositivePrefix(java.lang.String)[setPositivePrefix](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormat.html#setPositiveSuffix(java.lang.String)[setPositiveSuffix](String) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setRoundingMode(java.math.RoundingMode)[setRoundingMode](RoundingMode) +* String {java11-javadoc}/java.base/java/text/DecimalFormat.html#toLocalizedPattern()[toLocalizedPattern]() +* String {java11-javadoc}/java.base/java/text/DecimalFormat.html#toPattern()[toPattern]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DecimalFormatSymbols]] +==== DecimalFormatSymbols +* static Locale[] {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getAvailableLocales()[getAvailableLocales]() +* static DecimalFormatSymbols {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getInstance()[getInstance]() +* static DecimalFormatSymbols {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getInstance(java.util.Locale)[getInstance](Locale) +* {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#()[DecimalFormatSymbols]() +* {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#(java.util.Locale)[DecimalFormatSymbols](Locale) +* def {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Currency {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getCurrency()[getCurrency]() +* String {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getCurrencySymbol()[getCurrencySymbol]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getDecimalSeparator()[getDecimalSeparator]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getDigit()[getDigit]() +* String {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getExponentSeparator()[getExponentSeparator]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getGroupingSeparator()[getGroupingSeparator]() +* String {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getInfinity()[getInfinity]() +* String {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getInternationalCurrencySymbol()[getInternationalCurrencySymbol]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getMinusSign()[getMinusSign]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getMonetaryDecimalSeparator()[getMonetaryDecimalSeparator]() +* String {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getNaN()[getNaN]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getPatternSeparator()[getPatternSeparator]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getPerMill()[getPerMill]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getPercent()[getPercent]() +* char {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#getZeroDigit()[getZeroDigit]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setCurrency(java.util.Currency)[setCurrency](Currency) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setCurrencySymbol(java.lang.String)[setCurrencySymbol](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setDecimalSeparator(char)[setDecimalSeparator](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setDigit(char)[setDigit](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setExponentSeparator(java.lang.String)[setExponentSeparator](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setGroupingSeparator(char)[setGroupingSeparator](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setInfinity(java.lang.String)[setInfinity](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setInternationalCurrencySymbol(java.lang.String)[setInternationalCurrencySymbol](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setMinusSign(char)[setMinusSign](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setMonetaryDecimalSeparator(char)[setMonetaryDecimalSeparator](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setNaN(java.lang.String)[setNaN](String) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setPatternSeparator(char)[setPatternSeparator](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setPerMill(char)[setPerMill](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setPercent(char)[setPercent](char) +* void {java11-javadoc}/java.base/java/text/DecimalFormatSymbols.html#setZeroDigit(char)[setZeroDigit](char) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-FieldPosition]] +==== FieldPosition +* {java11-javadoc}/java.base/java/text/FieldPosition.html#(int)[FieldPosition](int) +* {java11-javadoc}/java.base/java/text/FieldPosition.html#(java.text.Format$Field,int)[FieldPosition](Format.Field, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/FieldPosition.html#getBeginIndex()[getBeginIndex]() +* int {java11-javadoc}/java.base/java/text/FieldPosition.html#getEndIndex()[getEndIndex]() +* int {java11-javadoc}/java.base/java/text/FieldPosition.html#getField()[getField]() +* Format.Field {java11-javadoc}/java.base/java/text/FieldPosition.html#getFieldAttribute()[getFieldAttribute]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/text/FieldPosition.html#setBeginIndex(int)[setBeginIndex](int) +* void {java11-javadoc}/java.base/java/text/FieldPosition.html#setEndIndex(int)[setEndIndex](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Format]] +==== Format +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Format-Field]] +==== Format.Field +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MessageFormat]] +==== MessageFormat +* static String {java11-javadoc}/java.base/java/text/MessageFormat.html#format(java.lang.String,java.lang.Object%5B%5D)[format](String, Object[]) +* void {java11-javadoc}/java.base/java/text/MessageFormat.html#applyPattern(java.lang.String)[applyPattern](String) +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* Format[] {java11-javadoc}/java.base/java/text/MessageFormat.html#getFormats()[getFormats]() +* Format[] {java11-javadoc}/java.base/java/text/MessageFormat.html#getFormatsByArgumentIndex()[getFormatsByArgumentIndex]() +* Locale {java11-javadoc}/java.base/java/text/MessageFormat.html#getLocale()[getLocale]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Object[] {java11-javadoc}/java.base/java/text/MessageFormat.html#parse(java.lang.String)[parse](String) +* Object[] {java11-javadoc}/java.base/java/text/MessageFormat.html#parse(java.lang.String,java.text.ParsePosition)[parse](String, ParsePosition) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* void {java11-javadoc}/java.base/java/text/MessageFormat.html#setFormat(int,java.text.Format)[setFormat](int, Format) +* void {java11-javadoc}/java.base/java/text/MessageFormat.html#setFormatByArgumentIndex(int,java.text.Format)[setFormatByArgumentIndex](int, Format) +* void {java11-javadoc}/java.base/java/text/MessageFormat.html#setFormats(java.text.Format%5B%5D)[setFormats](Format[]) +* void {java11-javadoc}/java.base/java/text/MessageFormat.html#setFormatsByArgumentIndex(java.text.Format%5B%5D)[setFormatsByArgumentIndex](Format[]) +* void {java11-javadoc}/java.base/java/text/MessageFormat.html#setLocale(java.util.Locale)[setLocale](Locale) +* String {java11-javadoc}/java.base/java/text/MessageFormat.html#toPattern()[toPattern]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MessageFormat-Field]] +==== MessageFormat.Field +* static MessageFormat.Field {java11-javadoc}/java.base/java/text/MessageFormat$Field.html#ARGUMENT[ARGUMENT] +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Normalizer]] +==== Normalizer +* static boolean {java11-javadoc}/java.base/java/text/Normalizer.html#isNormalized(java.lang.CharSequence,java.text.Normalizer$Form)[isNormalized](CharSequence, Normalizer.Form) +* static String {java11-javadoc}/java.base/java/text/Normalizer.html#normalize(java.lang.CharSequence,java.text.Normalizer$Form)[normalize](CharSequence, Normalizer.Form) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Normalizer-Form]] +==== Normalizer.Form +* static Normalizer.Form {java11-javadoc}/java.base/java/text/Normalizer$Form.html#NFC[NFC] +* static Normalizer.Form {java11-javadoc}/java.base/java/text/Normalizer$Form.html#NFD[NFD] +* static Normalizer.Form {java11-javadoc}/java.base/java/text/Normalizer$Form.html#NFKC[NFKC] +* static Normalizer.Form {java11-javadoc}/java.base/java/text/Normalizer$Form.html#NFKD[NFKD] +* static Normalizer.Form {java11-javadoc}/java.base/java/text/Normalizer$Form.html#valueOf(java.lang.String)[valueOf](String) +* static Normalizer.Form[] {java11-javadoc}/java.base/java/text/Normalizer$Form.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NumberFormat]] +==== NumberFormat +* static int {java11-javadoc}/java.base/java/text/NumberFormat.html#FRACTION_FIELD[FRACTION_FIELD] +* static int {java11-javadoc}/java.base/java/text/NumberFormat.html#INTEGER_FIELD[INTEGER_FIELD] +* static Locale[] {java11-javadoc}/java.base/java/text/NumberFormat.html#getAvailableLocales()[getAvailableLocales]() +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getCurrencyInstance()[getCurrencyInstance]() +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getCurrencyInstance(java.util.Locale)[getCurrencyInstance](Locale) +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getInstance()[getInstance]() +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getInstance(java.util.Locale)[getInstance](Locale) +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getIntegerInstance()[getIntegerInstance]() +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getIntegerInstance(java.util.Locale)[getIntegerInstance](Locale) +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getNumberInstance()[getNumberInstance]() +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getNumberInstance(java.util.Locale)[getNumberInstance](Locale) +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getPercentInstance()[getPercentInstance]() +* static NumberFormat {java11-javadoc}/java.base/java/text/NumberFormat.html#getPercentInstance(java.util.Locale)[getPercentInstance](Locale) +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* Currency {java11-javadoc}/java.base/java/text/NumberFormat.html#getCurrency()[getCurrency]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMaximumFractionDigits()[getMaximumFractionDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMaximumIntegerDigits()[getMaximumIntegerDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMinimumFractionDigits()[getMinimumFractionDigits]() +* int {java11-javadoc}/java.base/java/text/NumberFormat.html#getMinimumIntegerDigits()[getMinimumIntegerDigits]() +* RoundingMode {java11-javadoc}/java.base/java/text/NumberFormat.html#getRoundingMode()[getRoundingMode]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/NumberFormat.html#isGroupingUsed()[isGroupingUsed]() +* boolean {java11-javadoc}/java.base/java/text/NumberFormat.html#isParseIntegerOnly()[isParseIntegerOnly]() +* Number {java11-javadoc}/java.base/java/text/NumberFormat.html#parse(java.lang.String)[parse](String) +* Number {java11-javadoc}/java.base/java/text/NumberFormat.html#parse(java.lang.String,java.text.ParsePosition)[parse](String, ParsePosition) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setCurrency(java.util.Currency)[setCurrency](Currency) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setGroupingUsed(boolean)[setGroupingUsed](boolean) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMaximumFractionDigits(int)[setMaximumFractionDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMaximumIntegerDigits(int)[setMaximumIntegerDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMinimumFractionDigits(int)[setMinimumFractionDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setMinimumIntegerDigits(int)[setMinimumIntegerDigits](int) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setParseIntegerOnly(boolean)[setParseIntegerOnly](boolean) +* void {java11-javadoc}/java.base/java/text/NumberFormat.html#setRoundingMode(java.math.RoundingMode)[setRoundingMode](RoundingMode) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NumberFormat-Field]] +==== NumberFormat.Field +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#CURRENCY[CURRENCY] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#DECIMAL_SEPARATOR[DECIMAL_SEPARATOR] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#EXPONENT[EXPONENT] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#EXPONENT_SIGN[EXPONENT_SIGN] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#EXPONENT_SYMBOL[EXPONENT_SYMBOL] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#FRACTION[FRACTION] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#GROUPING_SEPARATOR[GROUPING_SEPARATOR] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#INTEGER[INTEGER] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#PERCENT[PERCENT] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#PERMILLE[PERMILLE] +* static NumberFormat.Field {java11-javadoc}/java.base/java/text/NumberFormat$Field.html#SIGN[SIGN] +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ParseException]] +==== ParseException +* {java11-javadoc}/java.base/java/text/ParseException.html#(java.lang.String,int)[ParseException](String, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/ParseException.html#getErrorOffset()[getErrorOffset]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ParsePosition]] +==== ParsePosition +* {java11-javadoc}/java.base/java/text/ParsePosition.html#(int)[ParsePosition](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/text/ParsePosition.html#getErrorIndex()[getErrorIndex]() +* int {java11-javadoc}/java.base/java/text/ParsePosition.html#getIndex()[getIndex]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/text/ParsePosition.html#setErrorIndex(int)[setErrorIndex](int) +* void {java11-javadoc}/java.base/java/text/ParsePosition.html#setIndex(int)[setIndex](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-RuleBasedCollator]] +==== RuleBasedCollator +* {java11-javadoc}/java.base/java/text/RuleBasedCollator.html#(java.lang.String)[RuleBasedCollator](String) +* def {java11-javadoc}/java.base/java/text/Collator.html#clone()[clone]() +* int {java11-javadoc}/java.base/java/util/Comparator.html#compare(java.lang.Object,java.lang.Object)[compare](def, def) +* boolean {java11-javadoc}/java.base/java/util/Comparator.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/text/Collator.html#equals(java.lang.String,java.lang.String)[equals](String, String) +* CollationElementIterator {java11-javadoc}/java.base/java/text/RuleBasedCollator.html#getCollationElementIterator(java.lang.String)[getCollationElementIterator](String) +* CollationKey {java11-javadoc}/java.base/java/text/Collator.html#getCollationKey(java.lang.String)[getCollationKey](String) +* int {java11-javadoc}/java.base/java/text/Collator.html#getDecomposition()[getDecomposition]() +* String {java11-javadoc}/java.base/java/text/RuleBasedCollator.html#getRules()[getRules]() +* int {java11-javadoc}/java.base/java/text/Collator.html#getStrength()[getStrength]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#reversed()[reversed]() +* void {java11-javadoc}/java.base/java/text/Collator.html#setDecomposition(int)[setDecomposition](int) +* void {java11-javadoc}/java.base/java/text/Collator.html#setStrength(int)[setStrength](int) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparing(java.util.Comparator)[thenComparing](Comparator) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparing(java.util.function.Function,java.util.Comparator)[thenComparing](Function, Comparator) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingDouble(java.util.function.ToDoubleFunction)[thenComparingDouble](ToDoubleFunction) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingInt(java.util.function.ToIntFunction)[thenComparingInt](ToIntFunction) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingLong(java.util.function.ToLongFunction)[thenComparingLong](ToLongFunction) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-SimpleDateFormat]] +==== SimpleDateFormat +* {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#()[SimpleDateFormat]() +* {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#(java.lang.String)[SimpleDateFormat](String) +* {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#(java.lang.String,java.util.Locale)[SimpleDateFormat](String, Locale) +* void {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#applyLocalizedPattern(java.lang.String)[applyLocalizedPattern](String) +* void {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#applyPattern(java.lang.String)[applyPattern](String) +* def {java11-javadoc}/java.base/java/text/Format.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object)[format](Object) +* StringBuffer {java11-javadoc}/java.base/java/text/Format.html#format(java.lang.Object,java.lang.StringBuffer,java.text.FieldPosition)[format](Object, StringBuffer, FieldPosition) +* AttributedCharacterIterator {java11-javadoc}/java.base/java/text/Format.html#formatToCharacterIterator(java.lang.Object)[formatToCharacterIterator](Object) +* Date {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#get2DigitYearStart()[get2DigitYearStart]() +* Calendar {java11-javadoc}/java.base/java/text/DateFormat.html#getCalendar()[getCalendar]() +* DateFormatSymbols {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#getDateFormatSymbols()[getDateFormatSymbols]() +* NumberFormat {java11-javadoc}/java.base/java/text/DateFormat.html#getNumberFormat()[getNumberFormat]() +* TimeZone {java11-javadoc}/java.base/java/text/DateFormat.html#getTimeZone()[getTimeZone]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/text/DateFormat.html#isLenient()[isLenient]() +* Date {java11-javadoc}/java.base/java/text/DateFormat.html#parse(java.lang.String)[parse](String) +* Date {java11-javadoc}/java.base/java/text/DateFormat.html#parse(java.lang.String,java.text.ParsePosition)[parse](String, ParsePosition) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String)[parseObject](String) +* Object {java11-javadoc}/java.base/java/text/Format.html#parseObject(java.lang.String,java.text.ParsePosition)[parseObject](String, ParsePosition) +* void {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#set2DigitYearStart(java.util.Date)[set2DigitYearStart](Date) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setCalendar(java.util.Calendar)[setCalendar](Calendar) +* void {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#setDateFormatSymbols(java.text.DateFormatSymbols)[setDateFormatSymbols](DateFormatSymbols) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setLenient(boolean)[setLenient](boolean) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setNumberFormat(java.text.NumberFormat)[setNumberFormat](NumberFormat) +* void {java11-javadoc}/java.base/java/text/DateFormat.html#setTimeZone(java.util.TimeZone)[setTimeZone](TimeZone) +* String {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#toLocalizedPattern()[toLocalizedPattern]() +* String {java11-javadoc}/java.base/java/text/SimpleDateFormat.html#toPattern()[toPattern]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-StringCharacterIterator]] +==== StringCharacterIterator +* {java11-javadoc}/java.base/java/text/StringCharacterIterator.html#(java.lang.String)[StringCharacterIterator](String) +* {java11-javadoc}/java.base/java/text/StringCharacterIterator.html#(java.lang.String,int)[StringCharacterIterator](String, int) +* {java11-javadoc}/java.base/java/text/StringCharacterIterator.html#(java.lang.String,int,int,int)[StringCharacterIterator](String, int, int, int) +* def {java11-javadoc}/java.base/java/text/CharacterIterator.html#clone()[clone]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#current()[current]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#first()[first]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getBeginIndex()[getBeginIndex]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getEndIndex()[getEndIndex]() +* int {java11-javadoc}/java.base/java/text/CharacterIterator.html#getIndex()[getIndex]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#last()[last]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#next()[next]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#previous()[previous]() +* char {java11-javadoc}/java.base/java/text/CharacterIterator.html#setIndex(int)[setIndex](int) +* void {java11-javadoc}/java.base/java/text/StringCharacterIterator.html#setText(java.lang.String)[setText](String) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-time"] +=== Shared API for package java.time +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-Clock]] +==== Clock +* static Clock {java11-javadoc}/java.base/java/time/Clock.html#fixed(java.time.Instant,java.time.ZoneId)[fixed](Instant, ZoneId) +* static Clock {java11-javadoc}/java.base/java/time/Clock.html#offset(java.time.Clock,java.time.Duration)[offset](Clock, Duration) +* static Clock {java11-javadoc}/java.base/java/time/Clock.html#tick(java.time.Clock,java.time.Duration)[tick](Clock, Duration) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* ZoneId {java11-javadoc}/java.base/java/time/Clock.html#getZone()[getZone]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Instant {java11-javadoc}/java.base/java/time/Clock.html#instant()[instant]() +* long {java11-javadoc}/java.base/java/time/Clock.html#millis()[millis]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DateTimeException]] +==== DateTimeException +* {java11-javadoc}/java.base/java/time/DateTimeException.html#(java.lang.String)[DateTimeException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DayOfWeek]] +==== DayOfWeek +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#FRIDAY[FRIDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#MONDAY[MONDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#SATURDAY[SATURDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#SUNDAY[SUNDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#THURSDAY[THURSDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#TUESDAY[TUESDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#WEDNESDAY[WEDNESDAY] +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#of(int)[of](int) +* static DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#valueOf(java.lang.String)[valueOf](String) +* static DayOfWeek[] {java11-javadoc}/java.base/java/time/DayOfWeek.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/DayOfWeek.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/DayOfWeek.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#minus(long)[minus](long) +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* DayOfWeek {java11-javadoc}/java.base/java/time/DayOfWeek.html#plus(long)[plus](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Duration]] +==== Duration +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ZERO[ZERO] +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#between(java.time.temporal.Temporal,java.time.temporal.Temporal)[between](Temporal, Temporal) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#from(java.time.temporal.TemporalAmount)[from](TemporalAmount) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#of(long,java.time.temporal.TemporalUnit)[of](long, TemporalUnit) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofDays(long)[ofDays](long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofHours(long)[ofHours](long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofMillis(long)[ofMillis](long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofMinutes(long)[ofMinutes](long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofNanos(long)[ofNanos](long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofSeconds(long)[ofSeconds](long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#ofSeconds(long,long)[ofSeconds](long, long) +* static Duration {java11-javadoc}/java.base/java/time/Duration.html#parse(java.lang.CharSequence)[parse](CharSequence) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#abs()[abs]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#addTo(java.time.temporal.Temporal)[addTo](Temporal) +* int {java11-javadoc}/java.base/java/time/Duration.html#compareTo(java.time.Duration)[compareTo](Duration) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#dividedBy(long)[dividedBy](long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#get(java.time.temporal.TemporalUnit)[get](TemporalUnit) +* int {java11-javadoc}/java.base/java/time/Duration.html#getNano()[getNano]() +* long {java11-javadoc}/java.base/java/time/Duration.html#getSeconds()[getSeconds]() +* List {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#getUnits()[getUnits]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/Duration.html#isNegative()[isNegative]() +* boolean {java11-javadoc}/java.base/java/time/Duration.html#isZero()[isZero]() +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minus(java.time.Duration)[minus](Duration) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minusDays(long)[minusDays](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minusHours(long)[minusHours](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minusMillis(long)[minusMillis](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minusMinutes(long)[minusMinutes](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minusNanos(long)[minusNanos](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#minusSeconds(long)[minusSeconds](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#multipliedBy(long)[multipliedBy](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#negated()[negated]() +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plus(java.time.Duration)[plus](Duration) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plusDays(long)[plusDays](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plusHours(long)[plusHours](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plusMillis(long)[plusMillis](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plusMinutes(long)[plusMinutes](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plusNanos(long)[plusNanos](long) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#plusSeconds(long)[plusSeconds](long) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#subtractFrom(java.time.temporal.Temporal)[subtractFrom](Temporal) +* long {java11-javadoc}/java.base/java/time/Duration.html#toDays()[toDays]() +* long {java11-javadoc}/java.base/java/time/Duration.html#toHours()[toHours]() +* long {java11-javadoc}/java.base/java/time/Duration.html#toMillis()[toMillis]() +* long {java11-javadoc}/java.base/java/time/Duration.html#toMinutes()[toMinutes]() +* long {java11-javadoc}/java.base/java/time/Duration.html#toNanos()[toNanos]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Duration {java11-javadoc}/java.base/java/time/Duration.html#withNanos(int)[withNanos](int) +* Duration {java11-javadoc}/java.base/java/time/Duration.html#withSeconds(long)[withSeconds](long) + + +[[painless-api-reference-shared-Instant]] +==== Instant +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#EPOCH[EPOCH] +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#MAX[MAX] +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#MIN[MIN] +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#ofEpochMilli(long)[ofEpochMilli](long) +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#ofEpochSecond(long)[ofEpochSecond](long) +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#ofEpochSecond(long,long)[ofEpochSecond](long, long) +* static Instant {java11-javadoc}/java.base/java/time/Instant.html#parse(java.lang.CharSequence)[parse](CharSequence) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* OffsetDateTime {java11-javadoc}/java.base/java/time/Instant.html#atOffset(java.time.ZoneOffset)[atOffset](ZoneOffset) +* ZonedDateTime {java11-javadoc}/java.base/java/time/Instant.html#atZone(java.time.ZoneId)[atZone](ZoneId) +* int {java11-javadoc}/java.base/java/time/Instant.html#compareTo(java.time.Instant)[compareTo](Instant) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* long {java11-javadoc}/java.base/java/time/Instant.html#getEpochSecond()[getEpochSecond]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/Instant.html#getNano()[getNano]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/Instant.html#isAfter(java.time.Instant)[isAfter](Instant) +* boolean {java11-javadoc}/java.base/java/time/Instant.html#isBefore(java.time.Instant)[isBefore](Instant) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#minusMillis(long)[minusMillis](long) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#minusNanos(long)[minusNanos](long) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#minusSeconds(long)[minusSeconds](long) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#plusMillis(long)[plusMillis](long) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#plusNanos(long)[plusNanos](long) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#plusSeconds(long)[plusSeconds](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/Instant.html#toEpochMilli()[toEpochMilli]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Instant {java11-javadoc}/java.base/java/time/Instant.html#truncatedTo(java.time.temporal.TemporalUnit)[truncatedTo](TemporalUnit) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* Instant {java11-javadoc}/java.base/java/time/Instant.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-LocalDate]] +==== LocalDate +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#MAX[MAX] +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#MIN[MIN] +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#of(int,int,int)[of](int, int, int) +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#ofEpochDay(long)[ofEpochDay](long) +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#ofYearDay(int,int)[ofYearDay](int, int) +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDate.html#atStartOfDay()[atStartOfDay]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/LocalDate.html#atStartOfDay(java.time.ZoneId)[atStartOfDay](ZoneId) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDate.html#atTime(java.time.LocalTime)[atTime](LocalTime) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDate.html#atTime(int,int)[atTime](int, int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDate.html#atTime(int,int,int)[atTime](int, int, int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDate.html#atTime(int,int,int,int)[atTime](int, int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#compareTo(java.time.chrono.ChronoLocalDate)[compareTo](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* IsoChronology {java11-javadoc}/java.base/java/time/LocalDate.html#getChronology()[getChronology]() +* int {java11-javadoc}/java.base/java/time/LocalDate.html#getDayOfMonth()[getDayOfMonth]() +* DayOfWeek {java11-javadoc}/java.base/java/time/LocalDate.html#getDayOfWeek()[getDayOfWeek]() +* int {java11-javadoc}/java.base/java/time/LocalDate.html#getDayOfYear()[getDayOfYear]() +* Era {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#getEra()[getEra]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* Month {java11-javadoc}/java.base/java/time/LocalDate.html#getMonth()[getMonth]() +* int {java11-javadoc}/java.base/java/time/LocalDate.html#getMonthValue()[getMonthValue]() +* int {java11-javadoc}/java.base/java/time/LocalDate.html#getYear()[getYear]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isAfter(java.time.chrono.ChronoLocalDate)[isAfter](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isBefore(java.time.chrono.ChronoLocalDate)[isBefore](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isEqual(java.time.chrono.ChronoLocalDate)[isEqual](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfYear()[lengthOfYear]() +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#minusDays(long)[minusDays](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#minusMonths(long)[minusMonths](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#minusWeeks(long)[minusWeeks](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#minusYears(long)[minusYears](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#plusDays(long)[plusDays](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#plusMonths(long)[plusMonths](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#plusWeeks(long)[plusWeeks](long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#plusYears(long)[plusYears](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toEpochDay()[toEpochDay]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toString()[toString]() +* Period {java11-javadoc}/java.base/java/time/LocalDate.html#until(java.time.chrono.ChronoLocalDate)[until](ChronoLocalDate) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#withDayOfMonth(int)[withDayOfMonth](int) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#withDayOfYear(int)[withDayOfYear](int) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#withMonth(int)[withMonth](int) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDate.html#withYear(int)[withYear](int) + + +[[painless-api-reference-shared-LocalDateTime]] +==== LocalDateTime +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#MAX[MAX] +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#MIN[MIN] +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#of(java.time.LocalDate,java.time.LocalTime)[of](LocalDate, LocalTime) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#of(int,int,int,int,int)[of](int, int, int, int, int) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#of(int,int,int,int,int,int)[of](int, int, int, int, int, int) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#of(int,int,int,int,int,int,int)[of](int, int, int, int, int, int, int) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#ofEpochSecond(long,int,java.time.ZoneOffset)[ofEpochSecond](long, int, ZoneOffset) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#ofInstant(java.time.Instant,java.time.ZoneId)[ofInstant](Instant, ZoneId) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* OffsetDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#atOffset(java.time.ZoneOffset)[atOffset](ZoneOffset) +* ZonedDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#atZone(java.time.ZoneId)[atZone](ZoneId) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#compareTo(java.time.chrono.ChronoLocalDateTime)[compareTo](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* Chronology {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#getChronology()[getChronology]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getDayOfMonth()[getDayOfMonth]() +* DayOfWeek {java11-javadoc}/java.base/java/time/LocalDateTime.html#getDayOfWeek()[getDayOfWeek]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getDayOfYear()[getDayOfYear]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getHour()[getHour]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getMinute()[getMinute]() +* Month {java11-javadoc}/java.base/java/time/LocalDateTime.html#getMonth()[getMonth]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getMonthValue()[getMonthValue]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getNano()[getNano]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getSecond()[getSecond]() +* int {java11-javadoc}/java.base/java/time/LocalDateTime.html#getYear()[getYear]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#isAfter(java.time.chrono.ChronoLocalDateTime)[isAfter](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#isBefore(java.time.chrono.ChronoLocalDateTime)[isBefore](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#isEqual(java.time.chrono.ChronoLocalDateTime)[isEqual](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusDays(long)[minusDays](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusHours(long)[minusHours](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusMinutes(long)[minusMinutes](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusMonths(long)[minusMonths](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusNanos(long)[minusNanos](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusSeconds(long)[minusSeconds](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusWeeks(long)[minusWeeks](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#minusYears(long)[minusYears](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusDays(long)[plusDays](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusHours(long)[plusHours](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusMinutes(long)[plusMinutes](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusMonths(long)[plusMonths](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusNanos(long)[plusNanos](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusSeconds(long)[plusSeconds](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusWeeks(long)[plusWeeks](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#plusYears(long)[plusYears](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toEpochSecond(java.time.ZoneOffset)[toEpochSecond](ZoneOffset) +* Instant {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toInstant(java.time.ZoneOffset)[toInstant](ZoneOffset) +* LocalDate {java11-javadoc}/java.base/java/time/LocalDateTime.html#toLocalDate()[toLocalDate]() +* LocalTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toLocalTime()[toLocalTime]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toString()[toString]() +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#truncatedTo(java.time.temporal.TemporalUnit)[truncatedTo](TemporalUnit) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withDayOfMonth(int)[withDayOfMonth](int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withDayOfYear(int)[withDayOfYear](int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withHour(int)[withHour](int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withMinute(int)[withMinute](int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withMonth(int)[withMonth](int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withSecond(int)[withSecond](int) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalDateTime.html#withYear(int)[withYear](int) + + +[[painless-api-reference-shared-LocalTime]] +==== LocalTime +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#MAX[MAX] +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#MIDNIGHT[MIDNIGHT] +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#MIN[MIN] +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#NOON[NOON] +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#of(int,int)[of](int, int) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#of(int,int,int)[of](int, int, int) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#of(int,int,int,int)[of](int, int, int, int) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#ofNanoOfDay(long)[ofNanoOfDay](long) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#ofSecondOfDay(long)[ofSecondOfDay](long) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* LocalDateTime {java11-javadoc}/java.base/java/time/LocalTime.html#atDate(java.time.LocalDate)[atDate](LocalDate) +* OffsetTime {java11-javadoc}/java.base/java/time/LocalTime.html#atOffset(java.time.ZoneOffset)[atOffset](ZoneOffset) +* int {java11-javadoc}/java.base/java/time/LocalTime.html#compareTo(java.time.LocalTime)[compareTo](LocalTime) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/LocalTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* int {java11-javadoc}/java.base/java/time/LocalTime.html#getHour()[getHour]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/LocalTime.html#getMinute()[getMinute]() +* int {java11-javadoc}/java.base/java/time/LocalTime.html#getNano()[getNano]() +* int {java11-javadoc}/java.base/java/time/LocalTime.html#getSecond()[getSecond]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/LocalTime.html#isAfter(java.time.LocalTime)[isAfter](LocalTime) +* boolean {java11-javadoc}/java.base/java/time/LocalTime.html#isBefore(java.time.LocalTime)[isBefore](LocalTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#minusHours(long)[minusHours](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#minusMinutes(long)[minusMinutes](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#minusNanos(long)[minusNanos](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#minusSeconds(long)[minusSeconds](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#plusHours(long)[plusHours](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#plusMinutes(long)[plusMinutes](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#plusNanos(long)[plusNanos](long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#plusSeconds(long)[plusSeconds](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/LocalTime.html#toNanoOfDay()[toNanoOfDay]() +* int {java11-javadoc}/java.base/java/time/LocalTime.html#toSecondOfDay()[toSecondOfDay]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#truncatedTo(java.time.temporal.TemporalUnit)[truncatedTo](TemporalUnit) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#withHour(int)[withHour](int) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#withMinute(int)[withMinute](int) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#withNano(int)[withNano](int) +* LocalTime {java11-javadoc}/java.base/java/time/LocalTime.html#withSecond(int)[withSecond](int) + + +[[painless-api-reference-shared-Month]] +==== Month +* static Month {java11-javadoc}/java.base/java/time/Month.html#APRIL[APRIL] +* static Month {java11-javadoc}/java.base/java/time/Month.html#AUGUST[AUGUST] +* static Month {java11-javadoc}/java.base/java/time/Month.html#DECEMBER[DECEMBER] +* static Month {java11-javadoc}/java.base/java/time/Month.html#FEBRUARY[FEBRUARY] +* static Month {java11-javadoc}/java.base/java/time/Month.html#JANUARY[JANUARY] +* static Month {java11-javadoc}/java.base/java/time/Month.html#JULY[JULY] +* static Month {java11-javadoc}/java.base/java/time/Month.html#JUNE[JUNE] +* static Month {java11-javadoc}/java.base/java/time/Month.html#MARCH[MARCH] +* static Month {java11-javadoc}/java.base/java/time/Month.html#MAY[MAY] +* static Month {java11-javadoc}/java.base/java/time/Month.html#NOVEMBER[NOVEMBER] +* static Month {java11-javadoc}/java.base/java/time/Month.html#OCTOBER[OCTOBER] +* static Month {java11-javadoc}/java.base/java/time/Month.html#SEPTEMBER[SEPTEMBER] +* static Month {java11-javadoc}/java.base/java/time/Month.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Month {java11-javadoc}/java.base/java/time/Month.html#of(int)[of](int) +* static Month {java11-javadoc}/java.base/java/time/Month.html#valueOf(java.lang.String)[valueOf](String) +* static Month[] {java11-javadoc}/java.base/java/time/Month.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/Month.html#firstDayOfYear(boolean)[firstDayOfYear](boolean) +* Month {java11-javadoc}/java.base/java/time/Month.html#firstMonthOfQuarter()[firstMonthOfQuarter]() +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/Month.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/Month.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/Month.html#length(boolean)[length](boolean) +* int {java11-javadoc}/java.base/java/time/Month.html#maxLength()[maxLength]() +* int {java11-javadoc}/java.base/java/time/Month.html#minLength()[minLength]() +* Month {java11-javadoc}/java.base/java/time/Month.html#minus(long)[minus](long) +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* Month {java11-javadoc}/java.base/java/time/Month.html#plus(long)[plus](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MonthDay]] +==== MonthDay +* static MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#of(int,int)[of](int, int) +* static MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* LocalDate {java11-javadoc}/java.base/java/time/MonthDay.html#atYear(int)[atYear](int) +* int {java11-javadoc}/java.base/java/time/MonthDay.html#compareTo(java.time.MonthDay)[compareTo](MonthDay) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/MonthDay.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* int {java11-javadoc}/java.base/java/time/MonthDay.html#getDayOfMonth()[getDayOfMonth]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* Month {java11-javadoc}/java.base/java/time/MonthDay.html#getMonth()[getMonth]() +* int {java11-javadoc}/java.base/java/time/MonthDay.html#getMonthValue()[getMonthValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/MonthDay.html#isAfter(java.time.MonthDay)[isAfter](MonthDay) +* boolean {java11-javadoc}/java.base/java/time/MonthDay.html#isBefore(java.time.MonthDay)[isBefore](MonthDay) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* boolean {java11-javadoc}/java.base/java/time/MonthDay.html#isValidYear(int)[isValidYear](int) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#with(java.time.Month)[with](Month) +* MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#withDayOfMonth(int)[withDayOfMonth](int) +* MonthDay {java11-javadoc}/java.base/java/time/MonthDay.html#withMonth(int)[withMonth](int) + + +[[painless-api-reference-shared-OffsetDateTime]] +==== OffsetDateTime +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#MAX[MAX] +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#MIN[MIN] +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#of(java.time.LocalDateTime,java.time.ZoneOffset)[of](LocalDateTime, ZoneOffset) +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#of(java.time.LocalDate,java.time.LocalTime,java.time.ZoneOffset)[of](LocalDate, LocalTime, ZoneOffset) +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#of(int,int,int,int,int,int,int,java.time.ZoneOffset)[of](int, int, int, int, int, int, int, ZoneOffset) +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#ofInstant(java.time.Instant,java.time.ZoneId)[ofInstant](Instant, ZoneId) +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* static Comparator {java11-javadoc}/java.base/java/time/OffsetDateTime.html#timeLineOrder()[timeLineOrder]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ZonedDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#atZoneSameInstant(java.time.ZoneId)[atZoneSameInstant](ZoneId) +* ZonedDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#atZoneSimilarLocal(java.time.ZoneId)[atZoneSimilarLocal](ZoneId) +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#compareTo(java.time.OffsetDateTime)[compareTo](OffsetDateTime) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/OffsetDateTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getDayOfMonth()[getDayOfMonth]() +* DayOfWeek {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getDayOfWeek()[getDayOfWeek]() +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getDayOfYear()[getDayOfYear]() +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getHour()[getHour]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getMinute()[getMinute]() +* Month {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getMonth()[getMonth]() +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getMonthValue()[getMonthValue]() +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getNano()[getNano]() +* ZoneOffset {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getOffset()[getOffset]() +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getSecond()[getSecond]() +* int {java11-javadoc}/java.base/java/time/OffsetDateTime.html#getYear()[getYear]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/OffsetDateTime.html#isAfter(java.time.OffsetDateTime)[isAfter](OffsetDateTime) +* boolean {java11-javadoc}/java.base/java/time/OffsetDateTime.html#isBefore(java.time.OffsetDateTime)[isBefore](OffsetDateTime) +* boolean {java11-javadoc}/java.base/java/time/OffsetDateTime.html#isEqual(java.time.OffsetDateTime)[isEqual](OffsetDateTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusDays(long)[minusDays](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusHours(long)[minusHours](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusMinutes(long)[minusMinutes](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusMonths(long)[minusMonths](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusNanos(long)[minusNanos](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusSeconds(long)[minusSeconds](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusWeeks(long)[minusWeeks](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#minusYears(long)[minusYears](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusDays(long)[plusDays](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusHours(long)[plusHours](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusMinutes(long)[plusMinutes](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusMonths(long)[plusMonths](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusNanos(long)[plusNanos](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusSeconds(long)[plusSeconds](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusWeeks(long)[plusWeeks](long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#plusYears(long)[plusYears](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toEpochSecond()[toEpochSecond]() +* Instant {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toInstant()[toInstant]() +* LocalDate {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toLocalDate()[toLocalDate]() +* LocalDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toLocalDateTime()[toLocalDateTime]() +* LocalTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toLocalTime()[toLocalTime]() +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toOffsetTime()[toOffsetTime]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#toZonedDateTime()[toZonedDateTime]() +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#truncatedTo(java.time.temporal.TemporalUnit)[truncatedTo](TemporalUnit) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withDayOfMonth(int)[withDayOfMonth](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withDayOfYear(int)[withDayOfYear](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withHour(int)[withHour](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withMinute(int)[withMinute](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withMonth(int)[withMonth](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withNano(int)[withNano](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withOffsetSameInstant(java.time.ZoneOffset)[withOffsetSameInstant](ZoneOffset) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withOffsetSameLocal(java.time.ZoneOffset)[withOffsetSameLocal](ZoneOffset) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withSecond(int)[withSecond](int) +* OffsetDateTime {java11-javadoc}/java.base/java/time/OffsetDateTime.html#withYear(int)[withYear](int) + + +[[painless-api-reference-shared-OffsetTime]] +==== OffsetTime +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#MAX[MAX] +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#MIN[MIN] +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#of(java.time.LocalTime,java.time.ZoneOffset)[of](LocalTime, ZoneOffset) +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#of(int,int,int,int,java.time.ZoneOffset)[of](int, int, int, int, ZoneOffset) +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#ofInstant(java.time.Instant,java.time.ZoneId)[ofInstant](Instant, ZoneId) +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/time/OffsetTime.html#compareTo(java.time.OffsetTime)[compareTo](OffsetTime) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/OffsetTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* int {java11-javadoc}/java.base/java/time/OffsetTime.html#getHour()[getHour]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/OffsetTime.html#getMinute()[getMinute]() +* int {java11-javadoc}/java.base/java/time/OffsetTime.html#getNano()[getNano]() +* ZoneOffset {java11-javadoc}/java.base/java/time/OffsetTime.html#getOffset()[getOffset]() +* int {java11-javadoc}/java.base/java/time/OffsetTime.html#getSecond()[getSecond]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/OffsetTime.html#isAfter(java.time.OffsetTime)[isAfter](OffsetTime) +* boolean {java11-javadoc}/java.base/java/time/OffsetTime.html#isBefore(java.time.OffsetTime)[isBefore](OffsetTime) +* boolean {java11-javadoc}/java.base/java/time/OffsetTime.html#isEqual(java.time.OffsetTime)[isEqual](OffsetTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#minusHours(long)[minusHours](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#minusMinutes(long)[minusMinutes](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#minusNanos(long)[minusNanos](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#minusSeconds(long)[minusSeconds](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#plusHours(long)[plusHours](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#plusMinutes(long)[plusMinutes](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#plusNanos(long)[plusNanos](long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#plusSeconds(long)[plusSeconds](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* LocalTime {java11-javadoc}/java.base/java/time/OffsetTime.html#toLocalTime()[toLocalTime]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#truncatedTo(java.time.temporal.TemporalUnit)[truncatedTo](TemporalUnit) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#withHour(int)[withHour](int) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#withMinute(int)[withMinute](int) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#withNano(int)[withNano](int) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#withOffsetSameInstant(java.time.ZoneOffset)[withOffsetSameInstant](ZoneOffset) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#withOffsetSameLocal(java.time.ZoneOffset)[withOffsetSameLocal](ZoneOffset) +* OffsetTime {java11-javadoc}/java.base/java/time/OffsetTime.html#withSecond(int)[withSecond](int) + + +[[painless-api-reference-shared-Period]] +==== Period +* static Period {java11-javadoc}/java.base/java/time/Period.html#ZERO[ZERO] +* static Period {java11-javadoc}/java.base/java/time/Period.html#between(java.time.LocalDate,java.time.LocalDate)[between](LocalDate, LocalDate) +* static Period {java11-javadoc}/java.base/java/time/Period.html#from(java.time.temporal.TemporalAmount)[from](TemporalAmount) +* static Period {java11-javadoc}/java.base/java/time/Period.html#of(int,int,int)[of](int, int, int) +* static Period {java11-javadoc}/java.base/java/time/Period.html#ofDays(int)[ofDays](int) +* static Period {java11-javadoc}/java.base/java/time/Period.html#ofMonths(int)[ofMonths](int) +* static Period {java11-javadoc}/java.base/java/time/Period.html#ofWeeks(int)[ofWeeks](int) +* static Period {java11-javadoc}/java.base/java/time/Period.html#ofYears(int)[ofYears](int) +* static Period {java11-javadoc}/java.base/java/time/Period.html#parse(java.lang.CharSequence)[parse](CharSequence) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#addTo(java.time.temporal.Temporal)[addTo](Temporal) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#get(java.time.temporal.TemporalUnit)[get](TemporalUnit) +* IsoChronology {java11-javadoc}/java.base/java/time/Period.html#getChronology()[getChronology]() +* int {java11-javadoc}/java.base/java/time/Period.html#getDays()[getDays]() +* int {java11-javadoc}/java.base/java/time/Period.html#getMonths()[getMonths]() +* List {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#getUnits()[getUnits]() +* int {java11-javadoc}/java.base/java/time/Period.html#getYears()[getYears]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#isNegative()[isNegative]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#isZero()[isZero]() +* Period {java11-javadoc}/java.base/java/time/Period.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* Period {java11-javadoc}/java.base/java/time/Period.html#minusDays(long)[minusDays](long) +* Period {java11-javadoc}/java.base/java/time/Period.html#minusMonths(long)[minusMonths](long) +* Period {java11-javadoc}/java.base/java/time/Period.html#minusYears(long)[minusYears](long) +* Period {java11-javadoc}/java.base/java/time/Period.html#multipliedBy(int)[multipliedBy](int) +* Period {java11-javadoc}/java.base/java/time/Period.html#negated()[negated]() +* Period {java11-javadoc}/java.base/java/time/Period.html#normalized()[normalized]() +* Period {java11-javadoc}/java.base/java/time/Period.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* Period {java11-javadoc}/java.base/java/time/Period.html#plusDays(long)[plusDays](long) +* Period {java11-javadoc}/java.base/java/time/Period.html#plusMonths(long)[plusMonths](long) +* Period {java11-javadoc}/java.base/java/time/Period.html#plusYears(long)[plusYears](long) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#subtractFrom(java.time.temporal.Temporal)[subtractFrom](Temporal) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#toString()[toString]() +* long {java11-javadoc}/java.base/java/time/Period.html#toTotalMonths()[toTotalMonths]() +* Period {java11-javadoc}/java.base/java/time/Period.html#withDays(int)[withDays](int) +* Period {java11-javadoc}/java.base/java/time/Period.html#withMonths(int)[withMonths](int) +* Period {java11-javadoc}/java.base/java/time/Period.html#withYears(int)[withYears](int) + + +[[painless-api-reference-shared-Year]] +==== Year +* static int {java11-javadoc}/java.base/java/time/Year.html#MAX_VALUE[MAX_VALUE] +* static int {java11-javadoc}/java.base/java/time/Year.html#MIN_VALUE[MIN_VALUE] +* static Year {java11-javadoc}/java.base/java/time/Year.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static boolean {java11-javadoc}/java.base/java/time/Year.html#isLeap(long)[isLeap](long) +* static Year {java11-javadoc}/java.base/java/time/Year.html#of(int)[of](int) +* static Year {java11-javadoc}/java.base/java/time/Year.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static Year {java11-javadoc}/java.base/java/time/Year.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* LocalDate {java11-javadoc}/java.base/java/time/Year.html#atDay(int)[atDay](int) +* YearMonth {java11-javadoc}/java.base/java/time/Year.html#atMonth(int)[atMonth](int) +* LocalDate {java11-javadoc}/java.base/java/time/Year.html#atMonthDay(java.time.MonthDay)[atMonthDay](MonthDay) +* int {java11-javadoc}/java.base/java/time/Year.html#compareTo(java.time.Year)[compareTo](Year) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/Year.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/Year.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/Year.html#isAfter(java.time.Year)[isAfter](Year) +* boolean {java11-javadoc}/java.base/java/time/Year.html#isLeap()[isLeap]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* boolean {java11-javadoc}/java.base/java/time/Year.html#isValidMonthDay(java.time.MonthDay)[isValidMonthDay](MonthDay) +* int {java11-javadoc}/java.base/java/time/Year.html#length()[length]() +* Year {java11-javadoc}/java.base/java/time/Year.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* Year {java11-javadoc}/java.base/java/time/Year.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* Year {java11-javadoc}/java.base/java/time/Year.html#minusYears(long)[minusYears](long) +* Year {java11-javadoc}/java.base/java/time/Year.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* Year {java11-javadoc}/java.base/java/time/Year.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* Year {java11-javadoc}/java.base/java/time/Year.html#plusYears(long)[plusYears](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* Year {java11-javadoc}/java.base/java/time/Year.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* Year {java11-javadoc}/java.base/java/time/Year.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-YearMonth]] +==== YearMonth +* static YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#of(int,int)[of](int, int) +* static YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* LocalDate {java11-javadoc}/java.base/java/time/YearMonth.html#atDay(int)[atDay](int) +* LocalDate {java11-javadoc}/java.base/java/time/YearMonth.html#atEndOfMonth()[atEndOfMonth]() +* int {java11-javadoc}/java.base/java/time/YearMonth.html#compareTo(java.time.YearMonth)[compareTo](YearMonth) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/YearMonth.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* Month {java11-javadoc}/java.base/java/time/YearMonth.html#getMonth()[getMonth]() +* int {java11-javadoc}/java.base/java/time/YearMonth.html#getMonthValue()[getMonthValue]() +* int {java11-javadoc}/java.base/java/time/YearMonth.html#getYear()[getYear]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/YearMonth.html#isAfter(java.time.YearMonth)[isAfter](YearMonth) +* boolean {java11-javadoc}/java.base/java/time/YearMonth.html#isBefore(java.time.YearMonth)[isBefore](YearMonth) +* boolean {java11-javadoc}/java.base/java/time/YearMonth.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* boolean {java11-javadoc}/java.base/java/time/YearMonth.html#isValidDay(int)[isValidDay](int) +* int {java11-javadoc}/java.base/java/time/YearMonth.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/YearMonth.html#lengthOfYear()[lengthOfYear]() +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#minusMonths(long)[minusMonths](long) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#minusYears(long)[minusYears](long) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#plusMonths(long)[plusMonths](long) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#plusYears(long)[plusYears](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#withMonth(int)[withMonth](int) +* YearMonth {java11-javadoc}/java.base/java/time/YearMonth.html#withYear(int)[withYear](int) + + +[[painless-api-reference-shared-ZoneId]] +==== ZoneId +* static Map {java11-javadoc}/java.base/java/time/ZoneId.html#SHORT_IDS[SHORT_IDS] +* static ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Set {java11-javadoc}/java.base/java/time/ZoneId.html#getAvailableZoneIds()[getAvailableZoneIds]() +* static ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#of(java.lang.String)[of](String) +* static ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#of(java.lang.String,java.util.Map)[of](String, Map) +* static ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#ofOffset(java.lang.String,java.time.ZoneOffset)[ofOffset](String, ZoneOffset) +* static ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#systemDefault()[systemDefault]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/ZoneId.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/ZoneId.html#getId()[getId]() +* ZoneRules {java11-javadoc}/java.base/java/time/ZoneId.html#getRules()[getRules]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#normalized()[normalized]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZoneOffset]] +==== ZoneOffset +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#MAX[MAX] +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#MIN[MIN] +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#UTC[UTC] +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#of(java.lang.String)[of](String) +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#ofHours(int)[ofHours](int) +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#ofHoursMinutes(int,int)[ofHoursMinutes](int, int) +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#ofHoursMinutesSeconds(int,int,int)[ofHoursMinutesSeconds](int, int, int) +* static ZoneOffset {java11-javadoc}/java.base/java/time/ZoneOffset.html#ofTotalSeconds(int)[ofTotalSeconds](int) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Comparable.html#compareTo(java.lang.Object)[compareTo](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/ZoneId.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/ZoneId.html#getId()[getId]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* ZoneRules {java11-javadoc}/java.base/java/time/ZoneId.html#getRules()[getRules]() +* int {java11-javadoc}/java.base/java/time/ZoneOffset.html#getTotalSeconds()[getTotalSeconds]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* ZoneId {java11-javadoc}/java.base/java/time/ZoneId.html#normalized()[normalized]() +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZonedDateTime]] +==== ZonedDateTime +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#of(java.time.LocalDateTime,java.time.ZoneId)[of](LocalDateTime, ZoneId) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#of(java.time.LocalDate,java.time.LocalTime,java.time.ZoneId)[of](LocalDate, LocalTime, ZoneId) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#of(int,int,int,int,int,int,int,java.time.ZoneId)[of](int, int, int, int, int, int, int, ZoneId) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#ofInstant(java.time.Instant,java.time.ZoneId)[ofInstant](Instant, ZoneId) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#ofInstant(java.time.LocalDateTime,java.time.ZoneOffset,java.time.ZoneId)[ofInstant](LocalDateTime, ZoneOffset, ZoneId) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#ofLocal(java.time.LocalDateTime,java.time.ZoneId,java.time.ZoneOffset)[ofLocal](LocalDateTime, ZoneId, ZoneOffset) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#ofStrict(java.time.LocalDateTime,java.time.ZoneOffset,java.time.ZoneId)[ofStrict](LocalDateTime, ZoneOffset, ZoneId) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#parse(java.lang.CharSequence)[parse](CharSequence) +* static ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#parse(java.lang.CharSequence,java.time.format.DateTimeFormatter)[parse](CharSequence, DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#compareTo(java.time.chrono.ChronoZonedDateTime)[compareTo](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* Chronology {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#getChronology()[getChronology]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getDayOfMonth()[getDayOfMonth]() +* DayOfWeek {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getDayOfWeek()[getDayOfWeek]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getDayOfYear()[getDayOfYear]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getHour()[getHour]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getMinute()[getMinute]() +* Month {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getMonth()[getMonth]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getMonthValue()[getMonthValue]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getNano()[getNano]() +* ZoneOffset {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#getOffset()[getOffset]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getSecond()[getSecond]() +* int {java11-javadoc}/java.base/java/time/ZonedDateTime.html#getYear()[getYear]() +* ZoneId {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#getZone()[getZone]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#isAfter(java.time.chrono.ChronoZonedDateTime)[isAfter](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#isBefore(java.time.chrono.ChronoZonedDateTime)[isBefore](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#isEqual(java.time.chrono.ChronoZonedDateTime)[isEqual](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusDays(long)[minusDays](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusHours(long)[minusHours](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusMinutes(long)[minusMinutes](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusMonths(long)[minusMonths](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusNanos(long)[minusNanos](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusSeconds(long)[minusSeconds](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusWeeks(long)[minusWeeks](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#minusYears(long)[minusYears](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusDays(long)[plusDays](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusHours(long)[plusHours](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusMinutes(long)[plusMinutes](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusMonths(long)[plusMonths](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusNanos(long)[plusNanos](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusSeconds(long)[plusSeconds](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusWeeks(long)[plusWeeks](long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#plusYears(long)[plusYears](long) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toEpochSecond()[toEpochSecond]() +* Instant {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toInstant()[toInstant]() +* LocalDate {java11-javadoc}/java.base/java/time/ZonedDateTime.html#toLocalDate()[toLocalDate]() +* LocalDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#toLocalDateTime()[toLocalDateTime]() +* LocalTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toLocalTime()[toLocalTime]() +* OffsetDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#toOffsetDateTime()[toOffsetDateTime]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toString()[toString]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#truncatedTo(java.time.temporal.TemporalUnit)[truncatedTo](TemporalUnit) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withDayOfMonth(int)[withDayOfMonth](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withDayOfYear(int)[withDayOfYear](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withEarlierOffsetAtOverlap()[withEarlierOffsetAtOverlap]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withFixedOffsetZone()[withFixedOffsetZone]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withHour(int)[withHour](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withLaterOffsetAtOverlap()[withLaterOffsetAtOverlap]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withMinute(int)[withMinute](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withMonth(int)[withMonth](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withNano(int)[withNano](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withSecond(int)[withSecond](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withYear(int)[withYear](int) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withZoneSameInstant(java.time.ZoneId)[withZoneSameInstant](ZoneId) +* ZonedDateTime {java11-javadoc}/java.base/java/time/ZonedDateTime.html#withZoneSameLocal(java.time.ZoneId)[withZoneSameLocal](ZoneId) + + +[role="exclude",id="painless-api-reference-shared-java-time-chrono"] +=== Shared API for package java.time.chrono +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-AbstractChronology]] +==== AbstractChronology +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#date(int,int,int)[date](int, int, int) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#dateEpochDay(long)[dateEpochDay](long) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* Era {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/Chronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-ChronoLocalDate]] +==== ChronoLocalDate +* static ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Comparator {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#timeLineOrder()[timeLineOrder]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#atTime(java.time.LocalTime)[atTime](LocalTime) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#compareTo(java.time.chrono.ChronoLocalDate)[compareTo](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* Chronology {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#getChronology()[getChronology]() +* Era {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#getEra()[getEra]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isAfter(java.time.chrono.ChronoLocalDate)[isAfter](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isBefore(java.time.chrono.ChronoLocalDate)[isBefore](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isEqual(java.time.chrono.ChronoLocalDate)[isEqual](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfYear()[lengthOfYear]() +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toEpochDay()[toEpochDay]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toString()[toString]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#until(java.time.chrono.ChronoLocalDate)[until](ChronoLocalDate) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-ChronoLocalDateTime]] +==== ChronoLocalDateTime +* static ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Comparator {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#timeLineOrder()[timeLineOrder]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#atZone(java.time.ZoneId)[atZone](ZoneId) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#compareTo(java.time.chrono.ChronoLocalDateTime)[compareTo](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* Chronology {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#getChronology()[getChronology]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#isAfter(java.time.chrono.ChronoLocalDateTime)[isAfter](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#isBefore(java.time.chrono.ChronoLocalDateTime)[isBefore](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#isEqual(java.time.chrono.ChronoLocalDateTime)[isEqual](ChronoLocalDateTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toEpochSecond(java.time.ZoneOffset)[toEpochSecond](ZoneOffset) +* Instant {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toInstant(java.time.ZoneOffset)[toInstant](ZoneOffset) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toLocalDate()[toLocalDate]() +* LocalTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toLocalTime()[toLocalTime]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#toString()[toString]() +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDateTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-ChronoPeriod]] +==== ChronoPeriod +* static ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#between(java.time.chrono.ChronoLocalDate,java.time.chrono.ChronoLocalDate)[between](ChronoLocalDate, ChronoLocalDate) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#addTo(java.time.temporal.Temporal)[addTo](Temporal) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#get(java.time.temporal.TemporalUnit)[get](TemporalUnit) +* Chronology {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#getChronology()[getChronology]() +* List {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#getUnits()[getUnits]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#isNegative()[isNegative]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#isZero()[isZero]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#multipliedBy(int)[multipliedBy](int) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#negated()[negated]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#normalized()[normalized]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#subtractFrom(java.time.temporal.Temporal)[subtractFrom](Temporal) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoPeriod.html#toString()[toString]() + + +[[painless-api-reference-shared-ChronoZonedDateTime]] +==== ChronoZonedDateTime +* static ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Comparator {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#timeLineOrder()[timeLineOrder]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#compareTo(java.time.chrono.ChronoZonedDateTime)[compareTo](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* Chronology {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#getChronology()[getChronology]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* ZoneOffset {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#getOffset()[getOffset]() +* ZoneId {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#getZone()[getZone]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#isAfter(java.time.chrono.ChronoZonedDateTime)[isAfter](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#isBefore(java.time.chrono.ChronoZonedDateTime)[isBefore](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#isEqual(java.time.chrono.ChronoZonedDateTime)[isEqual](ChronoZonedDateTime) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toEpochSecond()[toEpochSecond]() +* Instant {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toInstant()[toInstant]() +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toLocalDate()[toLocalDate]() +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toLocalDateTime()[toLocalDateTime]() +* LocalTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toLocalTime()[toLocalTime]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#toString()[toString]() +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#withEarlierOffsetAtOverlap()[withEarlierOffsetAtOverlap]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#withLaterOffsetAtOverlap()[withLaterOffsetAtOverlap]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#withZoneSameInstant(java.time.ZoneId)[withZoneSameInstant](ZoneId) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoZonedDateTime.html#withZoneSameLocal(java.time.ZoneId)[withZoneSameLocal](ZoneId) + + +[[painless-api-reference-shared-Chronology]] +==== Chronology +* static Chronology {java11-javadoc}/java.base/java/time/chrono/Chronology.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static Set {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getAvailableChronologies()[getAvailableChronologies]() +* static Chronology {java11-javadoc}/java.base/java/time/chrono/Chronology.html#of(java.lang.String)[of](String) +* static Chronology {java11-javadoc}/java.base/java/time/chrono/Chronology.html#ofLocale(java.util.Locale)[ofLocale](Locale) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#date(int,int,int)[date](int, int, int) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#dateEpochDay(long)[dateEpochDay](long) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* Era {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/Chronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* ChronoLocalDate {java11-javadoc}/java.base/java/time/chrono/Chronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-Era]] +==== Era +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/chrono/Era.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/Era.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-HijrahChronology]] +==== HijrahChronology +* static HijrahChronology {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#INSTANCE[INSTANCE] +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#date(int,int,int)[date](int, int, int) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#dateEpochDay(long)[dateEpochDay](long) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* HijrahEra {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/Chronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahChronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-HijrahDate]] +==== HijrahDate +* static HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#of(int,int,int)[of](int, int, int) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#atTime(java.time.LocalTime)[atTime](LocalTime) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#compareTo(java.time.chrono.ChronoLocalDate)[compareTo](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* HijrahChronology {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#getChronology()[getChronology]() +* HijrahEra {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#getEra()[getEra]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isAfter(java.time.chrono.ChronoLocalDate)[isAfter](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isBefore(java.time.chrono.ChronoLocalDate)[isBefore](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isEqual(java.time.chrono.ChronoLocalDate)[isEqual](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfYear()[lengthOfYear]() +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toEpochDay()[toEpochDay]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toString()[toString]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#until(java.time.chrono.ChronoLocalDate)[until](ChronoLocalDate) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) +* HijrahDate {java11-javadoc}/java.base/java/time/chrono/HijrahDate.html#withVariant(java.time.chrono.HijrahChronology)[withVariant](HijrahChronology) + + +[[painless-api-reference-shared-HijrahEra]] +==== HijrahEra +* static HijrahEra {java11-javadoc}/java.base/java/time/chrono/HijrahEra.html#AH[AH] +* static HijrahEra {java11-javadoc}/java.base/java/time/chrono/HijrahEra.html#of(int)[of](int) +* static HijrahEra {java11-javadoc}/java.base/java/time/chrono/HijrahEra.html#valueOf(java.lang.String)[valueOf](String) +* static HijrahEra[] {java11-javadoc}/java.base/java/time/chrono/HijrahEra.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/chrono/Era.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/HijrahEra.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IsoChronology]] +==== IsoChronology +* static IsoChronology {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#INSTANCE[INSTANCE] +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#date(int,int,int)[date](int, int, int) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#dateEpochDay(long)[dateEpochDay](long) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* IsoEra {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* LocalDateTime {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* Period {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* LocalDate {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ZonedDateTime {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ZonedDateTime {java11-javadoc}/java.base/java/time/chrono/IsoChronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-IsoEra]] +==== IsoEra +* static IsoEra {java11-javadoc}/java.base/java/time/chrono/IsoEra.html#BCE[BCE] +* static IsoEra {java11-javadoc}/java.base/java/time/chrono/IsoEra.html#CE[CE] +* static IsoEra {java11-javadoc}/java.base/java/time/chrono/IsoEra.html#of(int)[of](int) +* static IsoEra {java11-javadoc}/java.base/java/time/chrono/IsoEra.html#valueOf(java.lang.String)[valueOf](String) +* static IsoEra[] {java11-javadoc}/java.base/java/time/chrono/IsoEra.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/chrono/Era.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/IsoEra.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-JapaneseChronology]] +==== JapaneseChronology +* static JapaneseChronology {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#INSTANCE[INSTANCE] +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#date(int,int,int)[date](int, int, int) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#dateEpochDay(long)[dateEpochDay](long) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/Chronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseChronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-JapaneseDate]] +==== JapaneseDate +* static JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#of(int,int,int)[of](int, int, int) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#atTime(java.time.LocalTime)[atTime](LocalTime) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#compareTo(java.time.chrono.ChronoLocalDate)[compareTo](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* JapaneseChronology {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#getChronology()[getChronology]() +* JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#getEra()[getEra]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isAfter(java.time.chrono.ChronoLocalDate)[isAfter](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isBefore(java.time.chrono.ChronoLocalDate)[isBefore](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isEqual(java.time.chrono.ChronoLocalDate)[isEqual](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfYear()[lengthOfYear]() +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toEpochDay()[toEpochDay]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toString()[toString]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#until(java.time.chrono.ChronoLocalDate)[until](ChronoLocalDate) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* JapaneseDate {java11-javadoc}/java.base/java/time/chrono/JapaneseDate.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-JapaneseEra]] +==== JapaneseEra +* static JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#HEISEI[HEISEI] +* static JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#MEIJI[MEIJI] +* static JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#SHOWA[SHOWA] +* static JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#TAISHO[TAISHO] +* static JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#of(int)[of](int) +* static JapaneseEra {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#valueOf(java.lang.String)[valueOf](String) +* static JapaneseEra[] {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/chrono/Era.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/JapaneseEra.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MinguoChronology]] +==== MinguoChronology +* static MinguoChronology {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#INSTANCE[INSTANCE] +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#date(int,int,int)[date](int, int, int) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#dateEpochDay(long)[dateEpochDay](long) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* MinguoEra {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/Chronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoChronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-MinguoDate]] +==== MinguoDate +* static MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#of(int,int,int)[of](int, int, int) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#atTime(java.time.LocalTime)[atTime](LocalTime) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#compareTo(java.time.chrono.ChronoLocalDate)[compareTo](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* MinguoChronology {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#getChronology()[getChronology]() +* MinguoEra {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#getEra()[getEra]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isAfter(java.time.chrono.ChronoLocalDate)[isAfter](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isBefore(java.time.chrono.ChronoLocalDate)[isBefore](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isEqual(java.time.chrono.ChronoLocalDate)[isEqual](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfYear()[lengthOfYear]() +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toEpochDay()[toEpochDay]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toString()[toString]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#until(java.time.chrono.ChronoLocalDate)[until](ChronoLocalDate) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* MinguoDate {java11-javadoc}/java.base/java/time/chrono/MinguoDate.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-MinguoEra]] +==== MinguoEra +* static MinguoEra {java11-javadoc}/java.base/java/time/chrono/MinguoEra.html#BEFORE_ROC[BEFORE_ROC] +* static MinguoEra {java11-javadoc}/java.base/java/time/chrono/MinguoEra.html#ROC[ROC] +* static MinguoEra {java11-javadoc}/java.base/java/time/chrono/MinguoEra.html#of(int)[of](int) +* static MinguoEra {java11-javadoc}/java.base/java/time/chrono/MinguoEra.html#valueOf(java.lang.String)[valueOf](String) +* static MinguoEra[] {java11-javadoc}/java.base/java/time/chrono/MinguoEra.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/chrono/Era.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/MinguoEra.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ThaiBuddhistChronology]] +==== ThaiBuddhistChronology +* static ThaiBuddhistChronology {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#INSTANCE[INSTANCE] +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#compareTo(java.time.chrono.Chronology)[compareTo](Chronology) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#date(java.time.temporal.TemporalAccessor)[date](TemporalAccessor) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#date(int,int,int)[date](int, int, int) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#date(java.time.chrono.Era,int,int,int)[date](Era, int, int, int) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#dateEpochDay(long)[dateEpochDay](long) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#dateYearDay(int,int)[dateYearDay](int, int) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#dateYearDay(java.time.chrono.Era,int,int)[dateYearDay](Era, int, int) +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#equals(java.lang.Object)[equals](Object) +* ThaiBuddhistEra {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#eraOf(int)[eraOf](int) +* List {java11-javadoc}/java.base/java/time/chrono/Chronology.html#eras()[eras]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#getId()[getId]() +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/Chronology.html#isLeapYear(long)[isLeapYear](long) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#localDateTime(java.time.temporal.TemporalAccessor)[localDateTime](TemporalAccessor) +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/Chronology.html#period(int,int,int)[period](int, int, int) +* int {java11-javadoc}/java.base/java/time/chrono/Chronology.html#prolepticYear(java.time.chrono.Era,int)[prolepticYear](Era, int) +* ValueRange {java11-javadoc}/java.base/java/time/chrono/Chronology.html#range(java.time.temporal.ChronoField)[range](ChronoField) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistChronology.html#resolveDate(java.util.Map,java.time.format.ResolverStyle)[resolveDate](Map, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/chrono/Chronology.html#toString()[toString]() +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.temporal.TemporalAccessor)[zonedDateTime](TemporalAccessor) +* ChronoZonedDateTime {java11-javadoc}/java.base/java/time/chrono/Chronology.html#zonedDateTime(java.time.Instant,java.time.ZoneId)[zonedDateTime](Instant, ZoneId) + + +[[painless-api-reference-shared-ThaiBuddhistDate]] +==== ThaiBuddhistDate +* static ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#from(java.time.temporal.TemporalAccessor)[from](TemporalAccessor) +* static ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#of(int,int,int)[of](int, int, int) +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* ChronoLocalDateTime {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#atTime(java.time.LocalTime)[atTime](LocalTime) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#compareTo(java.time.chrono.ChronoLocalDate)[compareTo](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#format(java.time.format.DateTimeFormatter)[format](DateTimeFormatter) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* ThaiBuddhistChronology {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#getChronology()[getChronology]() +* ThaiBuddhistEra {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#getEra()[getEra]() +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isAfter(java.time.chrono.ChronoLocalDate)[isAfter](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isBefore(java.time.chrono.ChronoLocalDate)[isBefore](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isEqual(java.time.chrono.ChronoLocalDate)[isEqual](ChronoLocalDate) +* boolean {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#isLeapYear()[isLeapYear]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfMonth()[lengthOfMonth]() +* int {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#lengthOfYear()[lengthOfYear]() +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* long {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toEpochDay()[toEpochDay]() +* String {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#toString()[toString]() +* ChronoPeriod {java11-javadoc}/java.base/java/time/chrono/ChronoLocalDate.html#until(java.time.chrono.ChronoLocalDate)[until](ChronoLocalDate) +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* ThaiBuddhistDate {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistDate.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-ThaiBuddhistEra]] +==== ThaiBuddhistEra +* static ThaiBuddhistEra {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistEra.html#BE[BE] +* static ThaiBuddhistEra {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistEra.html#BEFORE_BE[BEFORE_BE] +* static ThaiBuddhistEra {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistEra.html#of(int)[of](int) +* static ThaiBuddhistEra {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistEra.html#valueOf(java.lang.String)[valueOf](String) +* static ThaiBuddhistEra[] {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistEra.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* String {java11-javadoc}/java.base/java/time/chrono/Era.html#getDisplayName(java.time.format.TextStyle,java.util.Locale)[getDisplayName](TextStyle, Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/time/chrono/ThaiBuddhistEra.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-time-format"] +=== Shared API for package java.time.format +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-DateTimeFormatter]] +==== DateTimeFormatter +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#BASIC_ISO_DATE[BASIC_ISO_DATE] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_DATE[ISO_DATE] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_DATE_TIME[ISO_DATE_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_INSTANT[ISO_INSTANT] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE[ISO_LOCAL_DATE] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE_TIME[ISO_LOCAL_DATE_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_LOCAL_TIME[ISO_LOCAL_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_OFFSET_DATE[ISO_OFFSET_DATE] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_OFFSET_DATE_TIME[ISO_OFFSET_DATE_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_OFFSET_TIME[ISO_OFFSET_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_ORDINAL_DATE[ISO_ORDINAL_DATE] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_TIME[ISO_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_WEEK_DATE[ISO_WEEK_DATE] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ISO_ZONED_DATE_TIME[ISO_ZONED_DATE_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#RFC_1123_DATE_TIME[RFC_1123_DATE_TIME] +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ofLocalizedDate(java.time.format.FormatStyle)[ofLocalizedDate](FormatStyle) +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ofLocalizedDateTime(java.time.format.FormatStyle)[ofLocalizedDateTime](FormatStyle) +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ofLocalizedDateTime(java.time.format.FormatStyle,java.time.format.FormatStyle)[ofLocalizedDateTime](FormatStyle, FormatStyle) +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ofLocalizedTime(java.time.format.FormatStyle)[ofLocalizedTime](FormatStyle) +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ofPattern(java.lang.String)[ofPattern](String) +* static DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#ofPattern(java.lang.String,java.util.Locale)[ofPattern](String, Locale) +* static TemporalQuery {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#parsedExcessDays()[parsedExcessDays]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#parsedLeapSecond()[parsedLeapSecond]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#format(java.time.temporal.TemporalAccessor)[format](TemporalAccessor) +* void {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#formatTo(java.time.temporal.TemporalAccessor,java.lang.Appendable)[formatTo](TemporalAccessor, Appendable) +* Chronology {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#getChronology()[getChronology]() +* DecimalStyle {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#getDecimalStyle()[getDecimalStyle]() +* Locale {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#getLocale()[getLocale]() +* Set {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#getResolverFields()[getResolverFields]() +* ResolverStyle {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#getResolverStyle()[getResolverStyle]() +* ZoneId {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#getZone()[getZone]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* TemporalAccessor {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#parse(java.lang.CharSequence)[parse](CharSequence) +* def {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#parse(java.lang.CharSequence,java.time.temporal.TemporalQuery)[parse](CharSequence, TemporalQuery) +* TemporalAccessor {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#parseBest(java.lang.CharSequence,java.time.temporal.TemporalQuery%5B%5D)[parseBest](CharSequence, TemporalQuery[]) +* TemporalAccessor {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#parseUnresolved(java.lang.CharSequence,java.text.ParsePosition)[parseUnresolved](CharSequence, ParsePosition) +* Format {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#toFormat()[toFormat]() +* Format {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#toFormat(java.time.temporal.TemporalQuery)[toFormat](TemporalQuery) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#withChronology(java.time.chrono.Chronology)[withChronology](Chronology) +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#withDecimalStyle(java.time.format.DecimalStyle)[withDecimalStyle](DecimalStyle) +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#withLocale(java.util.Locale)[withLocale](Locale) +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#withResolverFields(java.util.Set)[withResolverFields](Set) +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#withResolverStyle(java.time.format.ResolverStyle)[withResolverStyle](ResolverStyle) +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatter.html#withZone(java.time.ZoneId)[withZone](ZoneId) + + +[[painless-api-reference-shared-DateTimeFormatterBuilder]] +==== DateTimeFormatterBuilder +* static String {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#getLocalizedDateTimePattern(java.time.format.FormatStyle,java.time.format.FormatStyle,java.time.chrono.Chronology,java.util.Locale)[getLocalizedDateTimePattern](FormatStyle, FormatStyle, Chronology, Locale) +* {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#()[DateTimeFormatterBuilder]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#append(java.time.format.DateTimeFormatter)[append](DateTimeFormatter) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendChronologyId()[appendChronologyId]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendChronologyText(java.time.format.TextStyle)[appendChronologyText](TextStyle) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendFraction(java.time.temporal.TemporalField,int,int,boolean)[appendFraction](TemporalField, int, int, boolean) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendInstant()[appendInstant]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendInstant(int)[appendInstant](int) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendLiteral(java.lang.String)[appendLiteral](String) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendLocalized(java.time.format.FormatStyle,java.time.format.FormatStyle)[appendLocalized](FormatStyle, FormatStyle) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendLocalizedOffset(java.time.format.TextStyle)[appendLocalizedOffset](TextStyle) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendOffset(java.lang.String,java.lang.String)[appendOffset](String, String) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendOffsetId()[appendOffsetId]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendOptional(java.time.format.DateTimeFormatter)[appendOptional](DateTimeFormatter) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendPattern(java.lang.String)[appendPattern](String) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendText(java.time.temporal.TemporalField)[appendText](TemporalField) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendText(java.time.temporal.TemporalField,java.time.format.TextStyle)[appendText](TemporalField, TextStyle) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendValue(java.time.temporal.TemporalField)[appendValue](TemporalField) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendValue(java.time.temporal.TemporalField,int)[appendValue](TemporalField, int) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendValue(java.time.temporal.TemporalField,int,int,java.time.format.SignStyle)[appendValue](TemporalField, int, int, SignStyle) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendValueReduced(java.time.temporal.TemporalField,int,int,int)[appendValueReduced](TemporalField, int, int, int) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendZoneId()[appendZoneId]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendZoneOrOffsetId()[appendZoneOrOffsetId]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendZoneRegionId()[appendZoneRegionId]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendZoneText(java.time.format.TextStyle)[appendZoneText](TextStyle) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#appendZoneText(java.time.format.TextStyle,java.util.Set)[appendZoneText](TextStyle, Set) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#optionalEnd()[optionalEnd]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#optionalStart()[optionalStart]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#padNext(int)[padNext](int) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#padNext(int,char)[padNext](int, char) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#parseCaseInsensitive()[parseCaseInsensitive]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#parseCaseSensitive()[parseCaseSensitive]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#parseDefaulting(java.time.temporal.TemporalField,long)[parseDefaulting](TemporalField, long) +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#parseLenient()[parseLenient]() +* DateTimeFormatterBuilder {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#parseStrict()[parseStrict]() +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#toFormatter()[toFormatter]() +* DateTimeFormatter {java11-javadoc}/java.base/java/time/format/DateTimeFormatterBuilder.html#toFormatter(java.util.Locale)[toFormatter](Locale) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DateTimeParseException]] +==== DateTimeParseException +* {java11-javadoc}/java.base/java/time/format/DateTimeParseException.html#(java.lang.String,java.lang.CharSequence,int)[DateTimeParseException](String, CharSequence, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/format/DateTimeParseException.html#getErrorIndex()[getErrorIndex]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* String {java11-javadoc}/java.base/java/time/format/DateTimeParseException.html#getParsedString()[getParsedString]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DecimalStyle]] +==== DecimalStyle +* static DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#STANDARD[STANDARD] +* static Set {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#getAvailableLocales()[getAvailableLocales]() +* static DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#of(java.util.Locale)[of](Locale) +* static DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#ofDefaultLocale()[ofDefaultLocale]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* char {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#getDecimalSeparator()[getDecimalSeparator]() +* char {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#getNegativeSign()[getNegativeSign]() +* char {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#getPositiveSign()[getPositiveSign]() +* char {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#getZeroDigit()[getZeroDigit]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#withDecimalSeparator(char)[withDecimalSeparator](char) +* DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#withNegativeSign(char)[withNegativeSign](char) +* DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#withPositiveSign(char)[withPositiveSign](char) +* DecimalStyle {java11-javadoc}/java.base/java/time/format/DecimalStyle.html#withZeroDigit(char)[withZeroDigit](char) + + +[[painless-api-reference-shared-FormatStyle]] +==== FormatStyle +* static FormatStyle {java11-javadoc}/java.base/java/time/format/FormatStyle.html#FULL[FULL] +* static FormatStyle {java11-javadoc}/java.base/java/time/format/FormatStyle.html#LONG[LONG] +* static FormatStyle {java11-javadoc}/java.base/java/time/format/FormatStyle.html#MEDIUM[MEDIUM] +* static FormatStyle {java11-javadoc}/java.base/java/time/format/FormatStyle.html#SHORT[SHORT] +* static FormatStyle {java11-javadoc}/java.base/java/time/format/FormatStyle.html#valueOf(java.lang.String)[valueOf](String) +* static FormatStyle[] {java11-javadoc}/java.base/java/time/format/FormatStyle.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ResolverStyle]] +==== ResolverStyle +* static ResolverStyle {java11-javadoc}/java.base/java/time/format/ResolverStyle.html#LENIENT[LENIENT] +* static ResolverStyle {java11-javadoc}/java.base/java/time/format/ResolverStyle.html#SMART[SMART] +* static ResolverStyle {java11-javadoc}/java.base/java/time/format/ResolverStyle.html#STRICT[STRICT] +* static ResolverStyle {java11-javadoc}/java.base/java/time/format/ResolverStyle.html#valueOf(java.lang.String)[valueOf](String) +* static ResolverStyle[] {java11-javadoc}/java.base/java/time/format/ResolverStyle.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-SignStyle]] +==== SignStyle +* static SignStyle {java11-javadoc}/java.base/java/time/format/SignStyle.html#ALWAYS[ALWAYS] +* static SignStyle {java11-javadoc}/java.base/java/time/format/SignStyle.html#EXCEEDS_PAD[EXCEEDS_PAD] +* static SignStyle {java11-javadoc}/java.base/java/time/format/SignStyle.html#NEVER[NEVER] +* static SignStyle {java11-javadoc}/java.base/java/time/format/SignStyle.html#NORMAL[NORMAL] +* static SignStyle {java11-javadoc}/java.base/java/time/format/SignStyle.html#NOT_NEGATIVE[NOT_NEGATIVE] +* static SignStyle {java11-javadoc}/java.base/java/time/format/SignStyle.html#valueOf(java.lang.String)[valueOf](String) +* static SignStyle[] {java11-javadoc}/java.base/java/time/format/SignStyle.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TextStyle]] +==== TextStyle +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#FULL[FULL] +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#FULL_STANDALONE[FULL_STANDALONE] +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#NARROW[NARROW] +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#NARROW_STANDALONE[NARROW_STANDALONE] +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#SHORT[SHORT] +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#SHORT_STANDALONE[SHORT_STANDALONE] +* static TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#valueOf(java.lang.String)[valueOf](String) +* static TextStyle[] {java11-javadoc}/java.base/java/time/format/TextStyle.html#values()[values]() +* TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#asNormal()[asNormal]() +* TextStyle {java11-javadoc}/java.base/java/time/format/TextStyle.html#asStandalone()[asStandalone]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/format/TextStyle.html#isStandalone()[isStandalone]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-time-temporal"] +=== Shared API for package java.time.temporal +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-ChronoField]] +==== ChronoField +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#ALIGNED_DAY_OF_WEEK_IN_MONTH[ALIGNED_DAY_OF_WEEK_IN_MONTH] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#ALIGNED_DAY_OF_WEEK_IN_YEAR[ALIGNED_DAY_OF_WEEK_IN_YEAR] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#ALIGNED_WEEK_OF_MONTH[ALIGNED_WEEK_OF_MONTH] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#ALIGNED_WEEK_OF_YEAR[ALIGNED_WEEK_OF_YEAR] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#AMPM_OF_DAY[AMPM_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#CLOCK_HOUR_OF_AMPM[CLOCK_HOUR_OF_AMPM] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#CLOCK_HOUR_OF_DAY[CLOCK_HOUR_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#DAY_OF_MONTH[DAY_OF_MONTH] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#DAY_OF_WEEK[DAY_OF_WEEK] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#DAY_OF_YEAR[DAY_OF_YEAR] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#EPOCH_DAY[EPOCH_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#ERA[ERA] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#HOUR_OF_AMPM[HOUR_OF_AMPM] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#HOUR_OF_DAY[HOUR_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#INSTANT_SECONDS[INSTANT_SECONDS] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MICRO_OF_DAY[MICRO_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MICRO_OF_SECOND[MICRO_OF_SECOND] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MILLI_OF_DAY[MILLI_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MILLI_OF_SECOND[MILLI_OF_SECOND] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MINUTE_OF_DAY[MINUTE_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MINUTE_OF_HOUR[MINUTE_OF_HOUR] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#MONTH_OF_YEAR[MONTH_OF_YEAR] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#NANO_OF_DAY[NANO_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#NANO_OF_SECOND[NANO_OF_SECOND] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#OFFSET_SECONDS[OFFSET_SECONDS] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#PROLEPTIC_MONTH[PROLEPTIC_MONTH] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#SECOND_OF_DAY[SECOND_OF_DAY] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#SECOND_OF_MINUTE[SECOND_OF_MINUTE] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#YEAR[YEAR] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#YEAR_OF_ERA[YEAR_OF_ERA] +* static ChronoField {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#valueOf(java.lang.String)[valueOf](String) +* static ChronoField[] {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#adjustInto(java.time.temporal.Temporal,long)[adjustInto](Temporal, long) +* int {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#checkValidIntValue(long)[checkValidIntValue](long) +* long {java11-javadoc}/java.base/java/time/temporal/ChronoField.html#checkValidValue(long)[checkValidValue](long) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* TemporalUnit {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getBaseUnit()[getBaseUnit]() +* String {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getDisplayName(java.util.Locale)[getDisplayName](Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getFrom(java.time.temporal.TemporalAccessor)[getFrom](TemporalAccessor) +* TemporalUnit {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getRangeUnit()[getRangeUnit]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#isDateBased()[isDateBased]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#isSupportedBy(java.time.temporal.TemporalAccessor)[isSupportedBy](TemporalAccessor) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#isTimeBased()[isTimeBased]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#range()[range]() +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#rangeRefinedBy(java.time.temporal.TemporalAccessor)[rangeRefinedBy](TemporalAccessor) +* TemporalAccessor {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#resolve(java.util.Map,java.time.temporal.TemporalAccessor,java.time.format.ResolverStyle)[resolve](Map, TemporalAccessor, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#toString()[toString]() + + +[[painless-api-reference-shared-ChronoUnit]] +==== ChronoUnit +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#CENTURIES[CENTURIES] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#DAYS[DAYS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#DECADES[DECADES] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#ERAS[ERAS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#FOREVER[FOREVER] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#HALF_DAYS[HALF_DAYS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#HOURS[HOURS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#MICROS[MICROS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#MILLENNIA[MILLENNIA] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#MILLIS[MILLIS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#MINUTES[MINUTES] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#MONTHS[MONTHS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#NANOS[NANOS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#SECONDS[SECONDS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#WEEKS[WEEKS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#YEARS[YEARS] +* static ChronoUnit {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#valueOf(java.lang.String)[valueOf](String) +* static ChronoUnit[] {java11-javadoc}/java.base/java/time/temporal/ChronoUnit.html#values()[values]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#addTo(java.time.temporal.Temporal,long)[addTo](Temporal, long) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#between(java.time.temporal.Temporal,java.time.temporal.Temporal)[between](Temporal, Temporal) +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Duration {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#getDuration()[getDuration]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isDateBased()[isDateBased]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isDurationEstimated()[isDurationEstimated]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isSupportedBy(java.time.temporal.Temporal)[isSupportedBy](Temporal) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isTimeBased()[isTimeBased]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#toString()[toString]() + + +[[painless-api-reference-shared-IsoFields]] +==== IsoFields +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/IsoFields.html#DAY_OF_QUARTER[DAY_OF_QUARTER] +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/IsoFields.html#QUARTER_OF_YEAR[QUARTER_OF_YEAR] +* static TemporalUnit {java11-javadoc}/java.base/java/time/temporal/IsoFields.html#QUARTER_YEARS[QUARTER_YEARS] +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/IsoFields.html#WEEK_BASED_YEAR[WEEK_BASED_YEAR] +* static TemporalUnit {java11-javadoc}/java.base/java/time/temporal/IsoFields.html#WEEK_BASED_YEARS[WEEK_BASED_YEARS] +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/IsoFields.html#WEEK_OF_WEEK_BASED_YEAR[WEEK_OF_WEEK_BASED_YEAR] +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-JulianFields]] +==== JulianFields +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/JulianFields.html#JULIAN_DAY[JULIAN_DAY] +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/JulianFields.html#MODIFIED_JULIAN_DAY[MODIFIED_JULIAN_DAY] +* static TemporalField {java11-javadoc}/java.base/java/time/temporal/JulianFields.html#RATA_DIE[RATA_DIE] +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Temporal]] +==== Temporal +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* Temporal {java11-javadoc}/java.base/java/time/temporal/Temporal.html#minus(java.time.temporal.TemporalAmount)[minus](TemporalAmount) +* Temporal {java11-javadoc}/java.base/java/time/temporal/Temporal.html#minus(long,java.time.temporal.TemporalUnit)[minus](long, TemporalUnit) +* Temporal {java11-javadoc}/java.base/java/time/temporal/Temporal.html#plus(java.time.temporal.TemporalAmount)[plus](TemporalAmount) +* Temporal {java11-javadoc}/java.base/java/time/temporal/Temporal.html#plus(long,java.time.temporal.TemporalUnit)[plus](long, TemporalUnit) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* long {java11-javadoc}/java.base/java/time/temporal/Temporal.html#until(java.time.temporal.Temporal,java.time.temporal.TemporalUnit)[until](Temporal, TemporalUnit) +* Temporal {java11-javadoc}/java.base/java/time/temporal/Temporal.html#with(java.time.temporal.TemporalAdjuster)[with](TemporalAdjuster) +* Temporal {java11-javadoc}/java.base/java/time/temporal/Temporal.html#with(java.time.temporal.TemporalField,long)[with](TemporalField, long) + + +[[painless-api-reference-shared-TemporalAccessor]] +==== TemporalAccessor +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#get(java.time.temporal.TemporalField)[get](TemporalField) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#getLong(java.time.temporal.TemporalField)[getLong](TemporalField) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#isSupported(java.time.temporal.TemporalField)[isSupported](TemporalField) +* def {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#query(java.time.temporal.TemporalQuery)[query](TemporalQuery) +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalAccessor.html#range(java.time.temporal.TemporalField)[range](TemporalField) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalAdjuster]] +==== TemporalAdjuster +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAdjuster.html#adjustInto(java.time.temporal.Temporal)[adjustInto](Temporal) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalAdjusters]] +==== TemporalAdjusters +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#dayOfWeekInMonth(int,java.time.DayOfWeek)[dayOfWeekInMonth](int, DayOfWeek) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#firstDayOfMonth()[firstDayOfMonth]() +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#firstDayOfNextMonth()[firstDayOfNextMonth]() +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#firstDayOfNextYear()[firstDayOfNextYear]() +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#firstDayOfYear()[firstDayOfYear]() +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#firstInMonth(java.time.DayOfWeek)[firstInMonth](DayOfWeek) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#lastDayOfMonth()[lastDayOfMonth]() +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#lastDayOfYear()[lastDayOfYear]() +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#lastInMonth(java.time.DayOfWeek)[lastInMonth](DayOfWeek) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#next(java.time.DayOfWeek)[next](DayOfWeek) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#nextOrSame(java.time.DayOfWeek)[nextOrSame](DayOfWeek) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#ofDateAdjuster(java.util.function.UnaryOperator)[ofDateAdjuster](UnaryOperator) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#previous(java.time.DayOfWeek)[previous](DayOfWeek) +* static TemporalAdjuster {java11-javadoc}/java.base/java/time/temporal/TemporalAdjusters.html#previousOrSame(java.time.DayOfWeek)[previousOrSame](DayOfWeek) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalAmount]] +==== TemporalAmount +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#addTo(java.time.temporal.Temporal)[addTo](Temporal) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#get(java.time.temporal.TemporalUnit)[get](TemporalUnit) +* List {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#getUnits()[getUnits]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalAmount.html#subtractFrom(java.time.temporal.Temporal)[subtractFrom](Temporal) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalField]] +==== TemporalField +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#adjustInto(java.time.temporal.Temporal,long)[adjustInto](Temporal, long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* TemporalUnit {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getBaseUnit()[getBaseUnit]() +* String {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getDisplayName(java.util.Locale)[getDisplayName](Locale) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getFrom(java.time.temporal.TemporalAccessor)[getFrom](TemporalAccessor) +* TemporalUnit {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#getRangeUnit()[getRangeUnit]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#isDateBased()[isDateBased]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#isSupportedBy(java.time.temporal.TemporalAccessor)[isSupportedBy](TemporalAccessor) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#isTimeBased()[isTimeBased]() +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#range()[range]() +* ValueRange {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#rangeRefinedBy(java.time.temporal.TemporalAccessor)[rangeRefinedBy](TemporalAccessor) +* TemporalAccessor {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#resolve(java.util.Map,java.time.temporal.TemporalAccessor,java.time.format.ResolverStyle)[resolve](Map, TemporalAccessor, ResolverStyle) +* String {java11-javadoc}/java.base/java/time/temporal/TemporalField.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalQueries]] +==== TemporalQueries +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#chronology()[chronology]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#localDate()[localDate]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#localTime()[localTime]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#offset()[offset]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#precision()[precision]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#zone()[zone]() +* static TemporalQuery {java11-javadoc}/java.base/java/time/temporal/TemporalQueries.html#zoneId()[zoneId]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalQuery]] +==== TemporalQuery +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/time/temporal/TemporalQuery.html#queryFrom(java.time.temporal.TemporalAccessor)[queryFrom](TemporalAccessor) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TemporalUnit]] +==== TemporalUnit +* Temporal {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#addTo(java.time.temporal.Temporal,long)[addTo](Temporal, long) +* long {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#between(java.time.temporal.Temporal,java.time.temporal.Temporal)[between](Temporal, Temporal) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Duration {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#getDuration()[getDuration]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isDateBased()[isDateBased]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isDurationEstimated()[isDurationEstimated]() +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isSupportedBy(java.time.temporal.Temporal)[isSupportedBy](Temporal) +* boolean {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#isTimeBased()[isTimeBased]() +* String {java11-javadoc}/java.base/java/time/temporal/TemporalUnit.html#toString()[toString]() + + +[[painless-api-reference-shared-UnsupportedTemporalTypeException]] +==== UnsupportedTemporalTypeException +* {java11-javadoc}/java.base/java/time/temporal/UnsupportedTemporalTypeException.html#(java.lang.String)[UnsupportedTemporalTypeException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ValueRange]] +==== ValueRange +* static ValueRange {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#of(long,long)[of](long, long) +* static ValueRange {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#of(long,long,long)[of](long, long, long) +* static ValueRange {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#of(long,long,long,long)[of](long, long, long, long) +* int {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#checkValidIntValue(long,java.time.temporal.TemporalField)[checkValidIntValue](long, TemporalField) +* long {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#checkValidValue(long,java.time.temporal.TemporalField)[checkValidValue](long, TemporalField) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#getLargestMinimum()[getLargestMinimum]() +* long {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#getMaximum()[getMaximum]() +* long {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#getMinimum()[getMinimum]() +* long {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#getSmallestMaximum()[getSmallestMaximum]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#isFixed()[isFixed]() +* boolean {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#isIntValue()[isIntValue]() +* boolean {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#isValidIntValue(long)[isValidIntValue](long) +* boolean {java11-javadoc}/java.base/java/time/temporal/ValueRange.html#isValidValue(long)[isValidValue](long) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-WeekFields]] +==== WeekFields +* static WeekFields {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#ISO[ISO] +* static WeekFields {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#SUNDAY_START[SUNDAY_START] +* static TemporalUnit {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#WEEK_BASED_YEARS[WEEK_BASED_YEARS] +* static WeekFields {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#of(java.util.Locale)[of](Locale) +* static WeekFields {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#of(java.time.DayOfWeek,int)[of](DayOfWeek, int) +* TemporalField {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#dayOfWeek()[dayOfWeek]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* DayOfWeek {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#getFirstDayOfWeek()[getFirstDayOfWeek]() +* int {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#getMinimalDaysInFirstWeek()[getMinimalDaysInFirstWeek]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* TemporalField {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#weekBasedYear()[weekBasedYear]() +* TemporalField {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#weekOfMonth()[weekOfMonth]() +* TemporalField {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#weekOfWeekBasedYear()[weekOfWeekBasedYear]() +* TemporalField {java11-javadoc}/java.base/java/time/temporal/WeekFields.html#weekOfYear()[weekOfYear]() + + +[role="exclude",id="painless-api-reference-shared-java-time-zone"] +=== Shared API for package java.time.zone +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-ZoneOffsetTransition]] +==== ZoneOffsetTransition +* static ZoneOffsetTransition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#of(java.time.LocalDateTime,java.time.ZoneOffset,java.time.ZoneOffset)[of](LocalDateTime, ZoneOffset, ZoneOffset) +* int {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#compareTo(java.time.zone.ZoneOffsetTransition)[compareTo](ZoneOffsetTransition) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* LocalDateTime {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#getDateTimeAfter()[getDateTimeAfter]() +* LocalDateTime {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#getDateTimeBefore()[getDateTimeBefore]() +* Duration {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#getDuration()[getDuration]() +* Instant {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#getInstant()[getInstant]() +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#getOffsetAfter()[getOffsetAfter]() +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#getOffsetBefore()[getOffsetBefore]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#isGap()[isGap]() +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#isOverlap()[isOverlap]() +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#isValidOffset(java.time.ZoneOffset)[isValidOffset](ZoneOffset) +* long {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransition.html#toEpochSecond()[toEpochSecond]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZoneOffsetTransitionRule]] +==== ZoneOffsetTransitionRule +* static ZoneOffsetTransitionRule {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#of(java.time.Month,int,java.time.DayOfWeek,java.time.LocalTime,boolean,java.time.zone.ZoneOffsetTransitionRule$TimeDefinition,java.time.ZoneOffset,java.time.ZoneOffset,java.time.ZoneOffset)[of](Month, int, DayOfWeek, LocalTime, boolean, ZoneOffsetTransitionRule.TimeDefinition, ZoneOffset, ZoneOffset, ZoneOffset) +* ZoneOffsetTransition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#createTransition(int)[createTransition](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getDayOfMonthIndicator()[getDayOfMonthIndicator]() +* DayOfWeek {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getDayOfWeek()[getDayOfWeek]() +* LocalTime {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getLocalTime()[getLocalTime]() +* Month {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getMonth()[getMonth]() +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getOffsetAfter()[getOffsetAfter]() +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getOffsetBefore()[getOffsetBefore]() +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getStandardOffset()[getStandardOffset]() +* ZoneOffsetTransitionRule.TimeDefinition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#getTimeDefinition()[getTimeDefinition]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule.html#isMidnightEndOfDay()[isMidnightEndOfDay]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZoneOffsetTransitionRule-TimeDefinition]] +==== ZoneOffsetTransitionRule.TimeDefinition +* static ZoneOffsetTransitionRule.TimeDefinition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule$TimeDefinition.html#STANDARD[STANDARD] +* static ZoneOffsetTransitionRule.TimeDefinition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule$TimeDefinition.html#UTC[UTC] +* static ZoneOffsetTransitionRule.TimeDefinition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule$TimeDefinition.html#WALL[WALL] +* static ZoneOffsetTransitionRule.TimeDefinition {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule$TimeDefinition.html#valueOf(java.lang.String)[valueOf](String) +* static ZoneOffsetTransitionRule.TimeDefinition[] {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule$TimeDefinition.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* LocalDateTime {java11-javadoc}/java.base/java/time/zone/ZoneOffsetTransitionRule$TimeDefinition.html#createDateTime(java.time.LocalDateTime,java.time.ZoneOffset,java.time.ZoneOffset)[createDateTime](LocalDateTime, ZoneOffset, ZoneOffset) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZoneRules]] +==== ZoneRules +* static ZoneRules {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#of(java.time.ZoneOffset)[of](ZoneOffset) +* static ZoneRules {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#of(java.time.ZoneOffset,java.time.ZoneOffset,java.util.List,java.util.List,java.util.List)[of](ZoneOffset, ZoneOffset, List, List, List) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Duration {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getDaylightSavings(java.time.Instant)[getDaylightSavings](Instant) +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getOffset(java.time.Instant)[getOffset](Instant) +* ZoneOffset {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getStandardOffset(java.time.Instant)[getStandardOffset](Instant) +* ZoneOffsetTransition {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getTransition(java.time.LocalDateTime)[getTransition](LocalDateTime) +* List {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getTransitionRules()[getTransitionRules]() +* List {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getTransitions()[getTransitions]() +* List {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#getValidOffsets(java.time.LocalDateTime)[getValidOffsets](LocalDateTime) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#isDaylightSavings(java.time.Instant)[isDaylightSavings](Instant) +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#isFixedOffset()[isFixedOffset]() +* boolean {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#isValidOffset(java.time.LocalDateTime,java.time.ZoneOffset)[isValidOffset](LocalDateTime, ZoneOffset) +* ZoneOffsetTransition {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#nextTransition(java.time.Instant)[nextTransition](Instant) +* ZoneOffsetTransition {java11-javadoc}/java.base/java/time/zone/ZoneRules.html#previousTransition(java.time.Instant)[previousTransition](Instant) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZoneRulesException]] +==== ZoneRulesException +* {java11-javadoc}/java.base/java/time/zone/ZoneRulesException.html#(java.lang.String)[ZoneRulesException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ZoneRulesProvider]] +==== ZoneRulesProvider +* static Set {java11-javadoc}/java.base/java/time/zone/ZoneRulesProvider.html#getAvailableZoneIds()[getAvailableZoneIds]() +* static ZoneRules {java11-javadoc}/java.base/java/time/zone/ZoneRulesProvider.html#getRules(java.lang.String,boolean)[getRules](String, boolean) +* static NavigableMap {java11-javadoc}/java.base/java/time/zone/ZoneRulesProvider.html#getVersions(java.lang.String)[getVersions](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-util"] +=== Shared API for package java.util +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-AbstractCollection]] +==== AbstractCollection +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AbstractList]] +==== AbstractList +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AbstractMap]] +==== AbstractMap +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-AbstractMap-SimpleEntry]] +==== AbstractMap.SimpleEntry +* {java11-javadoc}/java.base/java/util/AbstractMap$SimpleEntry.html#(java.util.Map$Entry)[AbstractMap.SimpleEntry](Map.Entry) +* {java11-javadoc}/java.base/java/util/AbstractMap$SimpleEntry.html#(java.lang.Object,java.lang.Object)[AbstractMap.SimpleEntry](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map$Entry.html#equals(java.lang.Object)[equals](Object) +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#getKey()[getKey]() +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/util/Map$Entry.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#setValue(java.lang.Object)[setValue](def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AbstractMap-SimpleImmutableEntry]] +==== AbstractMap.SimpleImmutableEntry +* {java11-javadoc}/java.base/java/util/AbstractMap$SimpleImmutableEntry.html#(java.util.Map$Entry)[AbstractMap.SimpleImmutableEntry](Map.Entry) +* {java11-javadoc}/java.base/java/util/AbstractMap$SimpleImmutableEntry.html#(java.lang.Object,java.lang.Object)[AbstractMap.SimpleImmutableEntry](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map$Entry.html#equals(java.lang.Object)[equals](Object) +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#getKey()[getKey]() +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/util/Map$Entry.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#setValue(java.lang.Object)[setValue](def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AbstractQueue]] +==== AbstractQueue +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Queue.html#element()[element]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Queue.html#offer(java.lang.Object)[offer](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Queue.html#poll()[poll]() +* def {java11-javadoc}/java.base/java/util/Queue.html#remove()[remove]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AbstractSequentialList]] +==== AbstractSequentialList +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-AbstractSet]] +==== AbstractSet +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ArrayDeque]] +==== ArrayDeque +* {java11-javadoc}/java.base/java/util/ArrayDeque.html#()[ArrayDeque]() +* {java11-javadoc}/java.base/java/util/ArrayDeque.html#(java.util.Collection)[ArrayDeque](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* void {java11-javadoc}/java.base/java/util/Deque.html#addFirst(java.lang.Object)[addFirst](def) +* void {java11-javadoc}/java.base/java/util/Deque.html#addLast(java.lang.Object)[addLast](def) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* ArrayDeque {java11-javadoc}/java.base/java/util/ArrayDeque.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* Iterator {java11-javadoc}/java.base/java/util/Deque.html#descendingIterator()[descendingIterator]() +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Queue.html#element()[element]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/Deque.html#getFirst()[getFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#getLast()[getLast]() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Queue.html#offer(java.lang.Object)[offer](def) +* boolean {java11-javadoc}/java.base/java/util/Deque.html#offerFirst(java.lang.Object)[offerFirst](def) +* boolean {java11-javadoc}/java.base/java/util/Deque.html#offerLast(java.lang.Object)[offerLast](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Deque.html#peekFirst()[peekFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#peekLast()[peekLast]() +* def {java11-javadoc}/java.base/java/util/Queue.html#poll()[poll]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pollFirst()[pollFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pollLast()[pollLast]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pop()[pop]() +* void {java11-javadoc}/java.base/java/util/Deque.html#push(java.lang.Object)[push](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#remove()[remove]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* def {java11-javadoc}/java.base/java/util/Deque.html#removeFirst()[removeFirst]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#removeFirstOccurrence(java.lang.Object)[removeFirstOccurrence](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* def {java11-javadoc}/java.base/java/util/Deque.html#removeLast()[removeLast]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#removeLastOccurrence(java.lang.Object)[removeLastOccurrence](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ArrayList]] +==== ArrayList +* {java11-javadoc}/java.base/java/util/ArrayList.html#()[ArrayList]() +* {java11-javadoc}/java.base/java/util/ArrayList.html#(java.util.Collection)[ArrayList](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/ArrayList.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* void {java11-javadoc}/java.base/java/util/ArrayList.html#trimToSize()[trimToSize]() + + +[[painless-api-reference-shared-Arrays]] +==== Arrays +* static List {java11-javadoc}/java.base/java/util/Arrays.html#asList(java.lang.Object%5B%5D)[asList](Object[]) +* static boolean {java11-javadoc}/java.base/java/util/Arrays.html#deepEquals(java.lang.Object%5B%5D,java.lang.Object%5B%5D)[deepEquals](Object[], Object[]) +* static int {java11-javadoc}/java.base/java/util/Arrays.html#deepHashCode(java.lang.Object%5B%5D)[deepHashCode](Object[]) +* static String {java11-javadoc}/java.base/java/util/Arrays.html#deepToString(java.lang.Object%5B%5D)[deepToString](Object[]) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Base64]] +==== Base64 +* static Base64.Decoder {java11-javadoc}/java.base/java/util/Base64.html#getDecoder()[getDecoder]() +* static Base64.Encoder {java11-javadoc}/java.base/java/util/Base64.html#getEncoder()[getEncoder]() +* static Base64.Decoder {java11-javadoc}/java.base/java/util/Base64.html#getMimeDecoder()[getMimeDecoder]() +* static Base64.Encoder {java11-javadoc}/java.base/java/util/Base64.html#getMimeEncoder()[getMimeEncoder]() +* static Base64.Encoder {java11-javadoc}/java.base/java/util/Base64.html#getMimeEncoder(int,byte%5B%5D)[getMimeEncoder](int, byte[]) +* static Base64.Decoder {java11-javadoc}/java.base/java/util/Base64.html#getUrlDecoder()[getUrlDecoder]() +* static Base64.Encoder {java11-javadoc}/java.base/java/util/Base64.html#getUrlEncoder()[getUrlEncoder]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Base64-Decoder]] +==== Base64.Decoder +* byte[] {java11-javadoc}/java.base/java/util/Base64$Decoder.html#decode(java.lang.String)[decode](String) +* int {java11-javadoc}/java.base/java/util/Base64$Decoder.html#decode(byte%5B%5D,byte%5B%5D)[decode](byte[], byte[]) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Base64-Encoder]] +==== Base64.Encoder +* int {java11-javadoc}/java.base/java/util/Base64$Encoder.html#encode(byte%5B%5D,byte%5B%5D)[encode](byte[], byte[]) +* String {java11-javadoc}/java.base/java/util/Base64$Encoder.html#encodeToString(byte%5B%5D)[encodeToString](byte[]) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Base64.Encoder {java11-javadoc}/java.base/java/util/Base64$Encoder.html#withoutPadding()[withoutPadding]() + + +[[painless-api-reference-shared-BitSet]] +==== BitSet +* static BitSet {java11-javadoc}/java.base/java/util/BitSet.html#valueOf(long%5B%5D)[valueOf](long[]) +* {java11-javadoc}/java.base/java/util/BitSet.html#()[BitSet]() +* {java11-javadoc}/java.base/java/util/BitSet.html#(int)[BitSet](int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#and(java.util.BitSet)[and](BitSet) +* void {java11-javadoc}/java.base/java/util/BitSet.html#andNot(java.util.BitSet)[andNot](BitSet) +* int {java11-javadoc}/java.base/java/util/BitSet.html#cardinality()[cardinality]() +* void {java11-javadoc}/java.base/java/util/BitSet.html#clear()[clear]() +* void {java11-javadoc}/java.base/java/util/BitSet.html#clear(int)[clear](int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#clear(int,int)[clear](int, int) +* def {java11-javadoc}/java.base/java/util/BitSet.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/BitSet.html#flip(int)[flip](int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#flip(int,int)[flip](int, int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/BitSet.html#intersects(java.util.BitSet)[intersects](BitSet) +* boolean {java11-javadoc}/java.base/java/util/BitSet.html#isEmpty()[isEmpty]() +* int {java11-javadoc}/java.base/java/util/BitSet.html#length()[length]() +* int {java11-javadoc}/java.base/java/util/BitSet.html#nextClearBit(int)[nextClearBit](int) +* int {java11-javadoc}/java.base/java/util/BitSet.html#nextSetBit(int)[nextSetBit](int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#or(java.util.BitSet)[or](BitSet) +* int {java11-javadoc}/java.base/java/util/BitSet.html#previousClearBit(int)[previousClearBit](int) +* int {java11-javadoc}/java.base/java/util/BitSet.html#previousSetBit(int)[previousSetBit](int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#set(int)[set](int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#set(int,int)[set](int, int) +* void {java11-javadoc}/java.base/java/util/BitSet.html#set(int,int,boolean)[set](int, int, boolean) +* int {java11-javadoc}/java.base/java/util/BitSet.html#size()[size]() +* byte[] {java11-javadoc}/java.base/java/util/BitSet.html#toByteArray()[toByteArray]() +* long[] {java11-javadoc}/java.base/java/util/BitSet.html#toLongArray()[toLongArray]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* void {java11-javadoc}/java.base/java/util/BitSet.html#xor(java.util.BitSet)[xor](BitSet) + + +[[painless-api-reference-shared-Calendar]] +==== Calendar +* static int {java11-javadoc}/java.base/java/util/Calendar.html#ALL_STYLES[ALL_STYLES] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#AM[AM] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#AM_PM[AM_PM] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#APRIL[APRIL] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#AUGUST[AUGUST] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DATE[DATE] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DAY_OF_MONTH[DAY_OF_MONTH] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DAY_OF_WEEK[DAY_OF_WEEK] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DAY_OF_WEEK_IN_MONTH[DAY_OF_WEEK_IN_MONTH] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DAY_OF_YEAR[DAY_OF_YEAR] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DECEMBER[DECEMBER] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#DST_OFFSET[DST_OFFSET] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#ERA[ERA] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#FEBRUARY[FEBRUARY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#FIELD_COUNT[FIELD_COUNT] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#FRIDAY[FRIDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#HOUR[HOUR] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#HOUR_OF_DAY[HOUR_OF_DAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#JANUARY[JANUARY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#JULY[JULY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#JUNE[JUNE] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#LONG[LONG] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#LONG_FORMAT[LONG_FORMAT] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#LONG_STANDALONE[LONG_STANDALONE] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#MARCH[MARCH] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#MAY[MAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#MILLISECOND[MILLISECOND] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#MINUTE[MINUTE] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#MONDAY[MONDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#MONTH[MONTH] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#NARROW_FORMAT[NARROW_FORMAT] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#NARROW_STANDALONE[NARROW_STANDALONE] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#NOVEMBER[NOVEMBER] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#OCTOBER[OCTOBER] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#PM[PM] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SATURDAY[SATURDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SECOND[SECOND] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SEPTEMBER[SEPTEMBER] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SHORT[SHORT] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SHORT_FORMAT[SHORT_FORMAT] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SHORT_STANDALONE[SHORT_STANDALONE] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#SUNDAY[SUNDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#THURSDAY[THURSDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#TUESDAY[TUESDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#UNDECIMBER[UNDECIMBER] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#WEDNESDAY[WEDNESDAY] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#WEEK_OF_MONTH[WEEK_OF_MONTH] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#WEEK_OF_YEAR[WEEK_OF_YEAR] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#YEAR[YEAR] +* static int {java11-javadoc}/java.base/java/util/Calendar.html#ZONE_OFFSET[ZONE_OFFSET] +* static Set {java11-javadoc}/java.base/java/util/Calendar.html#getAvailableCalendarTypes()[getAvailableCalendarTypes]() +* static Locale[] {java11-javadoc}/java.base/java/util/Calendar.html#getAvailableLocales()[getAvailableLocales]() +* static Calendar {java11-javadoc}/java.base/java/util/Calendar.html#getInstance()[getInstance]() +* static Calendar {java11-javadoc}/java.base/java/util/Calendar.html#getInstance(java.util.TimeZone)[getInstance](TimeZone) +* static Calendar {java11-javadoc}/java.base/java/util/Calendar.html#getInstance(java.util.TimeZone,java.util.Locale)[getInstance](TimeZone, Locale) +* void {java11-javadoc}/java.base/java/util/Calendar.html#add(int,int)[add](int, int) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#after(java.lang.Object)[after](Object) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#before(java.lang.Object)[before](Object) +* void {java11-javadoc}/java.base/java/util/Calendar.html#clear()[clear]() +* void {java11-javadoc}/java.base/java/util/Calendar.html#clear(int)[clear](int) +* def {java11-javadoc}/java.base/java/util/Calendar.html#clone()[clone]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#compareTo(java.util.Calendar)[compareTo](Calendar) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/Calendar.html#get(int)[get](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getActualMaximum(int)[getActualMaximum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getActualMinimum(int)[getActualMinimum](int) +* String {java11-javadoc}/java.base/java/util/Calendar.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/util/Calendar.html#getDisplayName(int,int,java.util.Locale)[getDisplayName](int, int, Locale) +* Map {java11-javadoc}/java.base/java/util/Calendar.html#getDisplayNames(int,int,java.util.Locale)[getDisplayNames](int, int, Locale) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getFirstDayOfWeek()[getFirstDayOfWeek]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getGreatestMinimum(int)[getGreatestMinimum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getLeastMaximum(int)[getLeastMaximum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getMaximum(int)[getMaximum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getMinimalDaysInFirstWeek()[getMinimalDaysInFirstWeek]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getMinimum(int)[getMinimum](int) +* Date {java11-javadoc}/java.base/java/util/Calendar.html#getTime()[getTime]() +* long {java11-javadoc}/java.base/java/util/Calendar.html#getTimeInMillis()[getTimeInMillis]() +* TimeZone {java11-javadoc}/java.base/java/util/Calendar.html#getTimeZone()[getTimeZone]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getWeekYear()[getWeekYear]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getWeeksInWeekYear()[getWeeksInWeekYear]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#isLenient()[isLenient]() +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#isSet(int)[isSet](int) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#isWeekDateSupported()[isWeekDateSupported]() +* void {java11-javadoc}/java.base/java/util/Calendar.html#roll(int,int)[roll](int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int)[set](int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int,int)[set](int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int,int,int,int)[set](int, int, int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int,int,int,int,int)[set](int, int, int, int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setFirstDayOfWeek(int)[setFirstDayOfWeek](int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setLenient(boolean)[setLenient](boolean) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setMinimalDaysInFirstWeek(int)[setMinimalDaysInFirstWeek](int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setTime(java.util.Date)[setTime](Date) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setTimeInMillis(long)[setTimeInMillis](long) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setTimeZone(java.util.TimeZone)[setTimeZone](TimeZone) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setWeekDate(int,int,int)[setWeekDate](int, int, int) +* Instant {java11-javadoc}/java.base/java/util/Calendar.html#toInstant()[toInstant]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Calendar-Builder]] +==== Calendar.Builder +* {java11-javadoc}/java.base/java/util/Calendar$Builder.html#()[Calendar.Builder]() +* Calendar {java11-javadoc}/java.base/java/util/Calendar$Builder.html#build()[build]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#set(int,int)[set](int, int) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setCalendarType(java.lang.String)[setCalendarType](String) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setDate(int,int,int)[setDate](int, int, int) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setFields(int%5B%5D)[setFields](int[]) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setInstant(long)[setInstant](long) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setLenient(boolean)[setLenient](boolean) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setLocale(java.util.Locale)[setLocale](Locale) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setTimeOfDay(int,int,int)[setTimeOfDay](int, int, int) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setTimeOfDay(int,int,int,int)[setTimeOfDay](int, int, int, int) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setTimeZone(java.util.TimeZone)[setTimeZone](TimeZone) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setWeekDate(int,int,int)[setWeekDate](int, int, int) +* Calendar.Builder {java11-javadoc}/java.base/java/util/Calendar$Builder.html#setWeekDefinition(int,int)[setWeekDefinition](int, int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Collection]] +==== Collection +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Collections]] +==== Collections +* static List {java11-javadoc}/java.base/java/util/Collections.html#EMPTY_LIST[EMPTY_LIST] +* static Map {java11-javadoc}/java.base/java/util/Collections.html#EMPTY_MAP[EMPTY_MAP] +* static Set {java11-javadoc}/java.base/java/util/Collections.html#EMPTY_SET[EMPTY_SET] +* static boolean {java11-javadoc}/java.base/java/util/Collections.html#addAll(java.util.Collection,java.lang.Object%5B%5D)[addAll](Collection, def[]) +* static Queue {java11-javadoc}/java.base/java/util/Collections.html#asLifoQueue(java.util.Deque)[asLifoQueue](Deque) +* static int {java11-javadoc}/java.base/java/util/Collections.html#binarySearch(java.util.List,java.lang.Object)[binarySearch](List, def) +* static int {java11-javadoc}/java.base/java/util/Collections.html#binarySearch(java.util.List,java.lang.Object,java.util.Comparator)[binarySearch](List, def, Comparator) +* static void {java11-javadoc}/java.base/java/util/Collections.html#copy(java.util.List,java.util.List)[copy](List, List) +* static boolean {java11-javadoc}/java.base/java/util/Collections.html#disjoint(java.util.Collection,java.util.Collection)[disjoint](Collection, Collection) +* static Enumeration {java11-javadoc}/java.base/java/util/Collections.html#emptyEnumeration()[emptyEnumeration]() +* static Iterator {java11-javadoc}/java.base/java/util/Collections.html#emptyIterator()[emptyIterator]() +* static List {java11-javadoc}/java.base/java/util/Collections.html#emptyList()[emptyList]() +* static ListIterator {java11-javadoc}/java.base/java/util/Collections.html#emptyListIterator()[emptyListIterator]() +* static Map {java11-javadoc}/java.base/java/util/Collections.html#emptyMap()[emptyMap]() +* static NavigableMap {java11-javadoc}/java.base/java/util/Collections.html#emptyNavigableMap()[emptyNavigableMap]() +* static NavigableSet {java11-javadoc}/java.base/java/util/Collections.html#emptyNavigableSet()[emptyNavigableSet]() +* static Set {java11-javadoc}/java.base/java/util/Collections.html#emptySet()[emptySet]() +* static SortedMap {java11-javadoc}/java.base/java/util/Collections.html#emptySortedMap()[emptySortedMap]() +* static SortedSet {java11-javadoc}/java.base/java/util/Collections.html#emptySortedSet()[emptySortedSet]() +* static Enumeration {java11-javadoc}/java.base/java/util/Collections.html#enumeration(java.util.Collection)[enumeration](Collection) +* static void {java11-javadoc}/java.base/java/util/Collections.html#fill(java.util.List,java.lang.Object)[fill](List, def) +* static int {java11-javadoc}/java.base/java/util/Collections.html#frequency(java.util.Collection,java.lang.Object)[frequency](Collection, def) +* static int {java11-javadoc}/java.base/java/util/Collections.html#indexOfSubList(java.util.List,java.util.List)[indexOfSubList](List, List) +* static int {java11-javadoc}/java.base/java/util/Collections.html#lastIndexOfSubList(java.util.List,java.util.List)[lastIndexOfSubList](List, List) +* static ArrayList {java11-javadoc}/java.base/java/util/Collections.html#list(java.util.Enumeration)[list](Enumeration) +* static def {java11-javadoc}/java.base/java/util/Collections.html#max(java.util.Collection)[max](Collection) +* static def {java11-javadoc}/java.base/java/util/Collections.html#max(java.util.Collection,java.util.Comparator)[max](Collection, Comparator) +* static def {java11-javadoc}/java.base/java/util/Collections.html#min(java.util.Collection)[min](Collection) +* static def {java11-javadoc}/java.base/java/util/Collections.html#min(java.util.Collection,java.util.Comparator)[min](Collection, Comparator) +* static List {java11-javadoc}/java.base/java/util/Collections.html#nCopies(int,java.lang.Object)[nCopies](int, def) +* static Set {java11-javadoc}/java.base/java/util/Collections.html#newSetFromMap(java.util.Map)[newSetFromMap](Map) +* static boolean {java11-javadoc}/java.base/java/util/Collections.html#replaceAll(java.util.List,java.lang.Object,java.lang.Object)[replaceAll](List, def, def) +* static void {java11-javadoc}/java.base/java/util/Collections.html#reverse(java.util.List)[reverse](List) +* static Comparator {java11-javadoc}/java.base/java/util/Collections.html#reverseOrder()[reverseOrder]() +* static Comparator {java11-javadoc}/java.base/java/util/Collections.html#reverseOrder(java.util.Comparator)[reverseOrder](Comparator) +* static void {java11-javadoc}/java.base/java/util/Collections.html#rotate(java.util.List,int)[rotate](List, int) +* static void {java11-javadoc}/java.base/java/util/Collections.html#shuffle(java.util.List)[shuffle](List) +* static void {java11-javadoc}/java.base/java/util/Collections.html#shuffle(java.util.List,java.util.Random)[shuffle](List, Random) +* static Set {java11-javadoc}/java.base/java/util/Collections.html#singleton(java.lang.Object)[singleton](def) +* static List {java11-javadoc}/java.base/java/util/Collections.html#singletonList(java.lang.Object)[singletonList](def) +* static Map {java11-javadoc}/java.base/java/util/Collections.html#singletonMap(java.lang.Object,java.lang.Object)[singletonMap](def, def) +* static void {java11-javadoc}/java.base/java/util/Collections.html#sort(java.util.List)[sort](List) +* static void {java11-javadoc}/java.base/java/util/Collections.html#sort(java.util.List,java.util.Comparator)[sort](List, Comparator) +* static void {java11-javadoc}/java.base/java/util/Collections.html#swap(java.util.List,int,int)[swap](List, int, int) +* static Collection {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableCollection(java.util.Collection)[unmodifiableCollection](Collection) +* static List {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableList(java.util.List)[unmodifiableList](List) +* static Map {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableMap(java.util.Map)[unmodifiableMap](Map) +* static NavigableMap {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableNavigableMap(java.util.NavigableMap)[unmodifiableNavigableMap](NavigableMap) +* static NavigableSet {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableNavigableSet(java.util.NavigableSet)[unmodifiableNavigableSet](NavigableSet) +* static Set {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableSet(java.util.Set)[unmodifiableSet](Set) +* static SortedMap {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableSortedMap(java.util.SortedMap)[unmodifiableSortedMap](SortedMap) +* static SortedSet {java11-javadoc}/java.base/java/util/Collections.html#unmodifiableSortedSet(java.util.SortedSet)[unmodifiableSortedSet](SortedSet) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Comparator]] +==== Comparator +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#comparing(java.util.function.Function)[comparing](Function) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#comparing(java.util.function.Function,java.util.Comparator)[comparing](Function, Comparator) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#comparingDouble(java.util.function.ToDoubleFunction)[comparingDouble](ToDoubleFunction) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#comparingInt(java.util.function.ToIntFunction)[comparingInt](ToIntFunction) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#comparingLong(java.util.function.ToLongFunction)[comparingLong](ToLongFunction) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#naturalOrder()[naturalOrder]() +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#nullsFirst(java.util.Comparator)[nullsFirst](Comparator) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#nullsLast(java.util.Comparator)[nullsLast](Comparator) +* static Comparator {java11-javadoc}/java.base/java/util/Comparator.html#reverseOrder()[reverseOrder]() +* int {java11-javadoc}/java.base/java/util/Comparator.html#compare(java.lang.Object,java.lang.Object)[compare](def, def) +* boolean {java11-javadoc}/java.base/java/util/Comparator.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#reversed()[reversed]() +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparing(java.util.Comparator)[thenComparing](Comparator) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparing(java.util.function.Function,java.util.Comparator)[thenComparing](Function, Comparator) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingDouble(java.util.function.ToDoubleFunction)[thenComparingDouble](ToDoubleFunction) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingInt(java.util.function.ToIntFunction)[thenComparingInt](ToIntFunction) +* Comparator {java11-javadoc}/java.base/java/util/Comparator.html#thenComparingLong(java.util.function.ToLongFunction)[thenComparingLong](ToLongFunction) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ConcurrentModificationException]] +==== ConcurrentModificationException +* {java11-javadoc}/java.base/java/util/ConcurrentModificationException.html#()[ConcurrentModificationException]() +* {java11-javadoc}/java.base/java/util/ConcurrentModificationException.html#(java.lang.String)[ConcurrentModificationException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Currency]] +==== Currency +* static Set {java11-javadoc}/java.base/java/util/Currency.html#getAvailableCurrencies()[getAvailableCurrencies]() +* static Currency {java11-javadoc}/java.base/java/util/Currency.html#getInstance(java.lang.String)[getInstance](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/Currency.html#getCurrencyCode()[getCurrencyCode]() +* int {java11-javadoc}/java.base/java/util/Currency.html#getDefaultFractionDigits()[getDefaultFractionDigits]() +* String {java11-javadoc}/java.base/java/util/Currency.html#getDisplayName()[getDisplayName]() +* String {java11-javadoc}/java.base/java/util/Currency.html#getDisplayName(java.util.Locale)[getDisplayName](Locale) +* int {java11-javadoc}/java.base/java/util/Currency.html#getNumericCode()[getNumericCode]() +* String {java11-javadoc}/java.base/java/util/Currency.html#getSymbol()[getSymbol]() +* String {java11-javadoc}/java.base/java/util/Currency.html#getSymbol(java.util.Locale)[getSymbol](Locale) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Date]] +==== Date +* static Date {java11-javadoc}/java.base/java/util/Date.html#from(java.time.Instant)[from](Instant) +* {java11-javadoc}/java.base/java/util/Date.html#()[Date]() +* {java11-javadoc}/java.base/java/util/Date.html#(long)[Date](long) +* boolean {java11-javadoc}/java.base/java/util/Date.html#after(java.util.Date)[after](Date) +* boolean {java11-javadoc}/java.base/java/util/Date.html#before(java.util.Date)[before](Date) +* def {java11-javadoc}/java.base/java/util/Date.html#clone()[clone]() +* int {java11-javadoc}/java.base/java/util/Date.html#compareTo(java.util.Date)[compareTo](Date) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/Date.html#getTime()[getTime]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/util/Date.html#setTime(long)[setTime](long) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Deque]] +==== Deque +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* void {java11-javadoc}/java.base/java/util/Deque.html#addFirst(java.lang.Object)[addFirst](def) +* void {java11-javadoc}/java.base/java/util/Deque.html#addLast(java.lang.Object)[addLast](def) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* Iterator {java11-javadoc}/java.base/java/util/Deque.html#descendingIterator()[descendingIterator]() +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Queue.html#element()[element]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/Deque.html#getFirst()[getFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#getLast()[getLast]() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Queue.html#offer(java.lang.Object)[offer](def) +* boolean {java11-javadoc}/java.base/java/util/Deque.html#offerFirst(java.lang.Object)[offerFirst](def) +* boolean {java11-javadoc}/java.base/java/util/Deque.html#offerLast(java.lang.Object)[offerLast](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Deque.html#peekFirst()[peekFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#peekLast()[peekLast]() +* def {java11-javadoc}/java.base/java/util/Queue.html#poll()[poll]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pollFirst()[pollFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pollLast()[pollLast]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pop()[pop]() +* void {java11-javadoc}/java.base/java/util/Deque.html#push(java.lang.Object)[push](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#remove()[remove]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* def {java11-javadoc}/java.base/java/util/Deque.html#removeFirst()[removeFirst]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#removeFirstOccurrence(java.lang.Object)[removeFirstOccurrence](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* def {java11-javadoc}/java.base/java/util/Deque.html#removeLast()[removeLast]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#removeLastOccurrence(java.lang.Object)[removeLastOccurrence](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Dictionary]] +==== Dictionary +* Enumeration {java11-javadoc}/java.base/java/util/Dictionary.html#elements()[elements]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* def {java11-javadoc}/java.base/java/util/Dictionary.html#get(java.lang.Object)[get](def) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Dictionary.html#isEmpty()[isEmpty]() +* Enumeration {java11-javadoc}/java.base/java/util/Dictionary.html#keys()[keys]() +* def {java11-javadoc}/java.base/java/util/Dictionary.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* def {java11-javadoc}/java.base/java/util/Dictionary.html#remove(java.lang.Object)[remove](def) +* int {java11-javadoc}/java.base/java/util/Dictionary.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleSummaryStatistics]] +==== DoubleSummaryStatistics +* {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#()[DoubleSummaryStatistics]() +* void {java11-javadoc}/java.base/java/util/function/DoubleConsumer.html#accept(double)[accept](double) +* DoubleConsumer {java11-javadoc}/java.base/java/util/function/DoubleConsumer.html#andThen(java.util.function.DoubleConsumer)[andThen](DoubleConsumer) +* void {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#combine(java.util.DoubleSummaryStatistics)[combine](DoubleSummaryStatistics) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* double {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#getAverage()[getAverage]() +* long {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#getCount()[getCount]() +* double {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#getMax()[getMax]() +* double {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#getMin()[getMin]() +* double {java11-javadoc}/java.base/java/util/DoubleSummaryStatistics.html#getSum()[getSum]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DuplicateFormatFlagsException]] +==== DuplicateFormatFlagsException +* {java11-javadoc}/java.base/java/util/DuplicateFormatFlagsException.html#(java.lang.String)[DuplicateFormatFlagsException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/DuplicateFormatFlagsException.html#getFlags()[getFlags]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-EmptyStackException]] +==== EmptyStackException +* {java11-javadoc}/java.base/java/util/EmptyStackException.html#()[EmptyStackException]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Enumeration]] +==== Enumeration +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/util/Enumeration.html#hasMoreElements()[hasMoreElements]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Enumeration.html#nextElement()[nextElement]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-EventListener]] +==== EventListener +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-EventListenerProxy]] +==== EventListenerProxy +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* EventListener {java11-javadoc}/java.base/java/util/EventListenerProxy.html#getListener()[getListener]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-EventObject]] +==== EventObject +* {java11-javadoc}/java.base/java/util/EventObject.html#(java.lang.Object)[EventObject](Object) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Object {java11-javadoc}/java.base/java/util/EventObject.html#getSource()[getSource]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-FormatFlagsConversionMismatchException]] +==== FormatFlagsConversionMismatchException +* {java11-javadoc}/java.base/java/util/FormatFlagsConversionMismatchException.html#(java.lang.String,char)[FormatFlagsConversionMismatchException](String, char) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* char {java11-javadoc}/java.base/java/util/FormatFlagsConversionMismatchException.html#getConversion()[getConversion]() +* String {java11-javadoc}/java.base/java/util/FormatFlagsConversionMismatchException.html#getFlags()[getFlags]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Formattable]] +==== Formattable +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/Formattable.html#formatTo(java.util.Formatter,int,int,int)[formatTo](Formatter, int, int, int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-FormattableFlags]] +==== FormattableFlags +* static int {java11-javadoc}/java.base/java/util/FormattableFlags.html#ALTERNATE[ALTERNATE] +* static int {java11-javadoc}/java.base/java/util/FormattableFlags.html#LEFT_JUSTIFY[LEFT_JUSTIFY] +* static int {java11-javadoc}/java.base/java/util/FormattableFlags.html#UPPERCASE[UPPERCASE] +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Formatter]] +==== Formatter +* {java11-javadoc}/java.base/java/util/Formatter.html#()[Formatter]() +* {java11-javadoc}/java.base/java/util/Formatter.html#(java.lang.Appendable)[Formatter](Appendable) +* {java11-javadoc}/java.base/java/util/Formatter.html#(java.lang.Appendable,java.util.Locale)[Formatter](Appendable, Locale) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Formatter {java11-javadoc}/java.base/java/util/Formatter.html#format(java.lang.String,java.lang.Object%5B%5D)[format](String, def[]) +* Formatter {java11-javadoc}/java.base/java/util/Formatter.html#format(java.util.Locale,java.lang.String,java.lang.Object%5B%5D)[format](Locale, String, def[]) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Locale {java11-javadoc}/java.base/java/util/Formatter.html#locale()[locale]() +* Appendable {java11-javadoc}/java.base/java/util/Formatter.html#out()[out]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Formatter-BigDecimalLayoutForm]] +==== Formatter.BigDecimalLayoutForm +* static Formatter.BigDecimalLayoutForm {java11-javadoc}/java.base/java/util/Formatter$BigDecimalLayoutForm.html#DECIMAL_FLOAT[DECIMAL_FLOAT] +* static Formatter.BigDecimalLayoutForm {java11-javadoc}/java.base/java/util/Formatter$BigDecimalLayoutForm.html#SCIENTIFIC[SCIENTIFIC] +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-FormatterClosedException]] +==== FormatterClosedException +* {java11-javadoc}/java.base/java/util/FormatterClosedException.html#()[FormatterClosedException]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-GregorianCalendar]] +==== GregorianCalendar +* static int {java11-javadoc}/java.base/java/util/GregorianCalendar.html#AD[AD] +* static int {java11-javadoc}/java.base/java/util/GregorianCalendar.html#BC[BC] +* static GregorianCalendar {java11-javadoc}/java.base/java/util/GregorianCalendar.html#from(java.time.ZonedDateTime)[from](ZonedDateTime) +* {java11-javadoc}/java.base/java/util/GregorianCalendar.html#()[GregorianCalendar]() +* {java11-javadoc}/java.base/java/util/GregorianCalendar.html#(java.util.TimeZone)[GregorianCalendar](TimeZone) +* {java11-javadoc}/java.base/java/util/GregorianCalendar.html#(java.util.TimeZone,java.util.Locale)[GregorianCalendar](TimeZone, Locale) +* {java11-javadoc}/java.base/java/util/GregorianCalendar.html#(int,int,int)[GregorianCalendar](int, int, int) +* {java11-javadoc}/java.base/java/util/GregorianCalendar.html#(int,int,int,int,int)[GregorianCalendar](int, int, int, int, int) +* {java11-javadoc}/java.base/java/util/GregorianCalendar.html#(int,int,int,int,int,int)[GregorianCalendar](int, int, int, int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#add(int,int)[add](int, int) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#after(java.lang.Object)[after](Object) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#before(java.lang.Object)[before](Object) +* void {java11-javadoc}/java.base/java/util/Calendar.html#clear()[clear]() +* void {java11-javadoc}/java.base/java/util/Calendar.html#clear(int)[clear](int) +* def {java11-javadoc}/java.base/java/util/Calendar.html#clone()[clone]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#compareTo(java.util.Calendar)[compareTo](Calendar) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/Calendar.html#get(int)[get](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getActualMaximum(int)[getActualMaximum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getActualMinimum(int)[getActualMinimum](int) +* String {java11-javadoc}/java.base/java/util/Calendar.html#getCalendarType()[getCalendarType]() +* String {java11-javadoc}/java.base/java/util/Calendar.html#getDisplayName(int,int,java.util.Locale)[getDisplayName](int, int, Locale) +* Map {java11-javadoc}/java.base/java/util/Calendar.html#getDisplayNames(int,int,java.util.Locale)[getDisplayNames](int, int, Locale) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getFirstDayOfWeek()[getFirstDayOfWeek]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getGreatestMinimum(int)[getGreatestMinimum](int) +* Date {java11-javadoc}/java.base/java/util/GregorianCalendar.html#getGregorianChange()[getGregorianChange]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getLeastMaximum(int)[getLeastMaximum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getMaximum(int)[getMaximum](int) +* int {java11-javadoc}/java.base/java/util/Calendar.html#getMinimalDaysInFirstWeek()[getMinimalDaysInFirstWeek]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getMinimum(int)[getMinimum](int) +* Date {java11-javadoc}/java.base/java/util/Calendar.html#getTime()[getTime]() +* long {java11-javadoc}/java.base/java/util/Calendar.html#getTimeInMillis()[getTimeInMillis]() +* TimeZone {java11-javadoc}/java.base/java/util/Calendar.html#getTimeZone()[getTimeZone]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getWeekYear()[getWeekYear]() +* int {java11-javadoc}/java.base/java/util/Calendar.html#getWeeksInWeekYear()[getWeeksInWeekYear]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/GregorianCalendar.html#isLeapYear(int)[isLeapYear](int) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#isLenient()[isLenient]() +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#isSet(int)[isSet](int) +* boolean {java11-javadoc}/java.base/java/util/Calendar.html#isWeekDateSupported()[isWeekDateSupported]() +* void {java11-javadoc}/java.base/java/util/Calendar.html#roll(int,int)[roll](int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int)[set](int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int,int)[set](int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int,int,int,int)[set](int, int, int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#set(int,int,int,int,int,int)[set](int, int, int, int, int, int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setFirstDayOfWeek(int)[setFirstDayOfWeek](int) +* void {java11-javadoc}/java.base/java/util/GregorianCalendar.html#setGregorianChange(java.util.Date)[setGregorianChange](Date) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setLenient(boolean)[setLenient](boolean) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setMinimalDaysInFirstWeek(int)[setMinimalDaysInFirstWeek](int) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setTime(java.util.Date)[setTime](Date) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setTimeInMillis(long)[setTimeInMillis](long) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setTimeZone(java.util.TimeZone)[setTimeZone](TimeZone) +* void {java11-javadoc}/java.base/java/util/Calendar.html#setWeekDate(int,int,int)[setWeekDate](int, int, int) +* Instant {java11-javadoc}/java.base/java/util/Calendar.html#toInstant()[toInstant]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* ZonedDateTime {java11-javadoc}/java.base/java/util/GregorianCalendar.html#toZonedDateTime()[toZonedDateTime]() + + +[[painless-api-reference-shared-HashMap]] +==== HashMap +* {java11-javadoc}/java.base/java/util/HashMap.html#()[HashMap]() +* {java11-javadoc}/java.base/java/util/HashMap.html#(java.util.Map)[HashMap](Map) +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/HashMap.html#clone()[clone]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-HashSet]] +==== HashSet +* {java11-javadoc}/java.base/java/util/HashSet.html#()[HashSet]() +* {java11-javadoc}/java.base/java/util/HashSet.html#(java.util.Collection)[HashSet](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/HashSet.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Hashtable]] +==== Hashtable +* {java11-javadoc}/java.base/java/util/Hashtable.html#()[Hashtable]() +* {java11-javadoc}/java.base/java/util/Hashtable.html#(java.util.Map)[Hashtable](Map) +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/Hashtable.html#clone()[clone]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Enumeration {java11-javadoc}/java.base/java/util/Dictionary.html#elements()[elements]() +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* Enumeration {java11-javadoc}/java.base/java/util/Dictionary.html#keys()[keys]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-IdentityHashMap]] +==== IdentityHashMap +* {java11-javadoc}/java.base/java/util/IdentityHashMap.html#()[IdentityHashMap]() +* {java11-javadoc}/java.base/java/util/IdentityHashMap.html#(java.util.Map)[IdentityHashMap](Map) +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/IdentityHashMap.html#clone()[clone]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-IllegalFormatCodePointException]] +==== IllegalFormatCodePointException +* {java11-javadoc}/java.base/java/util/IllegalFormatCodePointException.html#(int)[IllegalFormatCodePointException](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/IllegalFormatCodePointException.html#getCodePoint()[getCodePoint]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalFormatConversionException]] +==== IllegalFormatConversionException +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* char {java11-javadoc}/java.base/java/util/IllegalFormatConversionException.html#getConversion()[getConversion]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalFormatException]] +==== IllegalFormatException +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalFormatFlagsException]] +==== IllegalFormatFlagsException +* {java11-javadoc}/java.base/java/util/IllegalFormatFlagsException.html#(java.lang.String)[IllegalFormatFlagsException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/IllegalFormatFlagsException.html#getFlags()[getFlags]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalFormatPrecisionException]] +==== IllegalFormatPrecisionException +* {java11-javadoc}/java.base/java/util/IllegalFormatPrecisionException.html#(int)[IllegalFormatPrecisionException](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* int {java11-javadoc}/java.base/java/util/IllegalFormatPrecisionException.html#getPrecision()[getPrecision]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllegalFormatWidthException]] +==== IllegalFormatWidthException +* {java11-javadoc}/java.base/java/util/IllegalFormatWidthException.html#(int)[IllegalFormatWidthException](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/util/IllegalFormatWidthException.html#getWidth()[getWidth]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IllformedLocaleException]] +==== IllformedLocaleException +* {java11-javadoc}/java.base/java/util/IllformedLocaleException.html#()[IllformedLocaleException]() +* {java11-javadoc}/java.base/java/util/IllformedLocaleException.html#(java.lang.String)[IllformedLocaleException](String) +* {java11-javadoc}/java.base/java/util/IllformedLocaleException.html#(java.lang.String,int)[IllformedLocaleException](String, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/IllformedLocaleException.html#getErrorIndex()[getErrorIndex]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-InputMismatchException]] +==== InputMismatchException +* {java11-javadoc}/java.base/java/util/InputMismatchException.html#()[InputMismatchException]() +* {java11-javadoc}/java.base/java/util/InputMismatchException.html#(java.lang.String)[InputMismatchException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntSummaryStatistics]] +==== IntSummaryStatistics +* {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#()[IntSummaryStatistics]() +* void {java11-javadoc}/java.base/java/util/function/IntConsumer.html#accept(int)[accept](int) +* IntConsumer {java11-javadoc}/java.base/java/util/function/IntConsumer.html#andThen(java.util.function.IntConsumer)[andThen](IntConsumer) +* void {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#combine(java.util.IntSummaryStatistics)[combine](IntSummaryStatistics) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* double {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#getAverage()[getAverage]() +* long {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#getCount()[getCount]() +* int {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#getMax()[getMax]() +* int {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#getMin()[getMin]() +* long {java11-javadoc}/java.base/java/util/IntSummaryStatistics.html#getSum()[getSum]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Iterator]] +==== Iterator +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/Iterator.html#forEachRemaining(java.util.function.Consumer)[forEachRemaining](Consumer) +* boolean {java11-javadoc}/java.base/java/util/Iterator.html#hasNext()[hasNext]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Iterator.html#next()[next]() +* void {java11-javadoc}/java.base/java/util/Iterator.html#remove()[remove]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LinkedHashMap]] +==== LinkedHashMap +* {java11-javadoc}/java.base/java/util/LinkedHashMap.html#()[LinkedHashMap]() +* {java11-javadoc}/java.base/java/util/LinkedHashMap.html#(java.util.Map)[LinkedHashMap](Map) +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/HashMap.html#clone()[clone]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-LinkedHashSet]] +==== LinkedHashSet +* {java11-javadoc}/java.base/java/util/LinkedHashSet.html#()[LinkedHashSet]() +* {java11-javadoc}/java.base/java/util/LinkedHashSet.html#(java.util.Collection)[LinkedHashSet](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/HashSet.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LinkedList]] +==== LinkedList +* {java11-javadoc}/java.base/java/util/LinkedList.html#()[LinkedList]() +* {java11-javadoc}/java.base/java/util/LinkedList.html#(java.util.Collection)[LinkedList](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* void {java11-javadoc}/java.base/java/util/Deque.html#addFirst(java.lang.Object)[addFirst](def) +* void {java11-javadoc}/java.base/java/util/Deque.html#addLast(java.lang.Object)[addLast](def) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/LinkedList.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* Iterator {java11-javadoc}/java.base/java/util/Deque.html#descendingIterator()[descendingIterator]() +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Queue.html#element()[element]() +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* def {java11-javadoc}/java.base/java/util/Deque.html#getFirst()[getFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#getLast()[getLast]() +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* boolean {java11-javadoc}/java.base/java/util/Queue.html#offer(java.lang.Object)[offer](def) +* boolean {java11-javadoc}/java.base/java/util/Deque.html#offerFirst(java.lang.Object)[offerFirst](def) +* boolean {java11-javadoc}/java.base/java/util/Deque.html#offerLast(java.lang.Object)[offerLast](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Deque.html#peekFirst()[peekFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#peekLast()[peekLast]() +* def {java11-javadoc}/java.base/java/util/Queue.html#poll()[poll]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pollFirst()[pollFirst]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pollLast()[pollLast]() +* def {java11-javadoc}/java.base/java/util/Deque.html#pop()[pop]() +* void {java11-javadoc}/java.base/java/util/Deque.html#push(java.lang.Object)[push](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#remove()[remove]() +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* def {java11-javadoc}/java.base/java/util/Deque.html#removeFirst()[removeFirst]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#removeFirstOccurrence(java.lang.Object)[removeFirstOccurrence](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* def {java11-javadoc}/java.base/java/util/Deque.html#removeLast()[removeLast]() +* boolean {java11-javadoc}/java.base/java/util/Deque.html#removeLastOccurrence(java.lang.Object)[removeLastOccurrence](def) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-List]] +==== List +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ListIterator]] +==== ListIterator +* void {java11-javadoc}/java.base/java/util/ListIterator.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/Iterator.html#forEachRemaining(java.util.function.Consumer)[forEachRemaining](Consumer) +* boolean {java11-javadoc}/java.base/java/util/Iterator.html#hasNext()[hasNext]() +* boolean {java11-javadoc}/java.base/java/util/ListIterator.html#hasPrevious()[hasPrevious]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Iterator.html#next()[next]() +* int {java11-javadoc}/java.base/java/util/ListIterator.html#nextIndex()[nextIndex]() +* int {java11-javadoc}/java.base/java/util/ListIterator.html#previousIndex()[previousIndex]() +* void {java11-javadoc}/java.base/java/util/Iterator.html#remove()[remove]() +* void {java11-javadoc}/java.base/java/util/ListIterator.html#set(java.lang.Object)[set](def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Locale]] +==== Locale +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#CANADA[CANADA] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#CANADA_FRENCH[CANADA_FRENCH] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#CHINA[CHINA] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#CHINESE[CHINESE] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#ENGLISH[ENGLISH] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#FRANCE[FRANCE] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#FRENCH[FRENCH] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#GERMAN[GERMAN] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#GERMANY[GERMANY] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#ITALIAN[ITALIAN] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#ITALY[ITALY] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#JAPAN[JAPAN] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#JAPANESE[JAPANESE] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#KOREA[KOREA] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#KOREAN[KOREAN] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#PRC[PRC] +* static char {java11-javadoc}/java.base/java/util/Locale.html#PRIVATE_USE_EXTENSION[PRIVATE_USE_EXTENSION] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#ROOT[ROOT] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#SIMPLIFIED_CHINESE[SIMPLIFIED_CHINESE] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#TAIWAN[TAIWAN] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#TRADITIONAL_CHINESE[TRADITIONAL_CHINESE] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#UK[UK] +* static char {java11-javadoc}/java.base/java/util/Locale.html#UNICODE_LOCALE_EXTENSION[UNICODE_LOCALE_EXTENSION] +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#US[US] +* static List {java11-javadoc}/java.base/java/util/Locale.html#filter(java.util.List,java.util.Collection)[filter](List, Collection) +* static List {java11-javadoc}/java.base/java/util/Locale.html#filterTags(java.util.List,java.util.Collection)[filterTags](List, Collection) +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#forLanguageTag(java.lang.String)[forLanguageTag](String) +* static Locale[] {java11-javadoc}/java.base/java/util/Locale.html#getAvailableLocales()[getAvailableLocales]() +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#getDefault()[getDefault]() +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#getDefault(java.util.Locale$Category)[getDefault](Locale.Category) +* static String[] {java11-javadoc}/java.base/java/util/Locale.html#getISOCountries()[getISOCountries]() +* static String[] {java11-javadoc}/java.base/java/util/Locale.html#getISOLanguages()[getISOLanguages]() +* static Locale {java11-javadoc}/java.base/java/util/Locale.html#lookup(java.util.List,java.util.Collection)[lookup](List, Collection) +* static String {java11-javadoc}/java.base/java/util/Locale.html#lookupTag(java.util.List,java.util.Collection)[lookupTag](List, Collection) +* {java11-javadoc}/java.base/java/util/Locale.html#(java.lang.String)[Locale](String) +* {java11-javadoc}/java.base/java/util/Locale.html#(java.lang.String,java.lang.String)[Locale](String, String) +* {java11-javadoc}/java.base/java/util/Locale.html#(java.lang.String,java.lang.String,java.lang.String)[Locale](String, String, String) +* def {java11-javadoc}/java.base/java/util/Locale.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/Locale.html#getCountry()[getCountry]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayCountry()[getDisplayCountry]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayCountry(java.util.Locale)[getDisplayCountry](Locale) +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayLanguage()[getDisplayLanguage]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayLanguage(java.util.Locale)[getDisplayLanguage](Locale) +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayName()[getDisplayName]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayName(java.util.Locale)[getDisplayName](Locale) +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayScript()[getDisplayScript]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayScript(java.util.Locale)[getDisplayScript](Locale) +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayVariant()[getDisplayVariant]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getDisplayVariant(java.util.Locale)[getDisplayVariant](Locale) +* String {java11-javadoc}/java.base/java/util/Locale.html#getExtension(char)[getExtension](char) +* Set {java11-javadoc}/java.base/java/util/Locale.html#getExtensionKeys()[getExtensionKeys]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getISO3Country()[getISO3Country]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getISO3Language()[getISO3Language]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getLanguage()[getLanguage]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getScript()[getScript]() +* Set {java11-javadoc}/java.base/java/util/Locale.html#getUnicodeLocaleAttributes()[getUnicodeLocaleAttributes]() +* Set {java11-javadoc}/java.base/java/util/Locale.html#getUnicodeLocaleKeys()[getUnicodeLocaleKeys]() +* String {java11-javadoc}/java.base/java/util/Locale.html#getUnicodeLocaleType(java.lang.String)[getUnicodeLocaleType](String) +* String {java11-javadoc}/java.base/java/util/Locale.html#getVariant()[getVariant]() +* boolean {java11-javadoc}/java.base/java/util/Locale.html#hasExtensions()[hasExtensions]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Locale {java11-javadoc}/java.base/java/util/Locale.html#stripExtensions()[stripExtensions]() +* String {java11-javadoc}/java.base/java/util/Locale.html#toLanguageTag()[toLanguageTag]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Locale-Builder]] +==== Locale.Builder +* {java11-javadoc}/java.base/java/util/Locale$Builder.html#()[Locale.Builder]() +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#addUnicodeLocaleAttribute(java.lang.String)[addUnicodeLocaleAttribute](String) +* Locale {java11-javadoc}/java.base/java/util/Locale$Builder.html#build()[build]() +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#clear()[clear]() +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#clearExtensions()[clearExtensions]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#removeUnicodeLocaleAttribute(java.lang.String)[removeUnicodeLocaleAttribute](String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setExtension(char,java.lang.String)[setExtension](char, String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setLanguage(java.lang.String)[setLanguage](String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setLanguageTag(java.lang.String)[setLanguageTag](String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setLocale(java.util.Locale)[setLocale](Locale) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setRegion(java.lang.String)[setRegion](String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setScript(java.lang.String)[setScript](String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setUnicodeLocaleKeyword(java.lang.String,java.lang.String)[setUnicodeLocaleKeyword](String, String) +* Locale.Builder {java11-javadoc}/java.base/java/util/Locale$Builder.html#setVariant(java.lang.String)[setVariant](String) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Locale-Category]] +==== Locale.Category +* static Locale.Category {java11-javadoc}/java.base/java/util/Locale$Category.html#DISPLAY[DISPLAY] +* static Locale.Category {java11-javadoc}/java.base/java/util/Locale$Category.html#FORMAT[FORMAT] +* static Locale.Category {java11-javadoc}/java.base/java/util/Locale$Category.html#valueOf(java.lang.String)[valueOf](String) +* static Locale.Category[] {java11-javadoc}/java.base/java/util/Locale$Category.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Locale-FilteringMode]] +==== Locale.FilteringMode +* static Locale.FilteringMode {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#AUTOSELECT_FILTERING[AUTOSELECT_FILTERING] +* static Locale.FilteringMode {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#EXTENDED_FILTERING[EXTENDED_FILTERING] +* static Locale.FilteringMode {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#IGNORE_EXTENDED_RANGES[IGNORE_EXTENDED_RANGES] +* static Locale.FilteringMode {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#MAP_EXTENDED_RANGES[MAP_EXTENDED_RANGES] +* static Locale.FilteringMode {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#REJECT_EXTENDED_RANGES[REJECT_EXTENDED_RANGES] +* static Locale.FilteringMode {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#valueOf(java.lang.String)[valueOf](String) +* static Locale.FilteringMode[] {java11-javadoc}/java.base/java/util/Locale$FilteringMode.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Locale-LanguageRange]] +==== Locale.LanguageRange +* static double {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#MAX_WEIGHT[MAX_WEIGHT] +* static double {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#MIN_WEIGHT[MIN_WEIGHT] +* static List {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#mapEquivalents(java.util.List,java.util.Map)[mapEquivalents](List, Map) +* static List {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#parse(java.lang.String)[parse](String) +* static List {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#parse(java.lang.String,java.util.Map)[parse](String, Map) +* {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#(java.lang.String)[Locale.LanguageRange](String) +* {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#(java.lang.String,double)[Locale.LanguageRange](String, double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#getRange()[getRange]() +* double {java11-javadoc}/java.base/java/util/Locale$LanguageRange.html#getWeight()[getWeight]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongSummaryStatistics]] +==== LongSummaryStatistics +* {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#()[LongSummaryStatistics]() +* void {java11-javadoc}/java.base/java/util/function/LongConsumer.html#accept(long)[accept](long) +* LongConsumer {java11-javadoc}/java.base/java/util/function/LongConsumer.html#andThen(java.util.function.LongConsumer)[andThen](LongConsumer) +* void {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#combine(java.util.LongSummaryStatistics)[combine](LongSummaryStatistics) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* double {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#getAverage()[getAverage]() +* long {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#getCount()[getCount]() +* long {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#getMax()[getMax]() +* long {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#getMin()[getMin]() +* long {java11-javadoc}/java.base/java/util/LongSummaryStatistics.html#getSum()[getSum]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Map]] +==== Map +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-Map-Entry]] +==== Map.Entry +* static Comparator {java11-javadoc}/java.base/java/util/Map$Entry.html#comparingByKey()[comparingByKey]() +* static Comparator {java11-javadoc}/java.base/java/util/Map$Entry.html#comparingByKey(java.util.Comparator)[comparingByKey](Comparator) +* static Comparator {java11-javadoc}/java.base/java/util/Map$Entry.html#comparingByValue()[comparingByValue]() +* static Comparator {java11-javadoc}/java.base/java/util/Map$Entry.html#comparingByValue(java.util.Comparator)[comparingByValue](Comparator) +* boolean {java11-javadoc}/java.base/java/util/Map$Entry.html#equals(java.lang.Object)[equals](Object) +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#getKey()[getKey]() +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#getValue()[getValue]() +* int {java11-javadoc}/java.base/java/util/Map$Entry.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Map$Entry.html#setValue(java.lang.Object)[setValue](def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MissingFormatArgumentException]] +==== MissingFormatArgumentException +* {java11-javadoc}/java.base/java/util/MissingFormatArgumentException.html#(java.lang.String)[MissingFormatArgumentException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/MissingFormatArgumentException.html#getFormatSpecifier()[getFormatSpecifier]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MissingFormatWidthException]] +==== MissingFormatWidthException +* {java11-javadoc}/java.base/java/util/MissingFormatWidthException.html#(java.lang.String)[MissingFormatWidthException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/MissingFormatWidthException.html#getFormatSpecifier()[getFormatSpecifier]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-MissingResourceException]] +==== MissingResourceException +* {java11-javadoc}/java.base/java/util/MissingResourceException.html#(java.lang.String,java.lang.String,java.lang.String)[MissingResourceException](String, String, String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/MissingResourceException.html#getClassName()[getClassName]() +* String {java11-javadoc}/java.base/java/util/MissingResourceException.html#getKey()[getKey]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NavigableMap]] +==== NavigableMap +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#ceilingEntry(java.lang.Object)[ceilingEntry](def) +* def {java11-javadoc}/java.base/java/util/NavigableMap.html#ceilingKey(java.lang.Object)[ceilingKey](def) +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* Comparator {java11-javadoc}/java.base/java/util/SortedMap.html#comparator()[comparator]() +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableMap.html#descendingKeySet()[descendingKeySet]() +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#descendingMap()[descendingMap]() +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#firstEntry()[firstEntry]() +* def {java11-javadoc}/java.base/java/util/SortedMap.html#firstKey()[firstKey]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#floorEntry(java.lang.Object)[floorEntry](def) +* def {java11-javadoc}/java.base/java/util/NavigableMap.html#floorKey(java.lang.Object)[floorKey](def) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#headMap(java.lang.Object)[headMap](def) +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#headMap(java.lang.Object,boolean)[headMap](def, boolean) +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#higherEntry(java.lang.Object)[higherEntry](def) +* def {java11-javadoc}/java.base/java/util/NavigableMap.html#higherKey(java.lang.Object)[higherKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#lastEntry()[lastEntry]() +* def {java11-javadoc}/java.base/java/util/SortedMap.html#lastKey()[lastKey]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#lowerEntry(java.lang.Object)[lowerEntry](def) +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableMap.html#navigableKeySet()[navigableKeySet]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#pollFirstEntry()[pollFirstEntry]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#pollLastEntry()[pollLastEntry]() +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#subMap(java.lang.Object,java.lang.Object)[subMap](def, def) +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#subMap(java.lang.Object,boolean,java.lang.Object,boolean)[subMap](def, boolean, def, boolean) +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#tailMap(java.lang.Object)[tailMap](def) +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#tailMap(java.lang.Object,boolean)[tailMap](def, boolean) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-NavigableSet]] +==== NavigableSet +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#ceiling(java.lang.Object)[ceiling](def) +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* Comparator {java11-javadoc}/java.base/java/util/SortedSet.html#comparator()[comparator]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* Iterator {java11-javadoc}/java.base/java/util/NavigableSet.html#descendingIterator()[descendingIterator]() +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#descendingSet()[descendingSet]() +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* def {java11-javadoc}/java.base/java/util/SortedSet.html#first()[first]() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#floor(java.lang.Object)[floor](def) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#headSet(java.lang.Object)[headSet](def) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#headSet(java.lang.Object,boolean)[headSet](def, boolean) +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#higher(java.lang.Object)[higher](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* def {java11-javadoc}/java.base/java/util/SortedSet.html#last()[last]() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#lower(java.lang.Object)[lower](def) +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#pollFirst()[pollFirst]() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#pollLast()[pollLast]() +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#subSet(java.lang.Object,java.lang.Object)[subSet](def, def) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#subSet(java.lang.Object,boolean,java.lang.Object,boolean)[subSet](def, boolean, def, boolean) +* double sum() +* double sum(ToDoubleFunction) +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#tailSet(java.lang.Object)[tailSet](def) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#tailSet(java.lang.Object,boolean)[tailSet](def, boolean) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-NoSuchElementException]] +==== NoSuchElementException +* {java11-javadoc}/java.base/java/util/NoSuchElementException.html#()[NoSuchElementException]() +* {java11-javadoc}/java.base/java/util/NoSuchElementException.html#(java.lang.String)[NoSuchElementException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Objects]] +==== Objects +* static int {java11-javadoc}/java.base/java/util/Objects.html#compare(java.lang.Object,java.lang.Object,java.util.Comparator)[compare](def, def, Comparator) +* static boolean {java11-javadoc}/java.base/java/util/Objects.html#deepEquals(java.lang.Object,java.lang.Object)[deepEquals](Object, Object) +* static boolean {java11-javadoc}/java.base/java/util/Objects.html#equals(java.lang.Object,java.lang.Object)[equals](Object, Object) +* static int {java11-javadoc}/java.base/java/util/Objects.html#hash(java.lang.Object%5B%5D)[hash](Object[]) +* static int {java11-javadoc}/java.base/java/util/Objects.html#hashCode(java.lang.Object)[hashCode](Object) +* static boolean {java11-javadoc}/java.base/java/util/Objects.html#isNull(java.lang.Object)[isNull](Object) +* static boolean {java11-javadoc}/java.base/java/util/Objects.html#nonNull(java.lang.Object)[nonNull](Object) +* static def {java11-javadoc}/java.base/java/util/Objects.html#requireNonNull(java.lang.Object)[requireNonNull](def) +* static def {java11-javadoc}/java.base/java/util/Objects.html#requireNonNull(java.lang.Object,java.lang.String)[requireNonNull](def, String) +* static String {java11-javadoc}/java.base/java/util/Objects.html#toString(java.lang.Object)[toString](Object) +* static String {java11-javadoc}/java.base/java/util/Objects.html#toString(java.lang.Object,java.lang.String)[toString](Object, String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Observable]] +==== Observable +* {java11-javadoc}/java.base/java/util/Observable.html#()[Observable]() +* void {java11-javadoc}/java.base/java/util/Observable.html#addObserver(java.util.Observer)[addObserver](Observer) +* int {java11-javadoc}/java.base/java/util/Observable.html#countObservers()[countObservers]() +* void {java11-javadoc}/java.base/java/util/Observable.html#deleteObserver(java.util.Observer)[deleteObserver](Observer) +* void {java11-javadoc}/java.base/java/util/Observable.html#deleteObservers()[deleteObservers]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/util/Observable.html#hasChanged()[hasChanged]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/util/Observable.html#notifyObservers()[notifyObservers]() +* void {java11-javadoc}/java.base/java/util/Observable.html#notifyObservers(java.lang.Object)[notifyObservers](Object) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Observer]] +==== Observer +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* void {java11-javadoc}/java.base/java/util/Observer.html#update(java.util.Observable,java.lang.Object)[update](Observable, Object) + + +[[painless-api-reference-shared-Optional]] +==== Optional +* static Optional {java11-javadoc}/java.base/java/util/Optional.html#empty()[empty]() +* static Optional {java11-javadoc}/java.base/java/util/Optional.html#of(java.lang.Object)[of](def) +* static Optional {java11-javadoc}/java.base/java/util/Optional.html#ofNullable(java.lang.Object)[ofNullable](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Optional {java11-javadoc}/java.base/java/util/Optional.html#filter(java.util.function.Predicate)[filter](Predicate) +* Optional {java11-javadoc}/java.base/java/util/Optional.html#flatMap(java.util.function.Function)[flatMap](Function) +* def {java11-javadoc}/java.base/java/util/Optional.html#get()[get]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/util/Optional.html#ifPresent(java.util.function.Consumer)[ifPresent](Consumer) +* boolean {java11-javadoc}/java.base/java/util/Optional.html#isPresent()[isPresent]() +* Optional {java11-javadoc}/java.base/java/util/Optional.html#map(java.util.function.Function)[map](Function) +* def {java11-javadoc}/java.base/java/util/Optional.html#orElse(java.lang.Object)[orElse](def) +* def {java11-javadoc}/java.base/java/util/Optional.html#orElseGet(java.util.function.Supplier)[orElseGet](Supplier) +* def {java11-javadoc}/java.base/java/util/Optional.html#orElseThrow(java.util.function.Supplier)[orElseThrow](Supplier) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-OptionalDouble]] +==== OptionalDouble +* static OptionalDouble {java11-javadoc}/java.base/java/util/OptionalDouble.html#empty()[empty]() +* static OptionalDouble {java11-javadoc}/java.base/java/util/OptionalDouble.html#of(double)[of](double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* double {java11-javadoc}/java.base/java/util/OptionalDouble.html#getAsDouble()[getAsDouble]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/util/OptionalDouble.html#ifPresent(java.util.function.DoubleConsumer)[ifPresent](DoubleConsumer) +* boolean {java11-javadoc}/java.base/java/util/OptionalDouble.html#isPresent()[isPresent]() +* double {java11-javadoc}/java.base/java/util/OptionalDouble.html#orElse(double)[orElse](double) +* double {java11-javadoc}/java.base/java/util/OptionalDouble.html#orElseGet(java.util.function.DoubleSupplier)[orElseGet](DoubleSupplier) +* double {java11-javadoc}/java.base/java/util/OptionalDouble.html#orElseThrow(java.util.function.Supplier)[orElseThrow](Supplier) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-OptionalInt]] +==== OptionalInt +* static OptionalInt {java11-javadoc}/java.base/java/util/OptionalInt.html#empty()[empty]() +* static OptionalInt {java11-javadoc}/java.base/java/util/OptionalInt.html#of(int)[of](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/OptionalInt.html#getAsInt()[getAsInt]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/util/OptionalInt.html#ifPresent(java.util.function.IntConsumer)[ifPresent](IntConsumer) +* boolean {java11-javadoc}/java.base/java/util/OptionalInt.html#isPresent()[isPresent]() +* int {java11-javadoc}/java.base/java/util/OptionalInt.html#orElse(int)[orElse](int) +* int {java11-javadoc}/java.base/java/util/OptionalInt.html#orElseGet(java.util.function.IntSupplier)[orElseGet](IntSupplier) +* int {java11-javadoc}/java.base/java/util/OptionalInt.html#orElseThrow(java.util.function.Supplier)[orElseThrow](Supplier) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-OptionalLong]] +==== OptionalLong +* static OptionalLong {java11-javadoc}/java.base/java/util/OptionalLong.html#empty()[empty]() +* static OptionalLong {java11-javadoc}/java.base/java/util/OptionalLong.html#of(long)[of](long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/OptionalLong.html#getAsLong()[getAsLong]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* void {java11-javadoc}/java.base/java/util/OptionalLong.html#ifPresent(java.util.function.LongConsumer)[ifPresent](LongConsumer) +* boolean {java11-javadoc}/java.base/java/util/OptionalLong.html#isPresent()[isPresent]() +* long {java11-javadoc}/java.base/java/util/OptionalLong.html#orElse(long)[orElse](long) +* long {java11-javadoc}/java.base/java/util/OptionalLong.html#orElseGet(java.util.function.LongSupplier)[orElseGet](LongSupplier) +* long {java11-javadoc}/java.base/java/util/OptionalLong.html#orElseThrow(java.util.function.Supplier)[orElseThrow](Supplier) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-PrimitiveIterator]] +==== PrimitiveIterator +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/PrimitiveIterator.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* boolean {java11-javadoc}/java.base/java/util/Iterator.html#hasNext()[hasNext]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Iterator.html#next()[next]() +* void {java11-javadoc}/java.base/java/util/Iterator.html#remove()[remove]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-PrimitiveIterator-OfDouble]] +==== PrimitiveIterator.OfDouble +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/PrimitiveIterator.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* boolean {java11-javadoc}/java.base/java/util/Iterator.html#hasNext()[hasNext]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Double {java11-javadoc}/java.base/java/util/PrimitiveIterator$OfDouble.html#next()[next]() +* double {java11-javadoc}/java.base/java/util/PrimitiveIterator$OfDouble.html#nextDouble()[nextDouble]() +* void {java11-javadoc}/java.base/java/util/Iterator.html#remove()[remove]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-PrimitiveIterator-OfInt]] +==== PrimitiveIterator.OfInt +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/PrimitiveIterator.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* boolean {java11-javadoc}/java.base/java/util/Iterator.html#hasNext()[hasNext]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Integer {java11-javadoc}/java.base/java/util/PrimitiveIterator$OfInt.html#next()[next]() +* int {java11-javadoc}/java.base/java/util/PrimitiveIterator$OfInt.html#nextInt()[nextInt]() +* void {java11-javadoc}/java.base/java/util/Iterator.html#remove()[remove]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-PrimitiveIterator-OfLong]] +==== PrimitiveIterator.OfLong +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* void {java11-javadoc}/java.base/java/util/PrimitiveIterator.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* boolean {java11-javadoc}/java.base/java/util/Iterator.html#hasNext()[hasNext]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Long {java11-javadoc}/java.base/java/util/PrimitiveIterator$OfLong.html#next()[next]() +* long {java11-javadoc}/java.base/java/util/PrimitiveIterator$OfLong.html#nextLong()[nextLong]() +* void {java11-javadoc}/java.base/java/util/Iterator.html#remove()[remove]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-PriorityQueue]] +==== PriorityQueue +* {java11-javadoc}/java.base/java/util/PriorityQueue.html#()[PriorityQueue]() +* {java11-javadoc}/java.base/java/util/PriorityQueue.html#(java.util.Comparator)[PriorityQueue](Comparator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Queue.html#element()[element]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Queue.html#offer(java.lang.Object)[offer](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Queue.html#poll()[poll]() +* def {java11-javadoc}/java.base/java/util/Queue.html#remove()[remove]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Queue]] +==== Queue +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Queue.html#element()[element]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Queue.html#offer(java.lang.Object)[offer](def) +* def {java11-javadoc}/java.base/java/util/Queue.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Queue.html#poll()[poll]() +* def {java11-javadoc}/java.base/java/util/Queue.html#remove()[remove]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Random]] +==== Random +* {java11-javadoc}/java.base/java/util/Random.html#()[Random]() +* {java11-javadoc}/java.base/java/util/Random.html#(long)[Random](long) +* DoubleStream {java11-javadoc}/java.base/java/util/Random.html#doubles(long)[doubles](long) +* DoubleStream {java11-javadoc}/java.base/java/util/Random.html#doubles(long,double,double)[doubles](long, double, double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* IntStream {java11-javadoc}/java.base/java/util/Random.html#ints(long)[ints](long) +* IntStream {java11-javadoc}/java.base/java/util/Random.html#ints(long,int,int)[ints](long, int, int) +* LongStream {java11-javadoc}/java.base/java/util/Random.html#longs(long)[longs](long) +* LongStream {java11-javadoc}/java.base/java/util/Random.html#longs(long,long,long)[longs](long, long, long) +* boolean {java11-javadoc}/java.base/java/util/Random.html#nextBoolean()[nextBoolean]() +* void {java11-javadoc}/java.base/java/util/Random.html#nextBytes(byte%5B%5D)[nextBytes](byte[]) +* double {java11-javadoc}/java.base/java/util/Random.html#nextDouble()[nextDouble]() +* float {java11-javadoc}/java.base/java/util/Random.html#nextFloat()[nextFloat]() +* double {java11-javadoc}/java.base/java/util/Random.html#nextGaussian()[nextGaussian]() +* int {java11-javadoc}/java.base/java/util/Random.html#nextInt()[nextInt]() +* int {java11-javadoc}/java.base/java/util/Random.html#nextInt(int)[nextInt](int) +* long {java11-javadoc}/java.base/java/util/Random.html#nextLong()[nextLong]() +* void {java11-javadoc}/java.base/java/util/Random.html#setSeed(long)[setSeed](long) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-RandomAccess]] +==== RandomAccess +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Set]] +==== Set +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-SimpleTimeZone]] +==== SimpleTimeZone +* static int {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#STANDARD_TIME[STANDARD_TIME] +* static int {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#UTC_TIME[UTC_TIME] +* static int {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#WALL_TIME[WALL_TIME] +* {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#(int,java.lang.String,int,int,int,int,int,int,int,int)[SimpleTimeZone](int, String, int, int, int, int, int, int, int, int) +* {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#(int,java.lang.String,int,int,int,int,int,int,int,int,int)[SimpleTimeZone](int, String, int, int, int, int, int, int, int, int, int) +* {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#(int,java.lang.String,int,int,int,int,int,int,int,int,int,int,int)[SimpleTimeZone](int, String, int, int, int, int, int, int, int, int, int, int, int) +* {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#(int,java.lang.String)[SimpleTimeZone](int, String) +* def {java11-javadoc}/java.base/java/util/TimeZone.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#getDSTSavings()[getDSTSavings]() +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName()[getDisplayName]() +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName(java.util.Locale)[getDisplayName](Locale) +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName(boolean,int)[getDisplayName](boolean, int) +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName(boolean,int,java.util.Locale)[getDisplayName](boolean, int, Locale) +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getID()[getID]() +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getOffset(long)[getOffset](long) +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getOffset(int,int,int,int,int,int)[getOffset](int, int, int, int, int, int) +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getRawOffset()[getRawOffset]() +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#hasSameRules(java.util.TimeZone)[hasSameRules](TimeZone) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#inDaylightTime(java.util.Date)[inDaylightTime](Date) +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#observesDaylightTime()[observesDaylightTime]() +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setDSTSavings(int)[setDSTSavings](int) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setEndRule(int,int,int)[setEndRule](int, int, int) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setEndRule(int,int,int,int)[setEndRule](int, int, int, int) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setEndRule(int,int,int,int,boolean)[setEndRule](int, int, int, int, boolean) +* void {java11-javadoc}/java.base/java/util/TimeZone.html#setRawOffset(int)[setRawOffset](int) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setStartRule(int,int,int)[setStartRule](int, int, int) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setStartRule(int,int,int,int)[setStartRule](int, int, int, int) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setStartRule(int,int,int,int,boolean)[setStartRule](int, int, int, int, boolean) +* void {java11-javadoc}/java.base/java/util/SimpleTimeZone.html#setStartYear(int)[setStartYear](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* ZoneId {java11-javadoc}/java.base/java/util/TimeZone.html#toZoneId()[toZoneId]() +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#useDaylightTime()[useDaylightTime]() + + +[[painless-api-reference-shared-SortedMap]] +==== SortedMap +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* Comparator {java11-javadoc}/java.base/java/util/SortedMap.html#comparator()[comparator]() +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* def {java11-javadoc}/java.base/java/util/SortedMap.html#firstKey()[firstKey]() +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#headMap(java.lang.Object)[headMap](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* def {java11-javadoc}/java.base/java/util/SortedMap.html#lastKey()[lastKey]() +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#subMap(java.lang.Object,java.lang.Object)[subMap](def, def) +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#tailMap(java.lang.Object)[tailMap](def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-SortedSet]] +==== SortedSet +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* Comparator {java11-javadoc}/java.base/java/util/SortedSet.html#comparator()[comparator]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* def {java11-javadoc}/java.base/java/util/SortedSet.html#first()[first]() +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#headSet(java.lang.Object)[headSet](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* def {java11-javadoc}/java.base/java/util/SortedSet.html#last()[last]() +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#subSet(java.lang.Object,java.lang.Object)[subSet](def, def) +* double sum() +* double sum(ToDoubleFunction) +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#tailSet(java.lang.Object)[tailSet](def) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Spliterator]] +==== Spliterator +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#CONCURRENT[CONCURRENT] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#DISTINCT[DISTINCT] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#IMMUTABLE[IMMUTABLE] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#NONNULL[NONNULL] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#ORDERED[ORDERED] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#SIZED[SIZED] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#SORTED[SORTED] +* static int {java11-javadoc}/java.base/java/util/Spliterator.html#SUBSIZED[SUBSIZED] +* int {java11-javadoc}/java.base/java/util/Spliterator.html#characteristics()[characteristics]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/Spliterator.html#estimateSize()[estimateSize]() +* void {java11-javadoc}/java.base/java/util/Spliterator.html#forEachRemaining(java.util.function.Consumer)[forEachRemaining](Consumer) +* Comparator {java11-javadoc}/java.base/java/util/Spliterator.html#getComparator()[getComparator]() +* long {java11-javadoc}/java.base/java/util/Spliterator.html#getExactSizeIfKnown()[getExactSizeIfKnown]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator.html#hasCharacteristics(int)[hasCharacteristics](int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator.html#tryAdvance(java.util.function.Consumer)[tryAdvance](Consumer) +* Spliterator {java11-javadoc}/java.base/java/util/Spliterator.html#trySplit()[trySplit]() + + +[[painless-api-reference-shared-Spliterator-OfDouble]] +==== Spliterator.OfDouble +* int {java11-javadoc}/java.base/java/util/Spliterator.html#characteristics()[characteristics]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/Spliterator.html#estimateSize()[estimateSize]() +* void {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* Comparator {java11-javadoc}/java.base/java/util/Spliterator.html#getComparator()[getComparator]() +* long {java11-javadoc}/java.base/java/util/Spliterator.html#getExactSizeIfKnown()[getExactSizeIfKnown]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator.html#hasCharacteristics(int)[hasCharacteristics](int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#tryAdvance(java.lang.Object)[tryAdvance](def) +* Spliterator.OfDouble {java11-javadoc}/java.base/java/util/Spliterator$OfDouble.html#trySplit()[trySplit]() + + +[[painless-api-reference-shared-Spliterator-OfInt]] +==== Spliterator.OfInt +* int {java11-javadoc}/java.base/java/util/Spliterator.html#characteristics()[characteristics]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/Spliterator.html#estimateSize()[estimateSize]() +* void {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* Comparator {java11-javadoc}/java.base/java/util/Spliterator.html#getComparator()[getComparator]() +* long {java11-javadoc}/java.base/java/util/Spliterator.html#getExactSizeIfKnown()[getExactSizeIfKnown]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator.html#hasCharacteristics(int)[hasCharacteristics](int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#tryAdvance(java.lang.Object)[tryAdvance](def) +* Spliterator.OfInt {java11-javadoc}/java.base/java/util/Spliterator$OfInt.html#trySplit()[trySplit]() + + +[[painless-api-reference-shared-Spliterator-OfLong]] +==== Spliterator.OfLong +* int {java11-javadoc}/java.base/java/util/Spliterator.html#characteristics()[characteristics]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/Spliterator.html#estimateSize()[estimateSize]() +* void {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* Comparator {java11-javadoc}/java.base/java/util/Spliterator.html#getComparator()[getComparator]() +* long {java11-javadoc}/java.base/java/util/Spliterator.html#getExactSizeIfKnown()[getExactSizeIfKnown]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator.html#hasCharacteristics(int)[hasCharacteristics](int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#tryAdvance(java.lang.Object)[tryAdvance](def) +* Spliterator.OfLong {java11-javadoc}/java.base/java/util/Spliterator$OfLong.html#trySplit()[trySplit]() + + +[[painless-api-reference-shared-Spliterator-OfPrimitive]] +==== Spliterator.OfPrimitive +* int {java11-javadoc}/java.base/java/util/Spliterator.html#characteristics()[characteristics]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/Spliterator.html#estimateSize()[estimateSize]() +* void {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#forEachRemaining(java.lang.Object)[forEachRemaining](def) +* Comparator {java11-javadoc}/java.base/java/util/Spliterator.html#getComparator()[getComparator]() +* long {java11-javadoc}/java.base/java/util/Spliterator.html#getExactSizeIfKnown()[getExactSizeIfKnown]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator.html#hasCharacteristics(int)[hasCharacteristics](int) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* boolean {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#tryAdvance(java.lang.Object)[tryAdvance](def) +* Spliterator.OfPrimitive {java11-javadoc}/java.base/java/util/Spliterator$OfPrimitive.html#trySplit()[trySplit]() + + +[[painless-api-reference-shared-Spliterators]] +==== Spliterators +* static Spliterator.OfDouble {java11-javadoc}/java.base/java/util/Spliterators.html#emptyDoubleSpliterator()[emptyDoubleSpliterator]() +* static Spliterator.OfInt {java11-javadoc}/java.base/java/util/Spliterators.html#emptyIntSpliterator()[emptyIntSpliterator]() +* static Spliterator.OfLong {java11-javadoc}/java.base/java/util/Spliterators.html#emptyLongSpliterator()[emptyLongSpliterator]() +* static Spliterator {java11-javadoc}/java.base/java/util/Spliterators.html#emptySpliterator()[emptySpliterator]() +* static Iterator {java11-javadoc}/java.base/java/util/Spliterators.html#iterator(java.util.Spliterator)[iterator](Spliterator) +* static Spliterator {java11-javadoc}/java.base/java/util/Spliterators.html#spliterator(java.util.Collection,int)[spliterator](Collection, int) +* static Spliterator {java11-javadoc}/java.base/java/util/Spliterators.html#spliterator(java.util.Iterator,long,int)[spliterator](Iterator, long, int) +* static Spliterator {java11-javadoc}/java.base/java/util/Spliterators.html#spliteratorUnknownSize(java.util.Iterator,int)[spliteratorUnknownSize](Iterator, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Stack]] +==== Stack +* {java11-javadoc}/java.base/java/util/Stack.html#()[Stack]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* void {java11-javadoc}/java.base/java/util/Vector.html#addElement(java.lang.Object)[addElement](def) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/Vector.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* void {java11-javadoc}/java.base/java/util/Vector.html#copyInto(java.lang.Object%5B%5D)[copyInto](Object[]) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Vector.html#elementAt(int)[elementAt](int) +* Enumeration {java11-javadoc}/java.base/java/util/Vector.html#elements()[elements]() +* boolean {java11-javadoc}/java.base/java/util/Stack.html#empty()[empty]() +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* def {java11-javadoc}/java.base/java/util/Vector.html#firstElement()[firstElement]() +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* void {java11-javadoc}/java.base/java/util/Vector.html#insertElementAt(java.lang.Object,int)[insertElementAt](def, int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* def {java11-javadoc}/java.base/java/util/Vector.html#lastElement()[lastElement]() +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* int {java11-javadoc}/java.base/java/util/Vector.html#lastIndexOf(java.lang.Object,int)[lastIndexOf](def, int) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/Stack.html#peek()[peek]() +* def {java11-javadoc}/java.base/java/util/Stack.html#pop()[pop]() +* def {java11-javadoc}/java.base/java/util/Stack.html#push(java.lang.Object)[push](def) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* void {java11-javadoc}/java.base/java/util/Vector.html#removeAllElements()[removeAllElements]() +* boolean {java11-javadoc}/java.base/java/util/Vector.html#removeElement(java.lang.Object)[removeElement](def) +* void {java11-javadoc}/java.base/java/util/Vector.html#removeElementAt(int)[removeElementAt](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Stack.html#search(java.lang.Object)[search](def) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* void {java11-javadoc}/java.base/java/util/Vector.html#setElementAt(java.lang.Object,int)[setElementAt](def, int) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-StringJoiner]] +==== StringJoiner +* {java11-javadoc}/java.base/java/util/StringJoiner.html#(java.lang.CharSequence)[StringJoiner](CharSequence) +* {java11-javadoc}/java.base/java/util/StringJoiner.html#(java.lang.CharSequence,java.lang.CharSequence,java.lang.CharSequence)[StringJoiner](CharSequence, CharSequence, CharSequence) +* StringJoiner {java11-javadoc}/java.base/java/util/StringJoiner.html#add(java.lang.CharSequence)[add](CharSequence) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/StringJoiner.html#length()[length]() +* StringJoiner {java11-javadoc}/java.base/java/util/StringJoiner.html#merge(java.util.StringJoiner)[merge](StringJoiner) +* StringJoiner {java11-javadoc}/java.base/java/util/StringJoiner.html#setEmptyValue(java.lang.CharSequence)[setEmptyValue](CharSequence) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-StringTokenizer]] +==== StringTokenizer +* {java11-javadoc}/java.base/java/util/StringTokenizer.html#(java.lang.String)[StringTokenizer](String) +* {java11-javadoc}/java.base/java/util/StringTokenizer.html#(java.lang.String,java.lang.String)[StringTokenizer](String, String) +* {java11-javadoc}/java.base/java/util/StringTokenizer.html#(java.lang.String,java.lang.String,boolean)[StringTokenizer](String, String, boolean) +* int {java11-javadoc}/java.base/java/util/StringTokenizer.html#countTokens()[countTokens]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/util/Enumeration.html#hasMoreElements()[hasMoreElements]() +* boolean {java11-javadoc}/java.base/java/util/StringTokenizer.html#hasMoreTokens()[hasMoreTokens]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* def {java11-javadoc}/java.base/java/util/Enumeration.html#nextElement()[nextElement]() +* String {java11-javadoc}/java.base/java/util/StringTokenizer.html#nextToken()[nextToken]() +* String {java11-javadoc}/java.base/java/util/StringTokenizer.html#nextToken(java.lang.String)[nextToken](String) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TimeZone]] +==== TimeZone +* static int {java11-javadoc}/java.base/java/util/TimeZone.html#LONG[LONG] +* static int {java11-javadoc}/java.base/java/util/TimeZone.html#SHORT[SHORT] +* static String[] {java11-javadoc}/java.base/java/util/TimeZone.html#getAvailableIDs()[getAvailableIDs]() +* static String[] {java11-javadoc}/java.base/java/util/TimeZone.html#getAvailableIDs(int)[getAvailableIDs](int) +* static TimeZone {java11-javadoc}/java.base/java/util/TimeZone.html#getDefault()[getDefault]() +* static TimeZone {java11-javadoc}/java.base/java/util/TimeZone.html#getTimeZone(java.lang.String)[getTimeZone](String) +* def {java11-javadoc}/java.base/java/util/TimeZone.html#clone()[clone]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getDSTSavings()[getDSTSavings]() +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName()[getDisplayName]() +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName(java.util.Locale)[getDisplayName](Locale) +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName(boolean,int)[getDisplayName](boolean, int) +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getDisplayName(boolean,int,java.util.Locale)[getDisplayName](boolean, int, Locale) +* String {java11-javadoc}/java.base/java/util/TimeZone.html#getID()[getID]() +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getOffset(long)[getOffset](long) +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getOffset(int,int,int,int,int,int)[getOffset](int, int, int, int, int, int) +* int {java11-javadoc}/java.base/java/util/TimeZone.html#getRawOffset()[getRawOffset]() +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#hasSameRules(java.util.TimeZone)[hasSameRules](TimeZone) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#inDaylightTime(java.util.Date)[inDaylightTime](Date) +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#observesDaylightTime()[observesDaylightTime]() +* void {java11-javadoc}/java.base/java/util/TimeZone.html#setRawOffset(int)[setRawOffset](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* ZoneId {java11-javadoc}/java.base/java/util/TimeZone.html#toZoneId()[toZoneId]() +* boolean {java11-javadoc}/java.base/java/util/TimeZone.html#useDaylightTime()[useDaylightTime]() + + +[[painless-api-reference-shared-TooManyListenersException]] +==== TooManyListenersException +* {java11-javadoc}/java.base/java/util/TooManyListenersException.html#()[TooManyListenersException]() +* {java11-javadoc}/java.base/java/util/TooManyListenersException.html#(java.lang.String)[TooManyListenersException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-TreeMap]] +==== TreeMap +* {java11-javadoc}/java.base/java/util/TreeMap.html#()[TreeMap]() +* {java11-javadoc}/java.base/java/util/TreeMap.html#(java.util.Comparator)[TreeMap](Comparator) +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#ceilingEntry(java.lang.Object)[ceilingEntry](def) +* def {java11-javadoc}/java.base/java/util/NavigableMap.html#ceilingKey(java.lang.Object)[ceilingKey](def) +* void {java11-javadoc}/java.base/java/util/Map.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/TreeMap.html#clone()[clone]() +* List collect(BiFunction) +* def collect(Collection, BiFunction) +* Comparator {java11-javadoc}/java.base/java/util/SortedMap.html#comparator()[comparator]() +* def {java11-javadoc}/java.base/java/util/Map.html#compute(java.lang.Object,java.util.function.BiFunction)[compute](def, BiFunction) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfAbsent(java.lang.Object,java.util.function.Function)[computeIfAbsent](def, Function) +* def {java11-javadoc}/java.base/java/util/Map.html#computeIfPresent(java.lang.Object,java.util.function.BiFunction)[computeIfPresent](def, BiFunction) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsKey(java.lang.Object)[containsKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#containsValue(java.lang.Object)[containsValue](def) +* int count(BiPredicate) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableMap.html#descendingKeySet()[descendingKeySet]() +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#descendingMap()[descendingMap]() +* def each(BiConsumer) +* Set {java11-javadoc}/java.base/java/util/Map.html#entrySet()[entrySet]() +* boolean {java11-javadoc}/java.base/java/util/Map.html#equals(java.lang.Object)[equals](Object) +* boolean every(BiPredicate) +* Map.Entry find(BiPredicate) +* Map findAll(BiPredicate) +* def findResult(BiFunction) +* def findResult(def, BiFunction) +* List findResults(BiFunction) +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#firstEntry()[firstEntry]() +* def {java11-javadoc}/java.base/java/util/SortedMap.html#firstKey()[firstKey]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#floorEntry(java.lang.Object)[floorEntry](def) +* def {java11-javadoc}/java.base/java/util/NavigableMap.html#floorKey(java.lang.Object)[floorKey](def) +* void {java11-javadoc}/java.base/java/util/Map.html#forEach(java.util.function.BiConsumer)[forEach](BiConsumer) +* def {java11-javadoc}/java.base/java/util/Map.html#get(java.lang.Object)[get](def) +* def {java11-javadoc}/java.base/java/util/Map.html#getOrDefault(java.lang.Object,java.lang.Object)[getOrDefault](def, def) +* Map groupBy(BiFunction) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#headMap(java.lang.Object)[headMap](def) +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#headMap(java.lang.Object,boolean)[headMap](def, boolean) +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#higherEntry(java.lang.Object)[higherEntry](def) +* def {java11-javadoc}/java.base/java/util/NavigableMap.html#higherKey(java.lang.Object)[higherKey](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#isEmpty()[isEmpty]() +* Set {java11-javadoc}/java.base/java/util/Map.html#keySet()[keySet]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#lastEntry()[lastEntry]() +* def {java11-javadoc}/java.base/java/util/SortedMap.html#lastKey()[lastKey]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#lowerEntry(java.lang.Object)[lowerEntry](def) +* def {java11-javadoc}/java.base/java/util/Map.html#merge(java.lang.Object,java.lang.Object,java.util.function.BiFunction)[merge](def, def, BiFunction) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableMap.html#navigableKeySet()[navigableKeySet]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#pollFirstEntry()[pollFirstEntry]() +* Map.Entry {java11-javadoc}/java.base/java/util/NavigableMap.html#pollLastEntry()[pollLastEntry]() +* def {java11-javadoc}/java.base/java/util/Map.html#put(java.lang.Object,java.lang.Object)[put](def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#putAll(java.util.Map)[putAll](Map) +* def {java11-javadoc}/java.base/java/util/Map.html#putIfAbsent(java.lang.Object,java.lang.Object)[putIfAbsent](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#remove(java.lang.Object,java.lang.Object)[remove](def, def) +* def {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object)[replace](def, def) +* boolean {java11-javadoc}/java.base/java/util/Map.html#replace(java.lang.Object,java.lang.Object,java.lang.Object)[replace](def, def, def) +* void {java11-javadoc}/java.base/java/util/Map.html#replaceAll(java.util.function.BiFunction)[replaceAll](BiFunction) +* int {java11-javadoc}/java.base/java/util/Map.html#size()[size]() +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#subMap(java.lang.Object,java.lang.Object)[subMap](def, def) +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#subMap(java.lang.Object,boolean,java.lang.Object,boolean)[subMap](def, boolean, def, boolean) +* SortedMap {java11-javadoc}/java.base/java/util/SortedMap.html#tailMap(java.lang.Object)[tailMap](def) +* NavigableMap {java11-javadoc}/java.base/java/util/NavigableMap.html#tailMap(java.lang.Object,boolean)[tailMap](def, boolean) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Collection {java11-javadoc}/java.base/java/util/Map.html#values()[values]() + + +[[painless-api-reference-shared-TreeSet]] +==== TreeSet +* {java11-javadoc}/java.base/java/util/TreeSet.html#()[TreeSet]() +* {java11-javadoc}/java.base/java/util/TreeSet.html#(java.util.Comparator)[TreeSet](Comparator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#ceiling(java.lang.Object)[ceiling](def) +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/TreeSet.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* Comparator {java11-javadoc}/java.base/java/util/SortedSet.html#comparator()[comparator]() +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* Iterator {java11-javadoc}/java.base/java/util/NavigableSet.html#descendingIterator()[descendingIterator]() +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#descendingSet()[descendingSet]() +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/Set.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* def {java11-javadoc}/java.base/java/util/SortedSet.html#first()[first]() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#floor(java.lang.Object)[floor](def) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/Set.html#hashCode()[hashCode]() +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#headSet(java.lang.Object)[headSet](def) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#headSet(java.lang.Object,boolean)[headSet](def, boolean) +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#higher(java.lang.Object)[higher](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* def {java11-javadoc}/java.base/java/util/SortedSet.html#last()[last]() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#lower(java.lang.Object)[lower](def) +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#pollFirst()[pollFirst]() +* def {java11-javadoc}/java.base/java/util/NavigableSet.html#pollLast()[pollLast]() +* boolean {java11-javadoc}/java.base/java/util/Set.html#remove(java.lang.Object)[remove](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#subSet(java.lang.Object,java.lang.Object)[subSet](def, def) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#subSet(java.lang.Object,boolean,java.lang.Object,boolean)[subSet](def, boolean, def, boolean) +* double sum() +* double sum(ToDoubleFunction) +* SortedSet {java11-javadoc}/java.base/java/util/SortedSet.html#tailSet(java.lang.Object)[tailSet](def) +* NavigableSet {java11-javadoc}/java.base/java/util/NavigableSet.html#tailSet(java.lang.Object,boolean)[tailSet](def, boolean) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-UUID]] +==== UUID +* static UUID {java11-javadoc}/java.base/java/util/UUID.html#fromString(java.lang.String)[fromString](String) +* static UUID {java11-javadoc}/java.base/java/util/UUID.html#nameUUIDFromBytes(byte%5B%5D)[nameUUIDFromBytes](byte[]) +* {java11-javadoc}/java.base/java/util/UUID.html#(long,long)[UUID](long, long) +* int {java11-javadoc}/java.base/java/util/UUID.html#clockSequence()[clockSequence]() +* int {java11-javadoc}/java.base/java/util/UUID.html#compareTo(java.util.UUID)[compareTo](UUID) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/UUID.html#getLeastSignificantBits()[getLeastSignificantBits]() +* long {java11-javadoc}/java.base/java/util/UUID.html#getMostSignificantBits()[getMostSignificantBits]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* long {java11-javadoc}/java.base/java/util/UUID.html#node()[node]() +* long {java11-javadoc}/java.base/java/util/UUID.html#timestamp()[timestamp]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* int {java11-javadoc}/java.base/java/util/UUID.html#variant()[variant]() +* int {java11-javadoc}/java.base/java/util/UUID.html#version()[version]() + + +[[painless-api-reference-shared-UnknownFormatConversionException]] +==== UnknownFormatConversionException +* {java11-javadoc}/java.base/java/util/UnknownFormatConversionException.html#(java.lang.String)[UnknownFormatConversionException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/UnknownFormatConversionException.html#getConversion()[getConversion]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-UnknownFormatFlagsException]] +==== UnknownFormatFlagsException +* {java11-javadoc}/java.base/java/util/UnknownFormatFlagsException.html#(java.lang.String)[UnknownFormatFlagsException](String) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* String {java11-javadoc}/java.base/java/util/UnknownFormatFlagsException.html#getFlags()[getFlags]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getLocalizedMessage()[getLocalizedMessage]() +* String {java11-javadoc}/java.base/java/lang/Throwable.html#getMessage()[getMessage]() +* StackTraceElement[] {java11-javadoc}/java.base/java/lang/Throwable.html#getStackTrace()[getStackTrace]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Vector]] +==== Vector +* {java11-javadoc}/java.base/java/util/Vector.html#()[Vector]() +* {java11-javadoc}/java.base/java/util/Vector.html#(java.util.Collection)[Vector](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* void {java11-javadoc}/java.base/java/util/Vector.html#addElement(java.lang.Object)[addElement](def) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* def {java11-javadoc}/java.base/java/util/Vector.html#clone()[clone]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* void {java11-javadoc}/java.base/java/util/Vector.html#copyInto(java.lang.Object%5B%5D)[copyInto](Object[]) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* def {java11-javadoc}/java.base/java/util/Vector.html#elementAt(int)[elementAt](int) +* Enumeration {java11-javadoc}/java.base/java/util/Vector.html#elements()[elements]() +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* def {java11-javadoc}/java.base/java/util/Vector.html#firstElement()[firstElement]() +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* def {java11-javadoc}/java.base/java/util/List.html#get(int)[get](int) +* int getLength() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* void {java11-javadoc}/java.base/java/util/Vector.html#insertElementAt(java.lang.Object,int)[insertElementAt](def, int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* def {java11-javadoc}/java.base/java/util/Vector.html#lastElement()[lastElement]() +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* int {java11-javadoc}/java.base/java/util/Vector.html#lastIndexOf(java.lang.Object,int)[lastIndexOf](def, int) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* void {java11-javadoc}/java.base/java/util/Vector.html#removeAllElements()[removeAllElements]() +* boolean {java11-javadoc}/java.base/java/util/Vector.html#removeElement(java.lang.Object)[removeElement](def) +* void {java11-javadoc}/java.base/java/util/Vector.html#removeElementAt(int)[removeElementAt](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* void {java11-javadoc}/java.base/java/util/Vector.html#setElementAt(java.lang.Object,int)[setElementAt](def, int) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-util-function"] +=== Shared API for package java.util.function +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-BiConsumer]] +==== BiConsumer +* void {java11-javadoc}/java.base/java/util/function/BiConsumer.html#accept(java.lang.Object,java.lang.Object)[accept](def, def) +* BiConsumer {java11-javadoc}/java.base/java/util/function/BiConsumer.html#andThen(java.util.function.BiConsumer)[andThen](BiConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-BiFunction]] +==== BiFunction +* BiFunction {java11-javadoc}/java.base/java/util/function/BiFunction.html#andThen(java.util.function.Function)[andThen](Function) +* def {java11-javadoc}/java.base/java/util/function/BiFunction.html#apply(java.lang.Object,java.lang.Object)[apply](def, def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-BiPredicate]] +==== BiPredicate +* BiPredicate {java11-javadoc}/java.base/java/util/function/BiPredicate.html#and(java.util.function.BiPredicate)[and](BiPredicate) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* BiPredicate {java11-javadoc}/java.base/java/util/function/BiPredicate.html#negate()[negate]() +* BiPredicate {java11-javadoc}/java.base/java/util/function/BiPredicate.html#or(java.util.function.BiPredicate)[or](BiPredicate) +* boolean {java11-javadoc}/java.base/java/util/function/BiPredicate.html#test(java.lang.Object,java.lang.Object)[test](def, def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-BinaryOperator]] +==== BinaryOperator +* static BinaryOperator {java11-javadoc}/java.base/java/util/function/BinaryOperator.html#maxBy(java.util.Comparator)[maxBy](Comparator) +* static BinaryOperator {java11-javadoc}/java.base/java/util/function/BinaryOperator.html#minBy(java.util.Comparator)[minBy](Comparator) +* BiFunction {java11-javadoc}/java.base/java/util/function/BiFunction.html#andThen(java.util.function.Function)[andThen](Function) +* def {java11-javadoc}/java.base/java/util/function/BiFunction.html#apply(java.lang.Object,java.lang.Object)[apply](def, def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-BooleanSupplier]] +==== BooleanSupplier +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/util/function/BooleanSupplier.html#getAsBoolean()[getAsBoolean]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Consumer]] +==== Consumer +* void {java11-javadoc}/java.base/java/util/function/Consumer.html#accept(java.lang.Object)[accept](def) +* Consumer {java11-javadoc}/java.base/java/util/function/Consumer.html#andThen(java.util.function.Consumer)[andThen](Consumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleBinaryOperator]] +==== DoubleBinaryOperator +* double {java11-javadoc}/java.base/java/util/function/DoubleBinaryOperator.html#applyAsDouble(double,double)[applyAsDouble](double, double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleConsumer]] +==== DoubleConsumer +* void {java11-javadoc}/java.base/java/util/function/DoubleConsumer.html#accept(double)[accept](double) +* DoubleConsumer {java11-javadoc}/java.base/java/util/function/DoubleConsumer.html#andThen(java.util.function.DoubleConsumer)[andThen](DoubleConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleFunction]] +==== DoubleFunction +* def {java11-javadoc}/java.base/java/util/function/DoubleFunction.html#apply(double)[apply](double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoublePredicate]] +==== DoublePredicate +* DoublePredicate {java11-javadoc}/java.base/java/util/function/DoublePredicate.html#and(java.util.function.DoublePredicate)[and](DoublePredicate) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* DoublePredicate {java11-javadoc}/java.base/java/util/function/DoublePredicate.html#negate()[negate]() +* DoublePredicate {java11-javadoc}/java.base/java/util/function/DoublePredicate.html#or(java.util.function.DoublePredicate)[or](DoublePredicate) +* boolean {java11-javadoc}/java.base/java/util/function/DoublePredicate.html#test(double)[test](double) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleSupplier]] +==== DoubleSupplier +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* double {java11-javadoc}/java.base/java/util/function/DoubleSupplier.html#getAsDouble()[getAsDouble]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleToIntFunction]] +==== DoubleToIntFunction +* int {java11-javadoc}/java.base/java/util/function/DoubleToIntFunction.html#applyAsInt(double)[applyAsInt](double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleToLongFunction]] +==== DoubleToLongFunction +* long {java11-javadoc}/java.base/java/util/function/DoubleToLongFunction.html#applyAsLong(double)[applyAsLong](double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleUnaryOperator]] +==== DoubleUnaryOperator +* static DoubleUnaryOperator {java11-javadoc}/java.base/java/util/function/DoubleUnaryOperator.html#identity()[identity]() +* DoubleUnaryOperator {java11-javadoc}/java.base/java/util/function/DoubleUnaryOperator.html#andThen(java.util.function.DoubleUnaryOperator)[andThen](DoubleUnaryOperator) +* double {java11-javadoc}/java.base/java/util/function/DoubleUnaryOperator.html#applyAsDouble(double)[applyAsDouble](double) +* DoubleUnaryOperator {java11-javadoc}/java.base/java/util/function/DoubleUnaryOperator.html#compose(java.util.function.DoubleUnaryOperator)[compose](DoubleUnaryOperator) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Function]] +==== Function +* static Function {java11-javadoc}/java.base/java/util/function/Function.html#identity()[identity]() +* Function {java11-javadoc}/java.base/java/util/function/Function.html#andThen(java.util.function.Function)[andThen](Function) +* def {java11-javadoc}/java.base/java/util/function/Function.html#apply(java.lang.Object)[apply](def) +* Function {java11-javadoc}/java.base/java/util/function/Function.html#compose(java.util.function.Function)[compose](Function) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntBinaryOperator]] +==== IntBinaryOperator +* int {java11-javadoc}/java.base/java/util/function/IntBinaryOperator.html#applyAsInt(int,int)[applyAsInt](int, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntConsumer]] +==== IntConsumer +* void {java11-javadoc}/java.base/java/util/function/IntConsumer.html#accept(int)[accept](int) +* IntConsumer {java11-javadoc}/java.base/java/util/function/IntConsumer.html#andThen(java.util.function.IntConsumer)[andThen](IntConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntFunction]] +==== IntFunction +* def {java11-javadoc}/java.base/java/util/function/IntFunction.html#apply(int)[apply](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntPredicate]] +==== IntPredicate +* IntPredicate {java11-javadoc}/java.base/java/util/function/IntPredicate.html#and(java.util.function.IntPredicate)[and](IntPredicate) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* IntPredicate {java11-javadoc}/java.base/java/util/function/IntPredicate.html#negate()[negate]() +* IntPredicate {java11-javadoc}/java.base/java/util/function/IntPredicate.html#or(java.util.function.IntPredicate)[or](IntPredicate) +* boolean {java11-javadoc}/java.base/java/util/function/IntPredicate.html#test(int)[test](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntSupplier]] +==== IntSupplier +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/function/IntSupplier.html#getAsInt()[getAsInt]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntToDoubleFunction]] +==== IntToDoubleFunction +* double {java11-javadoc}/java.base/java/util/function/IntToDoubleFunction.html#applyAsDouble(int)[applyAsDouble](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntToLongFunction]] +==== IntToLongFunction +* long {java11-javadoc}/java.base/java/util/function/IntToLongFunction.html#applyAsLong(int)[applyAsLong](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntUnaryOperator]] +==== IntUnaryOperator +* static IntUnaryOperator {java11-javadoc}/java.base/java/util/function/IntUnaryOperator.html#identity()[identity]() +* IntUnaryOperator {java11-javadoc}/java.base/java/util/function/IntUnaryOperator.html#andThen(java.util.function.IntUnaryOperator)[andThen](IntUnaryOperator) +* int {java11-javadoc}/java.base/java/util/function/IntUnaryOperator.html#applyAsInt(int)[applyAsInt](int) +* IntUnaryOperator {java11-javadoc}/java.base/java/util/function/IntUnaryOperator.html#compose(java.util.function.IntUnaryOperator)[compose](IntUnaryOperator) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongBinaryOperator]] +==== LongBinaryOperator +* long {java11-javadoc}/java.base/java/util/function/LongBinaryOperator.html#applyAsLong(long,long)[applyAsLong](long, long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongConsumer]] +==== LongConsumer +* void {java11-javadoc}/java.base/java/util/function/LongConsumer.html#accept(long)[accept](long) +* LongConsumer {java11-javadoc}/java.base/java/util/function/LongConsumer.html#andThen(java.util.function.LongConsumer)[andThen](LongConsumer) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongFunction]] +==== LongFunction +* def {java11-javadoc}/java.base/java/util/function/LongFunction.html#apply(long)[apply](long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongPredicate]] +==== LongPredicate +* LongPredicate {java11-javadoc}/java.base/java/util/function/LongPredicate.html#and(java.util.function.LongPredicate)[and](LongPredicate) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* LongPredicate {java11-javadoc}/java.base/java/util/function/LongPredicate.html#negate()[negate]() +* LongPredicate {java11-javadoc}/java.base/java/util/function/LongPredicate.html#or(java.util.function.LongPredicate)[or](LongPredicate) +* boolean {java11-javadoc}/java.base/java/util/function/LongPredicate.html#test(long)[test](long) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongSupplier]] +==== LongSupplier +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long {java11-javadoc}/java.base/java/util/function/LongSupplier.html#getAsLong()[getAsLong]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongToDoubleFunction]] +==== LongToDoubleFunction +* double {java11-javadoc}/java.base/java/util/function/LongToDoubleFunction.html#applyAsDouble(long)[applyAsDouble](long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongToIntFunction]] +==== LongToIntFunction +* int {java11-javadoc}/java.base/java/util/function/LongToIntFunction.html#applyAsInt(long)[applyAsInt](long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongUnaryOperator]] +==== LongUnaryOperator +* static LongUnaryOperator {java11-javadoc}/java.base/java/util/function/LongUnaryOperator.html#identity()[identity]() +* LongUnaryOperator {java11-javadoc}/java.base/java/util/function/LongUnaryOperator.html#andThen(java.util.function.LongUnaryOperator)[andThen](LongUnaryOperator) +* long {java11-javadoc}/java.base/java/util/function/LongUnaryOperator.html#applyAsLong(long)[applyAsLong](long) +* LongUnaryOperator {java11-javadoc}/java.base/java/util/function/LongUnaryOperator.html#compose(java.util.function.LongUnaryOperator)[compose](LongUnaryOperator) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ObjDoubleConsumer]] +==== ObjDoubleConsumer +* void {java11-javadoc}/java.base/java/util/function/ObjDoubleConsumer.html#accept(java.lang.Object,double)[accept](def, double) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ObjIntConsumer]] +==== ObjIntConsumer +* void {java11-javadoc}/java.base/java/util/function/ObjIntConsumer.html#accept(java.lang.Object,int)[accept](def, int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ObjLongConsumer]] +==== ObjLongConsumer +* void {java11-javadoc}/java.base/java/util/function/ObjLongConsumer.html#accept(java.lang.Object,long)[accept](def, long) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Predicate]] +==== Predicate +* static Predicate {java11-javadoc}/java.base/java/util/function/Predicate.html#isEqual(java.lang.Object)[isEqual](def) +* Predicate {java11-javadoc}/java.base/java/util/function/Predicate.html#and(java.util.function.Predicate)[and](Predicate) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Predicate {java11-javadoc}/java.base/java/util/function/Predicate.html#negate()[negate]() +* Predicate {java11-javadoc}/java.base/java/util/function/Predicate.html#or(java.util.function.Predicate)[or](Predicate) +* boolean {java11-javadoc}/java.base/java/util/function/Predicate.html#test(java.lang.Object)[test](def) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Supplier]] +==== Supplier +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* def {java11-javadoc}/java.base/java/util/function/Supplier.html#get()[get]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ToDoubleBiFunction]] +==== ToDoubleBiFunction +* double {java11-javadoc}/java.base/java/util/function/ToDoubleBiFunction.html#applyAsDouble(java.lang.Object,java.lang.Object)[applyAsDouble](def, def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ToDoubleFunction]] +==== ToDoubleFunction +* double {java11-javadoc}/java.base/java/util/function/ToDoubleFunction.html#applyAsDouble(java.lang.Object)[applyAsDouble](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ToIntBiFunction]] +==== ToIntBiFunction +* int {java11-javadoc}/java.base/java/util/function/ToIntBiFunction.html#applyAsInt(java.lang.Object,java.lang.Object)[applyAsInt](def, def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ToIntFunction]] +==== ToIntFunction +* int {java11-javadoc}/java.base/java/util/function/ToIntFunction.html#applyAsInt(java.lang.Object)[applyAsInt](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ToLongBiFunction]] +==== ToLongBiFunction +* long {java11-javadoc}/java.base/java/util/function/ToLongBiFunction.html#applyAsLong(java.lang.Object,java.lang.Object)[applyAsLong](def, def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ToLongFunction]] +==== ToLongFunction +* long {java11-javadoc}/java.base/java/util/function/ToLongFunction.html#applyAsLong(java.lang.Object)[applyAsLong](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-UnaryOperator]] +==== UnaryOperator +* static UnaryOperator {java11-javadoc}/java.base/java/util/function/UnaryOperator.html#identity()[identity]() +* Function {java11-javadoc}/java.base/java/util/function/Function.html#andThen(java.util.function.Function)[andThen](Function) +* def {java11-javadoc}/java.base/java/util/function/Function.html#apply(java.lang.Object)[apply](def) +* Function {java11-javadoc}/java.base/java/util/function/Function.html#compose(java.util.function.Function)[compose](Function) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-util-regex"] +=== Shared API for package java.util.regex +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-Matcher]] +==== Matcher +* static String {java11-javadoc}/java.base/java/util/regex/Matcher.html#quoteReplacement(java.lang.String)[quoteReplacement](String) +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#end()[end]() +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#end(int)[end](int) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#find()[find]() +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#find(int)[find](int) +* String {java11-javadoc}/java.base/java/util/regex/Matcher.html#group()[group]() +* String {java11-javadoc}/java.base/java/util/regex/Matcher.html#group(int)[group](int) +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#groupCount()[groupCount]() +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#hasAnchoringBounds()[hasAnchoringBounds]() +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#hasTransparentBounds()[hasTransparentBounds]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#hitEnd()[hitEnd]() +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#lookingAt()[lookingAt]() +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#matches()[matches]() +* String namedGroup(String) +* Pattern {java11-javadoc}/java.base/java/util/regex/Matcher.html#pattern()[pattern]() +* Matcher {java11-javadoc}/java.base/java/util/regex/Matcher.html#region(int,int)[region](int, int) +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#regionEnd()[regionEnd]() +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#regionStart()[regionStart]() +* String {java11-javadoc}/java.base/java/util/regex/Matcher.html#replaceAll(java.lang.String)[replaceAll](String) +* String {java11-javadoc}/java.base/java/util/regex/Matcher.html#replaceFirst(java.lang.String)[replaceFirst](String) +* boolean {java11-javadoc}/java.base/java/util/regex/Matcher.html#requireEnd()[requireEnd]() +* Matcher {java11-javadoc}/java.base/java/util/regex/Matcher.html#reset()[reset]() +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#start()[start]() +* int {java11-javadoc}/java.base/java/util/regex/Matcher.html#start(int)[start](int) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* Matcher {java11-javadoc}/java.base/java/util/regex/Matcher.html#useAnchoringBounds(boolean)[useAnchoringBounds](boolean) +* Matcher {java11-javadoc}/java.base/java/util/regex/Matcher.html#usePattern(java.util.regex.Pattern)[usePattern](Pattern) +* Matcher {java11-javadoc}/java.base/java/util/regex/Matcher.html#useTransparentBounds(boolean)[useTransparentBounds](boolean) + + +[[painless-api-reference-shared-Pattern]] +==== Pattern +* static String {java11-javadoc}/java.base/java/util/regex/Pattern.html#quote(java.lang.String)[quote](String) +* Predicate {java11-javadoc}/java.base/java/util/regex/Pattern.html#asPredicate()[asPredicate]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/util/regex/Pattern.html#flags()[flags]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Matcher {java11-javadoc}/java.base/java/util/regex/Pattern.html#matcher(java.lang.CharSequence)[matcher](CharSequence) +* String {java11-javadoc}/java.base/java/util/regex/Pattern.html#pattern()[pattern]() +* String[] {java11-javadoc}/java.base/java/util/regex/Pattern.html#split(java.lang.CharSequence)[split](CharSequence) +* String[] {java11-javadoc}/java.base/java/util/regex/Pattern.html#split(java.lang.CharSequence,int)[split](CharSequence, int) +* Stream {java11-javadoc}/java.base/java/util/regex/Pattern.html#splitAsStream(java.lang.CharSequence)[splitAsStream](CharSequence) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-java-util-stream"] +=== Shared API for package java.util.stream +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-BaseStream]] +==== BaseStream +* void {java11-javadoc}/java.base/java/util/stream/BaseStream.html#close()[close]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/stream/BaseStream.html#isParallel()[isParallel]() +* Iterator {java11-javadoc}/java.base/java/util/stream/BaseStream.html#iterator()[iterator]() +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#sequential()[sequential]() +* Spliterator {java11-javadoc}/java.base/java/util/stream/BaseStream.html#spliterator()[spliterator]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#unordered()[unordered]() + + +[[painless-api-reference-shared-Collector]] +==== Collector +* static Collector {java11-javadoc}/java.base/java/util/stream/Collector.html#of(java.util.function.Supplier,java.util.function.BiConsumer,java.util.function.BinaryOperator,java.util.stream.Collector$Characteristics%5B%5D)[of](Supplier, BiConsumer, BinaryOperator, Collector.Characteristics[]) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collector.html#of(java.util.function.Supplier,java.util.function.BiConsumer,java.util.function.BinaryOperator,java.util.function.Function,java.util.stream.Collector$Characteristics%5B%5D)[of](Supplier, BiConsumer, BinaryOperator, Function, Collector.Characteristics[]) +* BiConsumer {java11-javadoc}/java.base/java/util/stream/Collector.html#accumulator()[accumulator]() +* Set {java11-javadoc}/java.base/java/util/stream/Collector.html#characteristics()[characteristics]() +* BinaryOperator {java11-javadoc}/java.base/java/util/stream/Collector.html#combiner()[combiner]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Function {java11-javadoc}/java.base/java/util/stream/Collector.html#finisher()[finisher]() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* Supplier {java11-javadoc}/java.base/java/util/stream/Collector.html#supplier()[supplier]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Collector-Characteristics]] +==== Collector.Characteristics +* static Collector.Characteristics {java11-javadoc}/java.base/java/util/stream/Collector$Characteristics.html#CONCURRENT[CONCURRENT] +* static Collector.Characteristics {java11-javadoc}/java.base/java/util/stream/Collector$Characteristics.html#IDENTITY_FINISH[IDENTITY_FINISH] +* static Collector.Characteristics {java11-javadoc}/java.base/java/util/stream/Collector$Characteristics.html#UNORDERED[UNORDERED] +* static Collector.Characteristics {java11-javadoc}/java.base/java/util/stream/Collector$Characteristics.html#valueOf(java.lang.String)[valueOf](String) +* static Collector.Characteristics[] {java11-javadoc}/java.base/java/util/stream/Collector$Characteristics.html#values()[values]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#compareTo(java.lang.Enum)[compareTo](Enum) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Enum.html#name()[name]() +* int {java11-javadoc}/java.base/java/lang/Enum.html#ordinal()[ordinal]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Collectors]] +==== Collectors +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#averagingDouble(java.util.function.ToDoubleFunction)[averagingDouble](ToDoubleFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#averagingInt(java.util.function.ToIntFunction)[averagingInt](ToIntFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#averagingLong(java.util.function.ToLongFunction)[averagingLong](ToLongFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#collectingAndThen(java.util.stream.Collector,java.util.function.Function)[collectingAndThen](Collector, Function) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#counting()[counting]() +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#groupingBy(java.util.function.Function)[groupingBy](Function) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#groupingBy(java.util.function.Function,java.util.stream.Collector)[groupingBy](Function, Collector) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#groupingBy(java.util.function.Function,java.util.function.Supplier,java.util.stream.Collector)[groupingBy](Function, Supplier, Collector) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#joining()[joining]() +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#joining(java.lang.CharSequence)[joining](CharSequence) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#joining(java.lang.CharSequence,java.lang.CharSequence,java.lang.CharSequence)[joining](CharSequence, CharSequence, CharSequence) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#mapping(java.util.function.Function,java.util.stream.Collector)[mapping](Function, Collector) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#maxBy(java.util.Comparator)[maxBy](Comparator) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#minBy(java.util.Comparator)[minBy](Comparator) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#partitioningBy(java.util.function.Predicate)[partitioningBy](Predicate) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#partitioningBy(java.util.function.Predicate,java.util.stream.Collector)[partitioningBy](Predicate, Collector) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#reducing(java.util.function.BinaryOperator)[reducing](BinaryOperator) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#reducing(java.lang.Object,java.util.function.BinaryOperator)[reducing](def, BinaryOperator) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#reducing(java.lang.Object,java.util.function.Function,java.util.function.BinaryOperator)[reducing](def, Function, BinaryOperator) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#summarizingDouble(java.util.function.ToDoubleFunction)[summarizingDouble](ToDoubleFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#summarizingInt(java.util.function.ToIntFunction)[summarizingInt](ToIntFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#summarizingLong(java.util.function.ToLongFunction)[summarizingLong](ToLongFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#summingDouble(java.util.function.ToDoubleFunction)[summingDouble](ToDoubleFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#summingInt(java.util.function.ToIntFunction)[summingInt](ToIntFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#summingLong(java.util.function.ToLongFunction)[summingLong](ToLongFunction) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#toCollection(java.util.function.Supplier)[toCollection](Supplier) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#toList()[toList]() +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#toMap(java.util.function.Function,java.util.function.Function)[toMap](Function, Function) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#toMap(java.util.function.Function,java.util.function.Function,java.util.function.BinaryOperator)[toMap](Function, Function, BinaryOperator) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#toMap(java.util.function.Function,java.util.function.Function,java.util.function.BinaryOperator,java.util.function.Supplier)[toMap](Function, Function, BinaryOperator, Supplier) +* static Collector {java11-javadoc}/java.base/java/util/stream/Collectors.html#toSet()[toSet]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-DoubleStream]] +==== DoubleStream +* static DoubleStream.Builder {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#builder()[builder]() +* static DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#concat(java.util.stream.DoubleStream,java.util.stream.DoubleStream)[concat](DoubleStream, DoubleStream) +* static DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#empty()[empty]() +* static DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#of(double%5B%5D)[of](double[]) +* boolean {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#allMatch(java.util.function.DoublePredicate)[allMatch](DoublePredicate) +* boolean {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#anyMatch(java.util.function.DoublePredicate)[anyMatch](DoublePredicate) +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#average()[average]() +* Stream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#boxed()[boxed]() +* void {java11-javadoc}/java.base/java/util/stream/BaseStream.html#close()[close]() +* def {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#collect(java.util.function.Supplier,java.util.function.ObjDoubleConsumer,java.util.function.BiConsumer)[collect](Supplier, ObjDoubleConsumer, BiConsumer) +* long {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#count()[count]() +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#distinct()[distinct]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#filter(java.util.function.DoublePredicate)[filter](DoublePredicate) +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#findAny()[findAny]() +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#findFirst()[findFirst]() +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#flatMap(java.util.function.DoubleFunction)[flatMap](DoubleFunction) +* void {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#forEach(java.util.function.DoubleConsumer)[forEach](DoubleConsumer) +* void {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#forEachOrdered(java.util.function.DoubleConsumer)[forEachOrdered](DoubleConsumer) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/stream/BaseStream.html#isParallel()[isParallel]() +* PrimitiveIterator.OfDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#iterator()[iterator]() +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#limit(long)[limit](long) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#map(java.util.function.DoubleUnaryOperator)[map](DoubleUnaryOperator) +* IntStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#mapToInt(java.util.function.DoubleToIntFunction)[mapToInt](DoubleToIntFunction) +* LongStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#mapToLong(java.util.function.DoubleToLongFunction)[mapToLong](DoubleToLongFunction) +* Stream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#mapToObj(java.util.function.DoubleFunction)[mapToObj](DoubleFunction) +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#max()[max]() +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#min()[min]() +* boolean {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#noneMatch(java.util.function.DoublePredicate)[noneMatch](DoublePredicate) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#peek(java.util.function.DoubleConsumer)[peek](DoubleConsumer) +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#reduce(java.util.function.DoubleBinaryOperator)[reduce](DoubleBinaryOperator) +* double {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#reduce(double,java.util.function.DoubleBinaryOperator)[reduce](double, DoubleBinaryOperator) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#sequential()[sequential]() +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#skip(long)[skip](long) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#sorted()[sorted]() +* Spliterator.OfDouble {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#spliterator()[spliterator]() +* double {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#sum()[sum]() +* DoubleSummaryStatistics {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#summaryStatistics()[summaryStatistics]() +* double[] {java11-javadoc}/java.base/java/util/stream/DoubleStream.html#toArray()[toArray]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#unordered()[unordered]() + + +[[painless-api-reference-shared-DoubleStream-Builder]] +==== DoubleStream.Builder +* void {java11-javadoc}/java.base/java/util/function/DoubleConsumer.html#accept(double)[accept](double) +* DoubleStream.Builder {java11-javadoc}/java.base/java/util/stream/DoubleStream$Builder.html#add(double)[add](double) +* DoubleConsumer {java11-javadoc}/java.base/java/util/function/DoubleConsumer.html#andThen(java.util.function.DoubleConsumer)[andThen](DoubleConsumer) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/DoubleStream$Builder.html#build()[build]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-IntStream]] +==== IntStream +* static IntStream.Builder {java11-javadoc}/java.base/java/util/stream/IntStream.html#builder()[builder]() +* static IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#concat(java.util.stream.IntStream,java.util.stream.IntStream)[concat](IntStream, IntStream) +* static IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#empty()[empty]() +* static IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#of(int%5B%5D)[of](int[]) +* static IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#range(int,int)[range](int, int) +* static IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#rangeClosed(int,int)[rangeClosed](int, int) +* boolean {java11-javadoc}/java.base/java/util/stream/IntStream.html#allMatch(java.util.function.IntPredicate)[allMatch](IntPredicate) +* boolean {java11-javadoc}/java.base/java/util/stream/IntStream.html#anyMatch(java.util.function.IntPredicate)[anyMatch](IntPredicate) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#asDoubleStream()[asDoubleStream]() +* LongStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#asLongStream()[asLongStream]() +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/IntStream.html#average()[average]() +* Stream {java11-javadoc}/java.base/java/util/stream/IntStream.html#boxed()[boxed]() +* void {java11-javadoc}/java.base/java/util/stream/BaseStream.html#close()[close]() +* def {java11-javadoc}/java.base/java/util/stream/IntStream.html#collect(java.util.function.Supplier,java.util.function.ObjIntConsumer,java.util.function.BiConsumer)[collect](Supplier, ObjIntConsumer, BiConsumer) +* long {java11-javadoc}/java.base/java/util/stream/IntStream.html#count()[count]() +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#distinct()[distinct]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#filter(java.util.function.IntPredicate)[filter](IntPredicate) +* OptionalInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#findAny()[findAny]() +* OptionalInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#findFirst()[findFirst]() +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#flatMap(java.util.function.IntFunction)[flatMap](IntFunction) +* void {java11-javadoc}/java.base/java/util/stream/IntStream.html#forEach(java.util.function.IntConsumer)[forEach](IntConsumer) +* void {java11-javadoc}/java.base/java/util/stream/IntStream.html#forEachOrdered(java.util.function.IntConsumer)[forEachOrdered](IntConsumer) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/stream/BaseStream.html#isParallel()[isParallel]() +* PrimitiveIterator.OfInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#iterator()[iterator]() +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#limit(long)[limit](long) +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#map(java.util.function.IntUnaryOperator)[map](IntUnaryOperator) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#mapToDouble(java.util.function.IntToDoubleFunction)[mapToDouble](IntToDoubleFunction) +* LongStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#mapToLong(java.util.function.IntToLongFunction)[mapToLong](IntToLongFunction) +* Stream {java11-javadoc}/java.base/java/util/stream/IntStream.html#mapToObj(java.util.function.IntFunction)[mapToObj](IntFunction) +* OptionalInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#max()[max]() +* OptionalInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#min()[min]() +* boolean {java11-javadoc}/java.base/java/util/stream/IntStream.html#noneMatch(java.util.function.IntPredicate)[noneMatch](IntPredicate) +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#peek(java.util.function.IntConsumer)[peek](IntConsumer) +* OptionalInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#reduce(java.util.function.IntBinaryOperator)[reduce](IntBinaryOperator) +* int {java11-javadoc}/java.base/java/util/stream/IntStream.html#reduce(int,java.util.function.IntBinaryOperator)[reduce](int, IntBinaryOperator) +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#sequential()[sequential]() +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#skip(long)[skip](long) +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream.html#sorted()[sorted]() +* Spliterator.OfInt {java11-javadoc}/java.base/java/util/stream/IntStream.html#spliterator()[spliterator]() +* int {java11-javadoc}/java.base/java/util/stream/IntStream.html#sum()[sum]() +* IntSummaryStatistics {java11-javadoc}/java.base/java/util/stream/IntStream.html#summaryStatistics()[summaryStatistics]() +* int[] {java11-javadoc}/java.base/java/util/stream/IntStream.html#toArray()[toArray]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#unordered()[unordered]() + + +[[painless-api-reference-shared-IntStream-Builder]] +==== IntStream.Builder +* void {java11-javadoc}/java.base/java/util/function/IntConsumer.html#accept(int)[accept](int) +* IntStream.Builder {java11-javadoc}/java.base/java/util/stream/IntStream$Builder.html#add(int)[add](int) +* IntConsumer {java11-javadoc}/java.base/java/util/function/IntConsumer.html#andThen(java.util.function.IntConsumer)[andThen](IntConsumer) +* IntStream {java11-javadoc}/java.base/java/util/stream/IntStream$Builder.html#build()[build]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-LongStream]] +==== LongStream +* static LongStream.Builder {java11-javadoc}/java.base/java/util/stream/LongStream.html#builder()[builder]() +* static LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#concat(java.util.stream.LongStream,java.util.stream.LongStream)[concat](LongStream, LongStream) +* static LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#empty()[empty]() +* static LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#of(long%5B%5D)[of](long[]) +* static LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#range(long,long)[range](long, long) +* static LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#rangeClosed(long,long)[rangeClosed](long, long) +* boolean {java11-javadoc}/java.base/java/util/stream/LongStream.html#allMatch(java.util.function.LongPredicate)[allMatch](LongPredicate) +* boolean {java11-javadoc}/java.base/java/util/stream/LongStream.html#anyMatch(java.util.function.LongPredicate)[anyMatch](LongPredicate) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#asDoubleStream()[asDoubleStream]() +* OptionalDouble {java11-javadoc}/java.base/java/util/stream/LongStream.html#average()[average]() +* Stream {java11-javadoc}/java.base/java/util/stream/LongStream.html#boxed()[boxed]() +* void {java11-javadoc}/java.base/java/util/stream/BaseStream.html#close()[close]() +* def {java11-javadoc}/java.base/java/util/stream/LongStream.html#collect(java.util.function.Supplier,java.util.function.ObjLongConsumer,java.util.function.BiConsumer)[collect](Supplier, ObjLongConsumer, BiConsumer) +* long {java11-javadoc}/java.base/java/util/stream/LongStream.html#count()[count]() +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#distinct()[distinct]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#filter(java.util.function.LongPredicate)[filter](LongPredicate) +* OptionalLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#findAny()[findAny]() +* OptionalLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#findFirst()[findFirst]() +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#flatMap(java.util.function.LongFunction)[flatMap](LongFunction) +* void {java11-javadoc}/java.base/java/util/stream/LongStream.html#forEach(java.util.function.LongConsumer)[forEach](LongConsumer) +* void {java11-javadoc}/java.base/java/util/stream/LongStream.html#forEachOrdered(java.util.function.LongConsumer)[forEachOrdered](LongConsumer) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/stream/BaseStream.html#isParallel()[isParallel]() +* PrimitiveIterator.OfLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#iterator()[iterator]() +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#limit(long)[limit](long) +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#map(java.util.function.LongUnaryOperator)[map](LongUnaryOperator) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#mapToDouble(java.util.function.LongToDoubleFunction)[mapToDouble](LongToDoubleFunction) +* IntStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#mapToInt(java.util.function.LongToIntFunction)[mapToInt](LongToIntFunction) +* Stream {java11-javadoc}/java.base/java/util/stream/LongStream.html#mapToObj(java.util.function.LongFunction)[mapToObj](LongFunction) +* OptionalLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#max()[max]() +* OptionalLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#min()[min]() +* boolean {java11-javadoc}/java.base/java/util/stream/LongStream.html#noneMatch(java.util.function.LongPredicate)[noneMatch](LongPredicate) +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#peek(java.util.function.LongConsumer)[peek](LongConsumer) +* OptionalLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#reduce(java.util.function.LongBinaryOperator)[reduce](LongBinaryOperator) +* long {java11-javadoc}/java.base/java/util/stream/LongStream.html#reduce(long,java.util.function.LongBinaryOperator)[reduce](long, LongBinaryOperator) +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#sequential()[sequential]() +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#skip(long)[skip](long) +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream.html#sorted()[sorted]() +* Spliterator.OfLong {java11-javadoc}/java.base/java/util/stream/LongStream.html#spliterator()[spliterator]() +* long {java11-javadoc}/java.base/java/util/stream/LongStream.html#sum()[sum]() +* LongSummaryStatistics {java11-javadoc}/java.base/java/util/stream/LongStream.html#summaryStatistics()[summaryStatistics]() +* long[] {java11-javadoc}/java.base/java/util/stream/LongStream.html#toArray()[toArray]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#unordered()[unordered]() + + +[[painless-api-reference-shared-LongStream-Builder]] +==== LongStream.Builder +* void {java11-javadoc}/java.base/java/util/function/LongConsumer.html#accept(long)[accept](long) +* LongStream.Builder {java11-javadoc}/java.base/java/util/stream/LongStream$Builder.html#add(long)[add](long) +* LongConsumer {java11-javadoc}/java.base/java/util/function/LongConsumer.html#andThen(java.util.function.LongConsumer)[andThen](LongConsumer) +* LongStream {java11-javadoc}/java.base/java/util/stream/LongStream$Builder.html#build()[build]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-Stream]] +==== Stream +* static Stream.Builder {java11-javadoc}/java.base/java/util/stream/Stream.html#builder()[builder]() +* static Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#concat(java.util.stream.Stream,java.util.stream.Stream)[concat](Stream, Stream) +* static Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#empty()[empty]() +* static Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#of(java.lang.Object%5B%5D)[of](def[]) +* boolean {java11-javadoc}/java.base/java/util/stream/Stream.html#allMatch(java.util.function.Predicate)[allMatch](Predicate) +* boolean {java11-javadoc}/java.base/java/util/stream/Stream.html#anyMatch(java.util.function.Predicate)[anyMatch](Predicate) +* void {java11-javadoc}/java.base/java/util/stream/BaseStream.html#close()[close]() +* def {java11-javadoc}/java.base/java/util/stream/Stream.html#collect(java.util.stream.Collector)[collect](Collector) +* def {java11-javadoc}/java.base/java/util/stream/Stream.html#collect(java.util.function.Supplier,java.util.function.BiConsumer,java.util.function.BiConsumer)[collect](Supplier, BiConsumer, BiConsumer) +* long {java11-javadoc}/java.base/java/util/stream/Stream.html#count()[count]() +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#distinct()[distinct]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#filter(java.util.function.Predicate)[filter](Predicate) +* Optional {java11-javadoc}/java.base/java/util/stream/Stream.html#findAny()[findAny]() +* Optional {java11-javadoc}/java.base/java/util/stream/Stream.html#findFirst()[findFirst]() +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#flatMap(java.util.function.Function)[flatMap](Function) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/Stream.html#flatMapToDouble(java.util.function.Function)[flatMapToDouble](Function) +* IntStream {java11-javadoc}/java.base/java/util/stream/Stream.html#flatMapToInt(java.util.function.Function)[flatMapToInt](Function) +* LongStream {java11-javadoc}/java.base/java/util/stream/Stream.html#flatMapToLong(java.util.function.Function)[flatMapToLong](Function) +* void {java11-javadoc}/java.base/java/util/stream/Stream.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* void {java11-javadoc}/java.base/java/util/stream/Stream.html#forEachOrdered(java.util.function.Consumer)[forEachOrdered](Consumer) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean {java11-javadoc}/java.base/java/util/stream/BaseStream.html#isParallel()[isParallel]() +* Iterator {java11-javadoc}/java.base/java/util/stream/BaseStream.html#iterator()[iterator]() +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#limit(long)[limit](long) +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#map(java.util.function.Function)[map](Function) +* DoubleStream {java11-javadoc}/java.base/java/util/stream/Stream.html#mapToDouble(java.util.function.ToDoubleFunction)[mapToDouble](ToDoubleFunction) +* IntStream {java11-javadoc}/java.base/java/util/stream/Stream.html#mapToInt(java.util.function.ToIntFunction)[mapToInt](ToIntFunction) +* LongStream {java11-javadoc}/java.base/java/util/stream/Stream.html#mapToLong(java.util.function.ToLongFunction)[mapToLong](ToLongFunction) +* Optional {java11-javadoc}/java.base/java/util/stream/Stream.html#max(java.util.Comparator)[max](Comparator) +* Optional {java11-javadoc}/java.base/java/util/stream/Stream.html#min(java.util.Comparator)[min](Comparator) +* boolean {java11-javadoc}/java.base/java/util/stream/Stream.html#noneMatch(java.util.function.Predicate)[noneMatch](Predicate) +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#peek(java.util.function.Consumer)[peek](Consumer) +* Optional {java11-javadoc}/java.base/java/util/stream/Stream.html#reduce(java.util.function.BinaryOperator)[reduce](BinaryOperator) +* def {java11-javadoc}/java.base/java/util/stream/Stream.html#reduce(java.lang.Object,java.util.function.BinaryOperator)[reduce](def, BinaryOperator) +* def {java11-javadoc}/java.base/java/util/stream/Stream.html#reduce(java.lang.Object,java.util.function.BiFunction,java.util.function.BinaryOperator)[reduce](def, BiFunction, BinaryOperator) +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#sequential()[sequential]() +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#skip(long)[skip](long) +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#sorted()[sorted]() +* Stream {java11-javadoc}/java.base/java/util/stream/Stream.html#sorted(java.util.Comparator)[sorted](Comparator) +* Spliterator {java11-javadoc}/java.base/java/util/stream/BaseStream.html#spliterator()[spliterator]() +* def[] {java11-javadoc}/java.base/java/util/stream/Stream.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/stream/Stream.html#toArray(java.util.function.IntFunction)[toArray](IntFunction) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* BaseStream {java11-javadoc}/java.base/java/util/stream/BaseStream.html#unordered()[unordered]() + + +[[painless-api-reference-shared-Stream-Builder]] +==== Stream.Builder +* void {java11-javadoc}/java.base/java/util/function/Consumer.html#accept(java.lang.Object)[accept](def) +* Stream.Builder {java11-javadoc}/java.base/java/util/stream/Stream$Builder.html#add(java.lang.Object)[add](def) +* Consumer {java11-javadoc}/java.base/java/util/function/Consumer.html#andThen(java.util.function.Consumer)[andThen](Consumer) +* Stream {java11-javadoc}/java.base/java/util/stream/Stream$Builder.html#build()[build]() +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-apache-lucene-util"] +=== Shared API for package org.apache.lucene.util +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-BytesRef]] +==== BytesRef +* byte[] bytes +* int length +* int offset +* boolean bytesEquals(BytesRef) +* int {java11-javadoc}/java.base/java/lang/Comparable.html#compareTo(java.lang.Object)[compareTo](def) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() +* String utf8ToString() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-common-geo"] +=== Shared API for package org.elasticsearch.common.geo +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-GeoPoint]] +==== GeoPoint +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* double getLat() +* double getLon() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-index-fielddata"] +=== Shared API for package org.elasticsearch.index.fielddata +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-ScriptDocValues-Booleans]] +==== ScriptDocValues.Booleans +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Boolean get(int) +* int getLength() +* boolean getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptDocValues-BytesRefs]] +==== ScriptDocValues.BytesRefs +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* BytesRef get(int) +* int getLength() +* BytesRef getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptDocValues-Dates]] +==== ScriptDocValues.Dates +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* JodaCompatibleZonedDateTime get(int) +* int getLength() +* JodaCompatibleZonedDateTime getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptDocValues-Doubles]] +==== ScriptDocValues.Doubles +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Double get(int) +* int getLength() +* double getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptDocValues-GeoPoints]] +==== ScriptDocValues.GeoPoints +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* double arcDistance(double, double) +* double arcDistanceWithDefault(double, double, double) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* double geohashDistance(String) +* double geohashDistanceWithDefault(String, double) +* GeoPoint get(int) +* double getLat() +* double[] getLats() +* int getLength() +* double getLon() +* double[] getLons() +* GeoPoint getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* double planeDistance(double, double) +* double planeDistanceWithDefault(double, double, double) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptDocValues-Longs]] +==== ScriptDocValues.Longs +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* Long get(int) +* int getLength() +* long getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptDocValues-Strings]] +==== ScriptDocValues.Strings +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* String get(int) +* int getLength() +* String getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-index-mapper"] +=== Shared API for package org.elasticsearch.index.mapper +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-IpFieldMapper-IpFieldType-IpScriptDocValues]] +==== IpFieldMapper.IpFieldType.IpScriptDocValues +* boolean {java11-javadoc}/java.base/java/util/Collection.html#add(java.lang.Object)[add](def) +* void {java11-javadoc}/java.base/java/util/List.html#add(int,java.lang.Object)[add](int, def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#addAll(java.util.Collection)[addAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/List.html#addAll(int,java.util.Collection)[addAll](int, Collection) +* boolean any(Predicate) +* Collection asCollection() +* List asList() +* void {java11-javadoc}/java.base/java/util/Collection.html#clear()[clear]() +* List collect(Function) +* def collect(Collection, Function) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#contains(java.lang.Object)[contains](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#containsAll(java.util.Collection)[containsAll](Collection) +* def each(Consumer) +* def eachWithIndex(ObjIntConsumer) +* boolean {java11-javadoc}/java.base/java/util/List.html#equals(java.lang.Object)[equals](Object) +* boolean every(Predicate) +* def find(Predicate) +* List findAll(Predicate) +* def findResult(Function) +* def findResult(def, Function) +* List findResults(Function) +* void {java11-javadoc}/java.base/java/lang/Iterable.html#forEach(java.util.function.Consumer)[forEach](Consumer) +* String get(int) +* int getLength() +* String getValue() +* Map groupBy(Function) +* int {java11-javadoc}/java.base/java/util/List.html#hashCode()[hashCode]() +* int {java11-javadoc}/java.base/java/util/List.html#indexOf(java.lang.Object)[indexOf](def) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#isEmpty()[isEmpty]() +* Iterator {java11-javadoc}/java.base/java/lang/Iterable.html#iterator()[iterator]() +* String join(String) +* int {java11-javadoc}/java.base/java/util/List.html#lastIndexOf(java.lang.Object)[lastIndexOf](def) +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator()[listIterator]() +* ListIterator {java11-javadoc}/java.base/java/util/List.html#listIterator(int)[listIterator](int) +* def {java11-javadoc}/java.base/java/util/List.html#remove(int)[remove](int) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeAll(java.util.Collection)[removeAll](Collection) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#removeIf(java.util.function.Predicate)[removeIf](Predicate) +* void {java11-javadoc}/java.base/java/util/List.html#replaceAll(java.util.function.UnaryOperator)[replaceAll](UnaryOperator) +* boolean {java11-javadoc}/java.base/java/util/Collection.html#retainAll(java.util.Collection)[retainAll](Collection) +* def {java11-javadoc}/java.base/java/util/List.html#set(int,java.lang.Object)[set](int, def) +* int {java11-javadoc}/java.base/java/util/Collection.html#size()[size]() +* void {java11-javadoc}/java.base/java/util/List.html#sort(java.util.Comparator)[sort](Comparator) +* List split(Predicate) +* Spliterator {java11-javadoc}/java.base/java/util/Collection.html#spliterator()[spliterator]() +* Stream {java11-javadoc}/java.base/java/util/Collection.html#stream()[stream]() +* List {java11-javadoc}/java.base/java/util/List.html#subList(int,int)[subList](int, int) +* double sum() +* double sum(ToDoubleFunction) +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray()[toArray]() +* def[] {java11-javadoc}/java.base/java/util/Collection.html#toArray(java.lang.Object%5B%5D)[toArray](def[]) +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-index-query"] +=== Shared API for package org.elasticsearch.index.query +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-IntervalFilterScript-Interval]] +==== IntervalFilterScript.Interval +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int getEnd() +* int getGaps() +* int getStart() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-index-similarity"] +=== Shared API for package org.elasticsearch.index.similarity +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-ScriptedSimilarity-Doc]] +==== ScriptedSimilarity.Doc +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float getFreq() +* int getLength() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptedSimilarity-Field]] +==== ScriptedSimilarity.Field +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long getDocCount() +* long getSumDocFreq() +* long getSumTotalTermFreq() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptedSimilarity-Query]] +==== ScriptedSimilarity.Query +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* float getBoost() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[[painless-api-reference-shared-ScriptedSimilarity-Term]] +==== ScriptedSimilarity.Term +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* long getDocFreq() +* long getTotalTermFreq() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-painless-api"] +=== Shared API for package org.elasticsearch.painless.api +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-Debug]] +==== Debug +* static void explain(Object) +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-script"] +=== Shared API for package org.elasticsearch.script +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-JodaCompatibleZonedDateTime]] +==== JodaCompatibleZonedDateTime +* boolean equals(Object) +* int getCenturyOfEra() +* int getDayOfMonth() +* int getDayOfWeek() +* DayOfWeek getDayOfWeekEnum() +* int getDayOfYear() +* int getEra() +* int getHour() +* int getHourOfDay() +* long getMillis() +* int getMillisOfDay() +* int getMillisOfSecond() +* int getMinute() +* int getMinuteOfDay() +* int getMinuteOfHour() +* Month getMonth() +* int getMonthOfYear() +* int getMonthValue() +* int getNano() +* int getSecond() +* int getSecondOfDay() +* int getSecondOfMinute() +* int getWeekOfWeekyear() +* int getWeekyear() +* int getYear() +* int getYearOfCentury() +* int getYearOfEra() +* ZoneId getZone() +* int hashCode() +* boolean isAfter(JodaCompatibleZonedDateTime) +* boolean isBefore(JodaCompatibleZonedDateTime) +* boolean isEqual(JodaCompatibleZonedDateTime) +* ZonedDateTime minus(TemporalAmount) +* ZonedDateTime minus(long, TemporalUnit) +* ZonedDateTime minusDays(long) +* ZonedDateTime minusHours(long) +* ZonedDateTime minusMinutes(long) +* ZonedDateTime minusMonths(long) +* ZonedDateTime minusNanos(long) +* ZonedDateTime minusSeconds(long) +* ZonedDateTime minusWeeks(long) +* ZonedDateTime minusYears(long) +* ZonedDateTime plus(TemporalAmount) +* ZonedDateTime plus(long, TemporalUnit) +* ZonedDateTime plusDays(long) +* ZonedDateTime plusHours(long) +* ZonedDateTime plusMinutes(long) +* ZonedDateTime plusMonths(long) +* ZonedDateTime plusNanos(long) +* ZonedDateTime plusSeconds(long) +* ZonedDateTime plusWeeks(long) +* ZonedDateTime plusYears(long) +* Instant toInstant() +* LocalDate toLocalDate() +* LocalDateTime toLocalDateTime() +* OffsetDateTime toOffsetDateTime() +* String toString() +* String toString(String) +* String toString(String, Locale) +* ZonedDateTime truncatedTo(TemporalUnit) +* ZonedDateTime with(TemporalAdjuster) +* ZonedDateTime with(TemporalField, long) +* ZonedDateTime withDayOfMonth(int) +* ZonedDateTime withDayOfYear(int) +* ZonedDateTime withEarlierOffsetAtOverlap() +* ZonedDateTime withFixedOffsetZone() +* ZonedDateTime withHour(int) +* ZonedDateTime withLaterOffsetAtOverlap() +* ZonedDateTime withMinute(int) +* ZonedDateTime withMonth(int) +* ZonedDateTime withNano(int) +* ZonedDateTime withSecond(int) +* ZonedDateTime withYear(int) +* ZonedDateTime withZoneSameInstant(ZoneId) +* ZonedDateTime withZoneSameLocal(ZoneId) + + +[role="exclude",id="painless-api-reference-shared-org-elasticsearch-search-lookup"] +=== Shared API for package org.elasticsearch.search.lookup +See the <> for a high-level overview of all packages. + +[[painless-api-reference-shared-FieldLookup]] +==== FieldLookup +* boolean {java11-javadoc}/java.base/java/lang/Object.html#equals(java.lang.Object)[equals](Object) +* def getValue() +* List getValues() +* int {java11-javadoc}/java.base/java/lang/Object.html#hashCode()[hashCode]() +* boolean isEmpty() +* String {java11-javadoc}/java.base/java/lang/Object.html#toString()[toString]() + + diff --git a/docs/reference/redirects.asciidoc b/docs/reference/redirects.asciidoc index 6789395d44a31..7e00e10a52050 100644 --- a/docs/reference/redirects.asciidoc +++ b/docs/reference/redirects.asciidoc @@ -512,10 +512,10 @@ See {painless}/painless-debugging.html[Painless Debugging] in the guide to the {painless}/index.html[Painless Scripting Language]. [role="exclude",id="painless-api-reference"] -=== Painless API Reference +=== Painless Contexts API Reference -See the {painless}/painless-api-reference.html[Painless API Reference] in -the guide to the {painless}/index.html[Painless Scripting Language]. +See the {painless}/painless-api-reference.html[Painless Contexts API Reference] +in the guide to the {painless}/index.html[Painless Scripting Language]. [role="exclude", id="security-api-roles"] === Role management APIs diff --git a/modules/lang-painless/build.gradle b/modules/lang-painless/build.gradle index a8733fef90d56..1f6b722ec308c 100644 --- a/modules/lang-painless/build.gradle +++ b/modules/lang-painless/build.gradle @@ -17,6 +17,9 @@ * under the License. */ +import org.elasticsearch.gradle.test.ClusterConfiguration +import org.elasticsearch.gradle.test.ClusterFormationTasks + esplugin { description 'An easy, safe and fast scripting language for Elasticsearch' classname 'org.elasticsearch.painless.PainlessPlugin' @@ -49,17 +52,37 @@ task apiJavadoc(type: Javadoc) { include '**/org/elasticsearch/painless/api/' destinationDir = new File(docsDir, 'apiJavadoc') } + task apiJavadocJar(type: Jar) { classifier = 'apiJavadoc' from apiJavadoc } + assemble.dependsOn apiJavadocJar -// Reference documentation for Painless's public API. -task generatePainlessApi(type: JavaExec) { - main = 'org.elasticsearch.painless.PainlessDocGenerator' - classpath = sourceSets.test.runtimeClasspath - args file('../../docs/painless/painless-api-reference') +/********************************************** + * Context API Generation * + **********************************************/ + +sourceSets { + doc +} + +dependencies { + docCompile "org.elasticsearch:elasticsearch:${project.versions.elasticsearch}" + docCompile project(':modules:lang-painless') +} + +ClusterConfiguration clusterConfig = project.extensions.create("generateContextCluster", ClusterConfiguration.class, project) +gradle.projectsEvaluated { + project.ext.generateContextNodes = ClusterFormationTasks.setup(project, "generateContextCluster", generateContextDoc, clusterConfig) +} +clusterConfig.distribution = 'default' + +task generateContextDoc(type: JavaExec) { + main = 'org.elasticsearch.painless.ContextDocGenerator' + classpath = sourceSets.doc.runtimeClasspath + systemProperty "cluster.uri", "${-> project.ext.generateContextNodes.collect { it.httpUri() }.join(',') }" } /********************************************** diff --git a/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java b/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java new file mode 100644 index 0000000000000..fbea8d91726c9 --- /dev/null +++ b/modules/lang-painless/src/doc/java/org/elasticsearch/painless/ContextDocGenerator.java @@ -0,0 +1,670 @@ +/* + * 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.painless; + +import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.io.PathUtils; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.painless.action.PainlessContextClassInfo; +import org.elasticsearch.painless.action.PainlessContextConstructorInfo; +import org.elasticsearch.painless.action.PainlessContextFieldInfo; +import org.elasticsearch.painless.action.PainlessContextInfo; +import org.elasticsearch.painless.action.PainlessContextMethodInfo; + +import java.io.IOException; +import java.io.PrintStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * The gradle task generateContextDoc uses ContextDocGenerator to rebuild + * the Painless API documentation from a clean state after the + * existing documentation is deleted. The following pages are generated: + *
      + *
    • An index page with each context and links to the APIs
    • + *
    • A high-level overview page of shared API for all contexts
    • + *
    • A detailed page per package per context of shared API for all contexts
    • + *
    • A high-level overview page of specialized API for each context
    • + *
    • A detailed page per package per context of specialized API for each context
    • + *
    + * Use the docs build to generate HTML pages from the resultant asciidoc files. + */ +public final class ContextDocGenerator { + + private static final String SHARED_HEADER = "painless-api-reference-shared"; + private static final String SHARED_NAME = "Shared"; + + public static void main(String[] args) throws IOException { + List contextInfos = getContextInfos(); + Set sharedClassInfos = createShared(contextInfos); + + Path rootDir = resetRootDir(); + + Path sharedDir = createSharedDir(rootDir); + List classInfos = sortClassInfos(new ArrayList<>(sharedClassInfos), Collections.emptySet()); + Map javaNamesToDisplayNames = getDisplayNames(classInfos); + printSharedIndexPage(sharedDir, javaNamesToDisplayNames, classInfos); + printSharedPackagesPages(sharedDir, javaNamesToDisplayNames, classInfos); + + Set isSpecialized = new HashSet<>(); + + for (PainlessContextInfo contextInfo : contextInfos) { + Path contextDir = createContextDir(rootDir, contextInfo); + classInfos = sortClassInfos(new ArrayList<>(contextInfo.getClasses()), sharedClassInfos); + + if (classInfos.isEmpty() == false) { + isSpecialized.add(contextInfo); + javaNamesToDisplayNames = getDisplayNames(contextInfo.getClasses()); + printContextIndexPage(contextDir, javaNamesToDisplayNames, sharedClassInfos, contextInfo, classInfos); + printContextPackagesPages(contextDir, javaNamesToDisplayNames, sharedClassInfos, contextInfo, classInfos); + } + } + + printRootIndexPage(rootDir, contextInfos, isSpecialized); + } + + @SuppressForbidden(reason = "retrieving data from an internal API not exposed as part of the REST client") + private static List getContextInfos() throws IOException { + URLConnection getContextNames = new URL( + "http://" + System.getProperty("cluster.uri") + "/_scripts/painless/_context").openConnection(); + XContentParser parser = JsonXContent.jsonXContent.createParser(null, null, getContextNames.getInputStream()); + parser.nextToken(); + parser.nextToken(); + @SuppressWarnings("unchecked") + List contextNames = (List)(Object)parser.list(); + parser.close(); + ((HttpURLConnection)getContextNames).disconnect(); + + List contextInfos = new ArrayList<>(); + + for (String contextName : contextNames) { + URLConnection getContextInfo = new URL( + "http://" + System.getProperty("cluster.uri") + "/_scripts/painless/_context?context=" + contextName).openConnection(); + parser = JsonXContent.jsonXContent.createParser(null, null, getContextInfo.getInputStream()); + contextInfos.add(PainlessContextInfo.fromXContent(parser)); + ((HttpURLConnection)getContextInfo).disconnect(); + } + + contextInfos.sort(Comparator.comparing(PainlessContextInfo::getName)); + + return contextInfos; + } + + private static Set createShared(List contextInfos) { + Map classInfoCounts = new HashMap<>(); + + for (PainlessContextInfo contextInfo : contextInfos) { + for (PainlessContextClassInfo classInfo : contextInfo.getClasses()) { + classInfoCounts.compute(classInfo, (k, v) -> v == null ? 1 : v + 1); + } + } + + return classInfoCounts.entrySet().stream().filter( + e -> e.getValue() == contextInfos.size() + ).map(Map.Entry::getKey).collect(Collectors.toSet()); + } + + @SuppressForbidden(reason = "resolve api docs directory with environment") + private static Path resetRootDir() throws IOException { + Path rootDir = PathUtils.get("../../docs/painless/painless-api-reference"); + IOUtils.rm(rootDir); + Files.createDirectories(rootDir); + + return rootDir; + } + + private static Path createSharedDir(Path rootDir) throws IOException { + Path sharedDir = rootDir.resolve(SHARED_HEADER); + Files.createDirectories(sharedDir); + + return sharedDir; + } + + private static Path createContextDir(Path rootDir, PainlessContextInfo info) throws IOException { + Path contextDir = rootDir.resolve(getContextHeader(info)); + Files.createDirectories(contextDir); + + return contextDir; + } + + private static void printAutomatedMessage(PrintStream stream) { + stream.println("// This file is auto-generated. Do not edit."); + stream.println(); + } + + private static void printSharedIndexPage( + Path sharedDir, Map javaNamesToDisplayNames, List classInfos) throws IOException { + + Path sharedIndexPath = sharedDir.resolve("index.asciidoc"); + + try (PrintStream sharedIndexStream = new PrintStream( + Files.newOutputStream(sharedIndexPath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), + false, StandardCharsets.UTF_8.name())) { + + printAutomatedMessage(sharedIndexStream); + + sharedIndexStream.println("[[" + SHARED_HEADER + "]]"); + sharedIndexStream.println("=== " + SHARED_NAME + " API"); + sharedIndexStream.println(); + sharedIndexStream.println("The following API is available in all contexts."); + + printIndex(sharedIndexStream, SHARED_HEADER, javaNamesToDisplayNames, Collections.emptySet(), classInfos); + } + } + + private static void printContextIndexPage(Path contextDir, Map javaNamesToDisplayNames, + Set excludes, PainlessContextInfo contextInfo, List classInfos) + throws IOException { + + Path contextIndexPath = contextDir.resolve("index.asciidoc"); + + try (PrintStream contextIndexStream = new PrintStream( + Files.newOutputStream(contextIndexPath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), + false, StandardCharsets.UTF_8.name())) { + + printAutomatedMessage(contextIndexStream); + + contextIndexStream.println("[[" + getContextHeader(contextInfo) + "]]"); + contextIndexStream.println("=== " + getContextName(contextInfo) + " API"); + contextIndexStream.println(); + contextIndexStream.println("The following specialized API is available in the " + getContextName(contextInfo) + " context."); + contextIndexStream.println(); + contextIndexStream.println( + "* See the <<" + SHARED_HEADER + ", " + SHARED_NAME + " API>> for further API available in all contexts."); + + printIndex(contextIndexStream, getContextHeader(contextInfo), javaNamesToDisplayNames, excludes, classInfos); + } + } + + private static void printIndex(PrintStream indexStream, String contextHeader, Map javaNamesToDisplayNames, + Set excludes, List classInfos) { + + String currentPackageName = null; + + for (PainlessContextClassInfo classInfo : classInfos) { + if (excludes.contains(classInfo)) { + continue; + } + + String classPackageName = classInfo.getName().substring(0, classInfo.getName().lastIndexOf('.')); + + if (classPackageName.equals(currentPackageName) == false) { + currentPackageName = classPackageName; + + indexStream.println(); + indexStream.println("==== " + currentPackageName); + indexStream.println("<<" + getPackageHeader(contextHeader, currentPackageName) + ", " + + "Expand details for " + currentPackageName + ">>"); + indexStream.println(); + } + + String className = getType(javaNamesToDisplayNames, classInfo.getName()); + indexStream.println("* <<" + getClassHeader(contextHeader, className) + ", " + className + ">>"); + } + + indexStream.println(); + indexStream.println("include::packages.asciidoc[]"); + indexStream.println(); + } + + private static void printSharedPackagesPages( + Path sharedDir, Map javaNamesToDisplayNames, List classInfos) throws IOException { + + Path sharedClassesPath = sharedDir.resolve("packages.asciidoc"); + + try (PrintStream sharedPackagesStream = new PrintStream( + Files.newOutputStream(sharedClassesPath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), + false, StandardCharsets.UTF_8.name())) { + + printAutomatedMessage(sharedPackagesStream); + printPackages(sharedPackagesStream, SHARED_NAME, SHARED_HEADER, javaNamesToDisplayNames, Collections.emptySet(), classInfos); + } + } + + private static void printContextPackagesPages(Path contextDir, Map javaNamesToDisplayNames, + Set excludes, PainlessContextInfo contextInfo, List classInfos) + throws IOException { + + Path contextPackagesPath = contextDir.resolve("packages.asciidoc"); + + try (PrintStream contextPackagesStream = new PrintStream( + Files.newOutputStream(contextPackagesPath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), + false, StandardCharsets.UTF_8.name())) { + + printAutomatedMessage(contextPackagesStream); + printPackages(contextPackagesStream, + getContextName(contextInfo), getContextHeader(contextInfo), javaNamesToDisplayNames, excludes, classInfos); + } + } + + private static void printPackages(PrintStream packagesStream, String contextName, String contextHeader, + Map javaNamesToDisplayNames, Set excludes, List classInfos) + { + + String currentPackageName = null; + + for (PainlessContextClassInfo classInfo : classInfos) { + if (excludes.contains(classInfo)) { + continue; + } + + String classPackageName = classInfo.getName().substring(0, classInfo.getName().lastIndexOf('.')); + + if (classPackageName.equals(currentPackageName) == false) { + currentPackageName = classPackageName; + + packagesStream.println(); + packagesStream.println("[role=\"exclude\",id=\"" + getPackageHeader(contextHeader, currentPackageName) + "\"]"); + packagesStream.println("=== " + contextName + " API for package " + currentPackageName); + packagesStream.println( + "See the <<" + contextHeader + ", " + contextName + " API>> for a high-level overview of all packages."); + } + + String className = getType(javaNamesToDisplayNames, classInfo.getName()); + packagesStream.println(); + packagesStream.println("[[" + getClassHeader(contextHeader, className) + "]]"); + packagesStream.println("==== " + className + ""); + + for (PainlessContextFieldInfo fieldInfo : classInfo.getStaticFields()) { + printField(packagesStream, javaNamesToDisplayNames, true, fieldInfo); + } + + for (PainlessContextMethodInfo methodInfo : classInfo.getStaticMethods()) { + printMethod(packagesStream, javaNamesToDisplayNames, true, methodInfo); + } + + for (PainlessContextFieldInfo fieldInfo : classInfo.getFields()) { + printField(packagesStream, javaNamesToDisplayNames, false, fieldInfo); + } + + for (PainlessContextConstructorInfo constructorInfo : classInfo.getConstructors()) { + printConstructor(packagesStream, javaNamesToDisplayNames, className, constructorInfo); + } + + for (PainlessContextMethodInfo methodInfo : classInfo.getMethods()) { + printMethod(packagesStream, javaNamesToDisplayNames, false, methodInfo); + } + + packagesStream.println(); + } + + packagesStream.println(); + } + + private static void printRootIndexPage(Path rootDir, + List contextInfos, Set isSpecialized) throws IOException { + Path rootIndexPath = rootDir.resolve("index.asciidoc"); + + try (PrintStream rootIndexStream = new PrintStream( + Files.newOutputStream(rootIndexPath, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE), + false, StandardCharsets.UTF_8.name())) { + + printAutomatedMessage(rootIndexStream); + + rootIndexStream.println("[cols=\"<3,^3,^3\"]"); + rootIndexStream.println("|===="); + + for (PainlessContextInfo contextInfo : contextInfos) { + String contextName = getContextName(contextInfo); + String contextHeader = getContextHeader(contextInfo); + + rootIndexStream.print("|" + contextName + " "); + rootIndexStream.print("| <<" + SHARED_HEADER + ", " + SHARED_NAME + " API>> "); + + if (isSpecialized.contains(contextInfo)) { + rootIndexStream.println("| <<" + contextHeader + ", Specialized API>>"); + } else { + rootIndexStream.println("| "); + } + } + + rootIndexStream.println("|===="); + rootIndexStream.println(); + + rootIndexStream.println("include::" + SHARED_HEADER + "/index.asciidoc[]"); + + for (PainlessContextInfo contextInfo : contextInfos) { + if (isSpecialized.contains(contextInfo)) { + rootIndexStream.println("include::" + getContextHeader(contextInfo) + "/index.asciidoc[]"); + } + } + } + } + + private static void printConstructor( + PrintStream stream, Map javaNamesToDisplayNames, + String className, PainlessContextConstructorInfo constructorInfo) { + + stream.print("* "); + + if (constructorInfo.getDeclaring().startsWith("java.")) { + stream.print(getConstructorJavaDocLink(constructorInfo) + "[" + className + "]"); + } else { + stream.print(className); + } + + stream.print("("); + + for (int parameterIndex = 0; + parameterIndex < constructorInfo.getParameters().size(); + ++parameterIndex) { + + stream.print(getType(javaNamesToDisplayNames, constructorInfo.getParameters().get(parameterIndex))); + + if (parameterIndex + 1 < constructorInfo.getParameters().size()) { + stream.print(", "); + } + } + + stream.println(")"); + } + + private static void printMethod( + PrintStream stream, Map javaNamesToDisplayNames, + boolean isStatic, PainlessContextMethodInfo methodInfo) { + + stream.print("* " + (isStatic ? "static " : "")); + stream.print(getType(javaNamesToDisplayNames, methodInfo.getRtn()) + " "); + + if (methodInfo.getDeclaring().startsWith("java.")) { + stream.print(getMethodJavaDocLink(methodInfo) + "[" + methodInfo.getName() + "]"); + } else { + stream.print(methodInfo.getName()); + } + + stream.print("("); + + for (int parameterIndex = 0; + parameterIndex < methodInfo.getParameters().size(); + ++parameterIndex) { + + stream.print(getType(javaNamesToDisplayNames, methodInfo.getParameters().get(parameterIndex))); + + if (parameterIndex + 1 < methodInfo.getParameters().size()) { + stream.print(", "); + } + } + + stream.println(")"); + } + + private static void printField( + PrintStream stream, Map javaNamesToDisplayNames, + boolean isStatic, PainlessContextFieldInfo fieldInfo) { + + stream.print("* " + (isStatic ? "static " : "")); + stream.print(getType(javaNamesToDisplayNames, fieldInfo.getType()) + " "); + + if (fieldInfo.getDeclaring().startsWith("java.")) { + stream.println(getFieldJavaDocLink(fieldInfo) + "[" + fieldInfo.getName() + "]"); + } else { + stream.println(fieldInfo.getName()); + } + } + + private static String getType(Map javaNamesToDisplayNames, String javaType) { + int arrayDimensions = 0; + + while (javaType.charAt(arrayDimensions) == '[') { + ++arrayDimensions; + } + + if (arrayDimensions > 0) { + if (javaType.charAt(javaType.length() - 1) == ';') { + javaType = javaType.substring(arrayDimensions + 1, javaType.length() - 1); + } else { + javaType = javaType.substring(arrayDimensions); + } + } + + if ("Z".equals(javaType) || "boolean".equals(javaType)) { + javaType = "boolean"; + } else if ("V".equals(javaType) || "void".equals(javaType)) { + javaType = "void"; + } else if ("B".equals(javaType) || "byte".equals(javaType)) { + javaType = "byte"; + } else if ("S".equals(javaType) || "short".equals(javaType)) { + javaType = "short"; + } else if ("C".equals(javaType) || "char".equals(javaType)) { + javaType = "char"; + } else if ("I".equals(javaType) || "int".equals(javaType)) { + javaType = "int"; + } else if ("J".equals(javaType) || "long".equals(javaType)) { + javaType = "long"; + } else if ("F".equals(javaType) || "float".equals(javaType)) { + javaType = "float"; + } else if ("D".equals(javaType) || "double".equals(javaType)) { + javaType = "double"; + } else if ("org.elasticsearch.painless.lookup.def".equals(javaType)) { + javaType = "def"; + } else { + javaType = javaNamesToDisplayNames.get(javaType); + } + + while (arrayDimensions-- > 0) { + javaType += "[]"; + } + + return javaType; + } + + private static String getFieldJavaDocLink(PainlessContextFieldInfo fieldInfo) { + return "{java11-javadoc}/java.base/" + fieldInfo.getDeclaring().replace('.', '/') + ".html#" + fieldInfo.getName(); + } + + private static String getConstructorJavaDocLink(PainlessContextConstructorInfo constructorInfo) { + StringBuilder javaDocLink = new StringBuilder(); + + javaDocLink.append("{java11-javadoc}/java.base/"); + javaDocLink.append(constructorInfo.getDeclaring().replace('.', '/')); + javaDocLink.append(".html#("); + + for (int parameterIndex = 0; + parameterIndex < constructorInfo.getParameters().size(); + ++parameterIndex) { + + javaDocLink.append(getLinkType(constructorInfo.getParameters().get(parameterIndex))); + + if (parameterIndex + 1 < constructorInfo.getParameters().size()) { + javaDocLink.append(","); + } + } + + javaDocLink.append(")"); + + return javaDocLink.toString(); + } + + private static String getMethodJavaDocLink(PainlessContextMethodInfo methodInfo) { + StringBuilder javaDocLink = new StringBuilder(); + + javaDocLink.append("{java11-javadoc}/java.base/"); + javaDocLink.append(methodInfo.getDeclaring().replace('.', '/')); + javaDocLink.append(".html#"); + javaDocLink.append(methodInfo.getName()); + javaDocLink.append("("); + + for (int parameterIndex = 0; + parameterIndex < methodInfo.getParameters().size(); + ++parameterIndex) { + + javaDocLink.append(getLinkType(methodInfo.getParameters().get(parameterIndex))); + + if (parameterIndex + 1 < methodInfo.getParameters().size()) { + javaDocLink.append(","); + } + } + + javaDocLink.append(")"); + + return javaDocLink.toString(); + } + + private static String getLinkType(String javaType) { + int arrayDimensions = 0; + + while (javaType.charAt(arrayDimensions) == '[') { + ++arrayDimensions; + } + + if (arrayDimensions > 0) { + if (javaType.charAt(javaType.length() - 1) == ';') { + javaType = javaType.substring(arrayDimensions + 1, javaType.length() - 1); + } else { + javaType = javaType.substring(arrayDimensions); + } + } + + if ("Z".equals(javaType) || "boolean".equals(javaType)) { + javaType = "boolean"; + } else if ("V".equals(javaType) || "void".equals(javaType)) { + javaType = "void"; + } else if ("B".equals(javaType) || "byte".equals(javaType)) { + javaType = "byte"; + } else if ("S".equals(javaType) || "short".equals(javaType)) { + javaType = "short"; + } else if ("C".equals(javaType) || "char".equals(javaType)) { + javaType = "char"; + } else if ("I".equals(javaType) || "int".equals(javaType)) { + javaType = "int"; + } else if ("J".equals(javaType) || "long".equals(javaType)) { + javaType = "long"; + } else if ("F".equals(javaType) || "float".equals(javaType)) { + javaType = "float"; + } else if ("D".equals(javaType) || "double".equals(javaType)) { + javaType = "double"; + } else if ("org.elasticsearch.painless.lookup.def".equals(javaType)) { + javaType = "java.lang.Object"; + } + + while (arrayDimensions-- > 0) { + javaType += "%5B%5D"; + } + + return javaType; + } + + private static String getContextHeader(PainlessContextInfo contextInfo) { + return "painless-api-reference-" + contextInfo.getName().replace(" ", "-").replace("_", "-"); + } + + private static String getPackageHeader(String contextHeader, String packageName) { + return contextHeader + "-" + packageName.replace('.', '-'); + } + + private static String getClassHeader(String contextHeader, String className) { + return contextHeader + "-" + className.replace('.', '-'); + } + + private static String getContextName(PainlessContextInfo contextInfo) { + String[] split = contextInfo.getName().split("[_-]"); + StringBuilder contextNameBuilder = new StringBuilder(); + + for (String part : split) { + contextNameBuilder.append(Character.toUpperCase(part.charAt(0))); + contextNameBuilder.append(part.substring(1)); + contextNameBuilder.append(' '); + } + + return contextNameBuilder.substring(0, contextNameBuilder.length() - 1); + } + + private static List sortClassInfos( + List classInfos, Set excludes) { + classInfos = new ArrayList<>(classInfos); + classInfos.removeIf(v -> + "void".equals(v.getName()) || "boolean".equals(v.getName()) || "byte".equals(v.getName()) || + "short".equals(v.getName()) || "char".equals(v.getName()) || "int".equals(v.getName()) || + "long".equals(v.getName()) || "float".equals(v.getName()) || "double".equals(v.getName()) || + "org.elasticsearch.painless.lookup.def".equals(v.getName()) || + isInternalClass(v.getName()) || excludes.contains(v) + ); + + classInfos.sort((c1, c2) -> { + String n1 = c1.getName(); + String n2 = c2.getName(); + boolean i1 = c1.isImported(); + boolean i2 = c2.isImported(); + + String p1 = n1.substring(0, n1.lastIndexOf('.')); + String p2 = n2.substring(0, n2.lastIndexOf('.')); + + int compare = p1.compareTo(p2); + + if (compare == 0) { + if (i1 && i2) { + compare = n1.substring(n1.lastIndexOf('.') + 1).compareTo(n2.substring(n2.lastIndexOf('.') + 1)); + } else if (i1 == false && i2 == false) { + compare = n1.compareTo(n2); + } else { + compare = Boolean.compare(i1, i2) * -1; + } + } + + return compare; + }); + + return classInfos; + } + + private static Map getDisplayNames(List classInfos) { + Map javaNamesToDisplayNames = new HashMap<>(); + + for (PainlessContextClassInfo classInfo : classInfos) { + String className = classInfo.getName(); + + if (classInfo.isImported()) { + javaNamesToDisplayNames.put(className, + className.substring(className.lastIndexOf('.') + 1).replace('$', '.')); + } else { + javaNamesToDisplayNames.put(className, className.replace('$', '.')); + } + } + + return javaNamesToDisplayNames; + } + + private static boolean isInternalClass(String javaName) { + return javaName.equals("org.elasticsearch.script.ScoreScript") || + javaName.equals("org.elasticsearch.xpack.sql.expression.function.scalar.whitelist.InternalSqlScriptUtils") || + javaName.equals("org.elasticsearch.xpack.sql.expression.literal.IntervalDayTime") || + javaName.equals("org.elasticsearch.xpack.sql.expression.literal.IntervalYearMonth"); + } + + private ContextDocGenerator() { + + } +} diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassBindingInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassBindingInfo.java index 246c12f616a6d..33576dfb7341c 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassBindingInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassBindingInfo.java @@ -144,4 +144,35 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(declaring, name, rtn, readOnly, parameters); } + + @Override + public String toString() { + return "PainlessContextClassBindingInfo{" + + "declaring='" + declaring + '\'' + + ", name='" + name + '\'' + + ", rtn='" + rtn + '\'' + + ", readOnly=" + readOnly + + ", parameters=" + parameters + + '}'; + } + + public String getDeclaring() { + return declaring; + } + + public String getName() { + return name; + } + + public String getRtn() { + return rtn; + } + + public int getReadOnly() { + return readOnly; + } + + public List getParameters() { + return parameters; + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassInfo.java index 7cb3c33699603..c76fa1383b375 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextClassInfo.java @@ -183,4 +183,45 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(name, imported, constructors, staticMethods, methods, staticFields, fields); } + + @Override + public String toString() { + return "PainlessContextClassInfo{" + + "name='" + name + '\'' + + ", imported=" + imported + + ", constructors=" + constructors + + ", staticMethods=" + staticMethods + + ", methods=" + methods + + ", staticFields=" + staticFields + + ", fields=" + fields + + '}'; + } + + public String getName() { + return name; + } + + public boolean isImported() { + return imported; + } + + public List getConstructors() { + return constructors; + } + + public List getStaticMethods() { + return staticMethods; + } + + public List getMethods() { + return methods; + } + + public List getStaticFields() { + return staticFields; + } + + public List getFields() { + return fields; + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextConstructorInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextConstructorInfo.java index 13a1b2597e9dd..d5683bd621f1d 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextConstructorInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextConstructorInfo.java @@ -114,4 +114,20 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(declaring, parameters); } + + @Override + public String toString() { + return "PainlessContextConstructorInfo{" + + "declaring='" + declaring + '\'' + + ", parameters=" + parameters + + '}'; + } + + public String getDeclaring() { + return declaring; + } + + public List getParameters() { + return parameters; + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextFieldInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextFieldInfo.java index c87809e88342d..fbfac1e492ba8 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextFieldInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextFieldInfo.java @@ -119,4 +119,25 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(declaring, name, type); } + + @Override + public String toString() { + return "PainlessContextFieldInfo{" + + "declaring='" + declaring + '\'' + + ", name='" + name + '\'' + + ", type='" + type + '\'' + + '}'; + } + + public String getDeclaring() { + return declaring; + } + + public String getName() { + return name; + } + + public String getType() { + return type; + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInfo.java index 30fbf2581fe03..540c79a6d0309 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInfo.java @@ -184,4 +184,35 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(name, classes, importedMethods, classBindings, instanceBindings); } + + @Override + public String toString() { + return "PainlessContextInfo{" + + "name='" + name + '\'' + + ", classes=" + classes + + ", importedMethods=" + importedMethods + + ", classBindings=" + classBindings + + ", instanceBindings=" + instanceBindings + + '}'; + } + + public String getName() { + return name; + } + + public List getClasses() { + return classes; + } + + public List getImportedMethods() { + return importedMethods; + } + + public List getClassBindings() { + return classBindings; + } + + public List getInstanceBindings() { + return instanceBindings; + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInstanceBindingInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInstanceBindingInfo.java index 47a7834ff0fff..eb57517def369 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInstanceBindingInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextInstanceBindingInfo.java @@ -135,4 +135,30 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(declaring, name, rtn, parameters); } + + @Override + public String toString() { + return "PainlessContextInstanceBindingInfo{" + + "declaring='" + declaring + '\'' + + ", name='" + name + '\'' + + ", rtn='" + rtn + '\'' + + ", parameters=" + parameters + + '}'; + } + + public String getDeclaring() { + return declaring; + } + + public String getName() { + return name; + } + + public String getRtn() { + return rtn; + } + + public List getParameters() { + return parameters; + } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextMethodInfo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextMethodInfo.java index 4f814df7df4fe..342844b6d412d 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextMethodInfo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextMethodInfo.java @@ -134,4 +134,30 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(declaring, name, rtn, parameters); } + + @Override + public String toString() { + return "PainlessContextMethodInfo{" + + "declaring='" + declaring + '\'' + + ", name='" + name + '\'' + + ", rtn='" + rtn + '\'' + + ", parameters=" + parameters + + '}'; + } + + public String getDeclaring() { + return declaring; + } + + public String getName() { + return name; + } + + public String getRtn() { + return rtn; + } + + public List getParameters() { + return parameters; + } } From e77dfbc6987fb3293bc06e49cda908933f780cd4 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 23 Apr 2019 12:10:05 -0400 Subject: [PATCH 159/260] Adjust BWC version on aliases version upon backport This commit adjusts the BWC on the introduction of aliases version to index metadata after that functionality was backported to the 7.x branch, which is currently targeting 7.1.0. --- .../cluster/metadata/IndexMetaData.java | 14 +++++++------- .../metadata/MetaDataIndexAliasesServiceTests.java | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index f3c0af84a06c9..661cbb1bf8180 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -711,10 +711,10 @@ private static class IndexMetaDataDiff implements Diff { version = in.readLong(); mappingVersion = in.readVLong(); settingsVersion = in.readVLong(); - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { aliasesVersion = in.readVLong(); } else { - aliasesVersion = 0; + aliasesVersion = 1; } state = State.fromId(in.readByte()); settings = Settings.readSettingsFromStream(in); @@ -738,7 +738,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeVLong(mappingVersion); out.writeVLong(settingsVersion); - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { out.writeVLong(aliasesVersion); } out.writeByte(state.id); @@ -776,7 +776,7 @@ public static IndexMetaData readFrom(StreamInput in) throws IOException { builder.version(in.readLong()); builder.mappingVersion(in.readVLong()); builder.settingsVersion(in.readVLong()); - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { builder.aliasesVersion(in.readVLong()); } builder.setRoutingNumShards(in.readInt()); @@ -818,7 +818,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeVLong(mappingVersion); out.writeVLong(settingsVersion); - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { out.writeVLong(aliasesVersion); } out.writeInt(routingNumShards); @@ -1414,8 +1414,8 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_6_5_0)) { assert settingsVersion : "settings version should be present for indices created on or after 6.5.0"; } - if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_8_0_0)) { - assert aliasesVersion : "aliases version should be present for indices created on or after 8.0.0"; + if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_7_1_0)) { + assert aliasesVersion : "aliases version should be present for indices created on or after 7.1.0"; } return builder.build(); } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java index e07865f308ef5..7f7f26bcfbcbb 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexAliasesServiceTests.java @@ -102,14 +102,14 @@ public void testAddAndRemove() { public void testMultipleIndices() { final var length = randomIntBetween(2, 8); final var indices = new HashSet(length); - var before = ClusterState.builder(ClusterName.DEFAULT).build(); + ClusterState before = ClusterState.builder(ClusterName.DEFAULT).build(); final var addActions = new ArrayList(length); for (int i = 0; i < length; i++) { final String index = randomValueOtherThanMany(v -> indices.add(v) == false, () -> randomAlphaOfLength(8)); before = createIndex(before, index); addActions.add(new AliasAction.Add(index, "alias-" + index, null, null, null, null)); } - final var afterAddingAliasesToAll = service.innerExecute(before, addActions); + final ClusterState afterAddingAliasesToAll = service.innerExecute(before, addActions); assertAliasesVersionIncreased(indices.toArray(new String[0]), before, afterAddingAliasesToAll); // now add some aliases randomly @@ -121,7 +121,7 @@ public void testMultipleIndices() { randomIndices.add(index); } } - final var afterAddingRandomAliases = service.innerExecute(afterAddingAliasesToAll, randomAddActions); + final ClusterState afterAddingRandomAliases = service.innerExecute(afterAddingAliasesToAll, randomAddActions); assertAliasesVersionIncreased(randomIndices.toArray(new String[0]), afterAddingAliasesToAll, afterAddingRandomAliases); assertAliasesVersionUnchanged( Sets.difference(indices, randomIndices).toArray(new String[0]), From 4900c0dd6b82b47c20c04830c8325a58b1d7f019 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 23 Apr 2019 12:41:22 -0400 Subject: [PATCH 160/260] [DOCS] Fix broken link to elasticsearch-php security page --- x-pack/docs/en/security/ccs-clients-integrations/http.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/docs/en/security/ccs-clients-integrations/http.asciidoc b/x-pack/docs/en/security/ccs-clients-integrations/http.asciidoc index ca22ceeebbe22..aef7d093aefcd 100644 --- a/x-pack/docs/en/security/ccs-clients-integrations/http.asciidoc +++ b/x-pack/docs/en/security/ccs-clients-integrations/http.asciidoc @@ -54,7 +54,7 @@ specific clients, refer to https://github.com/elasticsearch/elasticsearch-ruby/tree/master/elasticsearch-transport#authentication[Ruby], http://elasticsearch-py.readthedocs.org/en/master/#ssl-and-authentication[Python], https://metacpan.org/pod/Search::Elasticsearch::Cxn::HTTPTiny#CONFIGURATION[Perl], -http://www.elastic.co/guide/en/elasticsearch/client/php-api/current/_security.html[PHP], +http://www.elastic.co/guide/en/elasticsearch/client/php-api/current/security.html[PHP], http://nest.azurewebsites.net/elasticsearch-net/security.html[.NET], http://www.elastic.co/guide/en/elasticsearch/client/javascript-api/current/auth-reference.html[JavaScript] From 378d74be00466ae7b2c06dbdd28adb9edf2db5b6 Mon Sep 17 00:00:00 2001 From: Guilherme Ferreira Date: Tue, 23 Apr 2019 19:12:41 +0200 Subject: [PATCH 161/260] [Docs] Correct default stop list constant (#41342) --- docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc | 2 +- docs/reference/analysis/analyzers/pattern-analyzer.asciidoc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc b/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc index cc82d2eb8179f..4d053884a4b6e 100644 --- a/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc @@ -68,7 +68,7 @@ The `fingerprint` analyzer accepts the following parameters: `stopwords`:: A pre-defined stop words list like `_english_` or an array containing a - list of stop words. Defaults to `\_none_`. + list of stop words. Defaults to `_none_`. `stopwords_path`:: diff --git a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc index 027f37280a67d..22fa534288ddd 100644 --- a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc @@ -159,7 +159,7 @@ The `pattern` analyzer accepts the following parameters: `stopwords`:: A pre-defined stop words list like `_english_` or an array containing a - list of stop words. Defaults to `\_none_`. + list of stop words. Defaults to `_none_`. `stopwords_path`:: From bd5877f659e7d51f091d0ca701b64d38b9206ded Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Tue, 23 Apr 2019 19:43:15 -0600 Subject: [PATCH 162/260] SSLDriver can transition to CLOSED in handshake (#41458) TLS 1.3 changes to the SSLEngine introduced a scenario where a UNWRAP call during a handshake can consume a close notify alerty without throwing an exception. This means that we continue down a codepath where we assert that we are still in handshaking mode. Transitioning to closed from handshaking is a valid scenario. This commit removes this assertion. --- .../xpack/security/transport/nio/SSLDriver.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java index 7314eadf9ec50..93978bcc6a359 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java @@ -439,8 +439,10 @@ private void maybeFinishHandshake() { // If the engine is partially closed, immediate transition to close mode. if (currentMode.isHandshake()) { currentMode = new CloseMode(true); - } else { - String message = "Expected to be in handshaking mode. Instead in non-handshaking mode: " + currentMode; + } else if (currentMode.isApplication()) { + // It is possible to be in CLOSED mode if the prior UNWRAP call returned CLOSE_NOTIFY. + // However we should not be in application mode at this point. + String message = "Expected to be in handshaking/closed mode. Instead in application mode."; throw new AssertionError(message); } } else if (hasFlushPending() == false) { From 24e3145fd002064e79f54aa644212999640d3565 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 23 Apr 2019 21:44:25 -0400 Subject: [PATCH 163/260] Recovery should not indefinitely retry on mapping error (#41099) A stuck peer recovery in #40913 reveals that we indefinitely retry on new cluster states if indexing translog operations hits a mapper exception. We should not wait and retry if the mapping on the target is as recent as the mapping that the primary used to index the replaying operations. Relates #40913 --- .../elasticsearch/index/shard/IndexShard.java | 2 +- .../recovery/PeerRecoveryTargetService.java | 12 +++- .../recovery/RecoverySourceHandler.java | 8 ++- .../indices/recovery/RecoveryTarget.java | 3 +- .../recovery/RecoveryTargetHandler.java | 6 ++ .../RecoveryTranslogOperationsRequest.java | 48 ++++++++++---- .../recovery/RemoteRecoveryTargetHandler.java | 4 +- .../RecoveryDuringReplicationTests.java | 6 +- .../index/shard/IndexShardTests.java | 6 ++ .../indices/recovery/IndexRecoveryIT.java | 65 ++++++++++++++++++- .../recovery/RecoverySourceHandlerTests.java | 14 ++-- .../indices/recovery/AsyncRecoveryTarget.java | 6 +- 12 files changed, 149 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index c5dc2d024aaa8..c0b482b176853 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -3148,7 +3148,7 @@ public long getMaxSeqNoOfUpdatesOrDeletes() { * which is at least the value of the max_seq_no_of_updates marker on the primary after that operation was executed on the primary. * * @see #acquireReplicaOperationPermit(long, long, long, ActionListener, String, Object) - * @see RecoveryTarget#indexTranslogOperations(List, int, long, long, RetentionLeases, ActionListener) + * @see RecoveryTarget#indexTranslogOperations(List, int, long, long, RetentionLeases, long, ActionListener) */ public void advanceMaxSeqNoOfUpdatesOrDeletes(long seqNo) { assert seqNo != UNASSIGNED_SEQ_NO diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index e3125ce5be97a..1ba854fdb2b13 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -33,6 +33,7 @@ import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; @@ -119,8 +120,8 @@ public PeerRecoveryTargetService(ThreadPool threadPool, TransportService transpo RecoveryCleanFilesRequest::new, new CleanFilesRequestHandler()); transportService.registerRequestHandler(Actions.PREPARE_TRANSLOG, ThreadPool.Names.GENERIC, RecoveryPrepareForTranslogOperationsRequest::new, new PrepareForTranslogOperationsRequestHandler()); - transportService.registerRequestHandler(Actions.TRANSLOG_OPS, RecoveryTranslogOperationsRequest::new, ThreadPool.Names.GENERIC, - new TranslogOperationsRequestHandler()); + transportService.registerRequestHandler(Actions.TRANSLOG_OPS, ThreadPool.Names.GENERIC, RecoveryTranslogOperationsRequest::new, + new TranslogOperationsRequestHandler()); transportService.registerRequestHandler(Actions.FINALIZE, RecoveryFinalizeRecoveryRequest::new, ThreadPool.Names.GENERIC, new FinalizeRecoveryRequestHandler()); transportService.registerRequestHandler( @@ -501,16 +502,21 @@ public void onTimeout(TimeValue timeout) { } }); }; + final IndexMetaData indexMetaData = clusterService.state().metaData().index(request.shardId().getIndex()); + final long mappingVersionOnTarget = indexMetaData != null ? indexMetaData.getMappingVersion() : 0L; recoveryTarget.indexTranslogOperations( request.operations(), request.totalTranslogOps(), request.maxSeenAutoIdTimestampOnPrimary(), request.maxSeqNoOfUpdatesOrDeletesOnPrimary(), request.retentionLeases(), + request.mappingVersionOnPrimary(), ActionListener.wrap( checkpoint -> listener.onResponse(new RecoveryTranslogOperationsResponse(checkpoint)), e -> { - if (e instanceof MapperException) { + // do not retry if the mapping on replica is at least as recent as the mapping + // that the primary used to index the operations in the request. + if (mappingVersionOnTarget < request.mappingVersionOnPrimary() && e instanceof MapperException) { retryOnMappingException.accept(e); } else { listener.onFailure(e); 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 22f64a9e5e3ca..aad460b821e62 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -219,8 +219,9 @@ public void recoverToTarget(ActionListener listener) { final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp(); final long maxSeqNoOfUpdatesOrDeletes = shard.getMaxSeqNoOfUpdatesOrDeletes(); final RetentionLeases retentionLeases = shard.getRetentionLeases(); + final long mappingVersionOnPrimary = shard.indexSettings().getIndexMetaData().getMappingVersion(); phase2(startingSeqNo, endingSeqNo, phase2Snapshot, maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, - retentionLeases, sendSnapshotStep); + retentionLeases, mappingVersionOnPrimary, sendSnapshotStep); sendSnapshotStep.whenComplete( r -> IOUtils.close(phase2Snapshot), e -> { @@ -510,6 +511,7 @@ void phase2( final long maxSeenAutoIdTimestamp, final long maxSeqNoOfUpdatesOrDeletes, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener) throws IOException { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); @@ -571,6 +573,7 @@ void phase2( maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, + mappingVersion, batchedListener); } @@ -582,6 +585,7 @@ private void sendBatch( final long maxSeenAutoIdTimestamp, final long maxSeqNoOfUpdatesOrDeletes, final RetentionLeases retentionLeases, + final long mappingVersionOnPrimary, final ActionListener listener) throws IOException { assert ThreadPool.assertCurrentMethodIsNotCalledRecursively(); final List operations = nextBatch.get(); @@ -594,6 +598,7 @@ private void sendBatch( maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, + mappingVersionOnPrimary, ActionListener.wrap( newCheckpoint -> { sendBatch( @@ -604,6 +609,7 @@ private void sendBatch( maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, + mappingVersionOnPrimary, listener); }, listener::onFailure diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index e15750b2feb06..a27ac8a352a32 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -320,6 +320,7 @@ public void indexTranslogOperations( final long maxSeenAutoIdTimestampOnPrimary, final long maxSeqNoOfDeletesOrUpdatesOnPrimary, final RetentionLeases retentionLeases, + final long mappingVersionOnPrimary, final ActionListener listener) { ActionListener.completeWith(listener, () -> { final RecoveryState.Translog translog = state().getTranslog(); @@ -351,7 +352,7 @@ public void indexTranslogOperations( throw new MapperException("mapping updates are not allowed [" + operation + "]"); } if (result.getFailure() != null) { - if (Assertions.ENABLED) { + if (Assertions.ENABLED && result.getFailure() instanceof MapperException == false) { throw new AssertionError("unexpected failure while replicating translog entry", result.getFailure()); } ExceptionsHelper.reThrowIfNotNull(result.getFailure()); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index a16fd4b6ab3a2..d03fe42d90146 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -65,6 +65,11 @@ public interface RecoveryTargetHandler { * the primary shard when capturing these operations. This value is at least as high as the * max_seq_no_of_updates on the primary was when any of these ops were processed on it. * @param retentionLeases the retention leases on the primary + * @param mappingVersionOnPrimary the mapping version which is at least as up to date as the mapping version that the + * primary used to index translog {@code operations} in this request. + * If the mapping version on the replica is not older this version, we should not retry on + * {@link org.elasticsearch.index.mapper.MapperException}; otherwise we should wait for a + * new mapping then retry. * @param listener a listener which will be notified with the local checkpoint on the target * after these operations are successfully indexed on the target. */ @@ -74,6 +79,7 @@ void indexTranslogOperations( long maxSeenAutoIdTimestampOnPrimary, long maxSeqNoOfUpdatesOrDeletesOnPrimary, RetentionLeases retentionLeases, + long mappingVersionOnPrimary, ActionListener listener); /** diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java index baf5dc4d99d99..a9484af5cc75a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices.recovery; +import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.seqno.RetentionLeases; @@ -31,16 +32,14 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest { - private long recoveryId; - private ShardId shardId; - private List operations; - private int totalTranslogOps = RecoveryState.Translog.UNKNOWN; - private long maxSeenAutoIdTimestampOnPrimary; - private long maxSeqNoOfUpdatesOrDeletesOnPrimary; - private RetentionLeases retentionLeases; - - public RecoveryTranslogOperationsRequest() { - } + private final long recoveryId; + private final ShardId shardId; + private final List operations; + private final int totalTranslogOps; + private final long maxSeenAutoIdTimestampOnPrimary; + private final long maxSeqNoOfUpdatesOrDeletesOnPrimary; + private final RetentionLeases retentionLeases; + private final long mappingVersionOnPrimary; RecoveryTranslogOperationsRequest( final long recoveryId, @@ -49,7 +48,8 @@ public RecoveryTranslogOperationsRequest() { final int totalTranslogOps, final long maxSeenAutoIdTimestampOnPrimary, final long maxSeqNoOfUpdatesOrDeletesOnPrimary, - final RetentionLeases retentionLeases) { + final RetentionLeases retentionLeases, + final long mappingVersionOnPrimary) { this.recoveryId = recoveryId; this.shardId = shardId; this.operations = operations; @@ -57,6 +57,7 @@ public RecoveryTranslogOperationsRequest() { this.maxSeenAutoIdTimestampOnPrimary = maxSeenAutoIdTimestampOnPrimary; this.maxSeqNoOfUpdatesOrDeletesOnPrimary = maxSeqNoOfUpdatesOrDeletesOnPrimary; this.retentionLeases = retentionLeases; + this.mappingVersionOnPrimary = mappingVersionOnPrimary; } public long recoveryId() { @@ -87,8 +88,16 @@ public RetentionLeases retentionLeases() { return retentionLeases; } - @Override - public void readFrom(StreamInput in) throws IOException { + /** + * Returns the mapping version which is at least as up to date as the mapping version that the primary used to index + * the translog operations in this request. If the mapping version on the replica is not older this version, we should not + * retry on {@link org.elasticsearch.index.mapper.MapperException}; otherwise we should wait for a new mapping then retry. + */ + long mappingVersionOnPrimary() { + return mappingVersionOnPrimary; + } + + RecoveryTranslogOperationsRequest(StreamInput in) throws IOException { super.readFrom(in); recoveryId = in.readLong(); shardId = ShardId.readShardId(in); @@ -97,6 +106,11 @@ public void readFrom(StreamInput in) throws IOException { maxSeenAutoIdTimestampOnPrimary = in.readZLong(); maxSeqNoOfUpdatesOrDeletesOnPrimary = in.readZLong(); retentionLeases = new RetentionLeases(in); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + mappingVersionOnPrimary = in.readVLong(); + } else { + mappingVersionOnPrimary = Long.MAX_VALUE; + } } @Override @@ -109,5 +123,13 @@ public void writeTo(StreamOutput out) throws IOException { out.writeZLong(maxSeenAutoIdTimestampOnPrimary); out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary); retentionLeases.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeVLong(mappingVersionOnPrimary); + } + } + + @Override + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 5deb6f6ff9d4b..230f53515754e 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -112,6 +112,7 @@ public void indexTranslogOperations( final long maxSeenAutoIdTimestampOnPrimary, final long maxSeqNoOfDeletesOrUpdatesOnPrimary, final RetentionLeases retentionLeases, + final long mappingVersionOnPrimary, final ActionListener listener) { final RecoveryTranslogOperationsRequest request = new RecoveryTranslogOperationsRequest( recoveryId, @@ -120,7 +121,8 @@ public void indexTranslogOperations( totalTranslogOps, maxSeenAutoIdTimestampOnPrimary, maxSeqNoOfDeletesOrUpdatesOnPrimary, - retentionLeases); + retentionLeases, + mappingVersionOnPrimary); transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.TRANSLOG_OPS, request, translogOpsRequestOptions, new ActionListenerResponseHandler<>(ActionListener.map(listener, r -> r.localCheckpoint), RecoveryTranslogOperationsResponse::new, ThreadPool.Names.GENERIC)); diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 7de353584235e..de6ab82892f51 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -564,6 +564,7 @@ public void indexTranslogOperations( final long maxAutoIdTimestamp, final long maxSeqNoOfUpdates, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener) { // index a doc which is not part of the snapshot, but also does not complete on replica replicaEngineFactory.latchIndexers(1); @@ -597,6 +598,7 @@ public void indexTranslogOperations( maxAutoIdTimestamp, maxSeqNoOfUpdates, retentionLeases, + mappingVersion, listener); } }); @@ -845,11 +847,13 @@ public void indexTranslogOperations( final long maxAutoIdTimestamp, final long maxSeqNoOfUpdates, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener) { if (hasBlocked() == false) { blockIfNeeded(RecoveryState.Stage.TRANSLOG); } - super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp, maxSeqNoOfUpdates, retentionLeases, listener); + super.indexTranslogOperations( + operations, totalTranslogOps, maxAutoIdTimestamp, maxSeqNoOfUpdates, retentionLeases, mappingVersion, listener); } @Override diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index c8f1777be246f..e7567ce495147 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2466,6 +2466,7 @@ public void indexTranslogOperations( final long maxSeenAutoIdTimestamp, final long maxSeqNoOfUpdatesOrDeletes, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener){ super.indexTranslogOperations( operations, @@ -2473,6 +2474,7 @@ public void indexTranslogOperations( maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, + mappingVersion, ActionListener.wrap( r -> { assertFalse(replica.isSyncNeeded()); @@ -2588,6 +2590,7 @@ public void indexTranslogOperations( final long maxAutoIdTimestamp, final long maxSeqNoOfUpdatesOrDeletes, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener){ super.indexTranslogOperations( operations, @@ -2595,6 +2598,7 @@ public void indexTranslogOperations( maxAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, + mappingVersion, ActionListener.wrap( checkpoint -> { listener.onResponse(checkpoint); @@ -2653,6 +2657,7 @@ public void indexTranslogOperations( final long maxAutoIdTimestamp, final long maxSeqNoOfUpdatesOrDeletes, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener) { super.indexTranslogOperations( operations, @@ -2660,6 +2665,7 @@ public void indexTranslogOperations( maxAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, + mappingVersion, ActionListener.wrap( r -> { assertListenerCalled.accept(replica); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 82d6c38becaec..4196472334ca9 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices.recovery; +import org.apache.lucene.analysis.TokenStream; import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; @@ -35,6 +36,7 @@ import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; @@ -46,12 +48,18 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; +import org.elasticsearch.index.analysis.AbstractTokenFilterFactory; +import org.elasticsearch.index.analysis.TokenFilterFactory; +import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.recovery.RecoveryStats; import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.indices.recovery.RecoveryState.Stage; import org.elasticsearch.node.RecoverySettingsChunkSizePlugin; +import org.elasticsearch.plugins.AnalysisPlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.test.BackgroundIndexer; @@ -78,10 +86,12 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import static java.util.Collections.singletonMap; import static org.elasticsearch.node.RecoverySettingsChunkSizePlugin.CHUNK_SIZE_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -109,7 +119,7 @@ public class IndexRecoveryIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { return Arrays.asList(MockTransportService.TestPlugin.class, MockFSIndexStore.TestPlugin.class, - RecoverySettingsChunkSizePlugin.class); + RecoverySettingsChunkSizePlugin.class, TestAnalysisPlugin.class); } @After @@ -863,4 +873,57 @@ public void testHistoryRetention() throws Exception { assertThat(recoveryStates.get(0).getIndex().totalFileCount(), is(0)); assertThat(recoveryStates.get(0).getTranslog().recoveredOperations(), greaterThan(0)); } + + public void testDoNotInfinitelyWaitForMapping() { + internalCluster().ensureAtLeastNumDataNodes(3); + createIndex("test", Settings.builder() + .put("index.analysis.analyzer.test_analyzer.type", "custom") + .put("index.analysis.analyzer.test_analyzer.tokenizer", "standard") + .putList("index.analysis.analyzer.test_analyzer.filter", "test_token_filter") + .put("index.number_of_replicas", 0).put("index.number_of_shards", 1).build()); + client().admin().indices().preparePutMapping("test") + .setType("_doc").setSource("test_field", "type=text,analyzer=test_analyzer").get(); + int numDocs = between(1, 10); + for (int i = 0; i < numDocs; i++) { + client().prepareIndex("test", "_doc", "u" + i) + .setSource(singletonMap("test_field", Integer.toString(i)), XContentType.JSON).get(); + } + Semaphore recoveryBlocked = new Semaphore(1); + for (DiscoveryNode node : clusterService().state().nodes()) { + MockTransportService transportService = (MockTransportService) internalCluster().getInstance( + TransportService.class, node.getName()); + transportService.addSendBehavior((connection, requestId, action, request, options) -> { + if (action.equals(PeerRecoverySourceService.Actions.START_RECOVERY)) { + if (recoveryBlocked.tryAcquire()) { + PluginsService pluginService = internalCluster().getInstance(PluginsService.class, node.getName()); + for (TestAnalysisPlugin plugin : pluginService.filterPlugins(TestAnalysisPlugin.class)) { + plugin.throwParsingError.set(true); + } + } + } + connection.sendRequest(requestId, action, request, options); + }); + } + client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put("index.number_of_replicas", 1)).get(); + ensureGreen("test"); + client().admin().indices().prepareRefresh("test").get(); + assertHitCount(client().prepareSearch().get(), numDocs); + } + + public static final class TestAnalysisPlugin extends Plugin implements AnalysisPlugin { + final AtomicBoolean throwParsingError = new AtomicBoolean(); + @Override + public Map> getTokenFilters() { + return singletonMap("test_token_filter", + (indexSettings, environment, name, settings) -> new AbstractTokenFilterFactory(indexSettings, name, settings) { + @Override + public TokenStream create(TokenStream tokenStream) { + if (throwParsingError.get()) { + throw new MapperParsingException("simulate mapping parsing error"); + } + return tokenStream; + } + }); + } + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 4ce402163d2b0..b63c7a2e0e8f6 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -237,7 +237,7 @@ public void testSendSnapshotSendsOps() throws IOException { RecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { @Override public void indexTranslogOperations(List operations, int totalTranslogOps, long timestamp, long msu, - RetentionLeases retentionLeases, ActionListener listener) { + RetentionLeases retentionLeases, long mappingVersion, ActionListener listener) { shippedOps.addAll(operations); checkpointOnTarget.set(randomLongBetween(checkpointOnTarget.get(), Long.MAX_VALUE)); listener.onResponse(checkpointOnTarget.get()); } @@ -246,7 +246,7 @@ public void indexTranslogOperations(List operations, int tot shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), request, fileChunkSizeInBytes, between(1, 10)); PlainActionFuture future = new PlainActionFuture<>(); handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()), - randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, future); + randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1); RecoverySourceHandler.SendSnapshotResult result = future.actionGet(); assertThat(result.totalOperations, equalTo(expectedOps)); @@ -272,7 +272,8 @@ public void testSendSnapshotStopOnError() throws Exception { RecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { @Override public void indexTranslogOperations(List operations, int totalTranslogOps, long timestamp, - long msu, RetentionLeases retentionLeases, ActionListener listener) { + long msu, RetentionLeases retentionLeases, long mappingVersion, + ActionListener listener) { if (randomBoolean()) { listener.onResponse(SequenceNumbers.NO_OPS_PERFORMED); } else { @@ -287,7 +288,7 @@ public void indexTranslogOperations(List operations, int tot final long startingSeqNo = randomLongBetween(0, ops.size() - 1L); final long endingSeqNo = randomLongBetween(startingSeqNo, ops.size() - 1L); handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(ops, Collections.emptyList()), - randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, future); + randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); if (wasFailed.get()) { assertThat(expectThrows(RuntimeException.class, () -> future.actionGet()).getMessage(), equalTo("test - failed to index")); } @@ -487,10 +488,10 @@ void prepareTargetForTranslog(boolean fileBasedRecovery, int totalTranslogOps, A @Override void phase2(long startingSeqNo, long endingSeqNo, Translog.Snapshot snapshot, long maxSeenAutoIdTimestamp, long maxSeqNoOfUpdatesOrDeletes, RetentionLeases retentionLeases, - ActionListener listener) throws IOException { + long mappingVersion, ActionListener listener) throws IOException { phase2Called.set(true); super.phase2(startingSeqNo, endingSeqNo, snapshot, - maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, listener); + maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, mappingVersion, listener); } }; @@ -706,6 +707,7 @@ public void indexTranslogOperations( final long timestamp, final long msu, final RetentionLeases retentionLeases, + final long mappingVersion, final ActionListener listener) { } diff --git a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java index 0622bce2013e6..d5a7ab8109e12 100644 --- a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java @@ -63,9 +63,9 @@ public void handoffPrimaryContext(ReplicationTracker.PrimaryContext primaryConte @Override public void indexTranslogOperations(List operations, int totalTranslogOps, long maxSeenAutoIdTimestampOnPrimary, long maxSeqNoOfDeletesOrUpdatesOnPrimary, - RetentionLeases retentionLeases, ActionListener listener) { - executor.execute(() -> target.indexTranslogOperations( - operations, totalTranslogOps, maxSeenAutoIdTimestampOnPrimary, maxSeqNoOfDeletesOrUpdatesOnPrimary, retentionLeases, listener)); + RetentionLeases retentionLeases, long mappingVersionOnPrimary, ActionListener listener) { + executor.execute(() -> target.indexTranslogOperations(operations, totalTranslogOps, maxSeenAutoIdTimestampOnPrimary, + maxSeqNoOfDeletesOrUpdatesOnPrimary, retentionLeases, mappingVersionOnPrimary, listener)); } @Override From 74ea91df4288c4cb324a900652ec0aadc127b08e Mon Sep 17 00:00:00 2001 From: Yogesh Gaikwad <902768+bizybot@users.noreply.github.com> Date: Wed, 24 Apr 2019 17:01:45 +1000 Subject: [PATCH 164/260] Remove deprecated stashWithOrigin calls and use the alternative (#40847) This commit removes the deprecated `stashWithOrigin` and modifies its usage to use the alternative. --- .../elasticsearch/xpack/core/ClientHelper.java | 15 +++------------ .../xpack/ml/datafeed/DatafeedJob.java | 11 +++++------ .../DatafeedDelayedDataDetector.java | 7 +++---- .../xpack/ml/job/persistence/JobDataDeleter.java | 3 +-- .../JobRenormalizedResultsPersister.java | 7 +++---- .../ml/job/persistence/JobResultsPersister.java | 7 +++---- .../ml/job/persistence/JobResultsProvider.java | 5 ++--- .../xpack/ml/job/persistence/StateStreamer.java | 5 ++--- .../output/AutodetectStateProcessor.java | 3 +-- .../monitoring/collector/ccr/StatsCollector.java | 3 +-- .../collector/ml/JobStatsCollector.java | 3 +-- .../security/authc/esnative/NativeUsersStore.java | 3 +-- .../support/mapper/NativeRoleMappingStore.java | 3 +-- .../authz/store/NativePrivilegeStore.java | 3 +-- .../security/authz/store/NativeRolesStore.java | 3 +-- .../xpack/watcher/WatcherFeatureSet.java | 3 +-- .../xpack/watcher/execution/ExecutionService.java | 6 +++--- 17 files changed, 33 insertions(+), 57 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java index 306a22253242f..a0074f4231876 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java @@ -54,15 +54,6 @@ public final class ClientHelper { private ClientHelper() {} - /** - * Stashes the current context and sets the origin in the current context. The original context is returned as a stored context - * @deprecated use ThreadContext.stashWithOrigin - */ - @Deprecated - public static ThreadContext.StoredContext stashWithOrigin(ThreadContext threadContext, String origin) { - return threadContext.stashWithOrigin(origin); - } - /** * Returns a client that will always set the appropriate origin and ensure the proper context is restored by listeners * @deprecated use {@link OriginSettingClient} instead @@ -79,7 +70,7 @@ public static v ThreadContext threadContext, String origin, Request request, ActionListener listener, BiConsumer> consumer) { final Supplier supplier = threadContext.newRestorableContext(false); - try (ThreadContext.StoredContext ignore = stashWithOrigin(threadContext, origin)) { + try (ThreadContext.StoredContext ignore = threadContext.stashWithOrigin(origin)) { consumer.accept(request, new ContextPreservingActionListener<>(supplier, listener)); } } @@ -94,7 +85,7 @@ RequestBuilder extends ActionRequestBuilder> void executeAsyn ActionListener listener) { final ThreadContext threadContext = client.threadPool().getThreadContext(); final Supplier supplier = threadContext.newRestorableContext(false); - try (ThreadContext.StoredContext ignore = stashWithOrigin(threadContext, origin)) { + try (ThreadContext.StoredContext ignore = threadContext.stashWithOrigin(origin)) { client.execute(action, request, new ContextPreservingActionListener<>(supplier, listener)); } } @@ -121,7 +112,7 @@ public static T executeWithHeaders(Map checkBucketEvents(long start, long end) { request.setExcludeInterim(true); request.setPageParams(new PageParams(0, (int)((end - start)/bucketSpan))); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { GetBucketsAction.Response response = client.execute(GetBucketsAction.INSTANCE, request).actionGet(); return response.getBuckets().results(); } @@ -115,7 +114,7 @@ private Map checkCurrentBucketEventCount(long start, long end) { .query(ExtractorUtils.wrapInTimeRangeQuery(datafeedQuery, timeField, start, end)); SearchRequest searchRequest = new SearchRequest(datafeedIndices).source(searchSourceBuilder); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { SearchResponse response = client.execute(SearchAction.INSTANCE, searchRequest).actionGet(); List buckets = ((Histogram)response.getAggregations().get(DATE_BUCKETS)).getBuckets(); Map hashMap = new HashMap<>(buckets.size()); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java index c96388213c8c0..9380be0c15b78 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java @@ -33,7 +33,6 @@ import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; public class JobDataDeleter { @@ -115,7 +114,7 @@ public void deleteInterimResults() { QueryBuilder qb = QueryBuilders.termQuery(Result.IS_INTERIM.getPreferredName(), true); deleteByQueryHolder.dbqRequest.setQuery(new ConstantScoreQueryBuilder(qb)); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { client.execute(DeleteByQueryAction.INSTANCE, deleteByQueryHolder.dbqRequest).get(); } catch (Exception e) { LOGGER.error("[" + jobId + "] An error occurred while deleting interim results", e); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java index c5a6a46080057..118a3913ee435 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java @@ -15,17 +15,16 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.xpack.ml.job.process.normalizer.BucketNormalizable; -import org.elasticsearch.xpack.ml.job.process.normalizer.Normalizable; import org.elasticsearch.xpack.core.ml.job.results.Bucket; import org.elasticsearch.xpack.core.ml.job.results.BucketInfluencer; +import org.elasticsearch.xpack.ml.job.process.normalizer.BucketNormalizable; +import org.elasticsearch.xpack.ml.job.process.normalizer.Normalizable; import java.io.IOException; import java.util.List; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; /** @@ -101,7 +100,7 @@ public void executeRequest() { } logger.trace("[{}] ES API CALL: bulk request with {} actions", jobId, bulkRequest.numberOfActions()); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { BulkResponse addRecordsResponse = client.bulk(bulkRequest).actionGet(); if (addRecordsResponse.hasFailures()) { logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index 8588ee8dff03a..fc5b87d4afebe 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -45,7 +45,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; /** * Persists result types, Quantiles etc to Elasticsearch
    @@ -193,7 +192,7 @@ public void executeRequest() { } logger.trace("[{}] ES API CALL: bulk request with {} actions", jobId, bulkRequest.numberOfActions()); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { BulkResponse addRecordsResponse = client.bulk(bulkRequest).actionGet(); if (addRecordsResponse.hasFailures()) { logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); @@ -292,7 +291,7 @@ public void commitResultWrites(String jobId) { logger.trace("[{}] ES API CALL: refresh index {}", jobId, indexName); RefreshRequest refreshRequest = new RefreshRequest(indexName); refreshRequest.indicesOptions(IndicesOptions.lenientExpandOpen()); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { client.admin().indices().refresh(refreshRequest).actionGet(); } } @@ -309,7 +308,7 @@ public void commitStateWrites(String jobId) { logger.trace("[{}] ES API CALL: refresh index {}", jobId, indexName); RefreshRequest refreshRequest = new RefreshRequest(indexName); refreshRequest.indicesOptions(IndicesOptions.lenientExpandOpen()); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { client.admin().indices().refresh(refreshRequest).actionGet(); } } 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 f2c53f3851556..e5d343bb304b6 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 @@ -73,12 +73,12 @@ import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.xpack.core.action.util.QueryPage; import org.elasticsearch.xpack.core.ml.MlMetaIndex; import org.elasticsearch.xpack.core.ml.action.GetBucketsAction; import org.elasticsearch.xpack.core.ml.action.GetCategoriesAction; import org.elasticsearch.xpack.core.ml.action.GetInfluencersAction; import org.elasticsearch.xpack.core.ml.action.GetRecordsAction; -import org.elasticsearch.xpack.core.action.util.QueryPage; import org.elasticsearch.xpack.core.ml.calendars.Calendar; import org.elasticsearch.xpack.core.ml.calendars.ScheduledEvent; import org.elasticsearch.xpack.core.ml.job.config.Job; @@ -130,7 +130,6 @@ import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.clientWithOrigin; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; public class JobResultsProvider { private static final Logger LOGGER = LogManager.getLogger(JobResultsProvider.class); @@ -927,7 +926,7 @@ public QueryPage modelPlot(String jobId, int from, int size) { String indexName = AnomalyDetectorsIndex.jobResultsAliasedName(jobId); LOGGER.trace("ES API CALL: search model plots from index {} from {} size {}", indexName, from, size); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { searchResponse = client.prepareSearch(indexName) .setIndicesOptions(MlIndicesUtils.addIgnoreUnavailable(SearchRequest.DEFAULT_INDICES_OPTIONS)) .setQuery(new TermsQueryBuilder(Result.RESULT_TYPE.getPreferredName(), ModelPlot.RESULT_TYPE_VALUE)) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java index da221360d57ab..3d5f3d4ea91e3 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java @@ -23,7 +23,6 @@ import java.util.Objects; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; /** * A {@code StateStreamer} fetches the various state documents and @@ -72,7 +71,7 @@ public void restoreStateToStream(String jobId, ModelSnapshot modelSnapshot, Outp LOGGER.trace("ES API CALL: get ID {} from index {}", stateDocId, indexName); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { SearchResponse stateResponse = client.prepareSearch(indexName) .setSize(1) .setQuery(QueryBuilders.idsQuery().addIds(stateDocId)).get(); @@ -98,7 +97,7 @@ public void restoreStateToStream(String jobId, ModelSnapshot modelSnapshot, Outp LOGGER.trace("ES API CALL: get ID {} from index {}", docId, indexName); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { SearchResponse stateResponse = client.prepareSearch(indexName) .setSize(1) .setQuery(QueryBuilders.idsQuery().addIds(docId)).get(); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessor.java index 6fa1393bb02f5..1a418bfb2a1c2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessor.java @@ -23,7 +23,6 @@ import java.util.List; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; /** * Reads the autodetect state and persists via a bulk request @@ -100,7 +99,7 @@ void persist(BytesReference bytes) throws IOException { bulkRequest.add(bytes, AnomalyDetectorsIndex.jobStateIndexWriteAlias(), XContentType.JSON); if (bulkRequest.numberOfActions() > 0) { LOGGER.trace("[{}] Persisting job state document", jobId); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { client.bulk(bulkRequest).actionGet(); } } diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ccr/StatsCollector.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ccr/StatsCollector.java index bdccb5604a361..6aab3114b7807 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ccr/StatsCollector.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ccr/StatsCollector.java @@ -29,7 +29,6 @@ import java.util.stream.Collectors; import static org.elasticsearch.xpack.core.ClientHelper.MONITORING_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; import static org.elasticsearch.xpack.monitoring.collector.ccr.FollowStatsMonitoringDoc.TYPE; public final class StatsCollector extends Collector { @@ -75,7 +74,7 @@ protected Collection doCollect( final MonitoringDoc.Node node, final long interval, final ClusterState clusterState) throws Exception { - try (ThreadContext.StoredContext ignore = stashWithOrigin(threadContext, MONITORING_ORIGIN)) { + try (ThreadContext.StoredContext ignore = threadContext.stashWithOrigin(MONITORING_ORIGIN)) { final long timestamp = timestamp(); final String clusterUuid = clusterUuid(clusterState); diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsCollector.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsCollector.java index 8742e0b645f8e..855780d4836ae 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsCollector.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/collector/ml/JobStatsCollector.java @@ -25,7 +25,6 @@ import java.util.stream.Collectors; import static org.elasticsearch.xpack.core.ClientHelper.MONITORING_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; /** * Collector for Machine Learning Job Stats. @@ -73,7 +72,7 @@ protected List doCollect(final MonitoringDoc.Node node, final long interval, final ClusterState clusterState) throws Exception { // fetch details about all jobs - try (ThreadContext.StoredContext ignore = stashWithOrigin(threadContext, MONITORING_ORIGIN)) { + try (ThreadContext.StoredContext ignore = threadContext.stashWithOrigin(MONITORING_ORIGIN)) { final GetJobsStatsAction.Response jobs = client.getJobsStats(new GetJobsStatsAction.Request(MetaData.ALL)) .actionGet(getCollectionTimeout()); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java index 3ae7373d2e2e3..a0c579dd881ce 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java @@ -66,7 +66,6 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; /** @@ -146,7 +145,7 @@ public void getUsers(String[] userNames, final ActionListener> query = QueryBuilders.boolQuery().filter(QueryBuilders.idsQuery().addIds(users)); } final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java index e8d874bc9d481..ab860ed058d91 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java @@ -60,7 +60,6 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.isIndexDeleted; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.isMoveFromRedToNonRed; @@ -131,7 +130,7 @@ protected void loadMappings(ActionListener> listener } final QueryBuilder query = QueryBuilders.termQuery(DOC_TYPE_FIELD, DOC_TYPE_ROLE_MAPPING); final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java index 19694bb003314..6a29f1de0e18b 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java @@ -63,7 +63,6 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; import static org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor.DOC_TYPE_VALUE; import static org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor.Fields.APPLICATION; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; @@ -129,7 +128,7 @@ public void getPrivileges(Collection applications, Collection na query = QueryBuilders.boolQuery().filter(typeQuery).filter(QueryBuilders.idsQuery().addIds(docIds)); } final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java index c0ec72277d870..289a616e254d6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java @@ -65,7 +65,6 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; import static org.elasticsearch.xpack.core.security.authz.RoleDescriptor.ROLE_TYPE; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; @@ -115,7 +114,7 @@ public void getRoleDescriptors(Set names, final ActionListener { QueryBuilder query = QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE); final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { SearchRequest request = client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherFeatureSet.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherFeatureSet.java index 5d5a30344ab12..3f02cfde7c7e7 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherFeatureSet.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/WatcherFeatureSet.java @@ -28,7 +28,6 @@ import java.util.stream.Collectors; import static org.elasticsearch.xpack.core.ClientHelper.WATCHER_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; public class WatcherFeatureSet implements XPackFeatureSet { @@ -72,7 +71,7 @@ public Map nativeCodeInfo() { public void usage(ActionListener listener) { if (enabled) { try (ThreadContext.StoredContext ignore = - stashWithOrigin(client.threadPool().getThreadContext(), WATCHER_ORIGIN)) { + client.threadPool().getThreadContext().stashWithOrigin(WATCHER_ORIGIN)) { WatcherClient watcherClient = new WatcherClient(client); WatcherStatsRequest request = new WatcherStatsRequest(); request.includeStats(true); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java index 9b730db7ec59b..4e33e15044c3e 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.watcher.execution; import com.google.common.collect.Iterables; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -72,7 +73,6 @@ import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.xpack.core.ClientHelper.WATCHER_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.stashWithOrigin; public class ExecutionService { @@ -356,7 +356,7 @@ public void updateWatchStatus(Watch watch) throws IOException { updateRequest.doc(source); updateRequest.setIfSeqNo(watch.getSourceSeqNo()); updateRequest.setIfPrimaryTerm(watch.getSourcePrimaryTerm()); - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), WATCHER_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(WATCHER_ORIGIN)) { client.update(updateRequest).actionGet(indexDefaultTimeout); } catch (DocumentMissingException e) { // do not rethrow this exception, otherwise the watch history will contain an exception @@ -500,7 +500,7 @@ public void executeTriggeredWatches(Collection triggeredWatches) * @return The GetResponse of calling the get API of this watch */ private GetResponse getWatch(String id) { - try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), WATCHER_ORIGIN)) { + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(WATCHER_ORIGIN)) { GetRequest getRequest = new GetRequest(Watch.INDEX, id).preference(Preference.LOCAL.type()).realtime(true); PlainActionFuture future = PlainActionFuture.newFuture(); client.get(getRequest, future); From a8b68110fc98959c9496ccc200150a5dc68f1c70 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 24 Apr 2019 10:12:48 +0200 Subject: [PATCH 165/260] Fix Repository Base Path Matching in Azure ITs (#41457) * Added quotes so that "regexy" base paths like `7.0` that we use on CI don't break matching * closes #41405 --- .../rest-api-spec/test/repository_azure/10_repository.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml b/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml index df3e1b3216ab9..92866190959e6 100644 --- a/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml +++ b/plugins/repository-azure/qa/microsoft-azure-storage/src/test/resources/rest-api-spec/test/repository_azure/10_repository.yml @@ -23,7 +23,7 @@ setup: - match: { repository.settings.container: ${container} } - match: { repository.settings.client : "integration_test" } - - match: { repository.settings.base_path : ${base_path} } + - match: { repository.settings.base_path : "${base_path}" } # Index documents - do: From 72b66af0f3107fef1cc8d3f900a0d9b8ce1a0d43 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 24 Apr 2019 06:53:53 -0500 Subject: [PATCH 166/260] [ML] refactoring the ML plugin to use the common auditor code (#41419) --- .../core/common/notifications/Auditor.java | 6 +- .../core/ml/notifications/AuditMessage.java | 191 +++--------------- .../xpack/core/ml/notifications/Level.java | 42 ---- .../ml/notifications/AuditMessageTests.java | 77 ------- .../xpack/ml/notifications/Auditor.java | 62 +----- .../retention/ExpiredResultsRemoverTests.java | 2 +- .../xpack/ml/notifications/AuditorTests.java | 94 --------- .../xpack/ml/notifications/LevelTests.java | 114 ----------- 8 files changed, 39 insertions(+), 549 deletions(-) delete mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/Level.java delete mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessageTests.java delete mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/AuditorTests.java delete mode 100644 x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/LevelTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/Auditor.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/Auditor.java index 01acb18900b2d..d4a1f14f18855 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/Auditor.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/notifications/Auditor.java @@ -42,15 +42,15 @@ public Auditor(Client client, this.messageBuilder = Objects.requireNonNull(messageBuilder); } - public final void info(String resourceId, String message) { + public void info(String resourceId, String message) { indexDoc(messageBuilder.info(resourceId, message, nodeName)); } - public final void warning(String resourceId, String message) { + public void warning(String resourceId, String message) { indexDoc(messageBuilder.warning(resourceId, message, nodeName)); } - public final void error(String resourceId, String message) { + public void error(String resourceId, String message) { indexDoc(messageBuilder.error(resourceId, message, nodeName)); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessage.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessage.java index fcb7ed479cf38..c3328bb3263fd 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessage.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessage.java @@ -5,188 +5,61 @@ */ package org.elasticsearch.xpack.core.ml.notifications; -import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.ObjectParser.ValueType; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.common.notifications.AbstractAuditMessage; +import org.elasticsearch.xpack.core.common.notifications.Level; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.utils.time.TimeUtils; -import java.io.IOException; import java.util.Date; -import java.util.Objects; -public class AuditMessage implements ToXContentObject, Writeable { - private static final ParseField TYPE = new ParseField("audit_message"); +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; - public static final ParseField MESSAGE = new ParseField("message"); - public static final ParseField LEVEL = new ParseField("level"); - public static final ParseField TIMESTAMP = new ParseField("timestamp"); - public static final ParseField NODE_NAME = new ParseField("node_name"); +public class AuditMessage extends AbstractAuditMessage { - public static final ObjectParser PARSER = new ObjectParser<>(TYPE.getPreferredName(), true, AuditMessage::new); + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "ml_audit_message", + true, + a -> new AuditMessage((String)a[0], (String)a[1], (Level)a[2], (Date)a[3], (String)a[4])); static { - PARSER.declareString(AuditMessage::setJobId, Job.ID); - PARSER.declareString(AuditMessage::setMessage, MESSAGE); - PARSER.declareField(AuditMessage::setLevel, p -> { + PARSER.declareString(optionalConstructorArg(), Job.ID); + PARSER.declareString(constructorArg(), MESSAGE); + PARSER.declareField(constructorArg(), p -> { if (p.currentToken() == XContentParser.Token.VALUE_STRING) { return Level.fromString(p.text()); } throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); - }, LEVEL, ValueType.STRING); - PARSER.declareField(AuditMessage::setTimestamp, - p -> TimeUtils.parseTimeField(p, TIMESTAMP.getPreferredName()), TIMESTAMP, ValueType.VALUE); - PARSER.declareString(AuditMessage::setNodeName, NODE_NAME); + }, LEVEL, ObjectParser.ValueType.STRING); + PARSER.declareField(constructorArg(), + p -> TimeUtils.parseTimeField(p, TIMESTAMP.getPreferredName()), + TIMESTAMP, + ObjectParser.ValueType.VALUE); + PARSER.declareString(optionalConstructorArg(), NODE_NAME); } - private String jobId; - private String message; - private Level level; - private Date timestamp; - private String nodeName; - - private AuditMessage() { - - } - - AuditMessage(String jobId, String message, Level level, String nodeName) { - this.jobId = jobId; - this.message = message; - this.level = level; - timestamp = new Date(); - this.nodeName = nodeName; - } - - public AuditMessage(StreamInput in) throws IOException { - jobId = in.readOptionalString(); - message = in.readOptionalString(); - if (in.readBoolean()) { - level = Level.readFromStream(in); - } - if (in.readBoolean()) { - timestamp = new Date(in.readLong()); - } - nodeName = in.readOptionalString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalString(jobId); - out.writeOptionalString(message); - boolean hasLevel = level != null; - out.writeBoolean(hasLevel); - if (hasLevel) { - level.writeTo(out); - } - boolean hasTimestamp = timestamp != null; - out.writeBoolean(hasTimestamp); - if (hasTimestamp) { - out.writeLong(timestamp.getTime()); - } - out.writeOptionalString(nodeName); - } - - public String getJobId() { - return jobId; - } - - public void setJobId(String jobId) { - this.jobId = jobId; - } - - public String getMessage() { - return message; - } - - public void setMessage(String message) { - this.message = message; - } - - public Level getLevel() { - return level; - } - - public void setLevel(Level level) { - this.level = level; - } - - public Date getTimestamp() { - return timestamp; - } - - public void setTimestamp(Date timestamp) { - this.timestamp = timestamp; - } - - public String getNodeName() { - return nodeName; - } - - public void setNodeName(String nodeName) { - this.nodeName = nodeName; - } - - public static AuditMessage newInfo(String jobId, String message, String nodeName) { - return new AuditMessage(jobId, message, Level.INFO, nodeName); - } - - public static AuditMessage newWarning(String jobId, String message, String nodeName) { - return new AuditMessage(jobId, message, Level.WARNING, nodeName); - } - - public static AuditMessage newActivity(String jobId, String message, String nodeName) { - return new AuditMessage(jobId, message, Level.ACTIVITY, nodeName); + public AuditMessage(String resourceId, String message, Level level, String nodeName) { + super(resourceId, message, level, nodeName); } - public static AuditMessage newError(String jobId, String message, String nodeName) { - return new AuditMessage(jobId, message, Level.ERROR, nodeName); + protected AuditMessage(String resourceId, String message, Level level, Date timestamp, String nodeName) { + super(resourceId, message, level, timestamp, nodeName); } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - if (jobId != null) { - builder.field(Job.ID.getPreferredName(), jobId); - } - if (message != null) { - builder.field(MESSAGE.getPreferredName(), message); - } - if (level != null) { - builder.field(LEVEL.getPreferredName(), level); - } - if (timestamp != null) { - builder.field(TIMESTAMP.getPreferredName(), timestamp.getTime()); - } - if (nodeName != null) { - builder.field(NODE_NAME.getPreferredName(), nodeName); - } - builder.endObject(); - return builder; + protected String getResourceField() { + return Job.ID.getPreferredName(); } - @Override - public int hashCode() { - return Objects.hash(jobId, message, level, timestamp); - } - - @Override - public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - AuditMessage other = (AuditMessage) obj; - return Objects.equals(jobId, other.jobId) && - Objects.equals(message, other.message) && - Objects.equals(level, other.level) && - Objects.equals(timestamp, other.timestamp); + public static AbstractBuilder builder() { + return new AbstractBuilder() { + @Override + protected AuditMessage newMessage(Level level, String resourceId, String message, String nodeName) { + return new AuditMessage(resourceId, message, level, nodeName); + } + }; } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/Level.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/Level.java deleted file mode 100644 index f54f9cf268077..0000000000000 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/notifications/Level.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.core.ml.notifications; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; - -import java.io.IOException; -import java.util.Locale; - -public enum Level implements Writeable { - INFO, ACTIVITY, WARNING, ERROR; - - /** - * Case-insensitive from string method. - * - * @param value - * String representation - * @return The condition type - */ - public static Level fromString(String value) { - return Level.valueOf(value.toUpperCase(Locale.ROOT)); - } - - public static Level readFromStream(StreamInput in) throws IOException { - return in.readEnum(Level.class); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeEnum(this); - } - - @Override - public String toString() { - return name().toLowerCase(Locale.ROOT); - } -} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessageTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessageTests.java deleted file mode 100644 index f5c30c5900394..0000000000000 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/notifications/AuditMessageTests.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.core.ml.notifications; - -import org.elasticsearch.common.io.stream.Writeable.Reader; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.test.AbstractSerializingTestCase; -import org.junit.Before; - -import java.util.Date; - -public class AuditMessageTests extends AbstractSerializingTestCase { - private long startMillis; - - @Before - public void setStartTime() { - startMillis = System.currentTimeMillis(); - } - - public void testNewInfo() { - AuditMessage info = AuditMessage.newInfo("foo", "some info", "some_node"); - assertEquals("foo", info.getJobId()); - assertEquals("some info", info.getMessage()); - assertEquals(Level.INFO, info.getLevel()); - assertDateBetweenStartAndNow(info.getTimestamp()); - } - - public void testNewWarning() { - AuditMessage warning = AuditMessage.newWarning("bar", "some warning", "some_node"); - assertEquals("bar", warning.getJobId()); - assertEquals("some warning", warning.getMessage()); - assertEquals(Level.WARNING, warning.getLevel()); - assertDateBetweenStartAndNow(warning.getTimestamp()); - } - - - public void testNewError() { - AuditMessage error = AuditMessage.newError("foo", "some error", "some_node"); - assertEquals("foo", error.getJobId()); - assertEquals("some error", error.getMessage()); - assertEquals(Level.ERROR, error.getLevel()); - assertDateBetweenStartAndNow(error.getTimestamp()); - } - - public void testNewActivity() { - AuditMessage error = AuditMessage.newActivity("foo", "some error", "some_node"); - assertEquals("foo", error.getJobId()); - assertEquals("some error", error.getMessage()); - assertEquals(Level.ACTIVITY, error.getLevel()); - assertDateBetweenStartAndNow(error.getTimestamp()); - } - - private void assertDateBetweenStartAndNow(Date timestamp) { - long timestampMillis = timestamp.getTime(); - assertTrue(timestampMillis >= startMillis); - assertTrue(timestampMillis <= System.currentTimeMillis()); - } - - @Override - protected AuditMessage doParseInstance(XContentParser parser) { - return AuditMessage.PARSER.apply(parser, null); - } - - @Override - protected AuditMessage createTestInstance() { - return new AuditMessage(randomAlphaOfLengthBetween(1, 20), randomAlphaOfLengthBetween(1, 200), - randomFrom(Level.values()), randomAlphaOfLengthBetween(1, 20)); - } - - @Override - protected Reader instanceReader() { - return AuditMessage::new; - } -} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/notifications/Auditor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/notifications/Auditor.java index 4b5da8fae2cda..25c827829b24a 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/notifications/Auditor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/notifications/Auditor.java @@ -5,71 +5,15 @@ */ package org.elasticsearch.xpack.ml.notifications; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.client.Client; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.ToXContent; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.xpack.core.ml.notifications.AuditMessage; import org.elasticsearch.xpack.core.ml.notifications.AuditorField; +import org.elasticsearch.xpack.core.ml.notifications.AuditMessage; -import java.io.IOException; -import java.util.Objects; - -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; -import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; - -public class Auditor { - private static final Logger LOGGER = LogManager.getLogger(Auditor.class); - - private final Client client; - private final String nodeName; +public class Auditor extends org.elasticsearch.xpack.core.common.notifications.Auditor { public Auditor(Client client, String nodeName) { - this.client = Objects.requireNonNull(client); - this.nodeName = Objects.requireNonNull(nodeName); - } - - public void info(String jobId, String message) { - indexDoc(AuditMessage.newInfo(jobId, message, nodeName)); - } - - public void warning(String jobId, String message) { - indexDoc(AuditMessage.newWarning(jobId, message, nodeName)); - } - - public void error(String jobId, String message) { - indexDoc(AuditMessage.newError(jobId, message, nodeName)); - } - - private void indexDoc(ToXContent toXContent) { - IndexRequest indexRequest = new IndexRequest(AuditorField.NOTIFICATIONS_INDEX); - indexRequest.source(toXContentBuilder(toXContent)); - indexRequest.timeout(TimeValue.timeValueSeconds(5)); - executeAsyncWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN, indexRequest, new ActionListener() { - @Override - public void onResponse(IndexResponse indexResponse) { - LOGGER.trace("Successfully persisted audit message"); - } - - @Override - public void onFailure(Exception e) { - LOGGER.debug("Error writing audit message", e); - } - }, client::index); - } - - private XContentBuilder toXContentBuilder(ToXContent toXContent) { - try (XContentBuilder jsonBuilder = jsonBuilder()) { - return toXContent.toXContent(jsonBuilder, ToXContent.EMPTY_PARAMS); - } catch (IOException e) { - throw new RuntimeException(e); - } + super(client, nodeName, AuditorField.NOTIFICATIONS_INDEX, ML_ORIGIN, AuditMessage.builder()); } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java index 7dc258a322ac3..0398882a8e35e 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java @@ -162,4 +162,4 @@ private void givenJobs(List jobs) throws IOException { private ExpiredResultsRemover createExpiredResultsRemover() { return new ExpiredResultsRemover(client, mock(Auditor.class)); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/AuditorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/AuditorTests.java deleted file mode 100644 index 441125c931b7c..0000000000000 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/AuditorTests.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * 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.notifications; - -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.common.xcontent.DeprecationHandler; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.core.ml.notifications.AuditMessage; -import org.elasticsearch.xpack.core.ml.notifications.Level; -import org.junit.Before; -import org.mockito.ArgumentCaptor; - -import java.io.IOException; - -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class AuditorTests extends ESTestCase { - private Client client; - private ArgumentCaptor indexRequestCaptor; - - @Before - public void setUpMocks() { - client = mock(Client.class); - ThreadPool threadPool = mock(ThreadPool.class); - when(client.threadPool()).thenReturn(threadPool); - when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); - - indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class); - } - - public void testInfo() throws IOException { - Auditor auditor = new Auditor(client, "node_1"); - auditor.info("foo", "Here is my info"); - - verify(client).index(indexRequestCaptor.capture(), any()); - IndexRequest indexRequest = indexRequestCaptor.getValue(); - assertArrayEquals(new String[] {".ml-notifications"}, indexRequest.indices()); - assertEquals(TimeValue.timeValueSeconds(5), indexRequest.timeout()); - AuditMessage auditMessage = parseAuditMessage(indexRequest.source()); - assertEquals("foo", auditMessage.getJobId()); - assertEquals("Here is my info", auditMessage.getMessage()); - assertEquals(Level.INFO, auditMessage.getLevel()); - } - - public void testWarning() throws IOException { - Auditor auditor = new Auditor(client, "node_1"); - auditor.warning("bar", "Here is my warning"); - - verify(client).index(indexRequestCaptor.capture(), any()); - IndexRequest indexRequest = indexRequestCaptor.getValue(); - assertArrayEquals(new String[] {".ml-notifications"}, indexRequest.indices()); - assertEquals(TimeValue.timeValueSeconds(5), indexRequest.timeout()); - AuditMessage auditMessage = parseAuditMessage(indexRequest.source()); - assertEquals("bar", auditMessage.getJobId()); - assertEquals("Here is my warning", auditMessage.getMessage()); - assertEquals(Level.WARNING, auditMessage.getLevel()); - } - - public void testError() throws IOException { - Auditor auditor = new Auditor(client, "node_1"); - auditor.error("foobar", "Here is my error"); - - verify(client).index(indexRequestCaptor.capture(), any()); - IndexRequest indexRequest = indexRequestCaptor.getValue(); - assertArrayEquals(new String[] {".ml-notifications"}, indexRequest.indices()); - assertEquals(TimeValue.timeValueSeconds(5), indexRequest.timeout()); - AuditMessage auditMessage = parseAuditMessage(indexRequest.source()); - assertEquals("foobar", auditMessage.getJobId()); - assertEquals("Here is my error", auditMessage.getMessage()); - assertEquals(Level.ERROR, auditMessage.getLevel()); - } - - private AuditMessage parseAuditMessage(BytesReference msg) throws IOException { - XContentParser parser = XContentFactory.xContent(XContentHelper.xContentType(msg)) - .createParser(NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, msg.streamInput()); - return AuditMessage.PARSER.apply(parser, null); - } -} diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/LevelTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/LevelTests.java deleted file mode 100644 index c3e3a885e8209..0000000000000 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/notifications/LevelTests.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.notifications; - -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.core.ml.notifications.Level; - -import java.io.IOException; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; - -public class LevelTests extends ESTestCase { - - public void testFromString() { - assertEquals(Level.INFO, Level.fromString("info")); - assertEquals(Level.INFO, Level.fromString("INFO")); - assertEquals(Level.ACTIVITY, Level.fromString("activity")); - assertEquals(Level.ACTIVITY, Level.fromString("ACTIVITY")); - assertEquals(Level.WARNING, Level.fromString("warning")); - assertEquals(Level.WARNING, Level.fromString("WARNING")); - assertEquals(Level.ERROR, Level.fromString("error")); - assertEquals(Level.ERROR, Level.fromString("ERROR")); - } - - public void testToString() { - assertEquals("info", Level.INFO.toString()); - assertEquals("activity", Level.ACTIVITY.toString()); - assertEquals("warning", Level.WARNING.toString()); - assertEquals("error", Level.ERROR.toString()); - } - - public void testValidOrdinals() { - assertThat(Level.INFO.ordinal(), equalTo(0)); - assertThat(Level.ACTIVITY.ordinal(), equalTo(1)); - assertThat(Level.WARNING.ordinal(), equalTo(2)); - assertThat(Level.ERROR.ordinal(), equalTo(3)); - } - - public void testwriteTo() throws Exception { - try (BytesStreamOutput out = new BytesStreamOutput()) { - Level.INFO.writeTo(out); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(in.readVInt(), equalTo(0)); - } - } - - try (BytesStreamOutput out = new BytesStreamOutput()) { - Level.ACTIVITY.writeTo(out); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(in.readVInt(), equalTo(1)); - } - } - - try (BytesStreamOutput out = new BytesStreamOutput()) { - Level.WARNING.writeTo(out); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(in.readVInt(), equalTo(2)); - } - } - - try (BytesStreamOutput out = new BytesStreamOutput()) { - Level.ERROR.writeTo(out); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(in.readVInt(), equalTo(3)); - } - } - } - - public void testReadFrom() throws Exception { - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeVInt(0); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(Level.readFromStream(in), equalTo(Level.INFO)); - } - } - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeVInt(1); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(Level.readFromStream(in), equalTo(Level.ACTIVITY)); - } - } - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeVInt(2); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(Level.readFromStream(in), equalTo(Level.WARNING)); - } - } - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeVInt(3); - try (StreamInput in = out.bytes().streamInput()) { - assertThat(Level.readFromStream(in), equalTo(Level.ERROR)); - } - } - } - - public void testInvalidReadFrom() throws Exception { - try (BytesStreamOutput out = new BytesStreamOutput()) { - out.writeVInt(randomIntBetween(4, Integer.MAX_VALUE)); - try (StreamInput in = out.bytes().streamInput()) { - Level.readFromStream(in); - fail("Expected IOException"); - } catch (IOException e) { - assertThat(e.getMessage(), containsString("Unknown Level ordinal [")); - } - } - } - -} From 3a1606bc60a648038795c8f47f4def9299e591fd Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Wed, 24 Apr 2019 08:44:01 -0400 Subject: [PATCH 167/260] [DOCS] Remove inline callouts for Asciidoctor migration (#41460) --- docs/reference/sql/functions/conditional.asciidoc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/reference/sql/functions/conditional.asciidoc b/docs/reference/sql/functions/conditional.asciidoc index cfce98d803cb3..cad61f91f1997 100644 --- a/docs/reference/sql/functions/conditional.asciidoc +++ b/docs/reference/sql/functions/conditional.asciidoc @@ -229,7 +229,10 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[ifNullReturnSecond] .Synopsis: [source, sql] ---- -IIF(expression<1>, expression<2>, [expression<3>]) +IIF( + expression, <1> + expression, <2> + [expression]) <3> ---- *Input*: From f78834db03758f1c973a1c256aeca84c13d03750 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 24 Apr 2019 15:44:22 +0200 Subject: [PATCH 168/260] Reenable SnapshotResiliency Test (#41437) This was fixed in https://github.com/elastic/elasticsearch/pull/41332 but I forgot to reenable the test. --- .../org/elasticsearch/snapshots/SnapshotResiliencyTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 2661f958fc2a2..285234999564e 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -433,7 +433,6 @@ public void testConcurrentSnapshotCreateAndDelete() { * Simulates concurrent restarts of data and master nodes as well as relocating a primary shard, while starting and subsequently * deleting a snapshot. */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41326") public void testSnapshotPrimaryRelocations() { final int masterNodeCount = randomFrom(1, 3, 5); setupTestCluster(masterNodeCount, randomIntBetween(2, 10)); From b4e7008a729c47078dc9b9352b343a16e83ccef2 Mon Sep 17 00:00:00 2001 From: Hendrik Muhs Date: Wed, 24 Apr 2019 16:45:38 +0200 Subject: [PATCH 169/260] [ML-DataFrame] Replace Streamable w/ Writeable (#41477) replace usages of Streamable with Writeable Relates to #36176 --- .../action/AbstractGetResourcesRequest.java | 7 +++ .../action/AbstractGetResourcesResponse.java | 5 +++ .../DeleteDataFrameTransformAction.java | 25 +++++------ .../action/GetDataFrameTransformsAction.java | 11 +++-- .../GetDataFrameTransformsStatsAction.java | 22 ++++----- .../PreviewDataFrameTransformAction.java | 45 ++++++------------- .../action/PutDataFrameTransformAction.java | 34 ++++---------- .../action/StartDataFrameTransformAction.java | 30 +++++-------- .../StartDataFrameTransformTaskAction.java | 26 ++++------- .../action/StopDataFrameTransformAction.java | 31 ++++--------- ...wDataFrameTransformActionRequestTests.java | 9 ++-- ...ataFrameTransformsActionResponseTests.java | 9 ++-- ...tDataFrameTransformActionRequestTests.java | 9 ++-- ...DataFrameTransformActionResponseTests.java | 24 ---------- ...nsportPreviewDataFrameTransformAction.java | 4 +- .../TransportPutDataFrameTransformAction.java | 21 ++++----- ...ransportStartDataFrameTransformAction.java | 2 +- 17 files changed, 119 insertions(+), 195 deletions(-) delete mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionResponseTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesRequest.java index 992a103b26c12..214d84bb6b8d1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesRequest.java @@ -23,6 +23,13 @@ public abstract class AbstractGetResourcesRequest extends ActionRequest { public AbstractGetResourcesRequest() { } + public AbstractGetResourcesRequest(StreamInput in) throws IOException { + super(in); + resourceId = in.readOptionalString(); + pageParams = in.readOptionalWriteable(PageParams::new); + allowNoResources = in.readBoolean(); + } + // Allow child classes to provide their own defaults if necessary protected AbstractGetResourcesRequest(String resourceId, PageParams pageParams, boolean allowNoResources) { this.resourceId = resourceId; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesResponse.java index fcfdccb680365..3ad388bf2e920 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/AbstractGetResourcesResponse.java @@ -26,6 +26,11 @@ public abstract class AbstractGetResourcesResponse(in, getReader()); + } + protected AbstractGetResourcesResponse(QueryPage resources) { this.resources = Objects.requireNonNull(resources); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java index 0316153fbc822..8ac1e9c6c513f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java @@ -36,11 +36,16 @@ private DeleteDataFrameTransformAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends BaseTasksRequest { - private String id; + private final String id; public Request(String id) { this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName()); @@ -91,10 +96,12 @@ public boolean equals(Object obj) { } public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { - private boolean acknowledged; + + private final boolean acknowledged; + public Response(StreamInput in) throws IOException { super(in); - readFrom(in); + acknowledged = in.readBoolean(); } public Response(boolean acknowledged, List taskFailures, List nodeFailures) { @@ -106,20 +113,10 @@ public Response(boolean acknowledged) { this(acknowledged, Collections.emptyList(), Collections.emptyList()); } - public Response() { - this(false, Collections.emptyList(), Collections.emptyList()); - } - public boolean isDeleted() { return acknowledged; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - acknowledged = in.readBoolean(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsAction.java index ac1498c72a6da..7cee5dbcbf527 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsAction.java @@ -42,7 +42,12 @@ private GetDataFrameTransformsAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends AbstractGetResourcesRequest { @@ -58,7 +63,7 @@ public Request() { } public Request(StreamInput in) throws IOException { - readFrom(in); + super(in); } public String getId() { @@ -95,7 +100,7 @@ public Response() { } public Response(StreamInput in) throws IOException { - readFrom(in); + super(in); } public List getTransformConfigurations() { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java index d91f7a1a06964..96763153240e5 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java @@ -42,11 +42,16 @@ public GetDataFrameTransformsStatsAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends BaseTasksRequest { - private String id; + private final String id; private PageParams pageParams = PageParams.defaultParams(); public static final int MAX_SIZE_RETURN = 1000; @@ -145,26 +150,15 @@ public Response(List transformsStateAndStats, L this.transformsStateAndStats = transformsStateAndStats; } - public Response() { - super(Collections.emptyList(), Collections.emptyList()); - this.transformsStateAndStats = Collections.emptyList(); - } - public Response(StreamInput in) throws IOException { super(in); - readFrom(in); + transformsStateAndStats = in.readList(DataFrameTransformStateAndStats::new); } public List getTransformsStateAndStats() { return transformsStateAndStats; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - transformsStateAndStats = in.readList(DataFrameTransformStateAndStats::new); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformAction.java index 8116e1d1f23e5..90e0adba16269 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformAction.java @@ -10,12 +10,11 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.support.master.AcknowledgedRequest; -import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; -import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; @@ -44,18 +43,26 @@ private PreviewDataFrameTransformAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends AcknowledgedRequest implements ToXContentObject { - private DataFrameTransformConfig config; + private final DataFrameTransformConfig config; public Request(DataFrameTransformConfig config) { - this.setConfig(config); + this.config = config; } - public Request() { } + public Request(StreamInput in) throws IOException { + super(in); + this.config = new DataFrameTransformConfig(in); + } public static Request fromXContent(final XContentParser parser) throws IOException { Map content = parser.map(); @@ -86,16 +93,6 @@ public DataFrameTransformConfig getConfig() { return config; } - public void setConfig(DataFrameTransformConfig config) { - this.config = config; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - this.config = new DataFrameTransformConfig(in); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -120,13 +117,6 @@ public boolean equals(Object obj) { } } - public static class RequestBuilder extends MasterNodeOperationRequestBuilder { - - protected RequestBuilder(ElasticsearchClient client, PreviewDataFrameTransformAction action) { - super(client, action, new Request()); - } - } - public static class Response extends ActionResponse implements ToXContentObject { private List> docs; @@ -154,15 +144,6 @@ public void setDocs(List> docs) { this.docs = new ArrayList<>(docs); } - @Override - public void readFrom(StreamInput in) throws IOException { - int size = in.readInt(); - this.docs = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - this.docs.add(in.readMap()); - } - } - @Override public void writeTo(StreamOutput out) throws IOException { out.writeInt(docs.size()); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformAction.java index 51b5e0d4ec1d6..0f6cc63f98851 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformAction.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Objects; -public class PutDataFrameTransformAction extends Action { +public class PutDataFrameTransformAction extends Action { public static final PutDataFrameTransformAction INSTANCE = new PutDataFrameTransformAction(); public static final String NAME = "cluster:admin/data_frame/put"; @@ -30,20 +30,21 @@ private PutDataFrameTransformAction() { } @Override - public Response newResponse() { - return new Response(); + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); } public static class Request extends AcknowledgedRequest implements ToXContentObject { - private DataFrameTransformConfig config; + private final DataFrameTransformConfig config; public Request(DataFrameTransformConfig config) { - this.setConfig(config); + this.config = config; } - public Request() { - + public Request(StreamInput in) throws IOException { + super(in); + this.config = new DataFrameTransformConfig(in); } public static Request fromXContent(final XContentParser parser, final String id) throws IOException { @@ -64,16 +65,6 @@ public DataFrameTransformConfig getConfig() { return config; } - public void setConfig(DataFrameTransformConfig config) { - this.config = config; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - this.config = new DataFrameTransformConfig(in); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -98,13 +89,4 @@ public boolean equals(Object obj) { } } - public static class Response extends AcknowledgedResponse { - public Response() { - super(); - } - - public Response(boolean acknowledged) { - super(acknowledged); - } - } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java index b86a2339faa47..d2550ab2f410c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java @@ -33,25 +33,28 @@ private StartDataFrameTransformAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends AcknowledgedRequest { - private String id; - private boolean force; + private final String id; + private final boolean force; public Request(String id, boolean force) { this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName()); this.force = force; } - public Request() { - } - public Request(StreamInput in) throws IOException { super(in); id = in.readString(); + force = in.readBoolean(); } public String getId() { @@ -66,6 +69,7 @@ public boolean isForce() { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(id); + out.writeBoolean(force); } @Override @@ -92,15 +96,11 @@ public boolean equals(Object obj) { } public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { - private boolean started; - - public Response() { - super(Collections.emptyList(), Collections.emptyList()); - } + private final boolean started; public Response(StreamInput in) throws IOException { super(in); - readFrom(in); + started = in.readBoolean(); } public Response(boolean started) { @@ -112,12 +112,6 @@ public boolean isStarted() { return started; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - started = in.readBoolean(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java index 044d9d58aed30..934d1bae43319 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java @@ -34,20 +34,22 @@ private StartDataFrameTransformTaskAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends BaseTasksRequest { - private String id; + private final String id; public Request(String id) { this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName()); } - public Request() { - } - public Request(StreamInput in) throws IOException { super(in); id = in.readString(); @@ -92,15 +94,11 @@ public boolean equals(Object obj) { } public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { - private boolean started; - - public Response() { - super(Collections.emptyList(), Collections.emptyList()); - } + private final boolean started; public Response(StreamInput in) throws IOException { super(in); - readFrom(in); + started = in.readBoolean(); } public Response(boolean started) { @@ -112,12 +110,6 @@ public boolean isStarted() { return started; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - started = in.readBoolean(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java index 54153aab91ced..ee7bb830660f3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java @@ -41,11 +41,16 @@ private StopDataFrameTransformAction() { @Override public Response newResponse() { - return new Response(); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + public Writeable.Reader getResponseReader() { + return Response::new; } public static class Request extends BaseTasksRequest { - private String id; + private final String id; private final boolean waitForCompletion; private final boolean force; private Set expandedIds; @@ -59,10 +64,6 @@ public Request(String id, boolean waitForCompletion, boolean force, @Nullable Ti this.setTimeout(timeout == null ? DEFAULT_TIMEOUT : timeout); } - private Request() { - this(null, false, false, null); - } - public Request(StreamInput in) throws IOException { super(in); id = in.readString(); @@ -77,10 +78,6 @@ public String getId() { return id; } - public void setId(String id) { - this.id = id; - } - public boolean waitForCompletion() { return waitForCompletion; } @@ -158,15 +155,11 @@ public boolean match(Task task) { public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { - private boolean stopped; - - public Response() { - super(Collections.emptyList(), Collections.emptyList()); - } + private final boolean stopped; public Response(StreamInput in) throws IOException { super(in); - readFrom(in); + stopped = in.readBoolean(); } public Response(boolean stopped) { @@ -178,12 +171,6 @@ public boolean isStopped() { return stopped; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - stopped = in.readBoolean(); - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java index 1936dda9117e5..4cab28105a899 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java @@ -8,13 +8,14 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.search.SearchModule; -import org.elasticsearch.test.AbstractStreamableXContentTestCase; +import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.dataframe.action.PreviewDataFrameTransformAction.Request; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DestConfig; @@ -26,7 +27,7 @@ import static java.util.Collections.emptyList; import static org.elasticsearch.xpack.core.dataframe.transforms.SourceConfigTests.randomSourceConfig; -public class PreviewDataFrameTransformActionRequestTests extends AbstractStreamableXContentTestCase { +public class PreviewDataFrameTransformActionRequestTests extends AbstractSerializingTestCase { private NamedWriteableRegistry namedWriteableRegistry; private NamedXContentRegistry namedXContentRegistry; @@ -55,8 +56,8 @@ protected Request doParseInstance(XContentParser parser) throws IOException { } @Override - protected Request createBlankInstance() { - return new Request(); + protected Writeable.Reader instanceReader() { + return Request::new; } @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformsActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformsActionResponseTests.java index 8594490b17b50..96744b4cd8ced 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformsActionResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformsActionResponseTests.java @@ -6,8 +6,9 @@ package org.elasticsearch.xpack.core.dataframe.action; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.test.AbstractStreamableXContentTestCase; +import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.dataframe.action.PreviewDataFrameTransformAction.Response; import java.io.IOException; @@ -16,7 +17,7 @@ import java.util.List; import java.util.Map; -public class PreviewDataFrameTransformsActionResponseTests extends AbstractStreamableXContentTestCase { +public class PreviewDataFrameTransformsActionResponseTests extends AbstractSerializingTestCase { @Override @@ -25,8 +26,8 @@ protected Response doParseInstance(XContentParser parser) throws IOException { } @Override - protected Response createBlankInstance() { - return new Response(); + protected Writeable.Reader instanceReader() { + return Response::new; } @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionRequestTests.java index ea445a647d8c6..95d0d3f7e4c9a 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionRequestTests.java @@ -7,11 +7,12 @@ package org.elasticsearch.xpack.core.dataframe.action; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.SearchModule; -import org.elasticsearch.test.AbstractStreamableXContentTestCase; +import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction.Request; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfigTests; @@ -21,7 +22,7 @@ import static java.util.Collections.emptyList; -public class PutDataFrameTransformActionRequestTests extends AbstractStreamableXContentTestCase { +public class PutDataFrameTransformActionRequestTests extends AbstractSerializingTestCase { private String transformId; @@ -57,8 +58,8 @@ protected Request doParseInstance(XContentParser parser) throws IOException { } @Override - protected Request createBlankInstance() { - return new Request(); + protected Writeable.Reader instanceReader() { + return Request::new; } @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionResponseTests.java deleted file mode 100644 index d16b8fbf168fe..0000000000000 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PutDataFrameTransformActionResponseTests.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.core.dataframe.action; - -import org.elasticsearch.test.AbstractStreamableTestCase; -import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction.Response; - -public class PutDataFrameTransformActionResponseTests extends AbstractStreamableTestCase { - - @Override - protected Response createBlankInstance() { - return new Response(); - } - - @Override - protected Response createTestInstance() { - return new Response(randomBoolean()); - } - -} diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java index b5642310df369..5b36127305002 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPreviewDataFrameTransformAction.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.Client; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; @@ -27,7 +28,6 @@ import java.util.List; import java.util.Map; -import java.util.function.Supplier; import java.util.stream.Collectors; import static org.elasticsearch.xpack.dataframe.transforms.DataFrameIndexer.COMPOSITE_AGGREGATION_NAME; @@ -44,7 +44,7 @@ public class TransportPreviewDataFrameTransformAction extends public TransportPreviewDataFrameTransformAction(TransportService transportService, ActionFilters actionFilters, Client client, ThreadPool threadPool, XPackLicenseState licenseState) { super(PreviewDataFrameTransformAction.NAME,transportService, actionFilters, - (Supplier) PreviewDataFrameTransformAction.Request::new); + (Writeable.Reader) PreviewDataFrameTransformAction.Request::new); this.licenseState = licenseState; this.client = client; this.threadPool = threadPool; diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java index c7f750ecdb210..edcd06892904e 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportPutDataFrameTransformAction.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; @@ -38,7 +39,6 @@ import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction; import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction.Request; -import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction.Response; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.security.SecurityContext; import org.elasticsearch.xpack.core.security.action.user.HasPrivilegesAction; @@ -55,7 +55,7 @@ import java.util.stream.Collectors; public class TransportPutDataFrameTransformAction - extends TransportMasterNodeAction { + extends TransportMasterNodeAction { private static final Logger logger = LogManager.getLogger(TransportPutDataFrameTransformAction.class); @@ -69,8 +69,8 @@ public TransportPutDataFrameTransformAction(Settings settings, TransportService ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, ClusterService clusterService, XPackLicenseState licenseState, DataFrameTransformsConfigManager dataFrameTransformsConfigManager, Client client) { - super(PutDataFrameTransformAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, - PutDataFrameTransformAction.Request::new); + super(PutDataFrameTransformAction.NAME, transportService, clusterService, threadPool, actionFilters, + PutDataFrameTransformAction.Request::new, indexNameExpressionResolver); this.licenseState = licenseState; this.client = client; this.dataFrameTransformsConfigManager = dataFrameTransformsConfigManager; @@ -84,12 +84,13 @@ protected String executor() { } @Override - protected PutDataFrameTransformAction.Response newResponse() { - return new PutDataFrameTransformAction.Response(); + protected AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); } @Override - protected void masterOperation(Request request, ClusterState clusterState, ActionListener listener) throws Exception { + protected void masterOperation(Request request, ClusterState clusterState, ActionListener listener) + throws Exception { if (!licenseState.isDataFrameAllowed()) { listener.onFailure(LicenseUtils.newComplianceException(XPackField.DATA_FRAME)); @@ -169,7 +170,7 @@ protected ClusterBlockException checkBlock(PutDataFrameTransformAction.Request r private void handlePrivsResponse(String username, DataFrameTransformConfig config, HasPrivilegesResponse privilegesResponse, - ActionListener listener) throws IOException { + ActionListener listener) throws IOException { if (privilegesResponse.isCompleteMatch()) { putDataFrame(config, listener); } else { @@ -187,7 +188,7 @@ private void handlePrivsResponse(String username, } } - private void putDataFrame(DataFrameTransformConfig config, ActionListener listener) { + private void putDataFrame(DataFrameTransformConfig config, ActionListener listener) { final Pivot pivot = new Pivot(config.getSource().getIndex(), config.getSource().getQueryConfig().getQuery(), @@ -196,7 +197,7 @@ private void putDataFrame(DataFrameTransformConfig config, ActionListener Return the listener, or clean up destination index on failure. ActionListener putTransformConfigurationListener = ActionListener.wrap( - putTransformConfigurationResult -> listener.onResponse(new Response(true)), + putTransformConfigurationResult -> listener.onResponse(new AcknowledgedResponse(true)), listener::onFailure ); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java index 0aeb757e5625c..d8fcd15921e16 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java @@ -82,7 +82,7 @@ protected String executor() { @Override protected StartDataFrameTransformAction.Response newResponse() { - return new StartDataFrameTransformAction.Response(); + return new StartDataFrameTransformAction.Response(false); } @Override From 02ef53c8533fdb08c5db7af3f2d8ffd5bbe6da67 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Wed, 24 Apr 2019 12:18:52 -0400 Subject: [PATCH 170/260] [DOCS] Standardize docs for `url` setting (#41117) --- .../settings/security-settings.asciidoc | 28 +++++++++++++------ 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/docs/reference/settings/security-settings.asciidoc b/docs/reference/settings/security-settings.asciidoc index 04cef7e199f29..dd2af0eeda955 100644 --- a/docs/reference/settings/security-settings.asciidoc +++ b/docs/reference/settings/security-settings.asciidoc @@ -278,10 +278,13 @@ Defaults to `true`. The `type` setting must be set to `ldap`. In addition to the <>, you can specify the following settings: -`url`:: Specifies one or more LDAP URLs in the format -`ldap[s]://:`. Multiple URLs can be defined using a comma -separated value or array syntax: `[ "ldaps://server1:636", "ldaps://server2:636" ]`. -`ldaps` and `ldap` URL protocols cannot be mixed in the same realm. Required. +`url`:: +One or more LDAP URLs in the `ldap[s]://:` format. Required. ++ +To provide multiple URLs, use a YAML array (`["ldap://server1:636", "ldap://server2:636"]`) +or comma-separated string (`"ldap://server1:636, ldap://server2:636"`). ++ +While both are supported, you can't mix the `ldap` and `ldaps` protocols. `load_balance.type`:: The behavior to use when there are multiple LDAP URLs defined. For supported @@ -547,11 +550,18 @@ The `type` setting must be set to `active_directory`. In addition to the the following settings: `url`:: -An LDAP URL of the form `ldap[s]://:`. {es} attempts to -authenticate against this URL. If the URL is not specified, it is derived from -the `domain_name` setting and assumes an unencrypted connection to port 389. -Defaults to `ldap://:389`. This setting is required when connecting -using SSL/TLS or when using a custom port. +One or more LDAP URLs in the `ldap[s]://:` format. Defaults to +`ldap://:389`. This setting is required when connecting using +SSL/TLS or when using a custom port. ++ +To provide multiple URLs, use a YAML array (`["ldap://server1:636", "ldap://server2:636"]`) +or comma-separated string (`"ldap://server1:636, ldap://server2:636"`). ++ +While both are supported, you can't mix the `ldap` and `ldaps` protocols. ++ +If no URL is provided, {es} uses a default of `ldap://:389`. This +default uses the `domain_name` setting value and assumes an unencrypted +connection to port 389. `load_balance.type`:: The behavior to use when there are multiple LDAP URLs defined. For supported From 49397636d3bf22a8f85e64696611984fa4773453 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 24 Apr 2019 18:23:54 +0200 Subject: [PATCH 171/260] Fix BulkRejectionIT (#41446) * Due to #40866 one of the two parallel bulk requests can randomly be rejected outright when the write queue is full already, we can catch this situation and ignore it since we can still have the rejection for the dynamic mapping udate for the other reuqest and it's somewhat rare to run into this anyway * Closes #41363 --- .../org/elasticsearch/action/bulk/BulkRejectionIT.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java index 1d7a5245a225e..900f50a9be005 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkRejectionIT.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.test.ESIntegTestCase; import java.util.Collections; @@ -49,7 +50,6 @@ protected int numberOfShards() { return 5; } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41363") public void testBulkRejectionAfterDynamicMappingUpdate() throws Exception { final String index = "test"; assertAcked(prepareCreate(index)); @@ -68,8 +68,12 @@ public void testBulkRejectionAfterDynamicMappingUpdate() throws Exception { } final ActionFuture bulkFuture1 = client().bulk(request1); final ActionFuture bulkFuture2 = client().bulk(request2); - bulkFuture1.actionGet(); - bulkFuture2.actionGet(); + try { + bulkFuture1.actionGet(); + bulkFuture2.actionGet(); + } catch (EsRejectedExecutionException e) { + // ignored, one of the two bulk requests was rejected outright due to the write queue being full + } internalCluster().assertSeqNos(); } } From 5ccc0b5a32fcb01023cf09948e64340d637fa592 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Wed, 24 Apr 2019 13:22:06 -0400 Subject: [PATCH 172/260] Disallow null/empty or duplicate composite sources (#41359) Adds some validation to prevent duplicate source names from being used in the composite agg. Also refactored to use a ConstructingObjectParser and removed the private ctor and setter for sources, making it mandatory. --- .../bucket/composite-aggregation.asciidoc | 5 ++ .../test/search.aggregation/230_composite.yml | 62 ++++++++++++++++++ .../CompositeAggregationBuilder.java | 65 +++++++++++++------ .../composite/CompositeAggregatorTests.java | 34 +++++++++- .../rollup/job/RollupIndexerStateTests.java | 3 +- 5 files changed, 147 insertions(+), 22 deletions(-) diff --git a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc index 6d09379e16993..a771bd4645200 100644 --- a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc @@ -98,6 +98,11 @@ for the aggregation: ==== Values source The `sources` parameter controls the sources that should be used to build the composite buckets. +The order that the `sources` are defined is important because it also controls the order +the keys are returned. + +The name given to each sources must be unique. + There are three different types of values source: ===== Terms 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 67ab217006ab4..3ffcafe5e1e57 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 @@ -483,3 +483,65 @@ setup: - match: { aggregations.test.buckets.0.key.long: 1000 } - match: { aggregations.test.buckets.0.key.kw: "bar" } - match: { aggregations.test.buckets.0.doc_count: 1 } + +--- +"Missing source": + - skip: + version: " - 7.99.99" # TODO change after backport + reason: null/empty sources disallowed in 7.1 + + - do: + catch: /Composite \[sources\] cannot be null or empty/ + search: + rest_total_hits_as_int: true + index: test + body: + aggregations: + test: + composite: + sources: [] + + - do: + catch: /Required \[sources\]/ + search: + rest_total_hits_as_int: true + index: test + body: + aggregations: + test: + composite: + size: 1 + + +--- +"Duplicate sources": + - skip: + version: " - 7.99.99" # TODO change after backport + reason: duplicate names disallowed in 7.1 + + - do: + catch: /Composite source names must be unique, found duplicates[:] \[keyword\]/ + search: + rest_total_hits_as_int: true + index: test + body: + aggregations: + test: + composite: + sources: [ + { + "keyword": { + "terms": { + "field": "keyword", + } + } + }, + { + "keyword": { + "terms": { + "field": "keyword", + } + } + } + ] + diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java index 69910d21ed8ad..809dec737ff2b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java @@ -22,7 +22,7 @@ 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.ConstructingObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; @@ -34,9 +34,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; +import java.util.function.Function; public class CompositeAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "composite"; @@ -45,29 +48,36 @@ public class CompositeAggregationBuilder extends AbstractAggregationBuilder PARSER; - static { - PARSER = new ObjectParser<>(NAME); - PARSER.declareInt(CompositeAggregationBuilder::size, SIZE_FIELD_NAME); - PARSER.declareObject(CompositeAggregationBuilder::aggregateAfter, (parser, context) -> parser.map(), AFTER_FIELD_NAME); - PARSER.declareObjectArray(CompositeAggregationBuilder::setSources, + private static final Function> PARSER = name -> { + @SuppressWarnings("unchecked") + ConstructingObjectParser parser = new ConstructingObjectParser<>(NAME, a -> { + CompositeAggregationBuilder builder = new CompositeAggregationBuilder(name, (List>)a[0]); + if (a[1] != null) { + builder.size((Integer)a[1]); + } + if (a[2] != null) { + builder.aggregateAfter((Map)a[2]); + } + return builder; + }); + parser.declareObjectArray(ConstructingObjectParser.constructorArg(), (p, c) -> CompositeValuesSourceParserHelper.fromXContent(p), SOURCES_FIELD_NAME); - } + parser.declareInt(ConstructingObjectParser.optionalConstructorArg(), SIZE_FIELD_NAME); + parser.declareObject(ConstructingObjectParser.optionalConstructorArg(), (p, context) -> p.map(), AFTER_FIELD_NAME); + return parser; + }; + public static CompositeAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { - return PARSER.parse(parser, new CompositeAggregationBuilder(aggregationName), null); + return PARSER.apply(aggregationName).parse(parser, null); } private List> sources; private Map after; private int size = 10; - private CompositeAggregationBuilder(String name) { - this(name, null); - } - - public CompositeAggregationBuilder(String name, List> sources) { super(name); + validateSources(sources); this.sources = sources; } @@ -116,11 +126,6 @@ public String getType() { return NAME; } - private CompositeAggregationBuilder setSources(List> sources) { - this.sources = sources; - return this; - } - /** * Gets the list of {@link CompositeValuesSourceBuilder} for this aggregation. */ @@ -167,6 +172,28 @@ private AggregatorFactory checkParentIsNullOrNested(AggregatorFactory fact } } + private static void validateSources(List> sources) { + if (sources == null || sources.isEmpty()) { + throw new IllegalArgumentException("Composite [" + SOURCES_FIELD_NAME.getPreferredName() + "] cannot be null or empty"); + } + + Set names = new HashSet<>(); + Set duplicates = new HashSet<>(); + sources.forEach(source -> { + if (source == null) { + throw new IllegalArgumentException("Composite source cannot be null"); + } + boolean unique = names.add(source.name()); + if (unique == false) { + duplicates.add(source.name()); + } + }); + + if (duplicates.size() > 0) { + throw new IllegalArgumentException("Composite source names must be unique, found duplicates: " + duplicates); + } + } + @Override protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subfactoriesBuilder) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java index 52cff012b6473..02c53f3bd4164 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java @@ -965,7 +965,7 @@ public void testMultiValuedWithKeywordLongAndDouble() throws Exception { Arrays.asList( new TermsValuesSourceBuilder("keyword").field("keyword"), new TermsValuesSourceBuilder("long").field("long"), - new TermsValuesSourceBuilder("long").field("double") + new TermsValuesSourceBuilder("double").field("double") ) ).aggregateAfter(createAfterKey("keyword", "z", "long", 100L, "double", 0.4d)) , (result) -> { @@ -1641,6 +1641,38 @@ public void testRandomInts() throws IOException { testRandomTerms("price", () -> randomInt(), (v) -> ((Number) v).intValue()); } + public void testDuplicateNames() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> { + List> builders = new ArrayList<>(); + builders.add(new TermsValuesSourceBuilder("duplicate1").field("bar")); + builders.add(new TermsValuesSourceBuilder("duplicate1").field("baz")); + builders.add(new TermsValuesSourceBuilder("duplicate2").field("bar")); + builders.add(new TermsValuesSourceBuilder("duplicate2").field("baz")); + new CompositeAggregationBuilder("foo", builders); + }); + assertThat(e.getMessage(), equalTo("Composite source names must be unique, found duplicates: [duplicate2, duplicate1]")); + } + + public void testMissingSources() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> { + List> builders = new ArrayList<>(); + new CompositeAggregationBuilder("foo", builders); + }); + assertThat(e.getMessage(), equalTo("Composite [sources] cannot be null or empty")); + + e = expectThrows(IllegalArgumentException.class, () -> new CompositeAggregationBuilder("foo", null)); + assertThat(e.getMessage(), equalTo("Composite [sources] cannot be null or empty")); + } + + public void testNullSourceNonNullCollection() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> { + List> builders = new ArrayList<>(); + builders.add(null); + new CompositeAggregationBuilder("foo", builders); + }); + assertThat(e.getMessage(), equalTo("Composite source cannot be null")); + } + private , V extends Comparable> void testRandomTerms(String field, Supplier randomSupplier, Function transformKey) throws IOException { diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java index db03ed0fb66ee..67bea4ba6ccd8 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerStateTests.java @@ -46,7 +46,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.startsWith; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -290,7 +289,7 @@ public void testStateChangeMidTrigger() throws Exception { RollupIndexerJobStats stats = new RollupIndexerJobStats(); RollupIndexerJobStats spyStats = spy(stats); - RollupJobConfig config = mock(RollupJobConfig.class); + RollupJobConfig config = ConfigTestHelpers.randomRollupJobConfig(random()); // We call stats before a final state check, so this allows us to flip the state // and make sure the appropriate error is thrown From 07710f86468e1842a67bab39c07f65fa15dcd162 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Wed, 24 Apr 2019 13:25:40 -0400 Subject: [PATCH 173/260] Update 230_composite yaml skips after backport --- .../rest-api-spec/test/search.aggregation/230_composite.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 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 3ffcafe5e1e57..32593896fa43a 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 @@ -487,7 +487,7 @@ setup: --- "Missing source": - skip: - version: " - 7.99.99" # TODO change after backport + version: " - 7.0.99" reason: null/empty sources disallowed in 7.1 - do: @@ -516,7 +516,7 @@ setup: --- "Duplicate sources": - skip: - version: " - 7.99.99" # TODO change after backport + version: " - 7.0.99" reason: duplicate names disallowed in 7.1 - do: From 82702cf4c8ebc6007ba93c860ca3dcaee791c4fc Mon Sep 17 00:00:00 2001 From: Dawid Weiss Date: Wed, 24 Apr 2019 20:15:53 +0200 Subject: [PATCH 174/260] Corrected typo (annotations -> assertions). (#41481) --- TESTING.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/TESTING.asciidoc b/TESTING.asciidoc index 3229b7e768137..8daaecfdf1d99 100644 --- a/TESTING.asciidoc +++ b/TESTING.asciidoc @@ -71,7 +71,7 @@ password: `elastic-password`. ==== Other useful arguments In order to start a node with a different max heap space add: `-Dtests.heap.size=4G` -In order to disable annotations add: `-Dtests.asserts=false` +In order to disable assertions add: `-Dtests.asserts=false` In order to set an Elasticsearch setting, provide a setting with the following prefix: `-Dtests.es.` === Test case filtering. From c3313962b4b1a8870729206752839a52e453a6b9 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 24 Apr 2019 17:38:50 -0400 Subject: [PATCH 175/260] Unmute testCloseWhileRelocatingShards Relates #39588 --- .../indices/state/CloseWhileRelocatingShardsIT.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java index de53e8cb756de..71db668c29f26 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java @@ -40,6 +40,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.StubbableTransport; import org.elasticsearch.transport.TransportService; @@ -86,7 +87,8 @@ protected int maximumNumberOfShards() { return 3; } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/39588") + @TestLogging("_root_:DEBUG,org.elasticsearch.indices.recovery:TRACE,org.elasticsearch.action.admin.indices.close:TRACE," + + "org.elasticsearch.cluster.metadata.MetaDataIndexStateService:DEBUG") public void testCloseWhileRelocatingShards() throws Exception { final String[] indices = new String[randomIntBetween(3, 5)]; final Map docsPerIndex = new HashMap<>(); From 168187002766e4955b1c6ab9b2b2605a1e118402 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Thu, 25 Apr 2019 05:36:52 +0200 Subject: [PATCH 176/260] Remove stale integrations in docs (#40764) Removed all integrations in documentation, who have not seen a commit for more than three years when the website returned an error. --- docs/plugins/integrations.asciidoc | 39 +----------------------------- 1 file changed, 1 insertion(+), 38 deletions(-) diff --git a/docs/plugins/integrations.asciidoc b/docs/plugins/integrations.asciidoc index efbc83f2f8f87..a107527cc8acf 100644 --- a/docs/plugins/integrations.asciidoc +++ b/docs/plugins/integrations.asciidoc @@ -108,13 +108,9 @@ releases 2.0 and later do not support rivers. * https://camel.apache.org/elasticsearch.html[Apache Camel Integration]: An Apache camel component to integrate Elasticsearch -* https://metacpan.org/release/Catmandu-Store-ElasticSearch[Catmanadu]: +* https://metacpan.org/pod/Catmandu::Store::ElasticSearch[Catmandu]: An Elasticsearch backend for the Catmandu framework. -* https://github.com/tlrx/elasticsearch-test[elasticsearch-test]: - Elasticsearch Java annotations for unit testing with - http://www.junit.org/[JUnit] - * https://github.com/FriendsOfSymfony/FOSElasticaBundle[FOSElasticaBundle]: Symfony2 Bundle wrapping Elastica. @@ -127,9 +123,6 @@ releases 2.0 and later do not support rivers. * http://hibernate.org/search/[Hibernate Search] Integration with Hibernate ORM, from the Hibernate team. Automatic synchronization of write operations, yet exposes full Elasticsearch capabilities for queries. Can return either Elasticsearch native or re-map queries back into managed entities loaded within transaction from the reference database. -* https://github.com/cleverage/play2-elasticsearch[play2-elasticsearch]: - Elasticsearch module for Play Framework 2.x - * https://github.com/spring-projects/spring-data-elasticsearch[Spring Data Elasticsearch]: Spring Data implementation for Elasticsearch @@ -159,25 +152,9 @@ releases 2.0 and later do not support rivers. [float] ==== Supported by the community: -* https://github.com/anchor/nagios-plugin-elasticsearch[check_elasticsearch]: - An Elasticsearch availability and performance monitoring plugin for - Nagios. - * https://github.com/radu-gheorghe/check-es[check-es]: Nagios/Shinken plugins for checking on Elasticsearch -* https://github.com/mattweber/es2graphite[es2graphite]: - Send cluster and indices stats and status to Graphite for monitoring and graphing. - - -* https://itunes.apple.com/us/app/elasticocean/id955278030?ls=1&mt=8[ElasticOcean]: - Elasticsearch & DigitalOcean iOS Real-Time Monitoring tool to keep an eye on DigitalOcean Droplets or Elasticsearch instances or both of them on-a-go. - -* https://github.com/rbramley/Opsview-elasticsearch[opsview-elasticsearch]: - Opsview plugin written in Perl for monitoring Elasticsearch - -* https://scoutapp.com[Scout]: Provides plugins for monitoring Elasticsearch https://scoutapp.com/plugin_urls/1331-elasticsearch-node-status[nodes], https://scoutapp.com/plugin_urls/1321-elasticsearch-cluster-status[clusters], and https://scoutapp.com/plugin_urls/1341-elasticsearch-index-status[indices]. - * http://sematext.com/spm/index.html[SPM for Elasticsearch]: Performance monitoring with live charts showing cluster and node stats, integrated alerts, email reports, etc. @@ -190,23 +167,9 @@ releases 2.0 and later do not support rivers. [float] ==== Supported by the community: -* https://github.com/kodcu/pes[Pes]: - A pluggable elastic JavaScript query DSL builder for Elasticsearch - * https://www.wireshark.org/[Wireshark]: Protocol dissection for HTTP and the transport protocol * https://www.itemsapi.com/[ItemsAPI]: Search backend for mobile and web - -These projects appear to have been abandoned: - -* http://www.github.com/neogenix/daikon[daikon]: - Daikon Elasticsearch CLI - -* https://github.com/fullscale/dangle[dangle]: - A set of AngularJS directives that provide common visualizations for Elasticsearch based on - D3. -* https://github.com/OlegKunitsyn/eslogd[eslogd]: - Linux daemon that replicates events to a central Elasticsearch server in realtime From cc48427e054866cf12d304b43e8992b56fa6d1ec Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Thu, 25 Apr 2019 08:06:55 +0200 Subject: [PATCH 177/260] Improve accuracy for Geo Centroid Aggregation (#41033) keeps the partial results as doubles and uses Kahan summation to help reduce floating point errors. --- .../metrics/geocentroid-aggregation.asciidoc | 10 ++-- .../metrics/GeoCentroidAggregator.java | 60 +++++++++++-------- .../metrics/InternalGeoCentroid.java | 18 ++++-- .../metrics/GeoCentroidAggregatorTests.java | 4 +- 4 files changed, 55 insertions(+), 37 deletions(-) diff --git a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc index af3274c1c09fe..9ce69a3dd395e 100644 --- a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc @@ -58,8 +58,8 @@ The response for the above aggregation: "aggregations": { "centroid": { "location": { - "lat": 51.009829603135586, - "lon": 3.9662130642682314 + "lat": 51.00982965203002, + "lon": 3.9662131341174245 }, "count": 6 } @@ -111,8 +111,8 @@ The response for the above aggregation: "doc_count": 3, "centroid": { "location": { - "lat": 52.371655642054975, - "lon": 4.9095632415264845 + "lat": 52.371655656024814, + "lon": 4.909563297405839 }, "count": 3 } @@ -123,7 +123,7 @@ The response for the above aggregation: "centroid": { "location": { "lat": 48.86055548675358, - "lon": 2.331694420427084 + "lon": 2.3316944623366 }, "count": 2 } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java index f0f570ebaced6..414679f6e2e42 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.LongArray; import org.elasticsearch.index.fielddata.MultiGeoPointValues; import org.elasticsearch.search.aggregations.Aggregator; @@ -42,7 +43,7 @@ */ final class GeoCentroidAggregator extends MetricsAggregator { private final ValuesSource.GeoPoint valuesSource; - private LongArray centroids; + private DoubleArray lonSum, lonCompensations, latSum, latCompensations; private LongArray counts; GeoCentroidAggregator(String name, SearchContext context, Aggregator parent, @@ -52,7 +53,10 @@ final class GeoCentroidAggregator extends MetricsAggregator { this.valuesSource = valuesSource; if (valuesSource != null) { final BigArrays bigArrays = context.bigArrays(); - centroids = bigArrays.newLongArray(1, true); + lonSum = bigArrays.newDoubleArray(1, true); + lonCompensations = bigArrays.newDoubleArray(1, true); + latSum = bigArrays.newDoubleArray(1, true); + latCompensations = bigArrays.newDoubleArray(1, true); counts = bigArrays.newLongArray(1, true); } } @@ -67,33 +71,41 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - centroids = bigArrays.grow(centroids, bucket + 1); + latSum = bigArrays.grow(latSum, bucket + 1); + lonSum = bigArrays.grow(lonSum, bucket + 1); + lonCompensations = bigArrays.grow(lonCompensations, bucket + 1); + latCompensations = bigArrays.grow(latCompensations, bucket + 1); counts = bigArrays.grow(counts, bucket + 1); if (values.advanceExact(doc)) { final int valueCount = values.docValueCount(); - double[] pt = new double[2]; - // get the previously accumulated number of counts - long prevCounts = counts.get(bucket); // increment by the number of points for this document counts.increment(bucket, valueCount); - // get the previous GeoPoint if a moving avg was - // computed - if (prevCounts > 0) { - final long mortonCode = centroids.get(bucket); - pt[0] = InternalGeoCentroid.decodeLongitude(mortonCode); - pt[1] = InternalGeoCentroid.decodeLatitude(mortonCode); - } - // update the moving average + // Compute the sum of double values with Kahan summation algorithm which is more + // accurate than naive summation. + double sumLat = latSum.get(bucket); + double compensationLat = latCompensations.get(bucket); + double sumLon = lonSum.get(bucket); + double compensationLon = lonCompensations.get(bucket); + + // update the sum for (int i = 0; i < valueCount; ++i) { GeoPoint value = values.nextValue(); - pt[0] = pt[0] + (value.getLon() - pt[0]) / ++prevCounts; - pt[1] = pt[1] + (value.getLat() - pt[1]) / prevCounts; + //latitude + double correctedLat = value.getLat() - compensationLat; + double newSumLat = sumLat + correctedLat; + compensationLat = (newSumLat - sumLat) - correctedLat; + sumLat = newSumLat; + //longitude + double correctedLon = value.getLon() - compensationLon; + double newSumLon = sumLon + correctedLon; + compensationLon = (newSumLon - sumLon) - correctedLon; + sumLon = newSumLon; } - // TODO: we do not need to interleave the lat and lon - // bits here - // should we just store them contiguously? - centroids.set(bucket, InternalGeoCentroid.encodeLatLon(pt[1], pt[0])); + lonSum.set(bucket, sumLon); + lonCompensations.set(bucket, compensationLon); + latSum.set(bucket, sumLat); + latCompensations.set(bucket, compensationLat); } } }; @@ -101,14 +113,12 @@ public void collect(int doc, long bucket) throws IOException { @Override public InternalAggregation buildAggregation(long bucket) { - if (valuesSource == null || bucket >= centroids.size()) { + if (valuesSource == null || bucket >= counts.size()) { return buildEmptyAggregation(); } final long bucketCount = counts.get(bucket); - final long mortonCode = centroids.get(bucket); final GeoPoint bucketCentroid = (bucketCount > 0) - ? new GeoPoint(InternalGeoCentroid.decodeLatitude(mortonCode), - InternalGeoCentroid.decodeLongitude(mortonCode)) + ? new GeoPoint(latSum.get(bucket) / bucketCount, lonSum.get(bucket) / bucketCount) : null; return new InternalGeoCentroid(name, bucketCentroid , bucketCount, pipelineAggregators(), metaData()); } @@ -120,6 +130,6 @@ public InternalAggregation buildEmptyAggregation() { @Override public void doClose() { - Releasables.close(centroids, counts); + Releasables.close(latSum, latCompensations, lonSum, lonCompensations, counts); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java index d5d537ab66e5a..b1ca571e7134b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalGeoCentroid.java @@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.metrics; import org.apache.lucene.geo.GeoEncodingUtils; +import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.io.stream.StreamInput; @@ -69,8 +70,13 @@ public InternalGeoCentroid(StreamInput in) throws IOException { super(in); count = in.readVLong(); if (in.readBoolean()) { - final long hash = in.readLong(); - centroid = new GeoPoint(decodeLatitude(hash), decodeLongitude(hash)); + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { + centroid = new GeoPoint(in.readDouble(), in.readDouble()); + } else { + final long hash = in.readLong(); + centroid = new GeoPoint(decodeLatitude(hash), decodeLongitude(hash)); + } + } else { centroid = null; } @@ -81,8 +87,12 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeVLong(count); if (centroid != null) { out.writeBoolean(true); - // should we just write lat and lon separately? - out.writeLong(encodeLatLon(centroid.lat(), centroid.lon())); + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { + out.writeDouble(centroid.lat()); + out.writeDouble(centroid.lon()); + } else { + out.writeLong(encodeLatLon(centroid.lat(), centroid.lon())); + } } else { out.writeBoolean(false); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorTests.java index 303ed65f44856..4acae1764953a 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregatorTests.java @@ -29,8 +29,6 @@ import org.elasticsearch.index.mapper.GeoPointFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.search.aggregations.AggregatorTestCase; -import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder; -import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroid; import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper; import org.elasticsearch.test.geo.RandomGeoGenerator; @@ -38,7 +36,7 @@ public class GeoCentroidAggregatorTests extends AggregatorTestCase { - private static final double GEOHASH_TOLERANCE = 1E-4D; + private static final double GEOHASH_TOLERANCE = 1E-6D; public void testEmpty() throws Exception { try (Directory dir = newDirectory(); From 5c25b93e8bf86c185ab0377c852767a5f9b1dc40 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Wed, 24 Apr 2019 23:39:40 -0700 Subject: [PATCH 178/260] Upgrade hamcrest to 2.1 (#41464) hamcrest has some improvements in newer versions, like FileMatchers that make assertions regarding file exists cleaner. This commit upgrades to the latest version of hamcrest so we can start using new and improved matchers. --- buildSrc/version.properties | 2 +- client/rest-high-level/build.gradle | 1 - client/rest/build.gradle | 2 +- client/sniffer/build.gradle | 1 - client/test/build.gradle | 2 +- client/transport/build.gradle | 2 +- distribution/tools/launchers/build.gradle | 2 +- libs/core/build.gradle | 2 +- libs/nio/build.gradle | 2 +- libs/secure-sm/build.gradle | 2 +- libs/ssl-config/build.gradle | 2 +- libs/x-content/build.gradle | 2 +- qa/vagrant/build.gradle | 3 +-- .../elasticsearch/packaging/util/FileMatcher.java | 3 +-- test/framework/build.gradle | 12 ++---------- x-pack/plugin/security/cli/build.gradle | 3 --- x-pack/transport-client/build.gradle | 2 +- 17 files changed, 15 insertions(+), 30 deletions(-) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index c723cb1c9e514..d08086603bb59 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -37,7 +37,7 @@ httpcore = 4.4.11 httpasyncclient = 4.1.4 commonslogging = 1.1.3 commonscodec = 1.11 -hamcrest = 1.3 +hamcrest = 2.1 securemock = 1.2 mocksocket = 1.2 diff --git a/client/rest-high-level/build.gradle b/client/rest-high-level/build.gradle index 420bd6d7414f4..a5035a70bcee2 100644 --- a/client/rest-high-level/build.gradle +++ b/client/rest-high-level/build.gradle @@ -61,7 +61,6 @@ dependencies { testCompile "org.elasticsearch.test:framework:${version}" testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" //this is needed to make RestHighLevelClientTests#testApiNamingConventions work from IDEs testCompile "org.elasticsearch:rest-api-spec:${version}" // Needed for serialization tests: diff --git a/client/rest/build.gradle b/client/rest/build.gradle index 7bbcb1df85694..ee0317457118d 100644 --- a/client/rest/build.gradle +++ b/client/rest/build.gradle @@ -47,7 +47,7 @@ dependencies { testCompile "org.elasticsearch.client:test:${version}" testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" testCompile "org.elasticsearch:securemock:${versions.securemock}" testCompile "org.elasticsearch:mocksocket:${versions.mocksocket}" } diff --git a/client/sniffer/build.gradle b/client/sniffer/build.gradle index fffc1b711b25c..2f9eeca6020c4 100644 --- a/client/sniffer/build.gradle +++ b/client/sniffer/build.gradle @@ -45,7 +45,6 @@ dependencies { testCompile "org.elasticsearch.client:test:${version}" testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" testCompile "org.elasticsearch:securemock:${versions.securemock}" testCompile "org.elasticsearch:mocksocket:${versions.mocksocket}" } diff --git a/client/test/build.gradle b/client/test/build.gradle index faf5fb7bddf46..184606e360791 100644 --- a/client/test/build.gradle +++ b/client/test/build.gradle @@ -27,7 +27,7 @@ dependencies { compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" compile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" compile "junit:junit:${versions.junit}" - compile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + compile "org.hamcrest:hamcrest:${versions.hamcrest}" } forbiddenApisMain { diff --git a/client/transport/build.gradle b/client/transport/build.gradle index e0292cd557438..c1e4503445bb7 100644 --- a/client/transport/build.gradle +++ b/client/transport/build.gradle @@ -32,7 +32,7 @@ dependencies { compile "org.elasticsearch.plugin:rank-eval-client:${version}" testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" } dependencyLicenses { diff --git a/distribution/tools/launchers/build.gradle b/distribution/tools/launchers/build.gradle index b7b12170f66e1..e60902cc591ec 100644 --- a/distribution/tools/launchers/build.gradle +++ b/distribution/tools/launchers/build.gradle @@ -24,7 +24,7 @@ dependencies { compile parent.project('java-version-checker') testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" } archivesBaseName = 'elasticsearch-launchers' diff --git a/libs/core/build.gradle b/libs/core/build.gradle index c6abf288230b5..86778c3d23744 100644 --- a/libs/core/build.gradle +++ b/libs/core/build.gradle @@ -34,7 +34,7 @@ publishing { dependencies { testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" if (isEclipse == false || project.path == ":libs:core-tests") { testCompile("org.elasticsearch.test:framework:${version}") { diff --git a/libs/nio/build.gradle b/libs/nio/build.gradle index f6a6ff652450f..66436bb040e18 100644 --- a/libs/nio/build.gradle +++ b/libs/nio/build.gradle @@ -34,7 +34,7 @@ dependencies { testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" if (isEclipse == false || project.path == ":libs:nio-tests") { testCompile("org.elasticsearch.test:framework:${version}") { diff --git a/libs/secure-sm/build.gradle b/libs/secure-sm/build.gradle index d9a6e30b83aab..bbd44afc70a6d 100644 --- a/libs/secure-sm/build.gradle +++ b/libs/secure-sm/build.gradle @@ -34,7 +34,7 @@ dependencies { testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" if (isEclipse == false || project.path == ":libs:secure-sm-tests") { testCompile("org.elasticsearch.test:framework:${version}") { diff --git a/libs/ssl-config/build.gradle b/libs/ssl-config/build.gradle index 6c59e9ddb15b5..860cdcd9e6fc8 100644 --- a/libs/ssl-config/build.gradle +++ b/libs/ssl-config/build.gradle @@ -29,7 +29,7 @@ dependencies { testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" } if (isEclipse) { diff --git a/libs/x-content/build.gradle b/libs/x-content/build.gradle index af8d5f20d3af7..0e99d80da1e00 100644 --- a/libs/x-content/build.gradle +++ b/libs/x-content/build.gradle @@ -42,7 +42,7 @@ dependencies { testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" if (isEclipse == false || project.path == ":libs:x-content-tests") { testCompile("org.elasticsearch.test:framework:${version}") { diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index ac0bfe78aadd3..7c342436dd3ca 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -26,8 +26,7 @@ plugins { dependencies { compile "junit:junit:${versions.junit}" - compile "org.hamcrest:hamcrest-core:${versions.hamcrest}" - compile "org.hamcrest:hamcrest-library:${versions.hamcrest}" + compile "org.hamcrest:hamcrest:${versions.hamcrest}" compile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" compile "org.apache.httpcomponents:httpcore:${versions.httpcore}" diff --git a/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/FileMatcher.java b/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/FileMatcher.java index 34bae68f97f3e..f6e598b5a0d55 100644 --- a/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/FileMatcher.java +++ b/qa/vagrant/src/main/java/org/elasticsearch/packaging/util/FileMatcher.java @@ -30,12 +30,11 @@ import java.util.Objects; import java.util.Set; +import static java.nio.file.attribute.PosixFilePermissions.fromString; import static org.elasticsearch.packaging.util.FileUtils.getBasicFileAttributes; import static org.elasticsearch.packaging.util.FileUtils.getFileOwner; import static org.elasticsearch.packaging.util.FileUtils.getPosixFileAttributes; -import static java.nio.file.attribute.PosixFilePermissions.fromString; - /** * Asserts that a file at a path matches its status as Directory/File, and its owner. If on a posix system, also matches the permission * set is what we expect. diff --git a/test/framework/build.gradle b/test/framework/build.gradle index e4a72f9da5fc5..686756c6e53ba 100644 --- a/test/framework/build.gradle +++ b/test/framework/build.gradle @@ -25,7 +25,7 @@ dependencies { compile "org.elasticsearch:elasticsearch-cli:${version}" compile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" compile "junit:junit:${versions.junit}" - compile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + compile "org.hamcrest:hamcrest:${versions.hamcrest}" compile "org.apache.lucene:lucene-test-framework:${versions.lucene}" compile "org.apache.lucene:lucene-codecs:${versions.lucene}" compile "commons-logging:commons-logging:${versions.commonslogging}" @@ -52,15 +52,7 @@ thirdPartyAudit.ignoreMissingClasses ( 'javax.servlet.ServletContextListener', 'org.apache.avalon.framework.logger.Logger', 'org.apache.log.Hierarchy', - 'org.apache.log.Logger', - // we intentionally exclude the ant tasks because people were depending on them from their tests!!!!!!! - 'org.apache.tools.ant.BuildException', - 'org.apache.tools.ant.DirectoryScanner', - 'org.apache.tools.ant.Task', - 'org.apache.tools.ant.types.FileSet', - 'org.easymock.EasyMock', - 'org.easymock.IArgumentMatcher', - 'org.jmock.core.Constraint' + 'org.apache.log.Logger' ) test { diff --git a/x-pack/plugin/security/cli/build.gradle b/x-pack/plugin/security/cli/build.gradle index 19a8d11dd6f0c..00321c77808cd 100644 --- a/x-pack/plugin/security/cli/build.gradle +++ b/x-pack/plugin/security/cli/build.gradle @@ -11,9 +11,6 @@ dependencies { compile "org.bouncycastle:bcpkix-jdk15on:${versions.bouncycastle}" compile "org.bouncycastle:bcprov-jdk15on:${versions.bouncycastle}" testImplementation 'com.google.jimfs:jimfs:1.1' - testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" - testCompile 'org.elasticsearch:securemock:1.2' testCompile "org.elasticsearch.test:framework:${version}" testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') } diff --git a/x-pack/transport-client/build.gradle b/x-pack/transport-client/build.gradle index 90e3f96418f08..d764ef897447a 100644 --- a/x-pack/transport-client/build.gradle +++ b/x-pack/transport-client/build.gradle @@ -12,7 +12,7 @@ dependencies { compile "org.elasticsearch.client:transport:${version}" testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" - testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" + testCompile "org.hamcrest:hamcrest:${versions.hamcrest}" } dependencyLicenses.enabled = false From df356f0269bda9af5436c2aa1c060a888a4c9f50 Mon Sep 17 00:00:00 2001 From: Przemyslaw Gomulka Date: Thu, 25 Apr 2019 09:11:05 +0200 Subject: [PATCH 179/260] Remove the test which is testing java and joda api (#41493) The test is testing the java time API and fails in case it hits daylight saving time changes. Java time has the right implementation and we don't need to test this. more details on how the test was affected by the DST change on this comment closes #39617 --- .../common/rounding/DateTimeUnitTests.java | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/common/rounding/DateTimeUnitTests.java b/server/src/test/java/org/elasticsearch/common/rounding/DateTimeUnitTests.java index 53f48becd300b..6c92d3d1f9a19 100644 --- a/server/src/test/java/org/elasticsearch/common/rounding/DateTimeUnitTests.java +++ b/server/src/test/java/org/elasticsearch/common/rounding/DateTimeUnitTests.java @@ -19,11 +19,6 @@ package org.elasticsearch.common.rounding; import org.elasticsearch.test.ESTestCase; -import org.joda.time.DateTimeZone; - -import java.time.Instant; -import java.time.ZoneId; -import java.time.ZonedDateTime; import static org.elasticsearch.common.rounding.DateTimeUnit.DAY_OF_MONTH; import static org.elasticsearch.common.rounding.DateTimeUnit.HOUR_OF_DAY; @@ -33,7 +28,6 @@ import static org.elasticsearch.common.rounding.DateTimeUnit.SECOND_OF_MINUTE; import static org.elasticsearch.common.rounding.DateTimeUnit.WEEK_OF_WEEKYEAR; import static org.elasticsearch.common.rounding.DateTimeUnit.YEAR_OF_CENTURY; -import static org.hamcrest.Matchers.is; public class DateTimeUnitTests extends ESTestCase { @@ -65,18 +59,4 @@ public void testEnumIds() { assertEquals(8, SECOND_OF_MINUTE.id()); assertEquals(SECOND_OF_MINUTE, DateTimeUnit.resolve((byte) 8)); } - - @AwaitsFix( bugUrl = "https://github.com/elastic/elasticsearch/issues/39617") - public void testConversion() { - long millis = randomLongBetween(0, Instant.now().toEpochMilli()); - DateTimeZone zone = randomDateTimeZone(); - ZoneId zoneId = zone.toTimeZone().toZoneId(); - - int offsetSeconds = zoneId.getRules().getOffset(Instant.ofEpochMilli(millis)).getTotalSeconds(); - long parsedMillisJavaTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(millis), zoneId) - .minusSeconds(offsetSeconds).toInstant().toEpochMilli(); - - long parsedMillisJodaTime = zone.convertLocalToUTC(millis, true); - assertThat(parsedMillisJavaTime, is(parsedMillisJodaTime)); - } } From 6236b3aee4c4ccf9a06ca94af4c4082b7fcf5cde Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Thu, 25 Apr 2019 12:02:09 +0300 Subject: [PATCH 180/260] Fix Has Privilege API check on restricted indices (#41226) The Has Privileges API allows to tap into the authorization process, to validate privileges without actually running the operations to be authorized. This commit fixes a bug, in which the Has Privilege API returned spurious results when checking for index privileges over restricted indices (currently .security, .security-6, .security-7). The actual authorization process is not affected by the bug. --- .../rest-api/security/has-privileges.asciidoc | 12 +- .../authz/permission/IndicesPermission.java | 49 ++++--- .../core/security/authz/permission/Role.java | 3 +- .../xpack/security/authz/RBACEngineTests.java | 125 ++++++++++++++++++ 4 files changed, 164 insertions(+), 25 deletions(-) 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 92f1081bc2b85..984a8e36f0a93 100644 --- a/x-pack/docs/en/rest-api/security/has-privileges.asciidoc +++ b/x-pack/docs/en/rest-api/security/has-privileges.asciidoc @@ -29,11 +29,13 @@ privilege is assigned to the user. `index`:: `names`::: (list) A list of indices. -`allow_restricted_indices`::: (boolean) If `names` contains internal restricted -that also have to be covered by the has-privilege check, then this has to be -set to `true`. By default this is `false` because restricted indices should -generaly not be "visible" to APIs. For most use cases it is safe to ignore -this parameter. +`allow_restricted_indices`::: (boolean) This needs to be set to `true` (default +is `false`) if using wildcards or regexps for patterns that cover restricted +indices. Implicitly, restricted indices do not match index patterns because +restricted indices usually have limited privileges and including them in +pattern tests would render most such tests `false`. If restricted indices are +explicitly included in the `names` list, privileges will be checked against +them regardless of the value of `allow_restricted_indices`. `privileges`::: (list) A list of the privileges that you want to check for the specified indices. diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java index 356e80c497522..aab6bab8110af 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java @@ -138,27 +138,40 @@ public ResourcePrivilegesMap checkResourcePrivileges(Set checkForIndexPa final ResourcePrivilegesMap.Builder resourcePrivilegesMapBuilder = ResourcePrivilegesMap.builder(); final Map predicateCache = new HashMap<>(); for (String forIndexPattern : checkForIndexPatterns) { - final Automaton checkIndexAutomaton = IndicesPermission.Group.buildIndexMatcherAutomaton(allowRestrictedIndices, - forIndexPattern); - Automaton allowedIndexPrivilegesAutomaton = null; - for (Group group : groups) { - final Automaton groupIndexAutomaton = predicateCache.computeIfAbsent(group, - g -> IndicesPermission.Group.buildIndexMatcherAutomaton(g.allowRestrictedIndices(), g.indices())); - if (Operations.subsetOf(checkIndexAutomaton, groupIndexAutomaton)) { - if (allowedIndexPrivilegesAutomaton != null) { - allowedIndexPrivilegesAutomaton = Automatons - .unionAndMinimize(Arrays.asList(allowedIndexPrivilegesAutomaton, group.privilege().getAutomaton())); + Automaton checkIndexAutomaton = Automatons.patterns(forIndexPattern); + if (false == allowRestrictedIndices && false == RestrictedIndicesNames.RESTRICTED_NAMES.contains(forIndexPattern)) { + checkIndexAutomaton = Automatons.minusAndMinimize(checkIndexAutomaton, RestrictedIndicesNames.NAMES_AUTOMATON); + } + if (false == Operations.isEmpty(checkIndexAutomaton)) { + Automaton allowedIndexPrivilegesAutomaton = null; + for (Group group : groups) { + final Automaton groupIndexAutomaton = predicateCache.computeIfAbsent(group, + g -> IndicesPermission.Group.buildIndexMatcherAutomaton(g.allowRestrictedIndices(), g.indices())); + if (Operations.subsetOf(checkIndexAutomaton, groupIndexAutomaton)) { + if (allowedIndexPrivilegesAutomaton != null) { + allowedIndexPrivilegesAutomaton = Automatons + .unionAndMinimize(Arrays.asList(allowedIndexPrivilegesAutomaton, group.privilege().getAutomaton())); + } else { + allowedIndexPrivilegesAutomaton = group.privilege().getAutomaton(); + } + } + } + for (String privilege : checkForPrivileges) { + IndexPrivilege indexPrivilege = IndexPrivilege.get(Collections.singleton(privilege)); + if (allowedIndexPrivilegesAutomaton != null + && Operations.subsetOf(indexPrivilege.getAutomaton(), allowedIndexPrivilegesAutomaton)) { + resourcePrivilegesMapBuilder.addResourcePrivilege(forIndexPattern, privilege, Boolean.TRUE); } else { - allowedIndexPrivilegesAutomaton = group.privilege().getAutomaton(); + resourcePrivilegesMapBuilder.addResourcePrivilege(forIndexPattern, privilege, Boolean.FALSE); } } - } - for (String privilege : checkForPrivileges) { - IndexPrivilege indexPrivilege = IndexPrivilege.get(Collections.singleton(privilege)); - if (allowedIndexPrivilegesAutomaton != null - && Operations.subsetOf(indexPrivilege.getAutomaton(), allowedIndexPrivilegesAutomaton)) { - resourcePrivilegesMapBuilder.addResourcePrivilege(forIndexPattern, privilege, Boolean.TRUE); - } else { + } else { + // the index pattern produced the empty automaton, presumably because the requested pattern expands exclusively inside the + // restricted indices namespace - a namespace of indices that are normally hidden when granting/checking privileges - and + // the pattern was not marked as `allowRestrictedIndices`. We try to anticipate this by considering _explicit_ restricted + // indices even if `allowRestrictedIndices` is false. + // TODO The `false` result is a _safe_ default but this is actually an error. Make it an error. + for (String privilege : checkForPrivileges) { resourcePrivilegesMapBuilder.addResourcePrivilege(forIndexPattern, privilege, Boolean.FALSE); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/Role.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/Role.java index 817a9e41eab71..207a5ab056709 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/Role.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/Role.java @@ -49,7 +49,6 @@ public class Role { this.runAs = Objects.requireNonNull(runAs); } - public String[] names() { return names; } @@ -116,7 +115,7 @@ public boolean checkIndicesAction(String action) { * @return an instance of {@link ResourcePrivilegesMap} */ public ResourcePrivilegesMap checkIndicesPrivileges(Set checkForIndexPatterns, boolean allowRestrictedIndices, - Set checkForPrivileges) { + Set checkForPrivileges) { return indices.checkResourcePrivileges(checkForIndexPatterns, allowRestrictedIndices, checkForPrivileges); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java index e43ca6bbc0b6f..5c2e964c743c6 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java @@ -48,6 +48,7 @@ import org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor; import org.elasticsearch.xpack.core.security.authz.privilege.ClusterPrivilege; import org.elasticsearch.xpack.core.security.authz.privilege.ConditionalClusterPrivileges.ManageApplicationPrivileges; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.core.security.authz.privilege.IndexPrivilege; import org.elasticsearch.xpack.core.security.authz.privilege.Privilege; import org.elasticsearch.xpack.core.security.user.User; @@ -493,6 +494,130 @@ public void testCheckingIndexPermissionsDefinedOnDifferentPatterns() throws Exce )); } + public void testCheckExplicitRestrictedIndexPermissions() throws Exception { + User user = new User(randomAlphaOfLengthBetween(4, 12)); + Authentication authentication = mock(Authentication.class); + when(authentication.getUser()).thenReturn(user); + final boolean restrictedIndexPermission = randomBoolean(); + final boolean restrictedMonitorPermission = randomBoolean(); + Role role = Role.builder("role") + .add(FieldPermissions.DEFAULT, null, IndexPrivilege.INDEX, restrictedIndexPermission, ".sec*") + .add(FieldPermissions.DEFAULT, null, IndexPrivilege.MONITOR, restrictedMonitorPermission, ".security*") + .build(); + RBACAuthorizationInfo authzInfo = new RBACAuthorizationInfo(role, null); + + String explicitRestrictedIndex = randomFrom(RestrictedIndicesNames.RESTRICTED_NAMES); + HasPrivilegesResponse response = hasPrivileges(RoleDescriptor.IndicesPrivileges.builder() + .indices(new String[] {".secret-non-restricted", explicitRestrictedIndex}) + .privileges("index", "monitor") + .allowRestrictedIndices(false) // explicit false for test + .build(), authentication, authzInfo, Collections.emptyList(), Strings.EMPTY_ARRAY); + assertThat(response.isCompleteMatch(), is(false)); + assertThat(response.getIndexPrivileges(), Matchers.iterableWithSize(2)); + assertThat(response.getIndexPrivileges(), containsInAnyOrder( + ResourcePrivileges.builder(".secret-non-restricted") // matches ".sec*" but not ".security*" + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", false).map()).build(), + ResourcePrivileges.builder(explicitRestrictedIndex) // matches both ".sec*" and ".security*" + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", restrictedIndexPermission).put("monitor", restrictedMonitorPermission).map()).build())); + + explicitRestrictedIndex = randomFrom(RestrictedIndicesNames.RESTRICTED_NAMES); + response = hasPrivileges(RoleDescriptor.IndicesPrivileges.builder() + .indices(new String[] {".secret-non-restricted", explicitRestrictedIndex}) + .privileges("index", "monitor") + .allowRestrictedIndices(true) // explicit true for test + .build(), authentication, authzInfo, Collections.emptyList(), Strings.EMPTY_ARRAY); + assertThat(response.isCompleteMatch(), is(false)); + assertThat(response.getIndexPrivileges(), Matchers.iterableWithSize(2)); + assertThat(response.getIndexPrivileges(), containsInAnyOrder( + ResourcePrivileges.builder(".secret-non-restricted") // matches ".sec*" but not ".security*" + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", false).map()).build(), + ResourcePrivileges.builder(explicitRestrictedIndex) // matches both ".sec*" and ".security*" + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", restrictedIndexPermission).put("monitor", restrictedMonitorPermission).map()).build())); + } + + public void testCheckRestrictedIndexWildcardPermissions() throws Exception { + User user = new User(randomAlphaOfLengthBetween(4, 12)); + Authentication authentication = mock(Authentication.class); + when(authentication.getUser()).thenReturn(user); + Role role = Role.builder("role") + .add(FieldPermissions.DEFAULT, null, IndexPrivilege.INDEX, false, ".sec*") + .add(FieldPermissions.DEFAULT, null, IndexPrivilege.MONITOR, true, ".security*") + .build(); + RBACAuthorizationInfo authzInfo = new RBACAuthorizationInfo(role, null); + + HasPrivilegesResponse response = hasPrivileges(RoleDescriptor.IndicesPrivileges.builder() + .indices(".sec*", ".security*") + .privileges("index", "monitor") + .build(), authentication, authzInfo, Collections.emptyList(), Strings.EMPTY_ARRAY); + assertThat(response.isCompleteMatch(), is(false)); + assertThat(response.getIndexPrivileges(), Matchers.iterableWithSize(2)); + assertThat(response.getIndexPrivileges(), containsInAnyOrder( + ResourcePrivileges.builder(".sec*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", false).map()).build(), + ResourcePrivileges.builder(".security*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", true).map()).build() + )); + + response = hasPrivileges(RoleDescriptor.IndicesPrivileges.builder() + .indices(".sec*", ".security*") + .privileges("index", "monitor") + .allowRestrictedIndices(true) + .build(), authentication, authzInfo, Collections.emptyList(), Strings.EMPTY_ARRAY); + assertThat(response.isCompleteMatch(), is(false)); + assertThat(response.getIndexPrivileges(), Matchers.iterableWithSize(2)); + assertThat(response.getIndexPrivileges(), containsInAnyOrder( + ResourcePrivileges.builder(".sec*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", false).put("monitor", false).map()).build(), + ResourcePrivileges.builder(".security*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", false).put("monitor", true).map()).build() + )); + + role = Role.builder("role") + .add(FieldPermissions.DEFAULT, null, IndexPrivilege.INDEX, true, ".sec*") + .add(FieldPermissions.DEFAULT, null, IndexPrivilege.MONITOR, false, ".security*") + .build(); + authzInfo = new RBACAuthorizationInfo(role, null); + + response = hasPrivileges(RoleDescriptor.IndicesPrivileges.builder() + .indices(".sec*", ".security*") + .privileges("index", "monitor") + .build(), authentication, authzInfo, Collections.emptyList(), Strings.EMPTY_ARRAY); + assertThat(response.isCompleteMatch(), is(false)); + assertThat(response.getIndexPrivileges(), Matchers.iterableWithSize(2)); + assertThat(response.getIndexPrivileges(), containsInAnyOrder( + ResourcePrivileges.builder(".sec*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", false).map()).build(), + ResourcePrivileges.builder(".security*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", true).map()).build() + )); + + response = hasPrivileges(RoleDescriptor.IndicesPrivileges.builder() + .indices(".sec*", ".security*") + .privileges("index", "monitor") + .allowRestrictedIndices(true) + .build(), authentication, authzInfo, Collections.emptyList(), Strings.EMPTY_ARRAY); + assertThat(response.isCompleteMatch(), is(false)); + assertThat(response.getIndexPrivileges(), Matchers.iterableWithSize(2)); + assertThat(response.getIndexPrivileges(), containsInAnyOrder( + ResourcePrivileges.builder(".sec*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", false).map()).build(), + ResourcePrivileges.builder(".security*") + .addPrivileges(MapBuilder.newMapBuilder(new LinkedHashMap()) + .put("index", true).put("monitor", false).map()).build() + )); + } + public void testCheckingApplicationPrivilegesOnDifferentApplicationsAndResources() throws Exception { List privs = new ArrayList<>(); final ApplicationPrivilege app1Read = defineApplicationPrivilege(privs, "app1", "read", "data:read/*"); From 9efc853aa668e285ede733d37b6fc7a0f4b02041 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Thu, 25 Apr 2019 12:04:35 +0100 Subject: [PATCH 181/260] Revert "Clean up clusters between tests (#41187)" This reverts commit 78e7f4a94c4a633100c86c02d073f7f6a26444b5. --- .../org/elasticsearch/gradle/test/ClusterFormationTasks.groovy | 2 -- 1 file changed, 2 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 1b5eb26f9d322..274a705ab8add 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -936,8 +936,6 @@ class ClusterFormationTasks { } doLast { project.delete(node.pidFile) - // Large tests can exhaust disk space, clean up on stop, but leave the data dir as some tests reuse it - project.delete(project.fileTree(node.baseDir).minus(project.fileTree(node.dataDir))) } } } From f26addc03c63c0c32ceba5c8538630d4d3c26252 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 25 Apr 2019 13:09:44 +0200 Subject: [PATCH 182/260] Update resiliency status page for 7.0 (#41522) Marks 2 items as done: - Documents indexed during a network partition cannot be uniquely identified - Replicas can fall out of sync when a primary shard fails --- docs/resiliency/index.asciidoc | 71 ++++++++++++++++++++-------------- 1 file changed, 41 insertions(+), 30 deletions(-) diff --git a/docs/resiliency/index.asciidoc b/docs/resiliency/index.asciidoc index 71a87ef57b424..70533b4e8fd86 100644 --- a/docs/resiliency/index.asciidoc +++ b/docs/resiliency/index.asciidoc @@ -21,7 +21,7 @@ been made and current in-progress work. We’ve also listed some historical improvements throughout this page to provide the full context. If you’re interested in more on how we approach ensuring resiliency in -Elasticsearch, you may be interested in Igor Motov’s recent talk +Elasticsearch, you may be interested in Igor Motov’s talk http://www.elastic.co/videos/improving-elasticsearch-resiliency[Improving Elasticsearch Resiliency]. You may also be interested in our blog post @@ -102,20 +102,6 @@ space. The following issues have been identified: Other safeguards are tracked in the meta-issue {GIT}11511[#11511]. -[float] -=== The _version field may not uniquely identify document content during a network partition (STATUS: ONGOING) - -When a primary has been partitioned away from the cluster there is a short period of time until it detects this. During that time it will continue -indexing writes locally, thereby updating document versions. When it tries to replicate the operation, however, it will discover that it is -partitioned away. It won't acknowledge the write and will wait until the partition is resolved to negotiate with the master on how to proceed. -The master will decide to either fail any replicas which failed to index the operations on the primary or tell the primary that it has to -step down because a new primary has been chosen in the meantime. Since the old primary has already written documents, clients may already have read from -the old primary before it shuts itself down. The version numbers of these reads may not be unique if the new primary has already accepted -writes for the same document (see {GIT}19269[#19269]). - -We are currently implementing Sequence numbers {GIT}10708[#10708] which better track primary changes. Sequence numbers thus provide a basis -for uniquely identifying writes even in the presence of network partitions and will replace `_version` in operations that require this. - [float] === Relocating shards omitted by reporting infrastructure (STATUS: ONGOING) @@ -136,24 +122,49 @@ We have ported the known scenarios in the Jepsen blogs that check loss of acknow The new tests are run continuously in our testing farm and are passing. We are also working on running Jepsen independently to verify that no failures are found. -[float] -=== Replicas can fall out of sync when a primary shard fails (STATUS: ONGOING) +== Completed -When a primary shard fails, a replica shard will be promoted to be the -primary shard. If there is more than one replica shard, it is possible -for the remaining replicas to be out of sync with the new primary -shard. This is caused by operations that were in-flight when the primary -shard failed and may not have been processed on all replica -shards. Currently, the discrepancies are not repaired on primary -promotion but instead would be repaired if replica shards are relocated -(e.g., from hot to cold nodes); this does mean that the length of time -which replicas can be out of sync with the primary shard is -unbounded. Sequence numbers {GIT}10708[#10708] will provide a mechanism -for syncing the remaining replicas with the newly-promoted primary +[float] +=== Documents indexed during a network partition cannot be uniquely identified (STATUS: DONE, v7.0.0) + +When a primary has been partitioned away from the cluster there is a short +period of time until it detects this. During that time it will continue +indexing writes locally, thereby updating document versions. When it tries +to replicate the operation, however, it will discover that it is partitioned +away. It won't acknowledge the write and will wait until the partition is +resolved to negotiate with the master on how to proceed. The master will +decide to either fail any replicas which failed to index the operations on +the primary or tell the primary that it has to step down because a new primary +has been chosen in the meantime. Since the old primary has already written +documents, clients may already have read from the old primary before it shuts +itself down. The `_version` field of these reads may not uniquely identify the +document's version if the new primary has already accepted writes for the same +document (see {GIT}19269[#19269]). + +The Sequence numbers infrastructure {GIT}10708[#10708] has introduced more +precise ways for tracking primary changes. This new infrastructure therefore +provides a way for uniquely identifying documents using their primary term +and sequence number fields, even in the presence of network partitions, and +has been used to replace the `_version` field in operations that require +uniquely identifying the document, such as optimistic concurrency control. + +[float] +=== Replicas can fall out of sync when a primary shard fails (STATUS: DONE, v7.0.0) + +When a primary shard fails, a replica shard will be promoted to be the primary +shard. If there is more than one replica shard, it is possible for the +remaining replicas to be out of sync with the new primary shard. This is caused +by operations that were in-flight when the primary shard failed and may not +have been processed on all replica shards. These discrepancies are not +repaired on primary promotion but instead delayed until replica shards are +relocated (e.g., from hot to cold nodes); this means that the length of time +in which replicas can be out of sync with the primary shard is unbounded. + +Sequence numbers {GIT}10708[#10708] provide a mechanism for identifying +the discrepancies between shard copies at the document level, which allows +to efficiently sync up the remaining replicas with the newly-promoted primary shard. -== Completed - [float] === Repeated network partitions can cause cluster state updates to be lost (STATUS: DONE, v7.0.0) From 9bf8b5a442328447a49b9652b8d5121cb982ebea Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 25 Apr 2019 07:54:06 -0500 Subject: [PATCH 183/260] [ML] Adds progress reporting for transforms (#41278) * [ML] Adds progress reporting for transforms * fixing after master merge * Addressing PR comments * removing unused imports * Adjusting afterKey handling and percentage to be 100* * Making sure it is a linked hashmap for serialization * removing unused import * addressing PR comments * removing unused import * simplifying code, only storing total docs and decrementing * adjusting for rewrite * removing initial progress gathering from executor --- .../DataFrameTransformProgress.java | 94 ++++ .../transforms/DataFrameTransformState.java | 47 +- .../DataFrameTransformStateAndStats.java | 2 +- .../client/DataFrameTransformIT.java | 5 + .../DataFrameTransformProgressTests.java | 55 +++ .../DataFrameTransformStateAndStatsTests.java | 6 +- .../DataFrameTransformStateTests.java | 11 +- .../hlrc/DataFrameTransformProgressTests.java | 57 +++ .../hlrc/DataFrameTransformStateTests.java | 12 +- .../DataFrameTransformDocumentationIT.java | 10 +- .../dataframe/get_data_frame_stats.asciidoc | 4 +- .../DataFrameTransformProgress.java | 135 ++++++ .../transforms/DataFrameTransformState.java | 66 ++- .../DataFrameTransformStateAndStats.java | 2 +- .../transforms/pivot/SingleGroupSource.java | 6 + .../dataframe/utils/ExceptionsHelper.java | 22 + .../DataFrameTransformProgressTests.java | 64 +++ .../DataFrameTransformStateTests.java | 5 +- .../qa/single-node-tests/build.gradle | 3 +- .../DataFrameGetAndGetStatsIT.java | 42 ++ .../DataFrameTransformProgressIT.java | 189 ++++++++ ...ransportStartDataFrameTransformAction.java | 67 ++- .../transforms/DataFrameIndexer.java | 21 +- ...FrameTransformPersistentTasksExecutor.java | 128 +++++- .../transforms/DataFrameTransformTask.java | 434 ++++++++++++------ .../transforms/TransformProgressGatherer.java | 53 +++ .../transforms/DataFrameIndexerTests.java | 6 + .../test/data_frame/transforms_stats.yml | 2 + 28 files changed, 1284 insertions(+), 264 deletions(-) create mode 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgress.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgressTests.java create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformProgressTests.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgress.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/ExceptionsHelper.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgressTests.java create mode 100644 x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java create mode 100644 x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/TransformProgressGatherer.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgress.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgress.java new file mode 100644 index 0000000000000..a4177a3348729 --- /dev/null +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgress.java @@ -0,0 +1,94 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.dataframe.transforms; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class DataFrameTransformProgress { + + public static final ParseField TOTAL_DOCS = new ParseField("total_docs"); + public static final ParseField DOCS_REMAINING = new ParseField("docs_remaining"); + public static final ParseField PERCENT_COMPLETE = new ParseField("percent_complete"); + + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "data_frame_transform_progress", + true, + a -> new DataFrameTransformProgress((Long) a[0], (Long)a[1], (Double)a[2])); + + static { + PARSER.declareLong(constructorArg(), TOTAL_DOCS); + PARSER.declareLong(optionalConstructorArg(), DOCS_REMAINING); + PARSER.declareDouble(optionalConstructorArg(), PERCENT_COMPLETE); + } + + public static DataFrameTransformProgress fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + private final long totalDocs; + private final long remainingDocs; + private final double percentComplete; + + public DataFrameTransformProgress(long totalDocs, Long remainingDocs, double percentComplete) { + this.totalDocs = totalDocs; + this.remainingDocs = remainingDocs == null ? totalDocs : remainingDocs; + this.percentComplete = percentComplete; + } + + public double getPercentComplete() { + return percentComplete; + } + + public long getTotalDocs() { + return totalDocs; + } + + public long getRemainingDocs() { + return remainingDocs; + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (other == null || other.getClass() != getClass()) { + return false; + } + + DataFrameTransformProgress that = (DataFrameTransformProgress) other; + return Objects.equals(this.remainingDocs, that.remainingDocs) + && Objects.equals(this.totalDocs, that.totalDocs) + && Objects.equals(this.percentComplete, that.percentComplete); + } + + @Override + public int hashCode(){ + return Objects.hash(remainingDocs, totalDocs, percentComplete); + } +} diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java index 6bbc7a00b1b05..352cbfb67fcde 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformState.java @@ -23,16 +23,14 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import java.util.Objects; -import java.util.SortedMap; -import java.util.TreeMap; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -44,33 +42,25 @@ public class DataFrameTransformState { private static final ParseField CURRENT_POSITION = new ParseField("current_position"); private static final ParseField CHECKPOINT = new ParseField("checkpoint"); private static final ParseField REASON = new ParseField("reason"); + private static final ParseField PROGRESS = new ParseField("progress"); @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("data_frame_transform_state", true, args -> new DataFrameTransformState((DataFrameTransformTaskState) args[0], (IndexerState) args[1], - (HashMap) args[2], + (Map) args[2], (long) args[3], - (String) args[4])); + (String) args[4], + (DataFrameTransformProgress) args[5])); static { - PARSER.declareField(constructorArg(), - p -> DataFrameTransformTaskState.fromString(p.text()), - TASK_STATE, - ObjectParser.ValueType.STRING); - PARSER.declareField(constructorArg(), p -> IndexerState.fromString(p.text()), INDEXER_STATE, ObjectParser.ValueType.STRING); - PARSER.declareField(optionalConstructorArg(), p -> { - if (p.currentToken() == XContentParser.Token.START_OBJECT) { - return p.map(); - } - if (p.currentToken() == XContentParser.Token.VALUE_NULL) { - return null; - } - throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); - }, CURRENT_POSITION, ObjectParser.ValueType.VALUE_OBJECT_ARRAY); + PARSER.declareField(constructorArg(), p -> DataFrameTransformTaskState.fromString(p.text()), TASK_STATE, ValueType.STRING); + PARSER.declareField(constructorArg(), p -> IndexerState.fromString(p.text()), INDEXER_STATE, ValueType.STRING); + PARSER.declareField(optionalConstructorArg(), (p, c) -> p.mapOrdered(), CURRENT_POSITION, ValueType.OBJECT); PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), CHECKPOINT); PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), REASON); + PARSER.declareField(optionalConstructorArg(), DataFrameTransformProgress::fromXContent, PROGRESS, ValueType.OBJECT); } public static DataFrameTransformState fromXContent(XContentParser parser) throws IOException { @@ -80,19 +70,22 @@ public static DataFrameTransformState fromXContent(XContentParser parser) throws private final DataFrameTransformTaskState taskState; private final IndexerState indexerState; private final long checkpoint; - private final SortedMap currentPosition; + private final Map currentPosition; private final String reason; + private final DataFrameTransformProgress progress; public DataFrameTransformState(DataFrameTransformTaskState taskState, IndexerState indexerState, @Nullable Map position, long checkpoint, - @Nullable String reason) { + @Nullable String reason, + @Nullable DataFrameTransformProgress progress) { this.taskState = taskState; this.indexerState = indexerState; - this.currentPosition = position == null ? null : Collections.unmodifiableSortedMap(new TreeMap<>(position)); + this.currentPosition = position == null ? null : Collections.unmodifiableMap(new LinkedHashMap<>(position)); this.checkpoint = checkpoint; this.reason = reason; + this.progress = progress; } public IndexerState getIndexerState() { @@ -117,6 +110,11 @@ public String getReason() { return reason; } + @Nullable + public DataFrameTransformProgress getProgress() { + return progress; + } + @Override public boolean equals(Object other) { if (this == other) { @@ -132,13 +130,14 @@ public boolean equals(Object other) { return Objects.equals(this.taskState, that.taskState) && Objects.equals(this.indexerState, that.indexerState) && Objects.equals(this.currentPosition, that.currentPosition) && + Objects.equals(this.progress, that.progress) && this.checkpoint == that.checkpoint && Objects.equals(this.reason, that.reason); } @Override public int hashCode() { - return Objects.hash(taskState, indexerState, currentPosition, checkpoint, reason); + return Objects.hash(taskState, indexerState, currentPosition, checkpoint, reason, progress); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStats.java index 938563796ca38..9914a0e6331bc 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStats.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStats.java @@ -57,7 +57,7 @@ public static DataFrameTransformStateAndStats fromXContent(XContentParser parser private final DataFrameTransformCheckpointingInfo checkpointingInfo; public DataFrameTransformStateAndStats(String id, DataFrameTransformState state, DataFrameIndexerTransformStats stats, - DataFrameTransformCheckpointingInfo checkpointingInfo) { + DataFrameTransformCheckpointingInfo checkpointingInfo) { this.id = id; this.transformState = state; this.transformStats = stats; diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java index 3e564a86207ba..3d7f5e3dbcbbd 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java @@ -71,6 +71,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; @@ -360,6 +361,10 @@ public void testGetStats() throws Exception { assertEquals(DataFrameTransformTaskState.STARTED, stateAndStats.getTransformState().getTaskState()); assertEquals(null, stateAndStats.getTransformState().getReason()); assertNotEquals(zeroIndexerStats, stateAndStats.getTransformStats()); + assertNotNull(stateAndStats.getTransformState().getProgress()); + assertThat(stateAndStats.getTransformState().getProgress().getPercentComplete(), equalTo(100.0)); + assertThat(stateAndStats.getTransformState().getProgress().getTotalDocs(), greaterThan(0L)); + assertThat(stateAndStats.getTransformState().getProgress().getRemainingDocs(), equalTo(0L)); }); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgressTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgressTests.java new file mode 100644 index 0000000000000..573e2ffdbb957 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformProgressTests.java @@ -0,0 +1,55 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.dataframe.transforms; + +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; + +public class DataFrameTransformProgressTests extends ESTestCase { + + public void testFromXContent() throws IOException { + xContentTester(this::createParser, + DataFrameTransformProgressTests::randomInstance, + DataFrameTransformProgressTests::toXContent, + DataFrameTransformProgress::fromXContent) + .supportsUnknownFields(true) + .randomFieldsExcludeFilter(field -> field.startsWith("state")) + .test(); + } + + public static DataFrameTransformProgress randomInstance() { + long totalDocs = randomNonNegativeLong(); + Long docsRemaining = randomBoolean() ? null : randomLongBetween(0, totalDocs); + double percentComplete = totalDocs == 0 ? 1.0 : docsRemaining == null ? 0.0 : 100.0*(double)(totalDocs - docsRemaining)/totalDocs; + return new DataFrameTransformProgress(totalDocs, docsRemaining, percentComplete); + } + + public static void toXContent(DataFrameTransformProgress progress, XContentBuilder builder) throws IOException { + builder.startObject(); + builder.field(DataFrameTransformProgress.TOTAL_DOCS.getPreferredName(), progress.getTotalDocs()); + builder.field(DataFrameTransformProgress.DOCS_REMAINING.getPreferredName(), progress.getRemainingDocs()); + builder.field(DataFrameTransformProgress.PERCENT_COMPLETE.getPreferredName(), progress.getPercentComplete()); + builder.endObject(); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStatsTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStatsTests.java index 8862869910422..6ebdec5a69009 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStatsTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateAndStatsTests.java @@ -40,9 +40,9 @@ public void testFromXContent() throws IOException { public static DataFrameTransformStateAndStats randomInstance() { return new DataFrameTransformStateAndStats(randomAlphaOfLength(10), - DataFrameTransformStateTests.randomDataFrameTransformState(), - DataFrameIndexerTransformStatsTests.randomStats(), - DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo()); + DataFrameTransformStateTests.randomDataFrameTransformState(), + DataFrameIndexerTransformStatsTests.randomStats(), + DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo()); } public static void toXContent(DataFrameTransformStateAndStats stateAndStats, XContentBuilder builder) throws IOException { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java index 7d1d713a1271f..4ada50c20d219 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformStateTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.test.ESTestCase; import java.io.IOException; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; @@ -46,7 +46,8 @@ public static DataFrameTransformState randomDataFrameTransformState() { randomFrom(IndexerState.values()), randomPositionMap(), randomLongBetween(0,10), - randomBoolean() ? null : randomAlphaOfLength(10)); + randomBoolean() ? null : randomAlphaOfLength(10), + randomBoolean() ? null : DataFrameTransformProgressTests.randomInstance()); } public static void toXContent(DataFrameTransformState state, XContentBuilder builder) throws IOException { @@ -60,6 +61,10 @@ public static void toXContent(DataFrameTransformState state, XContentBuilder bui if (state.getReason() != null) { builder.field("reason", state.getReason()); } + if (state.getProgress() != null) { + builder.field("progress"); + DataFrameTransformProgressTests.toXContent(state.getProgress(), builder); + } builder.endObject(); } @@ -68,7 +73,7 @@ private static Map randomPositionMap() { return null; } int numFields = randomIntBetween(1, 5); - Map position = new HashMap<>(); + Map position = new LinkedHashMap<>(); for (int i = 0; i < numFields; i++) { Object value; if (randomBoolean()) { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformProgressTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformProgressTests.java new file mode 100644 index 0000000000000..be589a63248b8 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformProgressTests.java @@ -0,0 +1,57 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.dataframe.transforms.hlrc; + +import org.elasticsearch.client.AbstractResponseTestCase; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; + +import static org.hamcrest.Matchers.equalTo; + +public class DataFrameTransformProgressTests extends AbstractResponseTestCase< + DataFrameTransformProgress, + org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress> { + + public static DataFrameTransformProgress fromHlrc( + org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress instance) { + if (instance == null) { + return null; + } + return new DataFrameTransformProgress(instance.getTotalDocs(), instance.getRemainingDocs()); + } + + @Override + protected DataFrameTransformProgress createServerTestInstance() { + return DataFrameTransformStateTests.randomDataFrameTransformProgress(); + } + + @Override + protected org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress doParseToClientInstance(XContentParser parser) { + return org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress.fromXContent(parser); + } + + @Override + protected void assertInstances(DataFrameTransformProgress serverTestInstance, + org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress clientInstance) { + assertThat(serverTestInstance.getTotalDocs(), equalTo(clientInstance.getTotalDocs())); + assertThat(serverTestInstance.getRemainingDocs(), equalTo(clientInstance.getRemainingDocs())); + assertThat(serverTestInstance.getPercentComplete(), equalTo(clientInstance.getPercentComplete())); + } +} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java index 457c68d593e5b..4c80365bc539a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/hlrc/DataFrameTransformStateTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; @@ -40,7 +41,7 @@ public class DataFrameTransformStateTests extends AbstractHlrcXContentTestCase randomPosition() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java index b7d6967206c2c..daa9dc06fee0a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/DataFrameTransformDocumentationIT.java @@ -40,6 +40,7 @@ import org.elasticsearch.client.dataframe.StopDataFrameTransformResponse; import org.elasticsearch.client.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformTaskState; import org.elasticsearch.client.dataframe.transforms.DestConfig; @@ -465,18 +466,21 @@ public void testGetStats() throws IOException, InterruptedException { // tag::get-data-frame-transform-stats-response DataFrameTransformStateAndStats stateAndStats = - response.getTransformsStateAndStats().get(0); // <1> + response.getTransformsStateAndStats().get(0); // <1> DataFrameTransformTaskState taskState = stateAndStats.getTransformState().getTaskState(); // <2> IndexerState indexerState = - stateAndStats.getTransformState().getIndexerState(); // <3> + stateAndStats.getTransformState().getIndexerState(); // <3> DataFrameIndexerTransformStats transformStats = - stateAndStats.getTransformStats(); // <4> + stateAndStats.getTransformStats(); // <4> + DataFrameTransformProgress progress = + stateAndStats.getTransformState().getProgress(); // <5> // end::get-data-frame-transform-stats-response assertEquals(IndexerState.STOPPED, indexerState); assertEquals(DataFrameTransformTaskState.STOPPED, taskState); assertNotNull(transformStats); + assertNull(progress); } { // tag::get-data-frame-transform-stats-execute-listener diff --git a/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc b/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc index 2b377d22c815a..cdc6254a4e443 100644 --- a/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc +++ b/docs/java-rest/high-level/dataframe/get_data_frame_stats.asciidoc @@ -37,4 +37,6 @@ include-tagged::{doc-tests-file}[{api}-response] <1> The response contains a list of `DataFrameTransformStateAndStats` objects <2> The running state of the transform task e.g `started` <3> The running state of the transform indexer e.g `started`, `indexing`, etc. -<4> The transform progress statistics recording the number of documents indexed etc \ No newline at end of file +<4> The overall transform statistics recording the number of documents indexed etc. +<5> The progress of the current run in the transform. Supplies the number of docs left until the next checkpoint +and the total number of docs expected. \ No newline at end of file diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgress.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgress.java new file mode 100644 index 0000000000000..5b7346bca2a38 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgress.java @@ -0,0 +1,135 @@ +/* + * 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.core.dataframe.transforms; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class DataFrameTransformProgress implements Writeable, ToXContentObject { + + private static final ParseField TOTAL_DOCS = new ParseField("total_docs"); + private static final ParseField DOCS_REMAINING = new ParseField("docs_remaining"); + private static final String PERCENT_COMPLETE = "percent_complete"; + + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "data_frame_transform_progress", + true, + a -> new DataFrameTransformProgress((Long) a[0], (Long)a[1])); + + static { + PARSER.declareLong(constructorArg(), TOTAL_DOCS); + PARSER.declareLong(optionalConstructorArg(), DOCS_REMAINING); + } + + private final long totalDocs; + private long remainingDocs; + + public DataFrameTransformProgress(long totalDocs, Long remainingDocs) { + if (totalDocs < 0) { + throw new IllegalArgumentException("[total_docs] must be >0."); + } + this.totalDocs = totalDocs; + if (remainingDocs != null && remainingDocs < 0) { + throw new IllegalArgumentException("[docs_remaining] must be >0."); + } + this.remainingDocs = remainingDocs == null ? totalDocs : remainingDocs; + } + + public DataFrameTransformProgress(DataFrameTransformProgress otherProgress) { + this.totalDocs = otherProgress.totalDocs; + this.remainingDocs = otherProgress.remainingDocs; + } + + public DataFrameTransformProgress(StreamInput in) throws IOException { + this.totalDocs = in.readLong(); + this.remainingDocs = in.readLong(); + } + + public Double getPercentComplete() { + if (totalDocs == 0) { + return 100.0; + } + long docsRead = totalDocs - remainingDocs; + if (docsRead < 0) { + return 100.0; + } + return 100.0*(double)docsRead/totalDocs; + } + + public long getTotalDocs() { + return totalDocs; + } + + public long getRemainingDocs() { + return remainingDocs; + } + + public void resetRemainingDocs() { + this.remainingDocs = totalDocs; + } + + public void docsProcessed(long docsProcessed) { + assert docsProcessed >= 0; + if (docsProcessed > remainingDocs) { + remainingDocs = 0; + } else { + remainingDocs -= docsProcessed; + } + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (other == null || other.getClass() != getClass()) { + return false; + } + + DataFrameTransformProgress that = (DataFrameTransformProgress) other; + return Objects.equals(this.remainingDocs, that.remainingDocs) && Objects.equals(this.totalDocs, that.totalDocs); + } + + @Override + public int hashCode(){ + return Objects.hash(remainingDocs, totalDocs); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(totalDocs); + out.writeLong(remainingDocs); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(TOTAL_DOCS.getPreferredName(), totalDocs); + builder.field(DOCS_REMAINING.getPreferredName(), remainingDocs); + builder.field(PERCENT_COMPLETE, getPercentComplete()); + builder.endObject(); + return builder; + } + + @Override + public String toString() { + return Strings.toString(this, true, true); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformState.java index 76d68cf178c19..bc1b710cd2e6f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformState.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformState.java @@ -12,7 +12,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.persistent.PersistentTaskState; @@ -22,10 +22,9 @@ import java.io.IOException; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.Map; import java.util.Objects; -import java.util.SortedMap; -import java.util.TreeMap; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -35,10 +34,11 @@ public class DataFrameTransformState implements Task.Status, PersistentTaskState private final DataFrameTransformTaskState taskState; private final IndexerState indexerState; + private final DataFrameTransformProgress progress; private final long checkpoint; @Nullable - private final SortedMap currentPosition; + private final Map currentPosition; @Nullable private final String reason; @@ -47,6 +47,7 @@ public class DataFrameTransformState implements Task.Status, PersistentTaskState private static final ParseField CURRENT_POSITION = new ParseField("current_position"); private static final ParseField CHECKPOINT = new ParseField("checkpoint"); private static final ParseField REASON = new ParseField("reason"); + private static final ParseField PROGRESS = new ParseField("progress"); @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, @@ -55,53 +56,40 @@ public class DataFrameTransformState implements Task.Status, PersistentTaskState (IndexerState) args[1], (Map) args[2], (long) args[3], - (String) args[4])); + (String) args[4], + (DataFrameTransformProgress) args[5])); static { - PARSER.declareField(constructorArg(), p -> { - if (p.currentToken() == XContentParser.Token.VALUE_STRING) { - return DataFrameTransformTaskState.fromString(p.text()); - } - throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); - }, TASK_STATE, ObjectParser.ValueType.STRING); - PARSER.declareField(constructorArg(), p -> { - if (p.currentToken() == XContentParser.Token.VALUE_STRING) { - return IndexerState.fromString(p.text()); - } - throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); - - }, INDEXER_STATE, ObjectParser.ValueType.STRING); - PARSER.declareField(optionalConstructorArg(), p -> { - if (p.currentToken() == XContentParser.Token.START_OBJECT) { - return p.map(); - } - if (p.currentToken() == XContentParser.Token.VALUE_NULL) { - return null; - } - throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); - }, CURRENT_POSITION, ObjectParser.ValueType.VALUE_OBJECT_ARRAY); + PARSER.declareField(constructorArg(), p -> DataFrameTransformTaskState.fromString(p.text()), TASK_STATE, ValueType.STRING); + PARSER.declareField(constructorArg(), p -> IndexerState.fromString(p.text()), INDEXER_STATE, ValueType.STRING); + PARSER.declareField(optionalConstructorArg(), XContentParser::mapOrdered, CURRENT_POSITION, ValueType.OBJECT); PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), CHECKPOINT); PARSER.declareString(optionalConstructorArg(), REASON); + PARSER.declareField(optionalConstructorArg(), DataFrameTransformProgress.PARSER::apply, PROGRESS, ValueType.OBJECT); } public DataFrameTransformState(DataFrameTransformTaskState taskState, IndexerState indexerState, @Nullable Map position, long checkpoint, - @Nullable String reason) { + @Nullable String reason, + @Nullable DataFrameTransformProgress progress) { this.taskState = taskState; this.indexerState = indexerState; - this.currentPosition = position == null ? null : Collections.unmodifiableSortedMap(new TreeMap<>(position)); + this.currentPosition = position == null ? null : Collections.unmodifiableMap(new LinkedHashMap<>(position)); this.checkpoint = checkpoint; this.reason = reason; + this.progress = progress; } public DataFrameTransformState(StreamInput in) throws IOException { taskState = DataFrameTransformTaskState.fromStream(in); indexerState = IndexerState.fromStream(in); - currentPosition = in.readBoolean() ? Collections.unmodifiableSortedMap(new TreeMap<>(in.readMap())) : null; + Map position = in.readMap(); + currentPosition = position == null ? null : Collections.unmodifiableMap(position); checkpoint = in.readLong(); reason = in.readOptionalString(); + progress = in.readOptionalWriteable(DataFrameTransformProgress::new); } public DataFrameTransformTaskState getTaskState() { @@ -120,6 +108,10 @@ public long getCheckpoint() { return checkpoint; } + public DataFrameTransformProgress getProgress() { + return progress; + } + /** * Get the in-progress checkpoint * @@ -153,6 +145,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (reason != null) { builder.field(REASON.getPreferredName(), reason); } + if (progress != null) { + builder.field(PROGRESS.getPreferredName(), progress); + } builder.endObject(); return builder; } @@ -166,12 +161,10 @@ public String getWriteableName() { public void writeTo(StreamOutput out) throws IOException { taskState.writeTo(out); indexerState.writeTo(out); - out.writeBoolean(currentPosition != null); - if (currentPosition != null) { - out.writeMap(currentPosition); - } + out.writeMap(currentPosition); out.writeLong(checkpoint); out.writeOptionalString(reason); + out.writeOptionalWriteable(progress); } @Override @@ -190,12 +183,13 @@ public boolean equals(Object other) { Objects.equals(this.indexerState, that.indexerState) && Objects.equals(this.currentPosition, that.currentPosition) && this.checkpoint == that.checkpoint && - Objects.equals(this.reason, that.reason); + Objects.equals(this.reason, that.reason) && + Objects.equals(this.progress, that.progress); } @Override public int hashCode() { - return Objects.hash(taskState, indexerState, currentPosition, checkpoint, reason); + return Objects.hash(taskState, indexerState, currentPosition, checkpoint, reason, progress); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateAndStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateAndStats.java index 57171fca43f96..2a145ba260f4e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateAndStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateAndStats.java @@ -53,7 +53,7 @@ public static DataFrameTransformStateAndStats initialStateAndStats(String id) { public static DataFrameTransformStateAndStats initialStateAndStats(String id, DataFrameIndexerTransformStats indexerTransformStats) { return new DataFrameTransformStateAndStats(id, - new DataFrameTransformState(DataFrameTransformTaskState.STOPPED, IndexerState.STOPPED, null, 0L, null), + new DataFrameTransformState(DataFrameTransformTaskState.STOPPED, IndexerState.STOPPED, null, 0L, null, null), indexerTransformStats, DataFrameTransformCheckpointingInfo.EMPTY); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java index 0cdef0e4c3a96..0a4cf2579460e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/SingleGroupSource.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.core.dataframe.transforms.pivot; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -116,4 +117,9 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(field); } + + @Override + public String toString() { + return Strings.toString(this, true, true); + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/ExceptionsHelper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/ExceptionsHelper.java new file mode 100644 index 0000000000000..8bfd558b209e1 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/ExceptionsHelper.java @@ -0,0 +1,22 @@ +/* + * 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.core.dataframe.utils; + +/** + * Collection of methods to aid in creating and checking for exceptions. + */ +public class ExceptionsHelper { + /** + * A more REST-friendly Object.requireNonNull() + */ + public static T requireNonNull(T obj, String paramName) { + if (obj == null) { + throw new IllegalArgumentException("[" + paramName + "] must not be null."); + } + return obj; + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgressTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgressTests.java new file mode 100644 index 0000000000000..8339669057ec6 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformProgressTests.java @@ -0,0 +1,64 @@ +/* + * 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.core.dataframe.transforms; + +import org.elasticsearch.common.io.stream.Writeable.Reader; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class DataFrameTransformProgressTests extends AbstractSerializingDataFrameTestCase { + public static DataFrameTransformProgress randomDataFrameTransformProgress() { + long totalDocs = randomNonNegativeLong(); + return new DataFrameTransformProgress(totalDocs, randomBoolean() ? null : randomLongBetween(0, totalDocs)); + } + + @Override + protected DataFrameTransformProgress doParseInstance(XContentParser parser) throws IOException { + return DataFrameTransformProgress.PARSER.apply(parser, null); + } + + @Override + protected DataFrameTransformProgress createTestInstance() { + return randomDataFrameTransformProgress(); + } + + @Override + protected Reader instanceReader() { + return DataFrameTransformProgress::new; + } + + public void testPercentComplete() { + DataFrameTransformProgress progress = new DataFrameTransformProgress(0L, 100L); + assertThat(progress.getPercentComplete(), equalTo(100.0)); + + progress = new DataFrameTransformProgress(100L, 0L); + assertThat(progress.getPercentComplete(), equalTo(100.0)); + + progress = new DataFrameTransformProgress(100L, 10000L); + assertThat(progress.getPercentComplete(), equalTo(100.0)); + + progress = new DataFrameTransformProgress(100L, null); + assertThat(progress.getPercentComplete(), equalTo(0.0)); + + progress = new DataFrameTransformProgress(100L, 50L); + assertThat(progress.getPercentComplete(), closeTo(50.0, 0.000001)); + } + + public void testConstructor() { + IllegalArgumentException ex = + expectThrows(IllegalArgumentException.class, () -> new DataFrameTransformProgress(-1, null)); + assertThat(ex.getMessage(), equalTo("[total_docs] must be >0.")); + + ex = expectThrows(IllegalArgumentException.class, () -> new DataFrameTransformProgress(1L, -1L)); + assertThat(ex.getMessage(), equalTo("[docs_remaining] must be >0.")); + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateTests.java index 341faafdf12a5..c978978b0589b 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformStateTests.java @@ -16,6 +16,8 @@ import java.util.Map; import java.util.function.Predicate; +import static org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgressTests.randomDataFrameTransformProgress; + public class DataFrameTransformStateTests extends AbstractSerializingTestCase { public static DataFrameTransformState randomDataFrameTransformState() { @@ -23,7 +25,8 @@ public static DataFrameTransformState randomDataFrameTransformState() { randomFrom(IndexerState.values()), randomPosition(), randomLongBetween(0,10), - randomBoolean() ? null : randomAlphaOfLength(10)); + randomBoolean() ? null : randomAlphaOfLength(10), + randomBoolean() ? null : randomDataFrameTransformProgress()); } @Override diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/build.gradle b/x-pack/plugin/data-frame/qa/single-node-tests/build.gradle index eb241713360c9..11014c764e330 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/build.gradle +++ b/x-pack/plugin/data-frame/qa/single-node-tests/build.gradle @@ -2,7 +2,8 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile "org.elasticsearch.plugin:x-pack-core:${version}" + testCompile project(path: xpackModule('core'), configuration: 'default') + testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') testCompile project(path: xpackModule('data-frame'), configuration: 'runtime') } 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 b348a9cef60b7..8de2fc5961a6c 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 @@ -18,6 +18,7 @@ import java.util.Map; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase { @@ -83,6 +84,19 @@ public void testGetAndGetStats() throws Exception { stats = entityAsMap(client().performRequest(getRequest)); assertEquals(2, XContentMapValues.extractValue("count", stats)); + List> transformsStats = (List>)XContentMapValues.extractValue("transforms", stats); + // Verify that both transforms have valid stats + for (Map transformStats : transformsStats) { + Map stat = (Map)transformStats.get("stats"); + assertThat("documents_processed is not > 0.", ((Integer)stat.get("documents_processed")), greaterThan(0)); + assertThat("search_total is not > 0.", ((Integer)stat.get("search_total")), greaterThan(0)); + assertThat("pages_processed is not > 0.", ((Integer)stat.get("pages_processed")), greaterThan(0)); + Map progress = (Map)XContentMapValues.extractValue("state.progress", transformStats); + assertThat("total_docs is not 1000", progress.get("total_docs"), equalTo(1000)); + assertThat("docs_remaining is not 0", progress.get("docs_remaining"), equalTo(0)); + assertThat("percent_complete is not 100.0", progress.get("percent_complete"), equalTo(100.0)); + } + // only pivot_1 getRequest = createRequestWithAuth("GET", DATAFRAME_ENDPOINT + "pivot_1/_stats", authHeader); stats = entityAsMap(client().performRequest(getRequest)); @@ -134,4 +148,32 @@ public void testGetPersistedStatsWithoutTask() throws Exception { assertThat(((Integer)stat.get("pages_processed")), greaterThan(0)); } } + + @SuppressWarnings("unchecked") + public void testGetProgressStatsWithPivotQuery() throws Exception { + String transformId = "simpleStatsPivotWithQuery"; + String dataFrameIndex = "pivot_stats_reviews_user_id_above_20"; + String query = "\"match\": {\"user_id\": \"user_26\"}"; + createPivotReviewsTransform(transformId, dataFrameIndex, query); + startAndWaitForTransform(transformId, dataFrameIndex); + + // Alternate testing between admin and lowly user, as both should be able to get the configs and stats + String authHeader = randomFrom(BASIC_AUTH_VALUE_DATA_FRAME_USER, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN); + + Request getRequest = createRequestWithAuth("GET", DATAFRAME_ENDPOINT + "simpleStatsPivotWithQuery/_stats", authHeader); + Map stats = entityAsMap(client().performRequest(getRequest)); + assertEquals(1, XContentMapValues.extractValue("count", stats)); + List> transformsStats = (List>)XContentMapValues.extractValue("transforms", stats); + // Verify that the transform has stats and the total docs process matches the expected + for (Map transformStats : transformsStats) { + Map stat = (Map)transformStats.get("stats"); + assertThat("documents_processed is not > 0.", ((Integer)stat.get("documents_processed")), greaterThan(0)); + assertThat("search_total is not > 0.", ((Integer)stat.get("search_total")), greaterThan(0)); + assertThat("pages_processed is not > 0.", ((Integer)stat.get("pages_processed")), greaterThan(0)); + Map progress = (Map)XContentMapValues.extractValue("state.progress", transformStats); + assertThat("total_docs is not 37", progress.get("total_docs"), equalTo(37)); + assertThat("docs_remaining is not 0", progress.get("docs_remaining"), equalTo(0)); + assertThat("percent_complete is not 100.0", progress.get("percent_complete"), equalTo(100.0)); + } + } } diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java new file mode 100644 index 0000000000000..e32842af0cc9f --- /dev/null +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java @@ -0,0 +1,189 @@ +/* + * 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.integration; + +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.SecuritySettingsSourceField; +import org.elasticsearch.transport.Netty4Plugin; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.core.XPackClientPlugin; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; +import org.elasticsearch.xpack.core.dataframe.transforms.DestConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.QueryConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.SourceConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.AggregationConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.GroupConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.HistogramGroupSource; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.PivotConfig; +import org.elasticsearch.xpack.core.security.SecurityField; +import org.elasticsearch.xpack.dataframe.transforms.TransformProgressGatherer; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.xpack.dataframe.integration.DataFrameRestTestCase.REVIEWS_INDEX_NAME; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class DataFrameTransformProgressIT extends ESIntegTestCase { + + protected void createReviewsIndex() throws Exception { + final int numDocs = 1000; + + // create mapping + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.startObject("properties") + .startObject("timestamp") + .field("type", "date") + .endObject() + .startObject("user_id") + .field("type", "keyword") + .endObject() + .startObject("count") + .field("type", "integer") + .endObject() + .startObject("business_id") + .field("type", "keyword") + .endObject() + .startObject("stars") + .field("type", "integer") + .endObject() + .endObject(); + } + builder.endObject(); + CreateIndexResponse response = client().admin() + .indices() + .prepareCreate(REVIEWS_INDEX_NAME) + .addMapping("_doc", builder) + .get(); + assertThat(response.isAcknowledged(), is(true)); + } + + // create index + BulkRequestBuilder bulk = client().prepareBulk(REVIEWS_INDEX_NAME, "_doc"); + int day = 10; + for (int i = 0; i < numDocs; i++) { + long user = i % 28; + int stars = (i + 20) % 5; + long business = (i + 100) % 50; + int hour = 10 + (i % 13); + int min = 10 + (i % 49); + int sec = 10 + (i % 49); + + String date_string = "2017-01-" + day + "T" + hour + ":" + min + ":" + sec + "Z"; + + StringBuilder sourceBuilder = new StringBuilder(); + sourceBuilder.append("{\"user_id\":\"") + .append("user_") + .append(user) + .append("\",\"count\":") + .append(i) + .append(",\"business_id\":\"") + .append("business_") + .append(business) + .append("\",\"stars\":") + .append(stars) + .append(",\"timestamp\":\"") + .append(date_string) + .append("\"}"); + bulk.add(new IndexRequest().source(sourceBuilder.toString(), XContentType.JSON)); + + if (i % 50 == 0) { + BulkResponse response = client().bulk(bulk.request()).get(); + assertThat(response.buildFailureMessage(), response.hasFailures(), is(false)); + bulk = client().prepareBulk(REVIEWS_INDEX_NAME, "_doc"); + day += 1; + } + } + client().bulk(bulk.request()).get(); + client().admin().indices().prepareRefresh(REVIEWS_INDEX_NAME).get(); + } + + public void testGetProgress() throws Exception { + createReviewsIndex(); + SourceConfig sourceConfig = new SourceConfig(REVIEWS_INDEX_NAME); + DestConfig destConfig = new DestConfig("unnecessary"); + GroupConfig histgramGroupConfig = new GroupConfig(Collections.emptyMap(), + Collections.singletonMap("every_50", new HistogramGroupSource("count", 50.0))); + AggregatorFactories.Builder aggs = new AggregatorFactories.Builder(); + aggs.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); + AggregationConfig aggregationConfig = new AggregationConfig(Collections.emptyMap(), aggs); + PivotConfig pivotConfig = new PivotConfig(histgramGroupConfig, aggregationConfig); + DataFrameTransformConfig config = new DataFrameTransformConfig("get_progress_transform", + sourceConfig, + destConfig, + null, + pivotConfig); + + PlainActionFuture progressFuture = new PlainActionFuture<>(); + TransformProgressGatherer.getInitialProgress(client(), config, progressFuture); + + DataFrameTransformProgress progress = progressFuture.get(); + + assertThat(progress.getTotalDocs(), equalTo(1000L)); + assertThat(progress.getRemainingDocs(), equalTo(1000L)); + assertThat(progress.getPercentComplete(), equalTo(0.0)); + + + QueryConfig queryConfig = new QueryConfig(Collections.emptyMap(), QueryBuilders.termQuery("user_id", "user_26")); + pivotConfig = new PivotConfig(histgramGroupConfig, aggregationConfig); + sourceConfig = new SourceConfig(new String[]{REVIEWS_INDEX_NAME}, queryConfig); + config = new DataFrameTransformConfig("get_progress_transform", + sourceConfig, + destConfig, + null, + pivotConfig); + + + progressFuture = new PlainActionFuture<>(); + + TransformProgressGatherer.getInitialProgress(client(), config, progressFuture); + progress = progressFuture.get(); + + assertThat(progress.getTotalDocs(), equalTo(35L)); + assertThat(progress.getRemainingDocs(), equalTo(35L)); + assertThat(progress.getPercentComplete(), equalTo(0.0)); + + client().admin().indices().prepareDelete(REVIEWS_INDEX_NAME).get(); + } + + @Override + protected Settings externalClusterClientSettings() { + Settings.Builder builder = Settings.builder(); + builder.put(NetworkModule.TRANSPORT_TYPE_KEY, SecurityField.NAME4); + builder.put(SecurityField.USER_SETTING.getKey(), "x_pack_rest_user:" + SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING); + return builder.build(); + } + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(LocalStateCompositeXPackPlugin.class, Netty4Plugin.class); + } + + @Override + protected Collection> transportClientPlugins() { + return Arrays.asList(XPackClientPlugin.class, Netty4Plugin.class); + } +} diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java index d8fcd15921e16..1e65639a89f4e 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java @@ -95,21 +95,16 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, } final DataFrameTransform transformTask = createDataFrameTransform(request.getId(), threadPool); - // <4> Set the allocated task's state to STARTED - ActionListener> persistentTaskActionListener = ActionListener.wrap( - task -> { - waitForDataFrameTaskAllocated(task.getId(), - transformTask, - request.timeout(), - ActionListener.wrap( - taskAssigned -> ClientHelper.executeAsyncWithOrigin(client, - ClientHelper.DATA_FRAME_ORIGIN, - StartDataFrameTransformTaskAction.INSTANCE, - new StartDataFrameTransformTaskAction.Request(request.getId()), - ActionListener.wrap( - r -> listener.onResponse(new StartDataFrameTransformAction.Response(true)), - listener::onFailure)), - listener::onFailure)); + // <3> Wait for the allocated task's state to STARTED + ActionListener> newPersistentTaskActionListener = + ActionListener.wrap( + task -> { + waitForDataFrameTaskStarted(task.getId(), + transformTask, + request.timeout(), + ActionListener.wrap( + taskStarted -> listener.onResponse(new StartDataFrameTransformAction.Response(true)), + listener::onFailure)); }, listener::onFailure ); @@ -120,10 +115,11 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, PersistentTasksCustomMetaData.PersistentTask existingTask = getExistingTask(transformTask.getId(), state); if (existingTask == null) { + // Create the allocated task and wait for it to be started persistentTasksService.sendStartRequest(transformTask.getId(), DataFrameTransform.NAME, transformTask, - persistentTaskActionListener); + newPersistentTaskActionListener); } else { DataFrameTransformState transformState = (DataFrameTransformState)existingTask.getState(); if(transformState.getTaskState() == DataFrameTransformTaskState.FAILED && request.isForce() == false) { @@ -138,7 +134,26 @@ protected void masterOperation(StartDataFrameTransformAction.Request request, "Unable to start data frame transform [" + request.getId() + "] as it is in state [" + transformState.getTaskState() + "]", RestStatus.CONFLICT)); } else { - persistentTaskActionListener.onResponse(existingTask); + // If the task already exists but is not assigned to a node, something is weird + // return a failure that includes the current assignment explanation (if one exists) + if (existingTask.isAssigned() == false) { + String assignmentExplanation = "unknown reason"; + if (existingTask.getAssignment() != null) { + assignmentExplanation = existingTask.getAssignment().getExplanation(); + } + listener.onFailure(new ElasticsearchStatusException("Unable to start data frame transform [" + + request.getId() + "] as it is not assigned to a node, explanation: " + assignmentExplanation, + RestStatus.CONFLICT)); + return; + } + // If the task already exists and is assigned to a node, simply attempt to set it to start + ClientHelper.executeAsyncWithOrigin(client, + ClientHelper.DATA_FRAME_ORIGIN, + StartDataFrameTransformTaskAction.INSTANCE, + new StartDataFrameTransformTaskAction.Request(request.getId()), + ActionListener.wrap( + r -> listener.onResponse(new StartDataFrameTransformAction.Response(true)), + listener::onFailure)); } } }, @@ -269,10 +284,10 @@ public void onFailure(Exception e) { ); } - private void waitForDataFrameTaskAllocated(String taskId, - DataFrameTransform params, - TimeValue timeout, - ActionListener listener) { + private void waitForDataFrameTaskStarted(String taskId, + DataFrameTransform params, + TimeValue timeout, + ActionListener listener) { DataFramePredicate predicate = new DataFramePredicate(); persistentTasksService.waitForPersistentTaskCondition(taskId, predicate, timeout, new PersistentTasksService.WaitForPersistentTaskListener() { @@ -324,7 +339,15 @@ public boolean test(PersistentTasksCustomMetaData.PersistentTask persistentTa return true; } // We just want it assigned so we can tell it to start working - return assignment != null && assignment.isAssigned(); + return assignment != null && assignment.isAssigned() && isNotStopped(persistentTask); + } + + // checking for `isNotStopped` as the state COULD be marked as failed for any number of reasons + // But if it is in a failed state, _stats will show as much and give good reason to the user. + // If it is not able to be assigned to a node all together, we should just close the task completely + private boolean isNotStopped(PersistentTasksCustomMetaData.PersistentTask task) { + DataFrameTransformState state = (DataFrameTransformState)task.getState(); + return state != null && state.getTaskState().equals(DataFrameTransformTaskState.STOPPED) == false; } } } 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 823ccaff71b1f..5fde9a1cac60e 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 @@ -14,6 +14,7 @@ import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.Nullable; import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -23,6 +24,7 @@ import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; import org.elasticsearch.xpack.core.indexing.AsyncTwoPhaseIndexer; import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.indexing.IterationResult; @@ -64,6 +66,9 @@ public DataFrameIndexer(Executor executor, protected abstract Map getFieldMappings(); + @Nullable + protected abstract DataFrameTransformProgress getProgress(); + protected abstract void failIndexer(String message); public int getPageSize() { @@ -87,7 +92,7 @@ protected void onStart(long now, ActionListener listener) { } // if run for the 1st time, create checkpoint - if (getPosition() == null) { + if (initialRun()) { createCheckpoint(listener); } else { listener.onResponse(null); @@ -97,6 +102,10 @@ protected void onStart(long now, ActionListener listener) { } } + protected boolean initialRun() { + return getPosition() == null; + } + @Override protected void onFinish(ActionListener listener) { // reset the page size, so we do not memorize a low page size forever, the pagesize will be re-calculated on start @@ -106,8 +115,14 @@ protected void onFinish(ActionListener listener) { @Override protected IterationResult> doProcess(SearchResponse searchResponse) { final CompositeAggregation agg = searchResponse.getAggregations().get(COMPOSITE_AGGREGATION_NAME); - return new IterationResult<>(processBucketsToIndexRequests(agg).collect(Collectors.toList()), agg.afterKey(), - agg.getBuckets().isEmpty()); + long docsBeforeProcess = getStats().getNumDocuments(); + IterationResult> result = new IterationResult<>(processBucketsToIndexRequests(agg).collect(Collectors.toList()), + agg.afterKey(), + agg.getBuckets().isEmpty()); + if (getProgress() != null) { + getProgress().docsProcessed(getStats().getNumDocuments() - docsBeforeProcess); + } + return result; } /* 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 b6f38a5dd2355..708585a8dc309 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 @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.client.Client; import org.elasticsearch.common.Nullable; import org.elasticsearch.persistent.AllocatedPersistentTask; @@ -19,21 +20,31 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.dataframe.DataFrameField; +import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; +import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState; +import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.scheduler.SchedulerEngine; import org.elasticsearch.xpack.dataframe.DataFrame; 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.transforms.pivot.SchemaUtil; import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksExecutor { private static final Logger logger = LogManager.getLogger(DataFrameTransformPersistentTasksExecutor.class); + // The amount of time we wait for the cluster state to respond when being marked as failed + private static final int MARK_AS_FAILED_TIMEOUT_SEC = 90; private final Client client; private final DataFrameTransformsConfigManager transformsConfigManager; private final DataFrameTransformsCheckpointService dataFrameTransformsCheckpointService; @@ -58,36 +69,116 @@ public DataFrameTransformPersistentTasksExecutor(Client client, @Override protected void nodeOperation(AllocatedPersistentTask task, @Nullable DataFrameTransform params, PersistentTaskState state) { - DataFrameTransformTask buildTask = (DataFrameTransformTask) task; - SchedulerEngine.Job schedulerJob = new SchedulerEngine.Job( - DataFrameTransformTask.SCHEDULE_NAME + "_" + params.getId(), next()); - DataFrameTransformState transformState = (DataFrameTransformState) state; - if (transformState != null && transformState.getTaskState() == DataFrameTransformTaskState.FAILED) { - logger.warn("Tried to start failed transform [" + params.getId() + "] failure reason: " + transformState.getReason()); - return; - } - transformsConfigManager.getTransformStats(params.getId(), ActionListener.wrap( + final String transformId = params.getId(); + final DataFrameTransformTask buildTask = (DataFrameTransformTask) task; + final SchedulerEngine.Job schedulerJob = new SchedulerEngine.Job(DataFrameTransformTask.SCHEDULE_NAME + "_" + transformId, + next()); + final DataFrameTransformState transformState = (DataFrameTransformState) state; + + final DataFrameTransformTask.ClientDataFrameIndexerBuilder indexerBuilder = + new DataFrameTransformTask.ClientDataFrameIndexerBuilder() + .setAuditor(auditor) + .setClient(client) + .setIndexerState(transformState == null ? IndexerState.STOPPED : transformState.getIndexerState()) + .setInitialPosition(transformState == null ? null : transformState.getPosition()) + // If the state is `null` that means this is a "first run". We can safely assume the + // task will attempt to gather the initial progress information + // if we have state, this may indicate the previous execution node crashed, so we should attempt to retrieve + // the progress from state to keep an accurate measurement of our progress + .setProgress(transformState == null ? null : transformState.getProgress()) + .setTransformsCheckpointService(dataFrameTransformsCheckpointService) + .setTransformsConfigManager(transformsConfigManager) + .setTransformId(transformId); + + ActionListener startTaskListener = ActionListener.wrap( + response -> logger.info("Successfully completed and scheduled task in node operation"), + failure -> logger.error("Failed to start task ["+ transformId +"] in node operation", failure) + ); + + // <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( stats -> { - // Initialize with the previously recorded stats - buildTask.initializePreviousStats(stats); - scheduleTask(buildTask, schedulerJob, params.getId()); + indexerBuilder.setInitialStats(stats); + buildTask.initializeIndexer(indexerBuilder); + scheduleAndStartTask(buildTask, schedulerJob, startTaskListener); }, error -> { if (error instanceof ResourceNotFoundException == false) { logger.error("Unable to load previously persisted statistics for transform [" + params.getId() + "]", error); } - scheduleTask(buildTask, schedulerJob, params.getId()); + indexerBuilder.setInitialStats(new DataFrameIndexerTransformStats(transformId)); + buildTask.initializeIndexer(indexerBuilder); + scheduleAndStartTask(buildTask, schedulerJob, startTaskListener); + } + ); + + // <2> set fieldmappings for the indexer, get the previous stats (if they exist) + ActionListener> getFieldMappingsListener = ActionListener.wrap( + fieldMappings -> { + indexerBuilder.setFieldMappings(fieldMappings); + transformsConfigManager.getTransformStats(transformId, transformStatsActionListener); + }, + error -> { + String msg = DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_UNABLE_TO_GATHER_FIELD_MAPPINGS, + indexerBuilder.getTransformConfig().getDestination().getIndex()); + logger.error(msg, error); + markAsFailed(buildTask, msg); } - )); + ); + + // <1> Validate the transform, assigning it to the indexer, and get the field mappings + ActionListener getTransformConfigListener = ActionListener.wrap( + config -> { + if (config.isValid()) { + indexerBuilder.setTransformConfig(config); + SchemaUtil.getDestinationFieldMappings(client, config.getDestination().getIndex(), getFieldMappingsListener); + } else { + markAsFailed(buildTask, + DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_INVALID, transformId)); + } + }, + error -> { + String msg = DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId); + logger.error(msg, error); + markAsFailed(buildTask, msg); + } + ); + // <0> Get the transform config + transformsConfigManager.getTransformConfiguration(transformId, getTransformConfigListener); } - private void scheduleTask(DataFrameTransformTask buildTask, SchedulerEngine.Job schedulerJob, String id) { + private void markAsFailed(DataFrameTransformTask task, String reason) { + CountDownLatch latch = new CountDownLatch(1); + + task.markAsFailed(reason, new LatchedActionListener<>(ActionListener.wrap( + nil -> {}, + failure -> logger.error("Failed to set task [" + task.getTransformId() +"] to failed", failure) + ), latch)); + try { + latch.await(MARK_AS_FAILED_TIMEOUT_SEC, TimeUnit.SECONDS); + } catch (InterruptedException e) { + logger.error("Timeout waiting for task [" + task.getTransformId() + "] to be marked as failed in cluster state", e); + } + } + + private void scheduleAndStartTask(DataFrameTransformTask buildTask, + SchedulerEngine.Job schedulerJob, + ActionListener listener) { // Note that while the task is added to the scheduler here, the internal state will prevent // it from doing any work until the task is "started" via the StartTransform api schedulerEngine.register(buildTask); schedulerEngine.add(schedulerJob); - - logger.info("Data frame transform [" + id + "] created."); + logger.info("Data frame transform [{}] created.", buildTask.getTransformId()); + // If we are stopped, and it is an initial run, this means we have never been started, + // attempt to start the task + if (buildTask.getState().getTaskState().equals(DataFrameTransformTaskState.STOPPED) && buildTask.isInitialRun()) { + buildTask.start(listener); + } else { + logger.debug("No need to start task. Its current state is: {}", buildTask.getState().getIndexerState()); + listener.onResponse(new StartDataFrameTransformTaskAction.Response(true)); + } } static SchedulerEngine.Schedule next() { @@ -100,7 +191,6 @@ static SchedulerEngine.Schedule next() { protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId, PersistentTasksCustomMetaData.PersistentTask persistentTask, Map headers) { return new DataFrameTransformTask(id, type, action, parentTaskId, persistentTask.getParams(), - (DataFrameTransformState) persistentTask.getState(), client, transformsConfigManager, - dataFrameTransformsCheckpointService, schedulerEngine, auditor, threadPool, headers); + (DataFrameTransformState) persistentTask.getState(), schedulerEngine, auditor, threadPool, headers); } } 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 f142fc36179f7..15a555da48859 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 @@ -8,9 +8,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -25,27 +25,25 @@ 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; import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction.Response; import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; 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.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.DataFrameTransformsCheckpointService; import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager; -import org.elasticsearch.xpack.dataframe.transforms.pivot.SchemaUtil; import java.util.Arrays; import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -62,21 +60,20 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S private final DataFrameTransform transform; private final SchedulerEngine schedulerEngine; private final ThreadPool threadPool; - private final DataFrameIndexer indexer; private final DataFrameAuditor auditor; - private final DataFrameIndexerTransformStats previousStats; + private final Map initialPosition; + private final IndexerState initialIndexerState; + + private final SetOnce indexer = new SetOnce<>(); private final AtomicReference taskState; private final AtomicReference stateReason; // 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; - private final AtomicInteger failureCount; public DataFrameTransformTask(long id, String type, String action, TaskId parentTask, DataFrameTransform transform, - DataFrameTransformState state, Client client, DataFrameTransformsConfigManager transformsConfigManager, - DataFrameTransformsCheckpointService transformsCheckpointService, - SchedulerEngine schedulerEngine, DataFrameAuditor auditor, + DataFrameTransformState state, SchedulerEngine schedulerEngine, DataFrameAuditor auditor, ThreadPool threadPool, Map headers) { super(id, type, action, DataFrameField.PERSISTENT_TASK_DESCRIPTION_PREFIX + transform.getId(), parentTask, headers); this.transform = transform; @@ -107,13 +104,11 @@ public DataFrameTransformTask(long id, String type, String action, TaskId parent initialGeneration = state.getCheckpoint(); } - this.indexer = new ClientDataFrameIndexer(transform.getId(), transformsConfigManager, transformsCheckpointService, - new AtomicReference<>(initialState), initialPosition, client, auditor); + this.initialIndexerState = initialState; + this.initialPosition = initialPosition; this.currentCheckpoint = new AtomicLong(initialGeneration); - this.previousStats = new DataFrameIndexerTransformStats(transform.getId()); this.taskState = new AtomicReference<>(initialTaskState); this.stateReason = new AtomicReference<>(initialReason); - this.failureCount = new AtomicInteger(0); } public String getTransformId() { @@ -128,21 +123,36 @@ public Status getStatus() { return getState(); } + private DataFrameIndexer getIndexer() { + return indexer.get(); + } + public DataFrameTransformState getState() { - return new DataFrameTransformState( + if (getIndexer() == null) { + return new DataFrameTransformState( taskState.get(), - indexer.getState(), - indexer.getPosition(), + initialIndexerState, + initialPosition, currentCheckpoint.get(), - stateReason.get()); - } - - void initializePreviousStats(DataFrameIndexerTransformStats stats) { - previousStats.merge(stats); + stateReason.get(), + null); + } else { + return new DataFrameTransformState( + taskState.get(), + indexer.get().getState(), + indexer.get().getPosition(), + currentCheckpoint.get(), + stateReason.get(), + getIndexer().getProgress()); + } } public DataFrameIndexerTransformStats getStats() { - return new DataFrameIndexerTransformStats(previousStats).merge(indexer.getStats()); + if (getIndexer() == null) { + return new DataFrameIndexerTransformStats(getTransformId()); + } else { + return getIndexer().getStats(); + } } public long getCheckpoint() { @@ -155,15 +165,29 @@ public long getCheckpoint() { * @return checkpoint in progress or 0 if task/indexer is not active */ public long getInProgressCheckpoint() { - return indexer.getState().equals(IndexerState.INDEXING) ? currentCheckpoint.get() + 1L : 0; + if (getIndexer() == null) { + return 0; + } else { + return indexer.get().getState().equals(IndexerState.INDEXING) ? currentCheckpoint.get() + 1L : 0; + } } public boolean isStopped() { - return indexer.getState().equals(IndexerState.STOPPED); + IndexerState currentState = getIndexer() == null ? initialIndexerState : getIndexer().getState(); + return currentState.equals(IndexerState.STOPPED); + } + + boolean isInitialRun() { + return getIndexer() != null && getIndexer().initialRun(); } public synchronized void start(ActionListener listener) { - final IndexerState newState = indexer.start(); + if (getIndexer() == null) { + listener.onFailure(new ElasticsearchException("Task for transform [{}] not fully initialized. Try again later", + 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 [{}]", transform.getId(), newState)); @@ -171,14 +195,14 @@ public synchronized void start(ActionListener listener) { } stateReason.set(null); taskState.set(DataFrameTransformTaskState.STARTED); - failureCount.set(0); final DataFrameTransformState state = new DataFrameTransformState( DataFrameTransformTaskState.STARTED, IndexerState.STOPPED, - indexer.getPosition(), + getIndexer().getPosition(), currentCheckpoint.get(), - null); + null, + getIndexer().getProgress()); logger.info("Updating state for data frame transform [{}] to [{}]", transform.getId(), state.toString()); persistStateToClusterState(state, ActionListener.wrap( @@ -187,7 +211,7 @@ public synchronized void start(ActionListener listener) { listener.onResponse(new StartDataFrameTransformTaskAction.Response(true)); }, exc -> { - indexer.stop(); + getIndexer().stop(); listener.onFailure(new ElasticsearchException("Error while updating state for data frame transform [" + transform.getId() + "] to [" + state.getIndexerState() + "].", exc)); } @@ -195,13 +219,18 @@ public synchronized void start(ActionListener listener) { } public synchronized void stop(ActionListener listener) { + if (getIndexer() == null) { + listener.onFailure(new ElasticsearchException("Task for transform [{}] not fully initialized. Try again later", + getTransformId())); + return; + } // taskState is initialized as STOPPED and is updated in tandem with the indexerState // Consequently, if it is STOPPED, we consider the whole task STOPPED. if (taskState.get() == DataFrameTransformTaskState.STOPPED) { listener.onResponse(new StopDataFrameTransformAction.Response(true)); return; } - final IndexerState newState = indexer.stop(); + final IndexerState newState = getIndexer().stop(); switch (newState) { case STOPPED: // Fall through to `STOPPING` as the behavior is the same for both, we should persist for both @@ -215,9 +244,10 @@ public synchronized void stop(ActionListener { auditor.info(transform.getId(), "Updated state to [" + state.getTaskState() + "]"); @@ -237,10 +267,14 @@ public synchronized void stop(ActionListener> listener) { updatePersistentTaskState(state, ActionListener.wrap( @@ -276,6 +321,17 @@ void persistStateToClusterState(DataFrameTransformState state, )); } + synchronized void markAsFailed(String reason, ActionListener listener) { + taskState.set(DataFrameTransformTaskState.FAILED); + stateReason.set(reason); + persistStateToClusterState(getState(), ActionListener.wrap( + r -> { + listener.onResponse(null); + }, + listener::onFailure + )); + } + /** * This is called when the persistent task signals that the allocated task should be terminated. * Termination in the task framework is essentially voluntary, as the allocated task can only be @@ -284,37 +340,174 @@ void persistStateToClusterState(DataFrameTransformState state, @Override public synchronized void onCancelled() { logger.info( - "Received cancellation request for data frame transform [" + transform.getId() + "], state: [" + indexer.getState() + "]"); - if (indexer.abort()) { + "Received cancellation request for data frame transform [" + transform.getId() + "], state: [" + taskState.get() + "]"); + if (getIndexer() != null && getIndexer().abort()) { // there is no background transform running, we can shutdown safely shutdown(); } } - protected class ClientDataFrameIndexer extends DataFrameIndexer { - private static final int LOAD_TRANSFORM_TIMEOUT_IN_SECONDS = 30; + synchronized void initializeIndexer(ClientDataFrameIndexerBuilder indexerBuilder) { + indexer.set(indexerBuilder.build(this)); + } + + 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 Map initialPosition; + private DataFrameTransformProgress progress; + + ClientDataFrameIndexer build(DataFrameTransformTask parentTask) { + return new ClientDataFrameIndexer(this.transformId, + this.transformsConfigManager, + this.transformsCheckpointService, + new AtomicReference<>(this.indexerState), + this.initialPosition, + this.client, + this.auditor, + this.initialStats, + this.transformConfig, + this.fieldMappings, + this.progress, + 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(Map initialPosition) { + this.initialPosition = initialPosition; + return this; + } + + ClientDataFrameIndexerBuilder setProgress(DataFrameTransformProgress progress) { + this.progress = progress; + return this; + } + } + static class ClientDataFrameIndexer extends DataFrameIndexer { private final Client client; private final DataFrameTransformsConfigManager transformsConfigManager; private final DataFrameTransformsCheckpointService transformsCheckpointService; private final String transformId; + private final DataFrameAuditor auditor; + private final DataFrameTransformTask transformTask; + private final Map fieldMappings; + private final DataFrameTransformConfig transformConfig; + private volatile DataFrameTransformProgress progress; private volatile DataFrameIndexerTransformStats previouslyPersistedStats = null; + private final AtomicInteger failureCount; // 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 Map fieldMappings = null; - private DataFrameTransformConfig transformConfig = null; + ClientDataFrameIndexer(String transformId, + DataFrameTransformsConfigManager transformsConfigManager, + DataFrameTransformsCheckpointService transformsCheckpointService, + AtomicReference initialState, + Map initialPosition, + Client client, + DataFrameAuditor auditor, + DataFrameIndexerTransformStats initialStats, + DataFrameTransformConfig transformConfig, + Map fieldMappings, + DataFrameTransformProgress transformProgress, + DataFrameTransformTask parentTask) { + super(ExceptionsHelper.requireNonNull(parentTask, "parentTask") + .threadPool + .executor(ThreadPool.Names.GENERIC), + ExceptionsHelper.requireNonNull(auditor, "auditor"), + ExceptionsHelper.requireNonNull(initialState, "initialState"), + initialPosition, + initialStats == null ? new DataFrameIndexerTransformStats(transformId) : initialStats); + this.transformId = ExceptionsHelper.requireNonNull(transformId, "transformId"); + this.transformsConfigManager = ExceptionsHelper.requireNonNull(transformsConfigManager, "transformsConfigManager"); + this.transformsCheckpointService = ExceptionsHelper.requireNonNull(transformsCheckpointService, + "transformsCheckpointService"); + this.client = ExceptionsHelper.requireNonNull(client, "client"); + this.auditor = auditor; + this.transformConfig = ExceptionsHelper.requireNonNull(transformConfig, "transformConfig"); + this.transformTask = parentTask; + this.fieldMappings = ExceptionsHelper.requireNonNull(fieldMappings, "fieldMappings"); + this.progress = transformProgress; + this.failureCount = new AtomicInteger(0); + } - public ClientDataFrameIndexer(String transformId, DataFrameTransformsConfigManager transformsConfigManager, - DataFrameTransformsCheckpointService transformsCheckpointService, - AtomicReference initialState, Map initialPosition, Client client, - DataFrameAuditor auditor) { - super(threadPool.executor(ThreadPool.Names.GENERIC), auditor, initialState, initialPosition, - new DataFrameIndexerTransformStats(transformId)); - this.transformId = transformId; - this.transformsConfigManager = transformsConfigManager; - this.transformsCheckpointService = transformsCheckpointService; - this.client = client; + @Override + protected void onStart(long now, ActionListener listener) { + // Reset our failure count as we are starting again + failureCount.set(0); + // 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. + if (initialRun()) { + TransformProgressGatherer.getInitialProgress(this.client, getConfig(), ActionListener.wrap( + newProgress -> { + progress = newProgress; + super.onStart(now, listener); + }, + failure -> { + progress = null; + logger.warn("Unable to load progress information for task [" + transformId + "]", failure); + super.onStart(now, listener); + } + )); + } else { + super.onStart(now, listener); + } } @Override @@ -327,6 +520,11 @@ protected Map getFieldMappings() { return fieldMappings; } + @Override + protected DataFrameTransformProgress getProgress() { + return progress; + } + @Override protected String getJobId() { return transformId; @@ -334,56 +532,11 @@ protected String getJobId() { @Override public synchronized boolean maybeTriggerAsyncJob(long now) { - if (taskState.get() == DataFrameTransformTaskState.FAILED) { + if (transformTask.taskState.get() == DataFrameTransformTaskState.FAILED) { logger.debug("Schedule was triggered for transform [{}] but task is failed. Ignoring trigger.", getJobId()); return false; } - if (transformConfig == null) { - CountDownLatch latch = new CountDownLatch(1); - - transformsConfigManager.getTransformConfiguration(transformId, new LatchedActionListener<>(ActionListener.wrap( - config -> transformConfig = config, - e -> { - throw new RuntimeException( - DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId), e); - }), latch)); - - try { - latch.await(LOAD_TRANSFORM_TIMEOUT_IN_SECONDS, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException( - DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_LOAD_TRANSFORM_CONFIGURATION, transformId), e); - } - } - - if (transformConfig.isValid() == false) { - DataFrameConfigurationException exception = new DataFrameConfigurationException(transformId); - handleFailure(exception); - throw exception; - } - - if (fieldMappings == null) { - CountDownLatch latch = new CountDownLatch(1); - SchemaUtil.getDestinationFieldMappings(client, transformConfig.getDestination().getIndex(), new LatchedActionListener<>( - ActionListener.wrap( - destinationMappings -> fieldMappings = destinationMappings, - e -> { - throw new RuntimeException( - DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_UNABLE_TO_GATHER_FIELD_MAPPINGS, - transformConfig.getDestination().getIndex()), - e); - }), latch)); - try { - latch.await(LOAD_TRANSFORM_TIMEOUT_IN_SECONDS, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException( - DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_UNABLE_TO_GATHER_FIELD_MAPPINGS, - transformConfig.getDestination().getIndex()), - e); - } - } - return super.maybeTriggerAsyncJob(now); } @@ -408,30 +561,28 @@ protected void doSaveState(IndexerState indexerState, Map positi } final DataFrameTransformState state = new DataFrameTransformState( - taskState.get(), + transformTask.taskState.get(), indexerState, getPosition(), - currentCheckpoint.get(), - stateReason.get()); - logger.info("Updating persistent state of transform [" + transform.getId() + "] to [" + state.toString() + "]"); + transformTask.currentCheckpoint.get(), + transformTask.stateReason.get(), + getProgress()); + logger.debug("Updating persistent state of transform [{}] to [{}]", transformConfig.getId(), state.toString()); // Persisting stats when we call `doSaveState` should be ok as we only call it on a state transition and // only every-so-often when doing the bulk indexing calls. See AsyncTwoPhaseIndexer#onBulkResponse for current periodicity ActionListener> updateClusterStateListener = ActionListener.wrap( task -> { - // Make a copy of the previousStats so that they are not constantly updated when `merge` is called - DataFrameIndexerTransformStats tempStats = new DataFrameIndexerTransformStats(previousStats).merge(getStats()); - // Only persist the stats if something has actually changed - if (previouslyPersistedStats == null || previouslyPersistedStats.equals(tempStats) == false) { - transformsConfigManager.putOrUpdateTransformStats(tempStats, + if (previouslyPersistedStats == null || previouslyPersistedStats.equals(getStats()) == false) { + transformsConfigManager.putOrUpdateTransformStats(getStats(), ActionListener.wrap( r -> { - previouslyPersistedStats = tempStats; + previouslyPersistedStats = getStats(); next.run(); }, statsExc -> { - logger.error("Updating stats of transform [" + transform.getId() + "] failed", statsExc); + logger.error("Updating stats of transform [" + transformConfig.getId() + "] failed", statsExc); next.run(); } )); @@ -441,24 +592,24 @@ protected void doSaveState(IndexerState indexerState, Map positi } }, exc -> { - logger.error("Updating persistent state of transform [" + transform.getId() + "] failed", exc); + logger.error("Updating persistent state of transform [" + transformConfig.getId() + "] failed", exc); next.run(); } ); - persistStateToClusterState(state, updateClusterStateListener); + transformTask.persistStateToClusterState(state, updateClusterStateListener); } @Override protected void onFailure(Exception exc) { // the failure handler must not throw an exception due to internal problems try { - logger.warn("Data frame transform [" + transform.getId() + "] encountered an exception: ", exc); + logger.warn("Data frame transform [" + transformTask.getTransformId() + "] encountered an exception: ", exc); // 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 (exc.getMessage().equals(lastAuditedExceptionMessage) == false) { - auditor.warning(transform.getId(), "Data frame transform encountered an exception: " + exc.getMessage()); + auditor.warning(transformTask.getTransformId(), "Data frame transform encountered an exception: " + exc.getMessage()); lastAuditedExceptionMessage = exc.getMessage(); } handleFailure(exc); @@ -471,9 +622,10 @@ protected void onFailure(Exception exc) { protected void onFinish(ActionListener listener) { try { super.onFinish(listener); - long checkpoint = currentCheckpoint.incrementAndGet(); - auditor.info(transform.getId(), "Finished indexing for data frame transform checkpoint [" + checkpoint + "]"); - logger.info("Finished indexing for data frame transform [" + transform.getId() + "] checkpoint [" + checkpoint + "]"); + long checkpoint = transformTask.currentCheckpoint.incrementAndGet(); + auditor.info(transformTask.getTransformId(), "Finished indexing for data frame transform checkpoint [" + checkpoint + "]"); + logger.info( + "Finished indexing for data frame transform [" + transformTask.getTransformId() + "] checkpoint [" + checkpoint + "]"); listener.onResponse(null); } catch (Exception e) { listener.onFailure(e); @@ -482,26 +634,29 @@ protected void onFinish(ActionListener listener) { @Override protected void onAbort() { - auditor.info(transform.getId(), "Received abort request, stopping indexer"); - logger.info("Data frame transform [" + transform.getId() + "] received abort request, stopping indexer"); - shutdown(); + auditor.info(transformConfig.getId(), "Received abort request, stopping indexer"); + logger.info("Data frame transform [" + transformConfig.getId() + "] received abort request, stopping indexer"); + transformTask.shutdown(); } @Override protected void createCheckpoint(ActionListener listener) { - transformsCheckpointService.getCheckpoint(transformConfig, currentCheckpoint.get() + 1, ActionListener.wrap(checkpoint -> { - transformsConfigManager.putTransformCheckpoint(checkpoint, ActionListener.wrap(putCheckPointResponse -> { - listener.onResponse(null); - }, createCheckpointException -> { - listener.onFailure(new RuntimeException("Failed to create checkpoint", createCheckpointException)); - })); - }, getCheckPointException -> { - listener.onFailure(new RuntimeException("Failed to retrieve checkpoint", getCheckPointException)); - })); + transformsCheckpointService.getCheckpoint(transformConfig, + transformTask.currentCheckpoint.get() + 1, + ActionListener.wrap( + checkpoint -> transformsConfigManager.putTransformCheckpoint(checkpoint, + ActionListener.wrap( + putCheckPointResponse -> listener.onResponse(null), + createCheckpointException -> + listener.onFailure(new RuntimeException("Failed to create checkpoint", createCheckpointException)) + )), + getCheckPointException -> + listener.onFailure(new RuntimeException("Failed to retrieve checkpoint", getCheckPointException)) + )); } private boolean isIrrecoverableFailure(Exception e) { - return e instanceof IndexNotFoundException || e instanceof DataFrameConfigurationException; + return e instanceof IndexNotFoundException; } synchronized void handleFailure(Exception e) { @@ -520,21 +675,12 @@ synchronized void handleFailure(Exception e) { @Override protected void failIndexer(String failureMessage) { logger.error("Data frame transform [" + getJobId() + "]:" + failureMessage); - auditor.error(transform.getId(), failureMessage); - stateReason.set(failureMessage); - taskState.set(DataFrameTransformTaskState.FAILED); - persistStateToClusterState(DataFrameTransformTask.this.getState(), ActionListener.wrap( - r -> failureCount.set(0), // Successfully marked as failed, reset counter so that task can be restarted - exception -> {} // Noop, internal method logs the failure to update the state - )); + 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 -> {})); } } - - class DataFrameConfigurationException extends RuntimeException { - - DataFrameConfigurationException(String transformId) { - super(DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_INVALID, transformId)); - } - - } } diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/TransformProgressGatherer.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/TransformProgressGatherer.java new file mode 100644 index 0000000000000..23168627d442e --- /dev/null +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/TransformProgressGatherer.java @@ -0,0 +1,53 @@ +/* + * 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.action.ActionListener; +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.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; + +/** + * Utility class to gather the progress information for a given config and its cursor position + */ +public final class TransformProgressGatherer { + + /** + * This gathers the total docs given the config and search + * + * TODO: Support checkpointing logic to restrict the query + * @param progressListener The listener to alert on completion + */ + public static void getInitialProgress(Client client, + DataFrameTransformConfig config, + ActionListener progressListener) { + SearchRequest request = client.prepareSearch(config.getSource().getIndex()) + .setSize(0) + .setAllowPartialSearchResults(false) + .setTrackTotalHits(true) + .setQuery(config.getSource().getQueryConfig().getQuery()) + .request(); + + ActionListener searchResponseActionListener = ActionListener.wrap( + searchResponse -> { + progressListener.onResponse(new DataFrameTransformProgress(searchResponse.getHits().getTotalHits().value, null)); + }, + progressListener::onFailure + ); + ClientHelper.executeWithHeadersAsync(config.getHeaders(), + ClientHelper.DATA_FRAME_ORIGIN, + client, + SearchAction.INSTANCE, + request, + searchResponseActionListener); + } + +} diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java index 89388d82e08a2..015eb4b65e3b0 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameIndexerTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfigTests; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress; import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.dataframe.notifications.DataFrameAuditor; import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; @@ -94,6 +95,11 @@ protected Map getFieldMappings() { return fieldMappings; } + @Override + protected DataFrameTransformProgress getProgress() { + return null; + } + @Override protected void createCheckpoint(ActionListener listener) { listener.onResponse(null); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_stats.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_stats.yml index 2453309853725..33b0f40863a79 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_stats.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_stats.yml @@ -34,6 +34,7 @@ teardown: - do: data_frame.stop_data_frame_transform: transform_id: "airline-transform-stats" + wait_for_completion: true - do: data_frame.delete_data_frame_transform: @@ -197,6 +198,7 @@ teardown: - match: { transforms.0.id: "airline-transform-stats-dos" } - match: { transforms.0.state.indexer_state: "stopped" } - match: { transforms.0.state.checkpoint: 0 } + - is_false: transforms.0.state.progress - match: { transforms.0.stats.pages_processed: 0 } - match: { transforms.0.stats.documents_processed: 0 } - match: { transforms.0.stats.documents_indexed: 0 } From 180deaa48e4ba3efe0bc94213d9abe561dfeaa1f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 25 Apr 2019 16:18:02 +0200 Subject: [PATCH 184/260] Remove Exists Check from S3 Repository Deletes (#40931) * The check doesn't add much if anything practically, since the S3 repository is eventually consistent and we only log the non-existence of a blob anyway * We don't do the check on writes for this very reason and documented it as such * Removing the check saves one API call per single delete speeding up the deletion process and lowering costs --- .../org/elasticsearch/repositories/s3/S3BlobContainer.java | 3 --- .../repositories/s3/S3BlobStoreContainerTests.java | 5 +++++ .../org/elasticsearch/common/blobstore/BlobContainer.java | 6 ++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index f98382e5526be..652fa6a36017e 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -119,9 +119,6 @@ public void writeBlobAtomic(String blobName, InputStream inputStream, long blobS @Override public void deleteBlob(String blobName) throws IOException { - if (blobExists(blobName) == false) { - throw new NoSuchFileException("Blob [" + blobName + "] does not exist"); - } deleteBlobIgnoringIfNotExists(blobName); } diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreContainerTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreContainerTests.java index b2afd826c5b8e..422fab45e43bc 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreContainerTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobStoreContainerTests.java @@ -65,6 +65,11 @@ protected BlobStore newBlobStore() { return randomMockS3BlobStore(); } + @Override + public void testDeleteBlob() { + assumeFalse("not implemented because of S3's weak consistency model", true); + } + @Override public void testVerifyOverwriteFails() { assumeFalse("not implemented because of S3's weak consistency model", true); diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index 19d3a66a87d32..9c1bacb51bac7 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -100,7 +100,7 @@ public interface BlobContainer { /** * Deletes the blob with the given name, if the blob exists. If the blob does not exist, - * this method throws a NoSuchFileException. + * this method may throw a {@link NoSuchFileException} if the underlying implementation supports an existence check before delete. * * @param blobName * The name of the blob to delete. @@ -120,9 +120,7 @@ default void deleteBlobsIgnoringIfNotExists(List blobNames) throws IOExc IOException ioe = null; for (String blobName : blobNames) { try { - deleteBlob(blobName); - } catch (NoSuchFileException e) { - // ignored + deleteBlobIgnoringIfNotExists(blobName); } catch (IOException e) { if (ioe == null) { ioe = e; From f89420e21674d2bf73b65ee267b22086553e255a Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Thu, 25 Apr 2019 18:12:43 +0200 Subject: [PATCH 185/260] field_caps: adapt bwc version after backport (#41427) --- build.gradle | 4 ++-- .../main/resources/rest-api-spec/test/field_caps/10_basic.yml | 2 +- .../action/fieldcaps/FieldCapabilitiesRequest.java | 4 ++-- .../action/fieldcaps/FieldCapabilitiesResponse.java | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/build.gradle b/build.gradle index 583569ef7cd45..932086423aa9b 100644 --- a/build.gradle +++ b/build.gradle @@ -162,8 +162,8 @@ task verifyVersions { * after the backport of the backcompat code is complete. */ -boolean bwc_tests_enabled = true -final String bwc_tests_disabled_issue = "" /* place a PR link here when committing bwc changes */ +boolean bwc_tests_enabled = false +final String bwc_tests_disabled_issue = "https://github.com/elastic/elasticsearch/pull/41426" /* place a PR link here when committing bwc changes */ if (bwc_tests_enabled == false) { if (bwc_tests_disabled_issue.isEmpty()) { throw new GradleException("bwc_tests_disabled_issue must be set when bwc_tests_enabled == false") diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml index 89cdaf4a842c6..3c59f5a929a8b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/field_caps/10_basic.yml @@ -298,7 +298,7 @@ setup: --- "Field caps with include_unmapped": - skip: - version: " - 7.99.99" + version: " - 7.0.99" reason: include_unmapped has been added in 7.1.0 - do: diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java index 250d656afdd6d..e34c55eb99fbc 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java @@ -84,7 +84,7 @@ public void readFrom(StreamInput in) throws IOException { indices = in.readStringArray(); indicesOptions = IndicesOptions.readIndicesOptions(in); mergeResults = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { includeUnmapped = in.readBoolean(); } else { includeUnmapped = false; @@ -98,7 +98,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(indices); indicesOptions.writeIndicesOptions(out); out.writeBoolean(mergeResults); - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { out.writeBoolean(includeUnmapped); } } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java index 4f7b75d879445..7999e1e70adf6 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java @@ -108,7 +108,7 @@ public Map getField(String field) { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { indices = in.readStringArray(); } else { indices = Strings.EMPTY_ARRAY; @@ -124,7 +124,7 @@ private static Map readField(StreamInput in) throws I @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { out.writeStringArray(indices); } out.writeMap(responseMap, StreamOutput::writeString, FieldCapabilitiesResponse::writeField); From bc957947f892bbebb0c9d8cc6b4685b0142db768 Mon Sep 17 00:00:00 2001 From: Hicham Mallah Date: Thu, 25 Apr 2019 19:13:00 +0300 Subject: [PATCH 186/260] Make 0 as invalid value for `min_children` in `has_child` query (#41347) * squashing multiple commits * fixing #32949 updated DEFAULT_MIN_CHILDREN to be 1, this way in case min_children is not provided it will be set to 1 by default. * Fix ChildQuerySearchIT --- .../join/query/HasChildQueryBuilder.java | 10 +-- .../join/query/ChildQuerySearchIT.java | 64 ++++--------------- .../join/query/HasChildQueryBuilderTests.java | 6 +- 3 files changed, 20 insertions(+), 60 deletions(-) diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java index 1c44daea4e982..14362afd2e9c6 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java @@ -66,7 +66,7 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder foo.minMaxChildren(randomIntBetween(Integer.MIN_VALUE, -1), positiveValue)); - assertEquals("[has_child] requires non-negative 'min_children' field", e.getMessage()); + assertEquals("[has_child] requires positive 'min_children' field", e.getMessage()); e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(positiveValue, randomIntBetween(Integer.MIN_VALUE, -1))); - assertEquals("[has_child] requires non-negative 'max_children' field", e.getMessage()); + assertEquals("[has_child] requires positive 'max_children' field", e.getMessage()); e = expectThrows(IllegalArgumentException.class, () -> foo.minMaxChildren(positiveValue, positiveValue - 10)); assertEquals("[has_child] 'max_children' is less than 'min_children'", e.getMessage()); From a8870ef98c9a25961914b408f25d27dabd3a5a11 Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Thu, 25 Apr 2019 11:59:13 -0500 Subject: [PATCH 187/260] Refactor GeoHashUtils (#40869) This commit refactors GeoHashUtils class into a new Geohash utility class located in the ES geo library. The intent is to not only better control what geo methods are whitelisted for painless scripting but to clean up the geo utility API in general. --- .../elasticsearch/geo/geometry/Rectangle.java | 8 + .../org/elasticsearch/geo/utils/BitUtil.java | 75 ++++ .../org/elasticsearch/geo/utils/Geohash.java | 411 ++++++++---------- .../geo/utils}/GeoHashTests.java | 51 +-- .../elasticsearch/common/geo/GeoPoint.java | 20 +- .../elasticsearch/common/geo/GeoUtils.java | 22 +- .../index/fielddata/ScriptDocValues.java | 6 +- .../query/GeoBoundingBoxQueryBuilder.java | 9 +- .../elasticsearch/search/DocValueFormat.java | 4 +- .../geogrid/GeoHashGridAggregatorFactory.java | 4 +- .../geogrid/InternalGeoHashGridBucket.java | 4 +- .../completion/context/GeoContextMapping.java | 4 +- .../common/geo/GeoDistanceTests.java | 18 +- .../mapper/GeoPointFieldMapperTests.java | 2 +- .../search/geo/GeoPointParsingTests.java | 2 +- .../index/search/geo/GeoUtilsTests.java | 4 +- .../aggregations/bucket/GeoHashGridIT.java | 4 +- .../aggregations/bucket/ShardReduceIT.java | 4 +- .../geogrid/GeoHashGridAggregatorTests.java | 2 +- .../bucket/geogrid/GeoHashGridTests.java | 4 +- .../metrics/AbstractGeoTestCase.java | 6 +- .../search/geo/GeoDistanceIT.java | 8 +- .../elasticsearch/search/geo/GeoFilterIT.java | 22 +- .../search/sort/GeoDistanceIT.java | 4 +- .../completion/GeoContextMappingTests.java | 26 +- 25 files changed, 385 insertions(+), 339 deletions(-) create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/utils/BitUtil.java rename server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java => libs/geo/src/main/java/org/elasticsearch/geo/utils/Geohash.java (61%) rename {server/src/test/java/org/elasticsearch/common/geo => libs/geo/src/test/java/org/elasticsearch/geo/utils}/GeoHashTests.java (66%) diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java index 0507a5e60cc15..120bf9e2eb862 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java @@ -163,6 +163,14 @@ public boolean crossesDateline() { return maxLon < minLon; } + /** returns true if rectangle (defined by minLat, maxLat, minLon, maxLon) contains the lat lon point */ + public boolean containsPoint(final double lat, final double lon) { + if (lat >= minLat && lat <= maxLat) { + return crossesDateline() ? lon >= minLon || lon <= maxLon : lon >= minLon && lon <= maxLon; + } + return false; + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/utils/BitUtil.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/BitUtil.java new file mode 100644 index 0000000000000..42ae200e055ba --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/BitUtil.java @@ -0,0 +1,75 @@ +/* + * 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.geo.utils; + +/** + * Utilities for common Bit twiddling methods. Borrowed heavily from Lucene (org.apache.lucene.util.BitUtil). + */ +public class BitUtil { // magic numbers for bit interleaving + private static final long MAGIC[] = { + 0x5555555555555555L, 0x3333333333333333L, + 0x0F0F0F0F0F0F0F0FL, 0x00FF00FF00FF00FFL, + 0x0000FFFF0000FFFFL, 0x00000000FFFFFFFFL, + 0xAAAAAAAAAAAAAAAAL + }; + // shift values for bit interleaving + private static final short SHIFT[] = {1, 2, 4, 8, 16}; + + /** + * Interleaves the first 32 bits of each long value + * + * Adapted from: http://graphics.stanford.edu/~seander/bithacks.html#InterleaveBMN + */ + public static long interleave(int even, int odd) { + long v1 = 0x00000000FFFFFFFFL & even; + long v2 = 0x00000000FFFFFFFFL & odd; + v1 = (v1 | (v1 << SHIFT[4])) & MAGIC[4]; + v1 = (v1 | (v1 << SHIFT[3])) & MAGIC[3]; + v1 = (v1 | (v1 << SHIFT[2])) & MAGIC[2]; + v1 = (v1 | (v1 << SHIFT[1])) & MAGIC[1]; + v1 = (v1 | (v1 << SHIFT[0])) & MAGIC[0]; + v2 = (v2 | (v2 << SHIFT[4])) & MAGIC[4]; + v2 = (v2 | (v2 << SHIFT[3])) & MAGIC[3]; + v2 = (v2 | (v2 << SHIFT[2])) & MAGIC[2]; + v2 = (v2 | (v2 << SHIFT[1])) & MAGIC[1]; + v2 = (v2 | (v2 << SHIFT[0])) & MAGIC[0]; + + return (v2<<1) | v1; + } + + /** + * Extract just the even-bits value as a long from the bit-interleaved value + */ + public static long deinterleave(long b) { + b &= MAGIC[0]; + b = (b ^ (b >>> SHIFT[0])) & MAGIC[1]; + b = (b ^ (b >>> SHIFT[1])) & MAGIC[2]; + b = (b ^ (b >>> SHIFT[2])) & MAGIC[3]; + b = (b ^ (b >>> SHIFT[3])) & MAGIC[4]; + b = (b ^ (b >>> SHIFT[4])) & MAGIC[5]; + return b; + } + + /** + * flip flops odd with even bits + */ + public static final long flipFlop(final long b) { + return ((b & MAGIC[6]) >>> 1) | ((b & MAGIC[0]) << 1 ); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/Geohash.java similarity index 61% rename from server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java rename to libs/geo/src/main/java/org/elasticsearch/geo/utils/Geohash.java index bf65162d215b9..e332000bfad96 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoHashUtils.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/Geohash.java @@ -16,197 +16,52 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.common.geo; +package org.elasticsearch.geo.utils; -import org.apache.lucene.geo.Rectangle; -import org.apache.lucene.spatial.util.MortonEncoder; -import org.apache.lucene.util.BitUtil; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Rectangle; import java.util.ArrayList; import java.util.Collection; -import static org.apache.lucene.geo.GeoUtils.MAX_LAT_INCL; - /** * Utilities for converting to/from the GeoHash standard * * The geohash long format is represented as lon/lat (x/y) interleaved with the 4 least significant bits * representing the level (1-12) [xyxy...xyxyllll] * - * This differs from a morton encoded value which interleaves lat/lon (y/x).* + * This differs from a morton encoded value which interleaves lat/lon (y/x). + * + * NOTE: this will replace {@code org.elasticsearch.common.geo.GeoHashUtils} */ -public class GeoHashUtils { +public class Geohash { private static final char[] BASE_32 = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'j', 'k', 'm', 'n', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z'}; private static final String BASE_32_STRING = new String(BASE_32); - /** maximum precision for geohash strings */ public static final int PRECISION = 12; /** number of bits used for quantizing latitude and longitude values */ - public static final short BITS = 31; - /** scaling factors to convert lat/lon into unsigned space */ - private static final double LAT_SCALE = (0x1L<>> 2; - } - - /** - * Encode lon/lat to the geohash based long format (lon/lat interleaved, 4 least significant bits = level) - */ - public static final long longEncode(final double lon, final double lat, final int level) { - // shift to appropriate level - final short msf = (short)(((12 - level) * 5) + MORTON_OFFSET); - return ((BitUtil.flipFlop(encodeLatLon(lat, lon)) >>> msf) << 4) | level; - } - - /** - * Encode from geohash string to the geohash based long format (lon/lat interleaved, 4 least significant bits = level) - */ - private static long longEncode(final String hash, int length) { - int level = length - 1; - long b; - long l = 0L; - for(char c : hash.toCharArray()) { - b = (long)(BASE_32_STRING.indexOf(c)); - l |= (b<<(level--*5)); - if (level < 0) { - // We cannot handle more than 12 levels - break; - } - } - return (l << 4) | length; - } - - /** - * Encode an existing geohash long to the provided precision - */ - public static long longEncode(long geohash, int level) { - final short precision = (short)(geohash & 15); - if (precision == level) { - return geohash; - } else if (precision > level) { - return ((geohash >>> (((precision - level) * 5) + 4)) << 4) | level; - } - return ((geohash >>> 4) << (((level - precision) * 5) + 4) | level); - } - - /** - * Convert from a morton encoded long from a geohash encoded long - */ - public static long fromMorton(long morton, int level) { - long mFlipped = BitUtil.flipFlop(morton); - mFlipped >>>= (((GeoHashUtils.PRECISION - level) * 5) + MORTON_OFFSET); - return (mFlipped << 4) | level; - } - - /** - * Encode to a geohash string from the geohash based long format - */ - public static final String stringEncode(long geoHashLong) { - int level = (int)geoHashLong&15; - geoHashLong >>>= 4; - char[] chars = new char[level]; - do { - chars[--level] = BASE_32[(int) (geoHashLong&31L)]; - geoHashLong>>>=5; - } while(level > 0); - return new String(chars); + // no instance: + private Geohash() { } - /** - * Encode to a geohash string from full resolution longitude, latitude) - */ - public static final String stringEncode(final double lon, final double lat) { - return stringEncode(lon, lat, 12); - } - - /** - * Encode to a level specific geohash string from full resolution longitude, latitude - */ - public static final String stringEncode(final double lon, final double lat, final int level) { - // convert to geohashlong - final long ghLong = fromMorton(encodeLatLon(lat, lon), level); - return stringEncode(ghLong); - - } - - /** - * Encode to a full precision geohash string from a given morton encoded long value - */ - public static final String stringEncodeFromMortonLong(final long hashedVal) throws Exception { - return stringEncode(hashedVal, PRECISION); - } - - /** - * Encode to a geohash string at a given level from a morton long - */ - public static final String stringEncodeFromMortonLong(long hashedVal, final int level) { - // bit twiddle to geohash (since geohash is a swapped (lon/lat) encoding) - hashedVal = BitUtil.flipFlop(hashedVal); - - StringBuilder geoHash = new StringBuilder(); - short precision = 0; - final short msf = (BITS<<1)-5; - long mask = 31L<>>(msf-(precision*5)))]); - // next 5 bits - mask >>>= 5; - } while (++precision < level); - return geoHash.toString(); - } - - /** - * Encode to a morton long value from a given geohash string - */ - public static final long mortonEncode(final String hash) { - if (hash.isEmpty()) { - throw new IllegalArgumentException("empty geohash"); - } - int level = 11; - long b; - long l = 0L; - for(char c : hash.toCharArray()) { - b = (long)(BASE_32_STRING.indexOf(c)); - if (b < 0) { - throw new IllegalArgumentException("unsupported symbol [" + c + "] in geohash [" + hash + "]"); - } - l |= (b<<((level--*5) + MORTON_OFFSET)); - if (level < 0) { - // We cannot handle more than 12 levels - break; - } - } - return BitUtil.flipFlop(l); - } - - /** - * Encode to a morton long value from a given geohash long value - */ - public static final long mortonEncode(final long geoHashLong) { - final int level = (int)(geoHashLong&15); - final short odd = (short)(level & 1); - - return BitUtil.flipFlop(((geoHashLong >>> 4) << odd) << (((12 - level) * 5) + (MORTON_OFFSET - odd))); - } - - private static char encode(int x, int y) { - return BASE_32[((x & 1) + ((y & 1) * 2) + ((x & 2) * 2) + ((y & 2) * 4) + ((x & 4) * 4)) % 32]; + /** Returns a {@link Point} instance from a geohash string */ + public static Point toPoint(final String geohash) throws IllegalArgumentException { + final long hash = mortonEncode(geohash); + return new Point(decodeLatitude(hash), decodeLongitude(hash)); } /** @@ -215,9 +70,9 @@ private static char encode(int x, int y) { * @param geohash Geohash of the defined cell * @return GeoRect rectangle defining the bounding box */ - public static Rectangle bbox(final String geohash) { + public static Rectangle toBoundingBox(final String geohash) { // bottom left is the coordinate - GeoPoint bottomLeft = GeoPoint.fromGeohash(geohash); + Point bottomLeft = toPoint(geohash); int len = Math.min(12, geohash.length()); long ghLong = longEncode(geohash, len); // shift away the level @@ -225,15 +80,20 @@ public static Rectangle bbox(final String geohash) { // deinterleave long lon = BitUtil.deinterleave(ghLong >>> 1); long lat = BitUtil.deinterleave(ghLong); + final int shift = (12 - len) * 5 + 2; if (lat < MAX_LAT_BITS) { // add 1 to lat and lon to get topRight - GeoPoint topRight = GeoPoint.fromGeohash(BitUtil.interleave((int)(lat + 1), (int)(lon + 1)) << 4 | len); - return new Rectangle(bottomLeft.lat(), topRight.lat(), bottomLeft.lon(), topRight.lon()); + ghLong = BitUtil.interleave((int)(lat + 1), (int)(lon + 1)) << 4 | len; + final long mortonHash = BitUtil.flipFlop((ghLong >>> 4) << shift); + Point topRight = new Point(decodeLatitude(mortonHash), decodeLongitude(mortonHash)); + return new Rectangle(bottomLeft.getLat(), topRight.getLat(), bottomLeft.getLon(), topRight.getLon()); } else { // We cannot go north of north pole, so just using 90 degrees instead of calculating it using // add 1 to lon to get lon of topRight, we are going to use 90 for lat - GeoPoint topRight = GeoPoint.fromGeohash(BitUtil.interleave((int)lat, (int)(lon + 1)) << 4 | len); - return new Rectangle(bottomLeft.lat(), MAX_LAT_INCL, bottomLeft.lon(), topRight.lon()); + ghLong = BitUtil.interleave((int)lat, (int)(lon + 1)) << 4 | len; + final long mortonHash = BitUtil.flipFlop((ghLong >>> 4) << shift); + Point topRight = new Point(decodeLatitude(mortonHash), decodeLongitude(mortonHash)); + return new Rectangle(bottomLeft.getLat(), 90D, bottomLeft.getLon(), topRight.getLon()); } } @@ -243,8 +103,48 @@ public static Rectangle bbox(final String geohash) { * @param geohash Geohash of the defined cell * @return geohashes of all neighbor cells */ - public static Collection neighbors(String geohash) { - return addNeighbors(geohash, geohash.length(), new ArrayList(8)); + public static Collection getNeighbors(String geohash) { + return addNeighborsAtLevel(geohash, geohash.length(), new ArrayList(8)); + } + /** + * Add all geohashes of the cells next to a given geohash to a list. + * + * @param geohash Geohash of a specified cell + * @param neighbors list to add the neighbors to + * @return the given list + */ + public static final > E addNeighbors(String geohash, E neighbors) { + return addNeighborsAtLevel(geohash, geohash.length(), neighbors); + } + + /** + * Add all geohashes of the cells next to a given geohash to a list. + * + * @param geohash Geohash of a specified cell + * @param level level of the given geohash + * @param neighbors list to add the neighbors to + * @return the given list + */ + public static final > E addNeighborsAtLevel(String geohash, + int level, E neighbors) { + String south = getNeighbor(geohash, level, 0, -1); + String north = getNeighbor(geohash, level, 0, +1); + if (north != null) { + neighbors.add(getNeighbor(north, level, -1, 0)); + neighbors.add(north); + neighbors.add(getNeighbor(north, level, +1, 0)); + } + + neighbors.add(getNeighbor(geohash, level, -1, 0)); + neighbors.add(getNeighbor(geohash, level, +1, 0)); + + if (south != null) { + neighbors.add(getNeighbor(south, level, -1, 0)); + neighbors.add(south); + neighbors.add(getNeighbor(south, level, +1, 0)); + } + + return neighbors; } /** @@ -256,7 +156,7 @@ public static Collection neighbors(String geohash) { * @param dy delta of the second grid coordinate (must be -1, 0 or +1) * @return geohash of the defined cell */ - public static final String neighbor(String geohash, int level, int dx, int dy) { + public static final String getNeighbor(String geohash, int level, int dx, int dy) { int cell = BASE_32_STRING.indexOf(geohash.charAt(level -1)); // Decoding the Geohash bit pattern to determine grid coordinates @@ -279,7 +179,7 @@ public static final String neighbor(String geohash, int level, int dx, int dy) { if ((dy < 0 && y == 0) || (dy > 0 && y == 3)) { return null; } else { - return Character.toString(encode(x + dx, y + dy)); + return Character.toString(encodeBase32(x + dx, y + dy)); } } else { // define grid coordinates for next level @@ -293,79 +193,154 @@ public static final String neighbor(String geohash, int level, int dx, int dy) { // xLimit and YLimit must always be respectively 7 and 3 // since x and y semantics are swapping on each level. if (nx >= 0 && nx <= 7 && ny >= 0 && ny <= 3) { - return geohash.substring(0, level - 1) + encode(nx, ny); + return geohash.substring(0, level - 1) + encodeBase32(nx, ny); } else { - String neighbor = neighbor(geohash, level - 1, dx, dy); - return (neighbor != null) ? neighbor + encode(nx, ny) : neighbor; + String neighbor = getNeighbor(geohash, level - 1, dx, dy); + return (neighbor != null) ? neighbor + encodeBase32(nx, ny) : neighbor; } } } /** - * Add all geohashes of the cells next to a given geohash to a list. - * - * @param geohash Geohash of a specified cell - * @param neighbors list to add the neighbors to - * @return the given list + * Encode lon/lat to the geohash based long format (lon/lat interleaved, 4 least significant bits = level) */ - public static final > E addNeighbors(String geohash, E neighbors) { - return addNeighbors(geohash, geohash.length(), neighbors); + public static final long longEncode(final double lon, final double lat, final int level) { + // shift to appropriate level + final short msf = (short)(((12 - level) * 5) + (MORTON_OFFSET - 2)); + return ((encodeLatLon(lat, lon) >>> msf) << 4) | level; } /** - * Add all geohashes of the cells next to a given geohash to a list. - * - * @param geohash Geohash of a specified cell - * @param length level of the given geohash - * @param neighbors list to add the neighbors to - * @return the given list + * Encode to a geohash string from full resolution longitude, latitude) */ - public static final > E addNeighbors(String geohash, int length, E neighbors) { - String south = neighbor(geohash, length, 0, -1); - String north = neighbor(geohash, length, 0, +1); - if (north != null) { - neighbors.add(neighbor(north, length, -1, 0)); - neighbors.add(north); - neighbors.add(neighbor(north, length, +1, 0)); - } + public static final String stringEncode(final double lon, final double lat) { + return stringEncode(lon, lat, 12); + } - neighbors.add(neighbor(geohash, length, -1, 0)); - neighbors.add(neighbor(geohash, length, +1, 0)); + /** + * Encode to a level specific geohash string from full resolution longitude, latitude + */ + public static final String stringEncode(final double lon, final double lat, final int level) { + // convert to geohashlong + long interleaved = encodeLatLon(lat, lon); + interleaved >>>= (((PRECISION - level) * 5) + (MORTON_OFFSET - 2)); + final long geohash = (interleaved << 4) | level; + return stringEncode(geohash); + } - if (south != null) { - neighbors.add(neighbor(south, length, -1, 0)); - neighbors.add(south); - neighbors.add(neighbor(south, length, +1, 0)); - } + /** + * Encode to a geohash string from the geohash based long format + */ + public static final String stringEncode(long geoHashLong) { + int level = (int)geoHashLong&15; + geoHashLong >>>= 4; + char[] chars = new char[level]; + do { + chars[--level] = BASE_32[(int) (geoHashLong&31L)]; + geoHashLong>>>=5; + } while(level > 0); - return neighbors; + return new String(chars); } - /** decode longitude value from morton encoded geo point */ - public static final double decodeLongitude(final long hash) { - return unscaleLon(BitUtil.deinterleave(hash)); + /** base32 encode at the given grid coordinate */ + private static char encodeBase32(int x, int y) { + return BASE_32[((x & 1) + ((y & 1) * 2) + ((x & 2) * 2) + ((y & 2) * 4) + ((x & 4) * 4)) % 32]; } - /** decode latitude value from morton encoded geo point */ - public static final double decodeLatitude(final long hash) { - return unscaleLat(BitUtil.deinterleave(hash >>> 1)); + /** + * Encode from geohash string to the geohash based long format (lon/lat interleaved, 4 least significant bits = level) + */ + private static long longEncode(final String hash, int length) { + int level = length - 1; + long b; + long l = 0L; + for(char c : hash.toCharArray()) { + b = (long)(BASE_32_STRING.indexOf(c)); + l |= (b<<(level--*5)); + if (level < 0) { + // We cannot handle more than 12 levels + break; + } + } + return (l << 4) | length; } - private static double unscaleLon(final long val) { - return (val / LON_SCALE) - 180; + /** + * Encode to a morton long value from a given geohash string + */ + public static long mortonEncode(final String hash) { + if (hash.isEmpty()) { + throw new IllegalArgumentException("empty geohash"); + } + int level = 11; + long b; + long l = 0L; + for(char c : hash.toCharArray()) { + b = (long)(BASE_32_STRING.indexOf(c)); + if (b < 0) { + throw new IllegalArgumentException("unsupported symbol [" + c + "] in geohash [" + hash + "]"); + } + l |= (b<<((level--*5) + (MORTON_OFFSET - 2))); + if (level < 0) { + // We cannot handle more than 12 levels + break; + } + } + return BitUtil.flipFlop(l); } - private static double unscaleLat(final long val) { - return (val / LAT_SCALE) - 90; + private static long encodeLatLon(final double lat, final double lon) { + // encode lat/lon flipping the sign bit so negative ints sort before positive ints + final int latEnc = encodeLatitude(lat) ^ 0x80000000; + final int lonEnc = encodeLongitude(lon) ^ 0x80000000; + return BitUtil.interleave(latEnc, lonEnc) >>> 2; + } + + + /** encode latitude to integer */ + public static int encodeLatitude(double latitude) { + // the maximum possible value cannot be encoded without overflow + if (latitude == 90.0D) { + latitude = Math.nextDown(latitude); + } + return (int) Math.floor(latitude / LAT_DECODE); + } + + /** encode longitude to integer */ + public static int encodeLongitude(double longitude) { + // the maximum possible value cannot be encoded without overflow + if (longitude == 180.0D) { + longitude = Math.nextDown(longitude); + } + return (int) Math.floor(longitude / LON_DECODE); } /** returns the latitude value from the string based geohash */ public static final double decodeLatitude(final String geohash) { - return decodeLatitude(mortonEncode(geohash)); + return decodeLatitude(Geohash.mortonEncode(geohash)); } /** returns the latitude value from the string based geohash */ public static final double decodeLongitude(final String geohash) { - return decodeLongitude(mortonEncode(geohash)); + return decodeLongitude(Geohash.mortonEncode(geohash)); + } + + /** decode longitude value from morton encoded geo point */ + public static double decodeLongitude(final long hash) { + return unscaleLon(BitUtil.deinterleave(hash)); + } + + /** decode latitude value from morton encoded geo point */ + public static double decodeLatitude(final long hash) { + return unscaleLat(BitUtil.deinterleave(hash >>> 1)); + } + + private static double unscaleLon(final long val) { + return (val / LON_SCALE) - 180; + } + + private static double unscaleLat(final long val) { + return (val / LAT_SCALE) - 90; } } diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/utils/GeoHashTests.java similarity index 66% rename from server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java rename to libs/geo/src/test/java/org/elasticsearch/geo/utils/GeoHashTests.java index b4a24cfc4fcd0..c919d072f0009 100644 --- a/server/src/test/java/org/elasticsearch/common/geo/GeoHashTests.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/utils/GeoHashTests.java @@ -16,13 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.common.geo; +package org.elasticsearch.geo.utils; -import org.apache.lucene.geo.Rectangle; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.geo.geometry.Rectangle; import org.elasticsearch.test.ESTestCase; /** - * Tests for {@link org.elasticsearch.common.geo.GeoHashUtils} + * Tests for {@link org.elasticsearch.geo.utils.Geohash} */ public class GeoHashTests extends ESTestCase { public void testGeohashAsLongRoutines() { @@ -37,13 +38,13 @@ public void testGeohashAsLongRoutines() { { for(int p=1;p<=12;p++) { - long geoAsLong = GeoHashUtils.longEncode(lng, lat, p); + long geoAsLong = Geohash.longEncode(lng, lat, p); // string encode from geohashlong encoded location - String geohashFromLong = GeoHashUtils.stringEncode(geoAsLong); + String geohashFromLong = Geohash.stringEncode(geoAsLong); // string encode from full res lat lon - String geohash = GeoHashUtils.stringEncode(lng, lat, p); + String geohash = Geohash.stringEncode(lng, lat, p); // ensure both strings are the same assertEquals(geohash, geohashFromLong); @@ -62,25 +63,25 @@ public void testGeohashAsLongRoutines() { public void testBboxFromHash() { String hash = randomGeohash(1, 12); int level = hash.length(); - Rectangle bbox = GeoHashUtils.bbox(hash); + Rectangle bbox = Geohash.toBoundingBox(hash); // check that the length is as expected double expectedLonDiff = 360.0 / (Math.pow(8.0, (level + 1) / 2) * Math.pow(4.0, level / 2)); double expectedLatDiff = 180.0 / (Math.pow(4.0, (level + 1) / 2) * Math.pow(8.0, level / 2)); - assertEquals(expectedLonDiff, bbox.maxLon - bbox.minLon, 0.00001); - assertEquals(expectedLatDiff, bbox.maxLat - bbox.minLat, 0.00001); - assertEquals(hash, GeoHashUtils.stringEncode(bbox.minLon, bbox.minLat, level)); + assertEquals(expectedLonDiff, bbox.getMaxLon() - bbox.getMinLon(), 0.00001); + assertEquals(expectedLatDiff, bbox.getMaxLat() - bbox.getMinLat(), 0.00001); + assertEquals(hash, Geohash.stringEncode(bbox.getMinLon(), bbox.getMinLat(), level)); } public void testGeohashExtremes() { - assertEquals("000000000000", GeoHashUtils.stringEncode(-180, -90)); - assertEquals("800000000000", GeoHashUtils.stringEncode(-180, 0)); - assertEquals("bpbpbpbpbpbp", GeoHashUtils.stringEncode(-180, 90)); - assertEquals("h00000000000", GeoHashUtils.stringEncode(0, -90)); - assertEquals("s00000000000", GeoHashUtils.stringEncode(0, 0)); - assertEquals("upbpbpbpbpbp", GeoHashUtils.stringEncode(0, 90)); - assertEquals("pbpbpbpbpbpb", GeoHashUtils.stringEncode(180, -90)); - assertEquals("xbpbpbpbpbpb", GeoHashUtils.stringEncode(180, 0)); - assertEquals("zzzzzzzzzzzz", GeoHashUtils.stringEncode(180, 90)); + assertEquals("000000000000", Geohash.stringEncode(-180, -90)); + assertEquals("800000000000", Geohash.stringEncode(-180, 0)); + assertEquals("bpbpbpbpbpbp", Geohash.stringEncode(-180, 90)); + assertEquals("h00000000000", Geohash.stringEncode(0, -90)); + assertEquals("s00000000000", Geohash.stringEncode(0, 0)); + assertEquals("upbpbpbpbpbp", Geohash.stringEncode(0, 90)); + assertEquals("pbpbpbpbpbpb", Geohash.stringEncode(180, -90)); + assertEquals("xbpbpbpbpbpb", Geohash.stringEncode(180, 0)); + assertEquals("zzzzzzzzzzzz", Geohash.stringEncode(180, 90)); } public void testLongGeohashes() { @@ -92,24 +93,24 @@ public void testLongGeohashes() { GeoPoint actual = GeoPoint.fromGeohash(extendedGeohash); assertEquals("Additional data points above 12 should be ignored [" + extendedGeohash + "]" , expected, actual); - Rectangle expectedBbox = GeoHashUtils.bbox(geohash); - Rectangle actualBbox = GeoHashUtils.bbox(extendedGeohash); + Rectangle expectedBbox = Geohash.toBoundingBox(geohash); + Rectangle actualBbox = Geohash.toBoundingBox(extendedGeohash); assertEquals("Additional data points above 12 should be ignored [" + extendedGeohash + "]" , expectedBbox, actualBbox); } } public void testNorthPoleBoundingBox() { - Rectangle bbox = GeoHashUtils.bbox("zzbxfpgzupbx"); // Bounding box with maximum precision touching north pole - assertEquals(90.0, bbox.maxLat, 0.0000001); // Should be 90 degrees + Rectangle bbox = Geohash.toBoundingBox("zzbxfpgzupbx"); // Bounding box with maximum precision touching north pole + assertEquals(90.0, bbox.getMaxLat(), 0.0000001); // Should be 90 degrees } public void testInvalidGeohashes() { IllegalArgumentException ex; - ex = expectThrows(IllegalArgumentException.class, () -> GeoHashUtils.mortonEncode("55.5")); + ex = expectThrows(IllegalArgumentException.class, () -> Geohash.mortonEncode("55.5")); assertEquals("unsupported symbol [.] in geohash [55.5]", ex.getMessage()); - ex = expectThrows(IllegalArgumentException.class, () -> GeoHashUtils.mortonEncode("")); + ex = expectThrows(IllegalArgumentException.class, () -> Geohash.mortonEncode("")); assertEquals("empty geohash", ex.getMessage()); } diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java b/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java index 0b7b4f7484b8c..b4039fdbd2825 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java @@ -28,12 +28,11 @@ import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.geo.utils.Geohash; import java.io.IOException; import java.util.Arrays; -import static org.elasticsearch.common.geo.GeoHashUtils.mortonEncode; -import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; import static org.elasticsearch.index.mapper.GeoPointFieldMapper.Names.IGNORE_Z_VALUE; public final class GeoPoint implements ToXContentFragment { @@ -117,8 +116,8 @@ public GeoPoint resetFromCoordinates(String value, final boolean ignoreZValue) { public GeoPoint resetFromIndexHash(long hash) { - lon = GeoHashUtils.decodeLongitude(hash); - lat = GeoHashUtils.decodeLatitude(hash); + lon = Geohash.decodeLongitude(hash); + lat = Geohash.decodeLatitude(hash); return this; } @@ -143,11 +142,11 @@ public GeoPoint resetFromIndexableField(IndexableField field) { public GeoPoint resetFromGeoHash(String geohash) { final long hash; try { - hash = mortonEncode(geohash); + hash = Geohash.mortonEncode(geohash); } catch (IllegalArgumentException ex) { throw new ElasticsearchParseException(ex.getMessage(), ex); } - return this.reset(GeoHashUtils.decodeLatitude(hash), GeoHashUtils.decodeLongitude(hash)); + return this.reset(Geohash.decodeLatitude(hash), Geohash.decodeLongitude(hash)); } public GeoPoint resetFromGeoHash(long geohashLong) { @@ -172,11 +171,11 @@ public double getLon() { } public String geohash() { - return stringEncode(lon, lat); + return Geohash.stringEncode(lon, lat); } public String getGeohash() { - return stringEncode(lon, lat); + return Geohash.stringEncode(lon, lat); } @Override @@ -208,11 +207,6 @@ public String toString() { return lat + ", " + lon; } - public static GeoPoint parseFromLatLon(String latLon) { - GeoPoint point = new GeoPoint(latLon); - return point; - } - public static GeoPoint fromGeohash(String geohash) { return new GeoPoint().resetFromGeoHash(geohash); } diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java index 6dcaaaa7d6a29..f990a9750e0e1 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java @@ -19,7 +19,6 @@ package org.elasticsearch.common.geo; -import org.apache.lucene.geo.Rectangle; import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree; import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree; import org.apache.lucene.util.SloppyMath; @@ -34,6 +33,8 @@ import org.elasticsearch.common.xcontent.XContentSubParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.geo.geometry.Rectangle; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.index.fielddata.FieldData; import org.elasticsearch.index.fielddata.GeoPointValues; import org.elasticsearch.index.fielddata.MultiGeoPointValues; @@ -535,14 +536,14 @@ private static GeoPoint parseGeoHash(GeoPoint point, String geohash, EffectivePo if (effectivePoint == EffectivePoint.BOTTOM_LEFT) { return point.resetFromGeoHash(geohash); } else { - Rectangle rectangle = GeoHashUtils.bbox(geohash); + Rectangle rectangle = Geohash.toBoundingBox(geohash); switch (effectivePoint) { case TOP_LEFT: - return point.reset(rectangle.maxLat, rectangle.minLon); + return point.reset(rectangle.getMaxLat(), rectangle.getMinLon()); case TOP_RIGHT: - return point.reset(rectangle.maxLat, rectangle.maxLon); + return point.reset(rectangle.getMaxLat(), rectangle.getMaxLon()); case BOTTOM_RIGHT: - return point.reset(rectangle.minLat, rectangle.maxLon); + return point.reset(rectangle.getMinLat(), rectangle.getMaxLon()); default: throw new IllegalArgumentException("Unsupported effective point " + effectivePoint); } @@ -639,17 +640,6 @@ public static double planeDistance(double lat1, double lon1, double lat2, double return Math.sqrt(x * x + y * y) * EARTH_MEAN_RADIUS; } - /** check if point is within a rectangle - * todo: move this to lucene Rectangle class - */ - public static boolean rectangleContainsPoint(Rectangle r, double lat, double lon) { - if (lat >= r.minLat && lat <= r.maxLat) { - // if rectangle crosses the dateline we only check if the lon is >= min or max - return r.crossesDateline() ? lon >= r.minLon || lon <= r.maxLon : lon >= r.minLon && lon <= r.maxLon; - } - return false; - } - /** * Return a {@link SortedNumericDoubleValues} instance that returns the distances to a list of geo-points * for each document. diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java b/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java index afd1d9e368480..69f48a74c13da 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java @@ -23,10 +23,10 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.common.time.DateUtils; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.script.JodaCompatibleZonedDateTime; import java.io.IOException; @@ -365,8 +365,8 @@ public double planeDistanceWithDefault(double lat, double lon, double defaultVal public double geohashDistance(String geohash) { GeoPoint point = getValue(); - return GeoUtils.arcDistance(point.lat(), point.lon(), GeoHashUtils.decodeLatitude(geohash), - GeoHashUtils.decodeLongitude(geohash)); + return GeoUtils.arcDistance(point.lat(), point.lon(), Geohash.decodeLatitude(geohash), + Geohash.decodeLongitude(geohash)); } public double geohashDistanceWithDefault(String geohash, double defaultValue) { diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java index 537127bc63507..a4633ea545344 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java @@ -21,7 +21,7 @@ import org.apache.lucene.document.LatLonDocValuesField; import org.apache.lucene.document.LatLonPoint; -import org.apache.lucene.geo.Rectangle; +//import org.apache.lucene.geo.Rectangle; import org.apache.lucene.search.IndexOrDocValuesQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; @@ -29,7 +29,6 @@ import org.elasticsearch.common.Numbers; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoShapeType; import org.elasticsearch.common.geo.GeoUtils; @@ -39,6 +38,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.geo.geometry.Rectangle; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.index.mapper.GeoPointFieldMapper.GeoPointFieldType; import org.elasticsearch.index.mapper.MappedFieldType; @@ -181,8 +182,8 @@ public GeoBoundingBoxQueryBuilder setCorners(GeoPoint topLeft, GeoPoint bottomRi */ public GeoBoundingBoxQueryBuilder setCorners(final String geohash) { // get the bounding box of the geohash and set topLeft and bottomRight - Rectangle ghBBox = GeoHashUtils.bbox(geohash); - return setCorners(new GeoPoint(ghBBox.maxLat, ghBBox.minLon), new GeoPoint(ghBBox.minLat, ghBBox.maxLon)); + Rectangle ghBBox = Geohash.toBoundingBox(geohash); + return setCorners(new GeoPoint(ghBBox.getMaxLat(), ghBBox.getMinLon()), new GeoPoint(ghBBox.getMinLat(), ghBBox.getMaxLon())); } /** diff --git a/server/src/main/java/org/elasticsearch/search/DocValueFormat.java b/server/src/main/java/org/elasticsearch/search/DocValueFormat.java index 9bdddfb2377a4..90de3063206a9 100644 --- a/server/src/main/java/org/elasticsearch/search/DocValueFormat.java +++ b/server/src/main/java/org/elasticsearch/search/DocValueFormat.java @@ -21,7 +21,6 @@ import org.apache.lucene.document.InetAddressPoint; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -29,6 +28,7 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateMathParser; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.index.mapper.DateFieldMapper; import java.io.IOException; @@ -237,7 +237,7 @@ public void writeTo(StreamOutput out) { @Override public String format(long value) { - return GeoHashUtils.stringEncode(value); + return Geohash.stringEncode(value); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java index e4b99b0c9656e..54253b9dea25c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorFactory.java @@ -19,7 +19,7 @@ package org.elasticsearch.search.aggregations.bucket.geogrid; -import org.elasticsearch.common.geo.GeoHashUtils; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -71,7 +71,7 @@ protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, context, parent); } - CellIdSource cellIdSource = new CellIdSource(valuesSource, precision, GeoHashUtils::longEncode); + CellIdSource cellIdSource = new CellIdSource(valuesSource, precision, Geohash::longEncode); return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, context, parent, pipelineAggregators, metaData); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridBucket.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridBucket.java index 1d77a54523b11..82f2f70c04c16 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridBucket.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGridBucket.java @@ -18,9 +18,9 @@ */ package org.elasticsearch.search.aggregations.bucket.geogrid; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.search.aggregations.InternalAggregations; import java.io.IOException; @@ -45,7 +45,7 @@ InternalGeoHashGridBucket buildBucket(InternalGeoGridBucket bucket, long hashAsL @Override public String getKeyAsString() { - return GeoHashUtils.stringEncode(hashAsLong); + return Geohash.stringEncode(hashAsLong); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java index b1b3c5e92cee4..1f19553572787 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java @@ -50,8 +50,8 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.elasticsearch.common.geo.GeoHashUtils.addNeighbors; -import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; +import static org.elasticsearch.geo.utils.Geohash.addNeighbors; +import static org.elasticsearch.geo.utils.Geohash.stringEncode; /** * A {@link ContextMapping} that uses a geo location/area as a diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoDistanceTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeoDistanceTests.java index 9b3a2d9ce44b0..7442d3c8d803d 100644 --- a/server/src/test/java/org/elasticsearch/common/geo/GeoDistanceTests.java +++ b/server/src/test/java/org/elasticsearch/common/geo/GeoDistanceTests.java @@ -18,10 +18,10 @@ */ package org.elasticsearch.common.geo; -import org.apache.lucene.geo.Rectangle; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.unit.DistanceUnit; +import org.elasticsearch.geo.geometry.Rectangle; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -70,15 +70,17 @@ public void testInvalidReadFrom() throws Exception { public void testDistanceCheck() { // Note, is within is an approximation, so, even though 0.52 is outside 50mi, we still get "true" double radius = DistanceUnit.convert(50, DistanceUnit.MILES, DistanceUnit.METERS); - Rectangle box = Rectangle.fromPointDistance(0, 0, radius); - assertThat(GeoUtils.rectangleContainsPoint(box, 0.5, 0.5), equalTo(true)); - assertThat(GeoUtils.rectangleContainsPoint(box, 0.52, 0.52), equalTo(true)); - assertThat(GeoUtils.rectangleContainsPoint(box, 1, 1), equalTo(false)); + org.apache.lucene.geo.Rectangle r = org.apache.lucene.geo.Rectangle.fromPointDistance(0, 0, radius); + Rectangle box = new Rectangle(r.minLat, r.maxLat, r.minLon, r.maxLon); + assertThat(box.containsPoint(0.5, 0.5), equalTo(true)); + assertThat(box.containsPoint(0.52, 0.52), equalTo(true)); + assertThat(box.containsPoint(1, 1), equalTo(false)); radius = DistanceUnit.convert(200, DistanceUnit.MILES, DistanceUnit.METERS); - box = Rectangle.fromPointDistance(0, 179, radius); - assertThat(GeoUtils.rectangleContainsPoint(box, 0, -179), equalTo(true)); - assertThat(GeoUtils.rectangleContainsPoint(box, 0, -178), equalTo(false)); + r = org.apache.lucene.geo.Rectangle.fromPointDistance(0, 179, radius); + box = new Rectangle(r.minLat, r.maxLat, r.minLon, r.maxLon); + assertThat(box.containsPoint(0, -179), equalTo(true)); + assertThat(box.containsPoint(0, -178), equalTo(false)); } private static double arcDistance(GeoPoint p1, GeoPoint p2) { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java index 2142fca565c9b..d80d51320403e 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java @@ -39,7 +39,7 @@ import java.util.Collection; import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; -import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; +import static org.elasticsearch.geo.utils.Geohash.stringEncode; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.mapper.GeoPointFieldMapper.Names.IGNORE_Z_VALUE; import static org.elasticsearch.index.mapper.GeoPointFieldMapper.Names.NULL_VALUE; diff --git a/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java b/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java index b116c61d27c28..9af651119e642 100644 --- a/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java +++ b/server/src/test/java/org/elasticsearch/index/search/geo/GeoPointParsingTests.java @@ -32,7 +32,7 @@ import java.io.IOException; import java.util.function.DoubleSupplier; -import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; +import static org.elasticsearch.geo.utils.Geohash.stringEncode; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; import static org.hamcrest.Matchers.is; diff --git a/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java b/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java index 1a85e29f02090..14052ea403aab 100644 --- a/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java @@ -23,12 +23,12 @@ import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree; import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.test.ESTestCase; import org.locationtech.spatial4j.context.SpatialContext; import org.locationtech.spatial4j.distance.DistanceUtils; @@ -457,7 +457,7 @@ public void testParseGeoPointArrayZValueError() throws IOException { public void testParseGeoPointGeohash() throws IOException { for (int i = 0; i < 100; i++) { - int geoHashLength = randomIntBetween(1, GeoHashUtils.PRECISION); + int geoHashLength = randomIntBetween(1, Geohash.PRECISION); StringBuilder geohashBuilder = new StringBuilder(geoHashLength); for (int j = 0; j < geoHashLength; j++) { geohashBuilder.append(BASE_32[randomInt(BASE_32.length - 1)]); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java index 4ea9db378e1ef..1067215445550 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoHashGridIT.java @@ -44,8 +44,8 @@ import java.util.Random; import java.util.Set; -import static org.elasticsearch.common.geo.GeoHashUtils.PRECISION; -import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; +import static org.elasticsearch.geo.utils.Geohash.PRECISION; +import static org.elasticsearch.geo.utils.Geohash.stringEncode; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.search.aggregations.AggregationBuilders.geohashGrid; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java index 8cb42e352156b..f0494d2f0515b 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java @@ -20,7 +20,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.common.geo.GeoHashUtils; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.bucket.filter.Filter; @@ -65,7 +65,7 @@ private IndexRequestBuilder indexDoc(String date, int value) throws Exception { .startObject() .field("value", value) .field("ip", "10.0.0." + value) - .field("location", GeoHashUtils.stringEncode(5, 52, GeoHashUtils.PRECISION)) + .field("location", Geohash.stringEncode(5, 52, Geohash.PRECISION)) .field("date", date) .field("term-l", 1) .field("term-d", 1.5) diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorTests.java index d01a1d3b6d098..5d0d684cd49a8 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregatorTests.java @@ -19,7 +19,7 @@ package org.elasticsearch.search.aggregations.bucket.geogrid; -import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; +import static org.elasticsearch.geo.utils.Geohash.stringEncode; public class GeoHashGridAggregatorTests extends GeoGridAggregatorTestCase { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridTests.java index 9d4079646bab9..ce4748422336f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridTests.java @@ -18,8 +18,8 @@ */ package org.elasticsearch.search.aggregations.bucket.geogrid; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; @@ -46,7 +46,7 @@ protected InternalGeoHashGridBucket createInternalGeoGridBucket(Long key, long d @Override protected long longEncode(double lng, double lat, int precision) { - return GeoHashUtils.longEncode(lng, lat, precision); + return Geohash.longEncode(lng, lat, precision); } @Override diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java index 3c60ad52b7b39..c32c919dcc78f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java @@ -28,12 +28,12 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.Strings; import org.elasticsearch.common.document.DocumentField; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; @@ -216,8 +216,8 @@ public void setupSuiteScopeCluster() throws Exception { } private void updateGeohashBucketsCentroid(final GeoPoint location) { - String hash = GeoHashUtils.stringEncode(location.lon(), location.lat(), GeoHashUtils.PRECISION); - for (int precision = GeoHashUtils.PRECISION; precision > 0; --precision) { + String hash = Geohash.stringEncode(location.lon(), location.lat(), Geohash.PRECISION); + for (int precision = Geohash.PRECISION; precision > 0; --precision) { final String h = hash.substring(0, precision); expectedDocCountsForGeoHash.put(h, expectedDocCountsForGeoHash.getOrDefault(h, 0) + 1); expectedCentroidsForGeoHash.put(h, updateHashCentroid(h, location)); diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java index e47ca503e020e..a8ef8c10116c4 100644 --- a/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoDistanceIT.java @@ -23,13 +23,13 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.DistanceUnit; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugins.Plugin; @@ -62,7 +62,7 @@ public class GeoDistanceIT extends ESIntegTestCase { private static final double src_lon = -117.151; private static final double tgt_lat = 32.81; private static final double tgt_lon = -117.21; - private static final String tgt_geohash = GeoHashUtils.stringEncode(tgt_lon, tgt_lat); + private static final String tgt_geohash = Geohash.stringEncode(tgt_lon, tgt_lat); @Override protected Collection> nodePlugins() { @@ -148,8 +148,8 @@ public void testDistanceScript() throws Exception { Collections.emptyMap())).get(); Double resultDistance4 = searchResponse4.getHits().getHits()[0].getFields().get("distance").getValue(); assertThat(resultDistance4, - closeTo(GeoUtils.arcDistance(src_lat, src_lon, GeoHashUtils.decodeLatitude(tgt_geohash), - GeoHashUtils.decodeLongitude(tgt_geohash)), 0.01d)); + closeTo(GeoUtils.arcDistance(src_lat, src_lon, Geohash.decodeLatitude(tgt_geohash), + Geohash.decodeLongitude(tgt_geohash)), 0.01d)); // Test doc['location'].arcDistance(lat, lon + 360)/1000d SearchResponse searchResponse5 = client().prepareSearch().addStoredField("_source") diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java index b21d6724693fc..518f1bc572982 100644 --- a/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoFilterIT.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.common.geo.builders.CoordinatesBuilder; @@ -67,6 +66,7 @@ import java.util.zip.GZIPInputStream; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.geo.utils.Geohash.addNeighbors; import static org.elasticsearch.index.query.QueryBuilders.geoBoundingBoxQuery; import static org.elasticsearch.index.query.QueryBuilders.geoDistanceQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; @@ -433,26 +433,26 @@ public void testBulk() throws Exception { public void testNeighbors() { // Simple root case - assertThat(GeoHashUtils.addNeighbors("7", new ArrayList()), containsInAnyOrder("4", "5", "6", "d", "e", "h", "k", "s")); + assertThat(addNeighbors("7", new ArrayList<>()), containsInAnyOrder("4", "5", "6", "d", "e", "h", "k", "s")); // Root cases (Outer cells) - assertThat(GeoHashUtils.addNeighbors("0", new ArrayList()), containsInAnyOrder("1", "2", "3", "p", "r")); - assertThat(GeoHashUtils.addNeighbors("b", new ArrayList()), containsInAnyOrder("8", "9", "c", "x", "z")); - assertThat(GeoHashUtils.addNeighbors("p", new ArrayList()), containsInAnyOrder("n", "q", "r", "0", "2")); - assertThat(GeoHashUtils.addNeighbors("z", new ArrayList()), containsInAnyOrder("8", "b", "w", "x", "y")); + assertThat(addNeighbors("0", new ArrayList<>()), containsInAnyOrder("1", "2", "3", "p", "r")); + assertThat(addNeighbors("b", new ArrayList<>()), containsInAnyOrder("8", "9", "c", "x", "z")); + assertThat(addNeighbors("p", new ArrayList<>()), containsInAnyOrder("n", "q", "r", "0", "2")); + assertThat(addNeighbors("z", new ArrayList<>()), containsInAnyOrder("8", "b", "w", "x", "y")); // Root crossing dateline - assertThat(GeoHashUtils.addNeighbors("2", new ArrayList()), containsInAnyOrder("0", "1", "3", "8", "9", "p", "r", "x")); - assertThat(GeoHashUtils.addNeighbors("r", new ArrayList()), containsInAnyOrder("0", "2", "8", "n", "p", "q", "w", "x")); + assertThat(addNeighbors("2", new ArrayList<>()), containsInAnyOrder("0", "1", "3", "8", "9", "p", "r", "x")); + assertThat(addNeighbors("r", new ArrayList<>()), containsInAnyOrder("0", "2", "8", "n", "p", "q", "w", "x")); // level1: simple case - assertThat(GeoHashUtils.addNeighbors("dk", new ArrayList()), + assertThat(addNeighbors("dk", new ArrayList<>()), containsInAnyOrder("d5", "d7", "de", "dh", "dj", "dm", "ds", "dt")); // Level1: crossing cells - assertThat(GeoHashUtils.addNeighbors("d5", new ArrayList()), + assertThat(addNeighbors("d5", new ArrayList<>()), containsInAnyOrder("d4", "d6", "d7", "dh", "dk", "9f", "9g", "9u")); - assertThat(GeoHashUtils.addNeighbors("d0", new ArrayList()), + assertThat(addNeighbors("d0", new ArrayList<>()), containsInAnyOrder("d1", "d2", "d3", "9b", "9c", "6p", "6r", "3z")); } diff --git a/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java b/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java index 53235997283a4..87d84484aca0a 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java +++ b/server/src/test/java/org/elasticsearch/search/sort/GeoDistanceIT.java @@ -23,11 +23,11 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.geo.GeoDistance; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.geo.utils.Geohash; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.VersionUtils; @@ -380,7 +380,7 @@ public void testGeoDistanceFilter() throws IOException { .startObject("pin").field("type", "geo_point"); mapping.endObject().endObject().endObject().endObject(); - XContentBuilder source = JsonXContent.contentBuilder().startObject().field("pin", GeoHashUtils.stringEncode(lon, lat)).endObject(); + XContentBuilder source = JsonXContent.contentBuilder().startObject().field("pin", Geohash.stringEncode(lon, lat)).endObject(); assertAcked(prepareCreate("locations").setSettings(settings).addMapping("location", mapping)); client().prepareIndex("locations", "location", "1").setCreate(true).setSource(source).get(); diff --git a/server/src/test/java/org/elasticsearch/search/suggest/completion/GeoContextMappingTests.java b/server/src/test/java/org/elasticsearch/search/suggest/completion/GeoContextMappingTests.java index 85e97d16e3b4f..bbffb3e8287dd 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/completion/GeoContextMappingTests.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/completion/GeoContextMappingTests.java @@ -40,7 +40,7 @@ import java.util.Collection; import java.util.List; -import static org.elasticsearch.common.geo.GeoHashUtils.addNeighbors; +import static org.elasticsearch.geo.utils.Geohash.addNeighborsAtLevel; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.search.suggest.completion.CategoryContextMappingTests.assertContextSuggestFields; import static org.hamcrest.Matchers.equalTo; @@ -273,7 +273,7 @@ public void testParsingQueryContextBasic() throws Exception { assertThat(internalQueryContexts.size(), equalTo(1 + 8)); Collection locations = new ArrayList<>(); locations.add("ezs42e"); - addNeighbors("ezs42e", GeoContextMapping.DEFAULT_PRECISION, locations); + addNeighborsAtLevel("ezs42e", GeoContextMapping.DEFAULT_PRECISION, locations); for (ContextMapping.InternalQueryContext internalQueryContext : internalQueryContexts) { assertThat(internalQueryContext.context, isIn(locations)); assertThat(internalQueryContext.boost, equalTo(1)); @@ -292,7 +292,7 @@ public void testParsingQueryContextGeoPoint() throws Exception { assertThat(internalQueryContexts.size(), equalTo(1 + 8)); Collection locations = new ArrayList<>(); locations.add("wh0n94"); - addNeighbors("wh0n94", GeoContextMapping.DEFAULT_PRECISION, locations); + addNeighborsAtLevel("wh0n94", GeoContextMapping.DEFAULT_PRECISION, locations); for (ContextMapping.InternalQueryContext internalQueryContext : internalQueryContexts) { assertThat(internalQueryContext.context, isIn(locations)); assertThat(internalQueryContext.boost, equalTo(1)); @@ -316,11 +316,11 @@ public void testParsingQueryContextObject() throws Exception { Collection locations = new ArrayList<>(); locations.add("wh0n94"); locations.add("w"); - addNeighbors("w", 1, locations); + addNeighborsAtLevel("w", 1, locations); locations.add("wh"); - addNeighbors("wh", 2, locations); + addNeighborsAtLevel("wh", 2, locations); locations.add("wh0"); - addNeighbors("wh0", 3, locations); + addNeighborsAtLevel("wh0", 3, locations); for (ContextMapping.InternalQueryContext internalQueryContext : internalQueryContexts) { assertThat(internalQueryContext.context, isIn(locations)); assertThat(internalQueryContext.boost, equalTo(10)); @@ -354,15 +354,15 @@ public void testParsingQueryContextObjectArray() throws Exception { Collection firstLocations = new ArrayList<>(); firstLocations.add("wh0n94"); firstLocations.add("w"); - addNeighbors("w", 1, firstLocations); + addNeighborsAtLevel("w", 1, firstLocations); firstLocations.add("wh"); - addNeighbors("wh", 2, firstLocations); + addNeighborsAtLevel("wh", 2, firstLocations); firstLocations.add("wh0"); - addNeighbors("wh0", 3, firstLocations); + addNeighborsAtLevel("wh0", 3, firstLocations); Collection secondLocations = new ArrayList<>(); secondLocations.add("w5cx04"); secondLocations.add("w5cx0"); - addNeighbors("w5cx0", 5, secondLocations); + addNeighborsAtLevel("w5cx0", 5, secondLocations); for (ContextMapping.InternalQueryContext internalQueryContext : internalQueryContexts) { if (firstLocations.contains(internalQueryContext.context)) { assertThat(internalQueryContext.boost, equalTo(10)); @@ -397,12 +397,12 @@ public void testParsingQueryContextMixed() throws Exception { Collection firstLocations = new ArrayList<>(); firstLocations.add("wh0n94"); firstLocations.add("w"); - addNeighbors("w", 1, firstLocations); + addNeighborsAtLevel("w", 1, firstLocations); firstLocations.add("wh"); - addNeighbors("wh", 2, firstLocations); + addNeighborsAtLevel("wh", 2, firstLocations); Collection secondLocations = new ArrayList<>(); secondLocations.add("w5cx04"); - addNeighbors("w5cx04", 6, secondLocations); + addNeighborsAtLevel("w5cx04", 6, secondLocations); for (ContextMapping.InternalQueryContext internalQueryContext : internalQueryContexts) { if (firstLocations.contains(internalQueryContext.context)) { assertThat(internalQueryContext.boost, equalTo(10)); From 94f7ee57e02d17d1cb4700ded58bae5a4474f70c Mon Sep 17 00:00:00 2001 From: Jash Gala Date: Thu, 25 Apr 2019 22:34:27 +0530 Subject: [PATCH 188/260] Fixed very small typo in date (#41398) --- docs/reference/api-conventions.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index aa6c447f693bc..0b7d9ce403466 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -126,7 +126,7 @@ The special characters used for date rounding must be URI encoded as follows: ====================================================== The following example shows different forms of date math index names and the final index names -they resolve to given the current time is 22rd March 2024 noon utc. +they resolve to given the current time is 22nd March 2024 noon utc. [options="header"] |====== From 32ca5122352a83d46f23689e3ca12d4c72cb1191 Mon Sep 17 00:00:00 2001 From: RomainGeffraye Date: Thu, 25 Apr 2019 19:28:36 +0200 Subject: [PATCH 189/260] Update configuring-ldap-realm.asciidoc (#40427) --- .../en/security/authentication/configuring-ldap-realm.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc index c023a8492b166..c7f793d92f32a 100644 --- a/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-ldap-realm.asciidoc @@ -132,7 +132,7 @@ See <>. . Map LDAP groups to roles. + -- -The `ldap` realm enables you to map LDAP users to to roles via their LDAP +The `ldap` realm enables you to map LDAP users to roles via their LDAP groups, or other metadata. This role mapping can be configured via the {ref}/security-api-put-role-mapping.html[add role mapping API] or by using a file stored on each node. When a user authenticates with LDAP, the privileges From bade776be6ed30cfc02ed98bf7d7acf506b545f0 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 25 Apr 2019 13:58:48 -0400 Subject: [PATCH 190/260] [DOCS] Note TESTRESPONSE can't be used immediately after TESTSETUP (#41542) --- docs/README.asciidoc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/README.asciidoc b/docs/README.asciidoc index 5aca8cb56c226..0a2840b02e7e4 100644 --- a/docs/README.asciidoc +++ b/docs/README.asciidoc @@ -60,6 +60,8 @@ 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[_cat]`: Add substitutions for testing `_cat` responses. Use this after all other substitutions so it doesn't make other substitutions difficult. From 34ae1f9c7d406d7b66d9f33723d63fe8984d12c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 25 Apr 2019 20:47:03 +0200 Subject: [PATCH 191/260] [Docs] Fix common word repetitions (#39703) --- .../org/elasticsearch/client/ml/job/process/DataCounts.java | 2 +- .../painless-bucket-selector-agg-context.asciidoc | 4 ++-- docs/reference/docs/index_.asciidoc | 2 +- docs/reference/ilm/apis/explain.asciidoc | 2 +- docs/reference/mapping/removal_of_types.asciidoc | 2 +- .../elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java | 2 +- .../main/java/org/elasticsearch/dissect/DissectParser.java | 2 +- .../java/org/elasticsearch/dissect/DissectParserTests.java | 2 +- .../src/main/java/org/elasticsearch/common/ssl/PemUtils.java | 2 +- .../main/java/org/elasticsearch/ingest/common/DateFormat.java | 2 +- .../org/elasticsearch/index/rankeval/RankEvalRequest.java | 2 +- .../org/elasticsearch/index/engine/SoftDeletesPolicy.java | 2 +- .../src/main/java/org/elasticsearch/index/mapper/Mapper.java | 2 +- .../elasticsearch/persistent/PersistentTasksExecutorIT.java | 4 ++-- .../persistent/PersistentTasksNodeServiceTests.java | 4 ++-- x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc | 2 +- x-pack/docs/en/security/fips-140-compliance.asciidoc | 2 +- .../ccr/repository/CcrRepositoryRetentionLeaseTests.java | 4 ++-- .../elasticsearch/snapshots/SourceOnlySnapshotRepository.java | 2 +- .../core/ml/job/process/autodetect/state/DataCounts.java | 2 +- .../elasticsearch/xpack/core/scheduler/SchedulerEngine.java | 2 +- .../core/security/authz/permission/IndicesPermission.java | 2 +- .../main/java/org/elasticsearch/xpack/core/ssl/PemUtils.java | 2 +- .../java/org/elasticsearch/xpack/security/PermissionsIT.java | 2 +- .../elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java | 2 +- .../org/elasticsearch/xpack/monitoring/MonitoringService.java | 2 +- .../xpack/monitoring/exporter/http/MultiHttpResource.java | 2 +- .../org/elasticsearch/xpack/security/authc/TokenService.java | 2 +- .../xpack/security/support/SecurityIndexManager.java | 2 +- .../transport/actions/get/TransportGetWatchAction.java | 2 +- .../watcher/test/integration/HttpSecretsIntegrationTests.java | 2 +- 31 files changed, 35 insertions(+), 35 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/DataCounts.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/DataCounts.java index 7afef0785fe38..c0e16622ba593 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/DataCounts.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/job/process/DataCounts.java @@ -34,7 +34,7 @@ * Job processed record counts. *

    * The getInput... methods return the actual number of - * fields/records sent the the API including invalid records. + * fields/records sent the API including invalid records. * The getProcessed... methods are the number sent to the * Engine. *

    diff --git a/docs/painless/painless-contexts/painless-bucket-selector-agg-context.asciidoc b/docs/painless/painless-contexts/painless-bucket-selector-agg-context.asciidoc index 8e20cf77c353d..69fbce1d0828f 100644 --- a/docs/painless/painless-contexts/painless-bucket-selector-agg-context.asciidoc +++ b/docs/painless/painless-contexts/painless-bucket-selector-agg-context.asciidoc @@ -15,7 +15,7 @@ to determine if a bucket should be retained or filtered out. ==== Return boolean:: - True if the the bucket should be retained, false if the bucket should be filtered out. + True if the bucket should be retained, false if the bucket should be filtered out. ==== API @@ -78,4 +78,4 @@ GET /seats/_search // TEST[setup:seats] <1> The `buckets_path` points to the max aggregations (`max_cost`) and adds `max` variables to the `params` map -<2> The user-specified `base_cost` is also added to the `params` map \ No newline at end of file +<2> The user-specified `base_cost` is also added to the `params` map diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 18b3589d5defc..2c617832488b5 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -367,7 +367,7 @@ PUT twitter/_doc/1?version=2&version_type=external near real time aspects of search operations. If no version is provided, then the operation is executed without any version checks. -The above will succeed since the the supplied version of 2 is higher than +The above will succeed since the supplied version of 2 is higher than the current document version of 1. If the document was already updated and its version was set to 2 or higher, the indexing command will fail and result in a conflict (409 http status code). diff --git a/docs/reference/ilm/apis/explain.asciidoc b/docs/reference/ilm/apis/explain.asciidoc index 66762ead9eb32..26c2cb2d26b75 100644 --- a/docs/reference/ilm/apis/explain.asciidoc +++ b/docs/reference/ilm/apis/explain.asciidoc @@ -227,7 +227,7 @@ information for the step that's being performed on the index. <1> Status of the step that's in progress. If the index is in the ERROR step, something went wrong while executing a -step in the policy and and you will need to take action for the index to proceed +step in the policy and you will need to take action for the index to proceed to the next step. To help you diagnose the problem, the explain response shows the step that failed and the step info provides information about the error. diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index a5b0a4e89413e..8e78f3f140596 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -431,7 +431,7 @@ In Elasticsearch 7.0, each API will support typeless requests, and specifying a type will produce a deprecation warning. NOTE: Typeless APIs work even if the target index contains a custom type. -For example, if an index has the the custom type name `my_type`, we can add +For example, if an index has the custom type name `my_type`, we can add documents to it using typeless `index` calls, and load documents with typeless `get` calls. diff --git a/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java b/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java index bbc6a64dcdb2b..e87df16264e94 100644 --- a/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java +++ b/docs/src/test/java/org/elasticsearch/smoketest/DocsClientYamlTestSuiteIT.java @@ -98,7 +98,7 @@ protected ClientYamlTestClient initClientYamlTestClient( } /** - * Compares the the results of running two analyzers against many random + * Compares the results of running two analyzers against many random * strings. The goal is to figure out if two anlayzers are "the same" by * comparing their results. This is far from perfect but should be fairly * accurate, especially for gross things like missing {@code decimal_digit} diff --git a/libs/dissect/src/main/java/org/elasticsearch/dissect/DissectParser.java b/libs/dissect/src/main/java/org/elasticsearch/dissect/DissectParser.java index 407d73134b611..66edf0646882e 100644 --- a/libs/dissect/src/main/java/org/elasticsearch/dissect/DissectParser.java +++ b/libs/dissect/src/main/java/org/elasticsearch/dissect/DissectParser.java @@ -182,7 +182,7 @@ public Map parse(String inputString) { * all of the bytes then progress the main cursor. Repeat till the end of the input string. Since the string being searching for * (the delimiter) is generally small and rare the naive approach is efficient. * - * In this case the the string that is walked is the input string, and the string being searched for is the current delimiter. + * In this case the string that is walked is the input string, and the string being searched for is the current delimiter. * For example for a dissect pattern of {@code %{a},%{b}:%{c}} the delimiters (comma then colon) are searched for in the * input string. At class construction the list of keys+delimiters are found (dissectPairs), which allows the use of that ordered * list to know which delimiter to use for the search. The delimiters is progressed once the current delimiter is matched. diff --git a/libs/dissect/src/test/java/org/elasticsearch/dissect/DissectParserTests.java b/libs/dissect/src/test/java/org/elasticsearch/dissect/DissectParserTests.java index c5d35bb9588c7..c604b8c089b84 100644 --- a/libs/dissect/src/test/java/org/elasticsearch/dissect/DissectParserTests.java +++ b/libs/dissect/src/test/java/org/elasticsearch/dissect/DissectParserTests.java @@ -67,7 +67,7 @@ public void testLogstashSpecs() { assertMatch("%{a} » %{b}»%{c}€%{d}", "foo » bar»baz€quux", Arrays.asList("a", "b", "c", "d"), Arrays.asList("foo", "bar", "baz", "quux")); assertMatch("%{a} %{b} %{+a}", "foo bar baz quux", Arrays.asList("a", "b"), Arrays.asList("foo baz quux", "bar"), " "); - //Logstash supports implicit ordering based anchored by the the key without the '+' + //Logstash supports implicit ordering based anchored by the key without the '+' //This implementation will only honor implicit ordering for appending right to left else explicit order (/N) is required. //The results of this test differ from Logstash. assertMatch("%{+a} %{a} %{+a} %{b}", "December 31 1999 quux", diff --git a/libs/ssl-config/src/main/java/org/elasticsearch/common/ssl/PemUtils.java b/libs/ssl-config/src/main/java/org/elasticsearch/common/ssl/PemUtils.java index aca7ba56b2ae9..79943870c52f6 100644 --- a/libs/ssl-config/src/main/java/org/elasticsearch/common/ssl/PemUtils.java +++ b/libs/ssl-config/src/main/java/org/elasticsearch/common/ssl/PemUtils.java @@ -397,7 +397,7 @@ private static byte[] possiblyDecryptPKCS1Key(Map pemHeaders, St * defined in RFC 1423. RFC 1423 only defines DES-CBS and triple DES (EDE) in CBC mode. AES in CBC mode is also widely used though ( 3 * different variants of 128, 192, 256 bit keys ) * - * @param dekHeaderValue The value of the the DEK-Info PEM header + * @param dekHeaderValue The value of the DEK-Info PEM header * @param password The password with which the key is encrypted * @return a cipher of the appropriate algorithm and parameters to be used for decryption * @throws GeneralSecurityException if the algorithm is not available in the used security provider, or if the key is inappropriate diff --git a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/DateFormat.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/DateFormat.java index ae3bc6b5d780d..65efdb40a5cc1 100644 --- a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/DateFormat.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/DateFormat.java @@ -92,7 +92,7 @@ Function getFunction(String format, ZoneId zoneId, Locale int year = LocalDate.now(ZoneOffset.UTC).getYear(); DateFormatter dateFormatter = DateFormatter.forPattern(format) .withLocale(locale); - // if UTC zone is set here, the the time zone specified in the format will be ignored, leading to wrong dates + // if UTC zone is set here, the time zone specified in the format will be ignored, leading to wrong dates if (isUtc == false) { dateFormatter = dateFormatter.withZone(zoneId); } diff --git a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java index 7d3ec94811c5a..0dbbb9f90f1fa 100644 --- a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java +++ b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java @@ -86,7 +86,7 @@ public RankEvalSpec getRankEvalSpec() { } /** - * Set the the specification of the ranking evaluation. + * Set the specification of the ranking evaluation. */ public void setRankEvalSpec(RankEvalSpec task) { this.rankingEvaluationSpec = task; diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java index 4c9ee0be92f46..2d9fc0077324d 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -120,7 +120,7 @@ synchronized long getMinRetainedSeqNo() { * localCheckpointOfSafeCommit. * - Changes APIs are driven by a combination of the global checkpoint, retention operations, and retention leases. Here we * prefer using the global checkpoint instead of the maximum sequence number because only operations up to the global - * checkpoint are exposed in the the changes APIs. + * checkpoint are exposed in the changes APIs. */ // calculate the minimum sequence number to retain based on retention leases diff --git a/server/src/main/java/org/elasticsearch/index/mapper/Mapper.java b/server/src/main/java/org/elasticsearch/index/mapper/Mapper.java index 9baaafe9c4e00..d98630e5f765e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/Mapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/Mapper.java @@ -171,7 +171,7 @@ public final String simpleName() { public abstract String name(); /** - * Returns a name representing the the type of this mapper. + * Returns a name representing the type of this mapper. */ public abstract String typeName(); diff --git a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java index 08c32665adc58..4acb391d9c0ee 100644 --- a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java +++ b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksExecutorIT.java @@ -162,7 +162,7 @@ public void testPersistentActionWithNoAvailableNode() throws Exception { TaskInfo taskInfo = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]") .get().getTasks().get(0); - // Verifying the the task runs on the new node + // Verifying the task runs on the new node assertThat(taskInfo.getTaskId().getNodeId(), equalTo(newNodeId)); internalCluster().stopRandomNode(settings -> "test".equals(settings.get("node.attr.test_attr"))); @@ -202,7 +202,7 @@ public void testPersistentActionWithNonClusterStateCondition() throws Exception TaskInfo taskInfo = client().admin().cluster().prepareListTasks().setActions(TestPersistentTasksExecutor.NAME + "[c]") .get().getTasks().get(0); - // Verifying the the task can now be assigned + // Verifying the task can now be assigned assertThat(taskInfo.getTaskId().getNodeId(), notNullValue()); // Remove the persistent task diff --git a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksNodeServiceTests.java b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksNodeServiceTests.java index 683d1bc00893b..735efb68091fd 100644 --- a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksNodeServiceTests.java +++ b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksNodeServiceTests.java @@ -269,7 +269,7 @@ public void sendCompletionRequest(final String taskId, final long taskAllocation newClusterState = addTask(state, "test", null, "this_node"); coordinator.clusterChanged(new ClusterChangedEvent("test", newClusterState, state)); - // Check the the task is know to the task manager + // Check the task is know to the task manager assertThat(taskManager.getTasks().size(), equalTo(1)); AllocatedPersistentTask runningTask = (AllocatedPersistentTask)taskManager.getTasks().values().iterator().next(); String persistentId = runningTask.getPersistentTaskId(); @@ -305,7 +305,7 @@ public void sendCompletionRequest(final String taskId, final long taskAllocation executor.get(0).task.markAsFailed(new IOException("test")); } - // Check the the task is now removed from task manager + // Check the task is now removed from task manager assertThat(taskManager.getTasks().values(), empty()); } 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 8c7f747969373..222fc2c8939c7 100644 --- a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc @@ -99,7 +99,7 @@ are five possible modes an action can be associated with: send it. In this mode, the action might be throttled if the current state of the watch indicates it should be. -| `force_simulate` | Similar to the the `simulate` mode, except the action is +| `force_simulate` | Similar to the `simulate` mode, except the action is not be throttled even if the current state of the watch indicates it should be. diff --git a/x-pack/docs/en/security/fips-140-compliance.asciidoc b/x-pack/docs/en/security/fips-140-compliance.asciidoc index 6bc9be512db4e..8c11503b3d3fb 100644 --- a/x-pack/docs/en/security/fips-140-compliance.asciidoc +++ b/x-pack/docs/en/security/fips-140-compliance.asciidoc @@ -89,7 +89,7 @@ the following table: {es} offers a number of algorithms for securely hashing credentials in memory and on disk. However, only the `PBKDF2` family of algorithms is compliant with FIPS -140-2 for password hashing. You must set the the `cache.hash_algo` realm settings +140-2 for password hashing. You must set the `cache.hash_algo` realm settings and the `xpack.security.authc.password_hashing.algorithm` setting to one of the available `PBKDF2` values. See <>. diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java index 2e382f739300b..8f49074b502a0 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRepositoryRetentionLeaseTests.java @@ -68,7 +68,7 @@ public void testWhenRetentionLeaseAlreadyExistsWeTryToRenewIt() { final String retentionLeaseId = retentionLeaseId("local-cluster", followerShardId.getIndex(), "remote-cluster", leaderShardId.getIndex()); - // simulate that the the retention lease already exists on the leader, and verify that we attempt to renew it + // simulate that the retention lease already exists on the leader, and verify that we attempt to renew it final Client remoteClient = mock(Client.class); final ArgumentCaptor addRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); @@ -133,7 +133,7 @@ public void testWhenRetentionLeaseExpiresBeforeWeCanRenewIt() { final String retentionLeaseId = retentionLeaseId("local-cluster", followerShardId.getIndex(), "remote-cluster", leaderShardId.getIndex()); - // simulate that the the retention lease already exists on the leader, expires before we renew, and verify that we attempt to add it + // simulate that the retention lease already exists on the leader, expires before we renew, and verify that we attempt to add it final Client remoteClient = mock(Client.class); final ArgumentCaptor addRequestCaptor = ArgumentCaptor.forClass(RetentionLeaseActions.AddRequest.class); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 3e36fc5977491..d7f70cf8ef2e1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -45,7 +45,7 @@ *

    * This is a filter snapshot repository that only snapshots the minimal required information * that is needed to recreate the index. In other words instead of snapshotting the entire shard - * with all it's lucene indexed fields, doc values, points etc. it only snapshots the the stored + * with all it's lucene indexed fields, doc values, points etc. it only snapshots the stored * fields including _source and _routing as well as the live docs in oder to distinguish between * live and deleted docs. *

    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/process/autodetect/state/DataCounts.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/process/autodetect/state/DataCounts.java index 08a6d8846f9a3..8d542ce25af93 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/process/autodetect/state/DataCounts.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/process/autodetect/state/DataCounts.java @@ -24,7 +24,7 @@ * Job processed record counts. *

    * The getInput... methods return the actual number of - * fields/records sent the the API including invalid records. + * fields/records sent the API including invalid records. * The getProcessed... methods are the number sent to the * Engine. *

    diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/SchedulerEngine.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/SchedulerEngine.java index 71784a8e9ebfd..95dca09661978 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/SchedulerEngine.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/scheduler/SchedulerEngine.java @@ -197,7 +197,7 @@ public void run() { } catch (final Throwable t) { /* * Allowing the throwable to escape here will lead to be it being caught in FutureTask#run and set as the outcome of this - * task; however, we never inspect the the outcomes of these scheduled tasks and so allowing the throwable to escape + * task; however, we never inspect the outcomes of these scheduled tasks and so allowing the throwable to escape * unhandled here could lead to us losing fatal errors. Instead, we rely on ExceptionsHelper#maybeDieOnAnotherThread to * appropriately dispatch any error to the uncaught exception handler. We should never see an exception here as these do * not escape from SchedulerEngine#notifyListeners. diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java index aab6bab8110af..2e51e07bc7676 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/IndicesPermission.java @@ -277,7 +277,7 @@ public static class Group { private final Set query; // by default certain restricted indices are exempted when granting privileges, as they should generally be hidden for ordinary // users. Setting this flag true eliminates the special status for the purpose of this permission - restricted indices still have - // to be covered by the the "indices" + // to be covered by the "indices" private final boolean allowRestrictedIndices; public Group(IndexPrivilege privilege, FieldPermissions fieldPermissions, @Nullable Set query, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/PemUtils.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/PemUtils.java index 421b30baac7b6..1e67e15a33c59 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/PemUtils.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/PemUtils.java @@ -374,7 +374,7 @@ private static byte[] possiblyDecryptPKCS1Key(Map pemHeaders, St * defined in RFC 1423. RFC 1423 only defines DES-CBS and triple DES (EDE) in CBC mode. AES in CBC mode is also widely used though ( 3 * different variants of 128, 192, 256 bit keys ) * - * @param dekHeaderValue The value of the the DEK-Info PEM header + * @param dekHeaderValue The value of the DEK-Info PEM header * @param password The password with which the key is encrypted * @return a cipher of the appropriate algorithm and parameters to be used for decryption * @throws GeneralSecurityException if the algorithm is not available in the used security provider, or if the key is inappropriate diff --git a/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java b/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java index 5503e12cb8b0e..40f322945dcb2 100644 --- a/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java +++ b/x-pack/plugin/ilm/qa/with-security/src/test/java/org/elasticsearch/xpack/security/PermissionsIT.java @@ -87,7 +87,7 @@ public void init() throws Exception { /** * Tests that a policy that simply deletes an index after 0s succeeds when an index * with user `test_admin` is created referencing a policy created by `test_ilm` when both - * users have read/write permissions on the the index. The goal is to verify that one + * users have read/write permissions on the index. The goal is to verify that one * does not need to be the same user who created both the policy and the index to have the * index be properly managed by ILM. */ diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java index 5af9c53649853..3ccf0f22da922 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java @@ -43,7 +43,7 @@ public interface ChainTask { * {@code true} means continue on to the next task. * Must be able to handle null values. * @param failureShortCircuitPredicate The predicate on whether to short circuit execution on a give exception. - * {@code true} means that no more tasks should execute and the the listener::onFailure should be + * {@code true} means that no more tasks should execute and the listener::onFailure should be * called. */ public TypedChainTaskExecutor(ExecutorService executorService, diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringService.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringService.java index 1aa7ef230a390..28cc769c46d7f 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringService.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringService.java @@ -233,7 +233,7 @@ protected void doRun() throws Exception { final Collection results = new ArrayList<>(); for (Collector collector : collectors) { if (isStarted() == false) { - // Do not collect more data if the the monitoring service is stopping + // Do not collect more data if the monitoring service is stopping // otherwise some collectors might just fail. return; } diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/MultiHttpResource.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/MultiHttpResource.java index d902abf71131e..44bda96a82d1d 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/MultiHttpResource.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/MultiHttpResource.java @@ -19,7 +19,7 @@ *

    * By telling the {@code MultiHttpResource} to become dirty, it effectively marks all of its sub-resources dirty as well. *

    - * Sub-resources should be the sole responsibility of the the {@code MultiHttpResource}; there should not be something using them directly + * Sub-resources should be the sole responsibility of the {@code MultiHttpResource}; there should not be something using them directly * if they are included in a {@code MultiHttpResource}. */ public class MultiHttpResource extends HttpResource { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java index be3aeba8d4833..19f7a50471043 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java @@ -349,7 +349,7 @@ public void getAuthenticationAndMetaData(String token, ActionListener listener) { if (securityIndex.isAvailable() == false) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java index 9a316a9ace4c6..31cf1f1568f24 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java @@ -73,7 +73,7 @@ import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; /** - * Manages the lifecycle of a single index, mapping and and data upgrades/migrations. + * Manages the lifecycle of a single index, mapping and data upgrades/migrations. */ public class SecurityIndexManager implements ClusterStateListener { diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/get/TransportGetWatchAction.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/get/TransportGetWatchAction.java index d0f257717c4ba..b8d6be360de9a 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/get/TransportGetWatchAction.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/transport/actions/get/TransportGetWatchAction.java @@ -61,7 +61,7 @@ protected void doExecute(GetWatchRequest request, ActionListener Date: Thu, 25 Apr 2019 14:00:44 -0500 Subject: [PATCH 192/260] [ML] add multi node integ tests for data frames (#41508) * [ML] adding native-multi-node-integTests for data frames' * addressing streaming issues * formatting fixes * Addressing PR comments --- .../xpack/core/XPackClientPlugin.java | 19 +- .../GetDataFrameTransformsStatsAction.java | 18 +- .../action/StartDataFrameTransformAction.java | 2 +- .../StartDataFrameTransformTaskAction.java | 2 +- ...ameTransformsStatsActionResponseTests.java | 16 +- .../qa/multi-node-tests/build.gradle | 55 +++ .../integration/DataFrameIntegTestCase.java | 332 ++++++++++++++++++ .../integration/DataFrameTransformIT.java | 60 ++++ ...portGetDataFrameTransformsStatsAction.java | 71 ++-- ...ransportStartDataFrameTransformAction.java | 9 +- 10 files changed, 531 insertions(+), 53 deletions(-) create mode 100644 x-pack/plugin/data-frame/qa/multi-node-tests/build.gradle create mode 100644 x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java create mode 100644 x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformIT.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index dc8403b7bd548..a145569898ee6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -44,6 +44,14 @@ import org.elasticsearch.xpack.core.ccr.CCRFeatureSet; import org.elasticsearch.xpack.core.dataframe.DataFrameFeatureSetUsage; import org.elasticsearch.xpack.core.dataframe.DataFrameField; +import org.elasticsearch.xpack.core.dataframe.action.DeleteDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsAction; +import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction; +import org.elasticsearch.xpack.core.dataframe.action.PreviewDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformTaskAction; +import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState; import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction; @@ -363,7 +371,16 @@ public List> getClientActions() { RemoveIndexLifecyclePolicyAction.INSTANCE, MoveToStepAction.INSTANCE, RetryAction.INSTANCE, - TransportFreezeIndexAction.FreezeIndexAction.INSTANCE + TransportFreezeIndexAction.FreezeIndexAction.INSTANCE, + // Data Frame + PutDataFrameTransformAction.INSTANCE, + StartDataFrameTransformAction.INSTANCE, + StartDataFrameTransformTaskAction.INSTANCE, + StopDataFrameTransformAction.INSTANCE, + DeleteDataFrameTransformAction.INSTANCE, + GetDataFrameTransformsAction.INSTANCE, + GetDataFrameTransformsStatsAction.INSTANCE, + PreviewDataFrameTransformAction.INSTANCE ); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java index 96763153240e5..e9c71d3718b22 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java @@ -56,7 +56,7 @@ public static class Request extends BaseTasksRequest { public static final int MAX_SIZE_RETURN = 1000; // used internally to expand the queried id expression - private List expandedIds = Collections.emptyList(); + private List expandedIds; public Request(String id) { if (Strings.isNullOrEmpty(id) || id.equals("*")) { @@ -64,13 +64,14 @@ public Request(String id) { } else { this.id = id; } + this.expandedIds = Collections.singletonList(id); } public Request(StreamInput in) throws IOException { super(in); id = in.readString(); - expandedIds = in.readList(StreamInput::readString); - pageParams = in.readOptionalWriteable(PageParams::new); + expandedIds = Collections.unmodifiableList(in.readStringList()); + pageParams = new PageParams(in); } @Override @@ -93,7 +94,7 @@ public void setExpandedIds(List expandedIds) { } public final void setPageParams(PageParams pageParams) { - this.pageParams = pageParams; + this.pageParams = Objects.requireNonNull(pageParams); } public final PageParams getPageParams() { @@ -105,7 +106,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(id); out.writeStringCollection(expandedIds); - out.writeOptionalWriteable(pageParams); + pageParams.writeTo(out); } @Override @@ -136,7 +137,7 @@ public boolean equals(Object obj) { } } - public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { + public static class Response extends BaseTasksResponse implements ToXContentObject { private List transformsStateAndStats; public Response(List transformsStateAndStats) { @@ -165,6 +166,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeList(transformsStateAndStats); } + @Override + public void readFrom(StreamInput in) { + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java index d2550ab2f410c..29529c1d67452 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java @@ -95,7 +95,7 @@ public boolean equals(Object obj) { } } - public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { + public static class Response extends BaseTasksResponse implements ToXContentObject { private final boolean started; public Response(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java index 934d1bae43319..52f061e380e82 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java @@ -93,7 +93,7 @@ public boolean equals(Object obj) { } } - public static class Response extends BaseTasksResponse implements Writeable, ToXContentObject { + public static class Response extends BaseTasksResponse implements ToXContentObject { private final boolean started; public Response(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsActionResponseTests.java index 4a293f42bb309..bbe53b59a8ace 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsActionResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsActionResponseTests.java @@ -6,6 +6,9 @@ package org.elasticsearch.xpack.core.dataframe.action; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.common.io.stream.Writeable.Reader; import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction.Response; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats; @@ -18,11 +21,18 @@ public class GetDataFrameTransformsStatsActionResponseTests extends AbstractWire @Override protected Response createTestInstance() { List stats = new ArrayList<>(); - for (int i = 0; i < randomInt(10); ++i) { + int totalStats = randomInt(10); + for (int i = 0; i < totalStats; ++i) { stats.add(DataFrameTransformStateAndStatsTests.randomDataFrameTransformStateAndStats()); } - - return new Response(stats); + int totalErrors = randomInt(10); + List taskFailures = new ArrayList<>(totalErrors); + List nodeFailures = new ArrayList<>(totalErrors); + for (int i = 0; i < totalErrors; i++) { + taskFailures.add(new TaskOperationFailure("node1", randomLongBetween(1, 10), new Exception("error"))); + nodeFailures.add(new FailedNodeException("node1", "message", new Exception("error"))); + } + return new Response(stats, taskFailures, nodeFailures); } @Override diff --git a/x-pack/plugin/data-frame/qa/multi-node-tests/build.gradle b/x-pack/plugin/data-frame/qa/multi-node-tests/build.gradle new file mode 100644 index 0000000000000..ab170d6be364f --- /dev/null +++ b/x-pack/plugin/data-frame/qa/multi-node-tests/build.gradle @@ -0,0 +1,55 @@ +apply plugin: 'elasticsearch.standalone-rest-test' +apply plugin: 'elasticsearch.rest-test' + +dependencies { + testCompile project(path: xpackModule('core'), configuration: 'default') + testCompile project(path: xpackModule('core'), configuration: 'testArtifacts') + testCompile project(path: xpackModule('data-frame'), configuration: 'runtime') +} + +// location for keys and certificates +File keystoreDir = new File(project.buildDir, 'keystore') +File nodeKey = file("$keystoreDir/testnode.pem") +File nodeCert = file("$keystoreDir/testnode.crt") +// Add key and certs to test classpath: it expects it there +task copyKeyCerts(type: Copy) { + from(project(':x-pack:plugin:core').file('src/test/resources/org/elasticsearch/xpack/security/transport/ssl/certs/simple/')) { + include 'testnode.crt', 'testnode.pem' + } + into keystoreDir +} +// Add keys and cets to test classpath: it expects it there +sourceSets.test.resources.srcDir(keystoreDir) +processTestResources.dependsOn(copyKeyCerts) + +integTestCluster { + dependsOn copyKeyCerts + setting 'xpack.security.enabled', 'true' + setting 'xpack.license.self_generated.type', 'trial' + setting 'xpack.monitoring.enabled', 'false' + setting 'xpack.security.authc.token.enabled', 'true' + setting 'xpack.security.transport.ssl.enabled', 'true' + setting 'xpack.security.transport.ssl.key', nodeKey.name + setting 'xpack.security.transport.ssl.certificate', nodeCert.name + setting 'xpack.security.transport.ssl.verification_mode', 'certificate' + setting 'xpack.security.audit.enabled', 'false' + setting 'xpack.license.self_generated.type', 'trial' + keystoreSetting 'bootstrap.password', 'x-pack-test-password' + keystoreSetting 'xpack.security.transport.ssl.secure_key_passphrase', 'testnode' + setupCommand 'setupDummyUser', + 'bin/elasticsearch-users', 'useradd', 'x_pack_rest_user', '-p', 'x-pack-test-password', '-r', 'superuser' + + numNodes = 3 + extraConfigFile nodeKey.name, nodeKey + extraConfigFile nodeCert.name, nodeCert + waitCondition = { node, ant -> + File tmpFile = new File(node.cwd, 'wait.success') + ant.get(src: "http://${node.httpUri()}/_cluster/health?wait_for_nodes=>=${numNodes}&wait_for_status=yellow", + dest: tmpFile.toString(), + username: 'x_pack_rest_user', + password: 'x-pack-test-password', + ignoreerrors: true, + retries: 10) + return tmpFile.exists() + } +} diff --git a/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java b/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java new file mode 100644 index 0000000000000..b45325fffaf08 --- /dev/null +++ b/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java @@ -0,0 +1,332 @@ +/* + * 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.integration; + +import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.PathUtils; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.DeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.SecuritySettingsSourceField; +import org.elasticsearch.transport.Netty4Plugin; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.core.XPackClientPlugin; +import org.elasticsearch.xpack.core.dataframe.action.DeleteDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction; +import org.elasticsearch.xpack.core.dataframe.action.PutDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.StartDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.action.StopDataFrameTransformAction; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DestConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.QueryConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.SourceConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.AggregationConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.DateHistogramGroupSource; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.GroupConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.PivotConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.SingleGroupSource; +import org.elasticsearch.xpack.core.security.SecurityField; + +import java.net.URISyntaxException; +import java.nio.file.Path; +import java.time.ZoneId; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.core.Is.is; + +abstract class DataFrameIntegTestCase extends ESIntegTestCase { + + protected static final String REVIEWS_INDEX_NAME = "data_frame_reviews"; + + private Map transformConfigs = new HashMap<>(); + + protected void cleanUp() { + cleanUpTransforms(); + waitForPendingTasks(); + } + + protected void cleanUpTransforms() { + for (DataFrameTransformConfig config : transformConfigs.values()) { + stopDataFrameTransform(config.getId()); + deleteDataFrameTransform(config.getId()); + } + transformConfigs.clear(); + } + + protected StopDataFrameTransformAction.Response stopDataFrameTransform(String id) { + return client().execute(StopDataFrameTransformAction.INSTANCE, + new StopDataFrameTransformAction.Request(id, true, false, null)).actionGet(); + } + + protected StartDataFrameTransformAction.Response startDataFrameTransform(String id) { + return client().execute(StartDataFrameTransformAction.INSTANCE, + new StartDataFrameTransformAction.Request(id, false)).actionGet(); + } + + protected DeleteDataFrameTransformAction.Response deleteDataFrameTransform(String id) { + DeleteDataFrameTransformAction.Response response = client().execute(DeleteDataFrameTransformAction.INSTANCE, + new DeleteDataFrameTransformAction.Request(id)) + .actionGet(); + if (response.isDeleted()) { + transformConfigs.remove(id); + } + return response; + } + + protected AcknowledgedResponse putDataFrameTransform(DataFrameTransformConfig config) { + if (transformConfigs.keySet().contains(config.getId())) { + throw new IllegalArgumentException("data frame transform [" + config.getId() + "] is already registered"); + } + AcknowledgedResponse response = client().execute(PutDataFrameTransformAction.INSTANCE, + new PutDataFrameTransformAction.Request(config)) + .actionGet(); + if (response.isAcknowledged()) { + transformConfigs.put(config.getId(), config); + } + return response; + } + + protected GetDataFrameTransformsStatsAction.Response getDataFrameTransformStats(String id) { + return client().execute(GetDataFrameTransformsStatsAction.INSTANCE, new GetDataFrameTransformsStatsAction.Request(id)).actionGet(); + } + + protected void waitUntilCheckpoint(String id, long checkpoint) throws Exception { + waitUntilCheckpoint(id, checkpoint, TimeValue.timeValueSeconds(30)); + } + + protected void waitUntilCheckpoint(String id, long checkpoint, TimeValue waitTime) throws Exception { + assertBusy(() -> + assertEquals(checkpoint, getDataFrameTransformStats(id) + .getTransformsStateAndStats() + .get(0) + .getTransformState() + .getCheckpoint()), + waitTime.getMillis(), + TimeUnit.MILLISECONDS); + } + + protected DateHistogramGroupSource createDateHistogramGroupSource(String field, long interval, ZoneId zone, String format) { + DateHistogramGroupSource source = new DateHistogramGroupSource(field); + source.setFormat(format); + source.setInterval(interval); + source.setTimeZone(zone); + return source; + } + + protected DateHistogramGroupSource createDateHistogramGroupSource(String field, + DateHistogramInterval interval, + ZoneId zone, + String format) { + DateHistogramGroupSource source = new DateHistogramGroupSource(field); + source.setFormat(format); + source.setDateHistogramInterval(interval); + source.setTimeZone(zone); + return source; + } + + protected GroupConfig createGroupConfig(Map groups) throws Exception { + Map lazyParsed = new HashMap<>(groups.size()); + for(Map.Entry sgs : groups.entrySet()) { + lazyParsed.put(sgs.getKey(), Collections.singletonMap(sgs.getValue().getType().value(), toLazy(sgs.getValue()))); + } + return new GroupConfig(lazyParsed, groups); + } + + protected QueryConfig createQueryConfig(QueryBuilder queryBuilder) throws Exception { + return new QueryConfig(toLazy(queryBuilder), queryBuilder); + } + + protected AggregationConfig createAggConfig(AggregatorFactories.Builder aggregations) throws Exception { + return new AggregationConfig(toLazy(aggregations), aggregations); + } + + protected PivotConfig createPivotConfig(Map groups, + AggregatorFactories.Builder aggregations) throws Exception { + return new PivotConfig(createGroupConfig(groups), createAggConfig(aggregations)); + } + + protected DataFrameTransformConfig createTransformConfig(String id, + Map groups, + AggregatorFactories.Builder aggregations, + String destinationIndex, + String... sourceIndices) throws Exception { + return createTransformConfig(id, groups, aggregations, destinationIndex, QueryBuilders.matchAllQuery(), sourceIndices); + } + + protected DataFrameTransformConfig createTransformConfig(String id, + Map groups, + AggregatorFactories.Builder aggregations, + String destinationIndex, + QueryBuilder queryBuilder, + String... sourceIndices) throws Exception { + return new DataFrameTransformConfig(id, + new SourceConfig(sourceIndices, createQueryConfig(queryBuilder)), + new DestConfig(destinationIndex), + Collections.emptyMap(), + createPivotConfig(groups, aggregations)); + } + + protected void createReviewsIndex() throws Exception { + final int numDocs = 1000; + + // create mapping + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.startObject("properties") + .startObject("timestamp") + .field("type", "date") + .endObject() + .startObject("user_id") + .field("type", "keyword") + .endObject() + .startObject("count") + .field("type", "integer") + .endObject() + .startObject("business_id") + .field("type", "keyword") + .endObject() + .startObject("stars") + .field("type", "integer") + .endObject() + .endObject(); + } + builder.endObject(); + CreateIndexResponse response = client().admin() + .indices() + .prepareCreate(REVIEWS_INDEX_NAME) + .addMapping("_doc", builder) + .get(); + assertThat(response.isAcknowledged(), is(true)); + } + + // create index + BulkRequestBuilder bulk = client().prepareBulk(REVIEWS_INDEX_NAME, "_doc"); + int day = 10; + for (int i = 0; i < numDocs; i++) { + long user = i % 28; + int stars = (i + 20) % 5; + long business = (i + 100) % 50; + int hour = 10 + (i % 13); + int min = 10 + (i % 49); + int sec = 10 + (i % 49); + + String date_string = "2017-01-" + day + "T" + hour + ":" + min + ":" + sec + "Z"; + + StringBuilder sourceBuilder = new StringBuilder(); + sourceBuilder.append("{\"user_id\":\"") + .append("user_") + .append(user) + .append("\",\"count\":") + .append(i) + .append(",\"business_id\":\"") + .append("business_") + .append(business) + .append("\",\"stars\":") + .append(stars) + .append(",\"timestamp\":\"") + .append(date_string) + .append("\"}"); + bulk.add(new IndexRequest().source(sourceBuilder.toString(), XContentType.JSON)); + + if (i % 50 == 0) { + BulkResponse response = client().bulk(bulk.request()).get(); + assertThat(response.buildFailureMessage(), response.hasFailures(), is(false)); + bulk = client().prepareBulk(REVIEWS_INDEX_NAME, "_doc"); + day += 1; + } + } + BulkResponse response = client().bulk(bulk.request()).get(); + assertThat(response.buildFailureMessage(), response.hasFailures(), is(false)); + client().admin().indices().prepareRefresh(REVIEWS_INDEX_NAME).get(); + } + + protected Map toLazy(ToXContent parsedObject) throws Exception { + BytesReference bytes = XContentHelper.toXContent(parsedObject, XContentType.JSON, false); + try(XContentParser parser = XContentHelper.createParser(xContentRegistry(), + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, + bytes, + XContentType.JSON)) { + return parser.mapOrdered(); + } + } + + private void waitForPendingTasks() { + ListTasksRequest listTasksRequest = new ListTasksRequest(); + listTasksRequest.setWaitForCompletion(true); + listTasksRequest.setDetailed(true); + listTasksRequest.setTimeout(TimeValue.timeValueSeconds(10)); + try { + admin().cluster().listTasks(listTasksRequest).get(); + } catch (Exception e) { + throw new AssertionError("Failed to wait for pending tasks to complete", e); + } + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList()); + return new NamedXContentRegistry(searchModule.getNamedXContents()); + } + + @Override + protected Settings externalClusterClientSettings() { + Path key; + Path certificate; + try { + key = PathUtils.get(getClass().getResource("/testnode.pem").toURI()); + certificate = PathUtils.get(getClass().getResource("/testnode.crt").toURI()); + } catch (URISyntaxException e) { + throw new IllegalStateException("error trying to get keystore path", e); + } + Settings.Builder builder = Settings.builder(); + builder.put(NetworkModule.TRANSPORT_TYPE_KEY, SecurityField.NAME4); + builder.put(SecurityField.USER_SETTING.getKey(), "x_pack_rest_user:" + SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING); + builder.put("xpack.security.transport.ssl.enabled", true); + builder.put("xpack.security.transport.ssl.key", key.toAbsolutePath().toString()); + builder.put("xpack.security.transport.ssl.certificate", certificate.toAbsolutePath().toString()); + builder.put("xpack.security.transport.ssl.key_passphrase", "testnode"); + builder.put("xpack.security.transport.ssl.verification_mode", "certificate"); + return builder.build(); + } + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(LocalStateCompositeXPackPlugin.class, Netty4Plugin.class); + } + + @Override + protected Collection> transportClientPlugins() { + return Arrays.asList(XPackClientPlugin.class, Netty4Plugin.class); + } +} diff --git a/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformIT.java b/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformIT.java new file mode 100644 index 0000000000000..ecb2025c6a9c5 --- /dev/null +++ b/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformIT.java @@ -0,0 +1,60 @@ +/* + * 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.integration; + +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.SingleGroupSource; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.TermsGroupSource; +import org.elasticsearch.xpack.core.indexing.IndexerState; +import org.junit.After; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class DataFrameTransformIT extends DataFrameIntegTestCase { + + @After + public void cleanTransforms() { + cleanUp(); + } + + public void testDataFrameTransformCrud() throws Exception { + createReviewsIndex(); + + Map groups = new HashMap<>(); + groups.put("by-day", createDateHistogramGroupSource("timestamp", DateHistogramInterval.DAY, null, null)); + groups.put("by-user", new TermsGroupSource("user_id")); + groups.put("by-business", new TermsGroupSource("business_id")); + + AggregatorFactories.Builder aggs = AggregatorFactories.builder() + .addAggregator(AggregationBuilders.avg("review_score").field("stars")) + .addAggregator(AggregationBuilders.max("timestamp").field("timestamp")); + + DataFrameTransformConfig config = createTransformConfig("data-frame-transform-crud", + groups, + aggs, + "reviews-by-user-business-day", + REVIEWS_INDEX_NAME); + + assertTrue(putDataFrameTransform(config).isAcknowledged()); + assertTrue(startDataFrameTransform(config.getId()).isStarted()); + + waitUntilCheckpoint(config.getId(), 1L); + + DataFrameTransformStateAndStats stats = getDataFrameTransformStats(config.getId()).getTransformsStateAndStats().get(0); + + assertThat(stats.getTransformState().getIndexerState(), equalTo(IndexerState.STARTED)); + } + + +} diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsStatsAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsStatsAction.java index b350300392cca..93658c2f4f427 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsStatsAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportGetDataFrameTransformsStatsAction.java @@ -12,16 +12,14 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.tasks.TransportTasksAction; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; @@ -30,7 +28,6 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.discovery.MasterNotDiscoveredException; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -81,7 +78,7 @@ public TransportGetDataFrameTransformsStatsAction(TransportService transportServ DataFrameTransformsConfigManager dataFrameTransformsConfigManager, DataFrameTransformsCheckpointService transformsCheckpointService) { super(GetDataFrameTransformsStatsAction.NAME, clusterService, transportService, actionFilters, Request::new, Response::new, - Response::new, ThreadPool.Names.SAME); + Response::new, ThreadPool.Names.SAME); this.client = client; this.dataFrameTransformsConfigManager = dataFrameTransformsConfigManager; this.transformsCheckpointService = transformsCheckpointService; @@ -94,7 +91,9 @@ protected Response newResponse(Request request, List tasks, List r.getTransformsStateAndStats().stream()) .sorted(Comparator.comparing(DataFrameTransformStateAndStats::getId)) .collect(Collectors.toList()); - return new Response(responses, taskOperationFailures, failedNodeExceptions); + List allFailedNodeExceptions = new ArrayList<>(failedNodeExceptions); + allFailedNodeExceptions.addAll(tasks.stream().flatMap(r -> r.getNodeFailures().stream()).collect(Collectors.toList())); + return new Response(responses, taskOperationFailures, allFailedNodeExceptions); } @Override @@ -110,7 +109,7 @@ protected void taskOperation(Request request, DataFrameTransformTask task, Actio Collections.singletonList(new DataFrameTransformStateAndStats(task.getTransformId(), task.getState(), task.getStats(), DataFrameTransformCheckpointingInfo.EMPTY)), Collections.emptyList(), - Collections.singletonList(new ElasticsearchException("Failed to retrieve checkpointing info", e)))); + Collections.singletonList(new FailedNodeException("", "Failed to retrieve checkpointing info", e)))); })); } else { listener.onResponse(new Response(Collections.emptyList())); @@ -119,37 +118,24 @@ protected void taskOperation(Request request, DataFrameTransformTask task, Actio @Override protected void doExecute(Task task, Request request, ActionListener finalListener) { - final ClusterState state = clusterService.state(); - final DiscoveryNodes nodes = state.nodes(); - if (nodes.isLocalNodeElectedMaster()) { - dataFrameTransformsConfigManager.expandTransformIds(request.getId(), request.getPageParams(), ActionListener.wrap( - ids -> { - request.setExpandedIds(ids); - super.doExecute(task, request, ActionListener.wrap( - response -> collectStatsForTransformsWithoutTasks(request, response, finalListener), - finalListener::onFailure - )); - }, - e -> { - // If the index to search, or the individual config is not there, just return empty - if (e instanceof ResourceNotFoundException) { - finalListener.onResponse(new Response(Collections.emptyList())); - } else { - finalListener.onFailure(e); - } + dataFrameTransformsConfigManager.expandTransformIds(request.getId(), request.getPageParams(), ActionListener.wrap( + ids -> { + request.setExpandedIds(ids); + super.doExecute(task, request, ActionListener.wrap( + response -> collectStatsForTransformsWithoutTasks(request, response, finalListener), + finalListener::onFailure + )); + }, + e -> { + // If the index to search, or the individual config is not there, just return empty + logger.error("failed to expand ids", e); + if (e instanceof ResourceNotFoundException) { + finalListener.onResponse(new Response(Collections.emptyList())); + } else { + finalListener.onFailure(e); } - )); - } else { - // Delegates GetTransforms to elected master node, so it becomes the coordinating node. - // Non-master nodes may have a stale cluster state that shows transforms which are cancelled - // on the master, which makes testing difficult. - if (nodes.getMasterNode() == null) { - finalListener.onFailure(new MasterNotDiscoveredException("no known master nodes")); - } else { - transportService.sendRequest(nodes.getMasterNode(), actionName, request, - new ActionListenerResponseHandler<>(finalListener, Response::new)); } - } + )); } private void collectStatsForTransformsWithoutTasks(Request request, @@ -172,10 +158,15 @@ private void collectStatsForTransformsWithoutTasks(Request request, searchResponse -> { List nodeFailures = new ArrayList<>(response.getNodeFailures()); if (searchResponse.getShardFailures().length > 0) { - String msg = "transform statistics document search returned shard failures: " + - Arrays.toString(searchResponse.getShardFailures()); - logger.error(msg); - nodeFailures.add(new ElasticsearchException(msg)); + for(ShardSearchFailure shardSearchFailure : searchResponse.getShardFailures()) { + String nodeId = ""; + if (shardSearchFailure.shard() != null) { + nodeId = shardSearchFailure.shard().getNodeId(); + } + nodeFailures.add(new FailedNodeException(nodeId, shardSearchFailure.toString(), shardSearchFailure.getCause())); + } + logger.error("transform statistics document search returned shard failures: {}", + Arrays.toString(searchResponse.getShardFailures())); } List allStateAndStats = response.getTransformsStateAndStats(); for(SearchHit hit : searchResponse.getHits().getHits()) { diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java index 1e65639a89f4e..98e70fa2578d1 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/action/TransportStartDataFrameTransformAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.XPackLicenseState; @@ -44,6 +45,7 @@ import org.elasticsearch.xpack.dataframe.persistence.DataframeIndex; import org.elasticsearch.xpack.dataframe.transforms.pivot.Pivot; +import java.io.IOException; import java.util.Collection; import java.util.Map; import java.util.function.Consumer; @@ -82,7 +84,12 @@ protected String executor() { @Override protected StartDataFrameTransformAction.Response newResponse() { - return new StartDataFrameTransformAction.Response(false); + throw new UnsupportedOperationException("usage of Streamable is to be replaced by Writeable"); + } + + @Override + protected StartDataFrameTransformAction.Response read(StreamInput in) throws IOException { + return new StartDataFrameTransformAction.Response(in); } @Override From bb62bc6d1c8f016cc1d0396ac36fb7b3ab5c31bf Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Thu, 25 Apr 2019 15:06:35 -0400 Subject: [PATCH 193/260] [DOCS] Explicitly set section IDs for Asciidoctor migration (#41547) * [DOCS] Explicitly set section ID for faster phrase queries * [DOCS] Explicitly set section ID for faster prefix queries --- docs/reference/how-to/search-speed.asciidoc | 2 ++ docs/reference/redirects.asciidoc | 8 ++++++++ 2 files changed, 10 insertions(+) diff --git a/docs/reference/how-to/search-speed.asciidoc b/docs/reference/how-to/search-speed.asciidoc index 097d791363249..017dd74afa084 100644 --- a/docs/reference/how-to/search-speed.asciidoc +++ b/docs/reference/how-to/search-speed.asciidoc @@ -411,6 +411,7 @@ Some caveats to the Profile API are that: - given the added overhead, the resulting took times are not reliable indicators of actual took time, but can be used comparatively between clauses for relative timing differences - the Profile API is best for exploring possible reasons behind the most costly clauses of a query but isn't intended for accurately measuring absolute timings of each clause +[[faster-phrase-queries]] === Faster phrase queries with `index_phrases` The <> field has an <> option that @@ -418,6 +419,7 @@ indexes 2-shingles and is automatically leveraged by query parsers to run phrase queries that don't have a slop. If your use-case involves running lots of phrase queries, this can speed up queries significantly. +[[faster-prefix-queries]] === Faster prefix queries with `index_prefixes` The <> field has an <> option that diff --git a/docs/reference/redirects.asciidoc b/docs/reference/redirects.asciidoc index 7e00e10a52050..1c6850542a971 100644 --- a/docs/reference/redirects.asciidoc +++ b/docs/reference/redirects.asciidoc @@ -594,3 +594,11 @@ formation module>>. === {xpack} settings in {es} include::{asciidoc-dir}/../../shared/settings.asciidoc[] + +[role="exclude",id="_faster_phrase_queries_with_literal_index_phrases_literal"] + +See <>. + +[role="exclude",id="_faster_prefix_queries_with_literal_index_prefixes_literal.html"] + +See <>. \ No newline at end of file From 3567b79e5f37308648973db8af6a9da8f3cad075 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Thu, 25 Apr 2019 21:09:32 +0200 Subject: [PATCH 194/260] Disable max score optimization for queries with unbounded max scores (#41361) Lucene 8 has the ability to skip blocks of non-competitive documents. However some queries don't track their maximum score (`script_score`, `span`, ...) so they always return Float.POSITIVE_INFINITY as maximum score. This can slow down some boolean queries if other clauses have bounded max scores. This commit disables the max score optimization when we detect a mandatory scoring clause with unbounded max scores. Optional clauses are not checked since they can still skip documents when the unbounded clause is after the current document. --- .../search/ESToParentBlockJoinQuery.java | 13 +- .../search/query/TopDocsCollectorContext.java | 74 +++++++++++- .../search/query/QueryPhaseTests.java | 111 ++++++++++++++++++ 3 files changed, 194 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java b/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java index 30cd6e52f0dbd..ea9c8793d88c3 100644 --- a/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java +++ b/server/src/main/java/org/elasticsearch/index/search/ESToParentBlockJoinQuery.java @@ -37,14 +37,16 @@ public final class ESToParentBlockJoinQuery extends Query { private final ToParentBlockJoinQuery query; private final String path; + private final ScoreMode scoreMode; public ESToParentBlockJoinQuery(Query childQuery, BitSetProducer parentsFilter, ScoreMode scoreMode, String path) { - this(new ToParentBlockJoinQuery(childQuery, parentsFilter, scoreMode), path); + this(new ToParentBlockJoinQuery(childQuery, parentsFilter, scoreMode), path, scoreMode); } - private ESToParentBlockJoinQuery(ToParentBlockJoinQuery query, String path) { + private ESToParentBlockJoinQuery(ToParentBlockJoinQuery query, String path, ScoreMode scoreMode) { this.query = query; this.path = path; + this.scoreMode = scoreMode; } /** Return the child query. */ @@ -57,6 +59,11 @@ public String getPath() { return path; } + /** Return the score mode for the matching children. **/ + public ScoreMode getScoreMode() { + return scoreMode; + } + @Override public Query rewrite(IndexReader reader) throws IOException { Query innerRewrite = query.rewrite(reader); @@ -68,7 +75,7 @@ public Query rewrite(IndexReader reader) throws IOException { // to a MatchNoDocsQuery. In that case it would be fine to lose information // about the nested path. if (innerRewrite instanceof ToParentBlockJoinQuery) { - return new ESToParentBlockJoinQuery((ToParentBlockJoinQuery) innerRewrite, path); + return new ESToParentBlockJoinQuery((ToParentBlockJoinQuery) innerRewrite, path, scoreMode); } else { return innerRewrite; } diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 1e2cd7541f944..5ae6cc739c362 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -27,16 +27,21 @@ import org.apache.lucene.index.PointValues; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; +import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Collector; import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.FilterCollector; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MultiCollector; import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; @@ -47,11 +52,15 @@ import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.grouping.CollapseTopFieldDocs; import org.apache.lucene.search.grouping.CollapsingTopDocsCollector; +import org.apache.lucene.search.spans.SpanQuery; import org.elasticsearch.action.search.MaxScoreCollector; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; +import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; +import org.elasticsearch.common.lucene.search.function.ScriptScoreQuery; import org.elasticsearch.common.util.CachedSupplier; +import org.elasticsearch.index.search.ESToParentBlockJoinQuery; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.collapse.CollapseContext; import org.elasticsearch.search.internal.ScrollContext; @@ -264,7 +273,29 @@ private SimpleTopDocsCollectorContext(IndexReader reader, } else { maxScoreSupplier = () -> Float.NaN; } - this.collector = MultiCollector.wrap(topDocsCollector, maxScoreCollector); + + final Collector collector = MultiCollector.wrap(topDocsCollector, maxScoreCollector); + if (sortAndFormats == null || + SortField.FIELD_SCORE.equals(sortAndFormats.sort.getSort()[0])) { + if (hasInfMaxScore(query)) { + // disable max score optimization since we have a mandatory clause + // that doesn't track the maximum score + this.collector = new FilterCollector(collector) { + @Override + public ScoreMode scoreMode() { + if (in.scoreMode() == ScoreMode.TOP_SCORES) { + return ScoreMode.COMPLETE; + } + return in.scoreMode(); + } + }; + } else { + this.collector = collector; + } + } else { + this.collector = collector; + } + } @Override @@ -437,4 +468,45 @@ boolean shouldRescore() { }; } } + + /** + * Return true if the provided query contains a mandatory clauses (MUST) + * that doesn't track the maximum scores per block + */ + static boolean hasInfMaxScore(Query query) { + MaxScoreQueryVisitor visitor = new MaxScoreQueryVisitor(); + query.visit(visitor); + return visitor.hasInfMaxScore; + } + + private static class MaxScoreQueryVisitor extends QueryVisitor { + private boolean hasInfMaxScore; + + @Override + public void visitLeaf(Query query) { + checkMaxScoreInfo(query); + } + + @Override + public QueryVisitor getSubVisitor(BooleanClause.Occur occur, Query parent) { + if (occur != BooleanClause.Occur.MUST) { + // boolean queries can skip documents even if they have some should + // clauses that don't track maximum scores + return QueryVisitor.EMPTY_VISITOR; + } + checkMaxScoreInfo(parent); + return this; + } + + void checkMaxScoreInfo(Query query) { + if (query instanceof FunctionScoreQuery + || query instanceof ScriptScoreQuery + || query instanceof SpanQuery) { + hasInfMaxScore = true; + } else if (query instanceof ESToParentBlockJoinQuery) { + ESToParentBlockJoinQuery q = (ESToParentBlockJoinQuery) query; + hasInfMaxScore |= (q.getScoreMode() != org.apache.lucene.search.join.ScoreMode.None); + } + } + } } diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java index e65b4aa377a40..a5061c35d7fcf 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.query; +import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field.Store; import org.apache.lucene.document.LatLonDocValuesField; @@ -26,6 +27,7 @@ import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.document.StringField; +import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriterConfig; @@ -52,12 +54,19 @@ import org.apache.lucene.search.SortField; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TotalHitCountCollector; +import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.Weight; +import org.apache.lucene.search.join.BitSetProducer; +import org.apache.lucene.search.join.ScoreMode; +import org.apache.lucene.search.spans.SpanNearQuery; +import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.action.search.SearchTask; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.query.ParsedQuery; +import org.elasticsearch.index.search.ESToParentBlockJoinQuery; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.search.DocValueFormat; @@ -559,6 +568,108 @@ public void testIndexSortScrollOptimization() throws Exception { dir.close(); } + + public void testDisableTopScoreCollection() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig(new StandardAnalyzer()); + RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc); + Document doc = new Document(); + for (int i = 0; i < 10; i++) { + doc.clear(); + if (i % 2 == 0) { + doc.add(new TextField("title", "foo bar", Store.NO)); + } else { + doc.add(new TextField("title", "foo", Store.NO)); + } + w.addDocument(doc); + } + w.close(); + + IndexReader reader = DirectoryReader.open(dir); + IndexSearcher contextSearcher = new IndexSearcher(reader); + TestSearchContext context = new TestSearchContext(null, indexShard); + context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); + Query q = new SpanNearQuery.Builder("title", true) + .addClause(new SpanTermQuery(new Term("title", "foo"))) + .addClause(new SpanTermQuery(new Term("title", "bar"))) + .build(); + + context.parsedQuery(new ParsedQuery(q)); + context.setSize(10); + TopDocsCollectorContext topDocsContext = + TopDocsCollectorContext.createTopDocsCollectorContext(context, reader, false); + assertEquals(topDocsContext.create(null).scoreMode(), org.apache.lucene.search.ScoreMode.COMPLETE); + QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + assertEquals(5, context.queryResult().topDocs().topDocs.totalHits.value); + assertEquals(context.queryResult().topDocs().topDocs.totalHits.relation, TotalHits.Relation.EQUAL_TO); + assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(5)); + + + context.sort(new SortAndFormats(new Sort(new SortField("other", SortField.Type.INT)), + new DocValueFormat[] { DocValueFormat.RAW })); + topDocsContext = + TopDocsCollectorContext.createTopDocsCollectorContext(context, reader, false); + assertEquals(topDocsContext.create(null).scoreMode(), org.apache.lucene.search.ScoreMode.COMPLETE_NO_SCORES); + QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + assertEquals(5, context.queryResult().topDocs().topDocs.totalHits.value); + assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(5)); + assertEquals(context.queryResult().topDocs().topDocs.totalHits.relation, TotalHits.Relation.EQUAL_TO); + + reader.close(); + dir.close(); + } + + public void testMaxScoreQueryVisitor() { + BitSetProducer producer = context -> new FixedBitSet(1); + Query query = new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"); + assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + + query = new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.None, "nested"); + assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + + + for (Occur occur : Occur.values()) { + query = new BooleanQuery.Builder() + .add(new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"), occur) + .build(); + if (occur == Occur.MUST) { + assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + } else { + assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + } + + query = new BooleanQuery.Builder() + .add(new BooleanQuery.Builder() + .add(new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"), occur) + .build(), occur) + .build(); + if (occur == Occur.MUST) { + assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + } else { + assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + } + + query = new BooleanQuery.Builder() + .add(new BooleanQuery.Builder() + .add(new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"), occur) + .build(), Occur.FILTER) + .build(); + assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + + query = new BooleanQuery.Builder() + .add(new BooleanQuery.Builder() + .add(new SpanTermQuery(new Term("field", "foo")), occur) + .add(new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"), occur) + .build(), occur) + .build(); + if (occur == Occur.MUST) { + assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + } else { + assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + } + } + } + private static IndexSearcher getAssertingEarlyTerminationSearcher(IndexReader reader, int size) { return new IndexSearcher(reader) { @Override From 0ca375ed42f420f4c7c4a8c489a2ecc328db3648 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 25 Apr 2019 14:30:16 -0600 Subject: [PATCH 195/260] Remove dedicated SSL network write buffer (#41283) This is related to #27260. Currently for the SSLDriver we allocate a dedicated network write buffer and encrypt the data into that buffer one buffer at a time. This requires constantly switching between encrypting and flushing. This commit adds a dedicated outbound buffer for SSL operations that will internally allocate new packet sized buffers as they are need (for writing encrypted data). This allows us to totally encrypt an operation before writing it to the network. Eventually it can be hooked up to buffer recycling. --- .../org/elasticsearch/nio/FlushOperation.java | 31 +++- .../elasticsearch/nio/FlushReadyWrite.java | 2 +- .../nio/InboundChannelBuffer.java | 62 +------ .../main/java/org/elasticsearch/nio/Page.java | 89 ++++++++++ .../nio/SocketChannelContext.java | 2 +- .../nio/BytesChannelContextTests.java | 13 +- .../nio/FlushOperationTests.java | 20 ++- .../nio/InboundChannelBufferTests.java | 18 +- .../nio/SocketChannelContextTests.java | 2 +- .../elasticsearch/http/nio/NettyAdaptor.java | 4 +- .../http/nio/NioHttpServerTransport.java | 5 +- .../elasticsearch/http/nio/PagedByteBuf.java | 10 +- .../transport/nio/NioTransport.java | 5 +- .../http/nio/PagedByteBufTests.java | 20 +-- .../transport/nio/MockNioTransport.java | 5 +- .../transport/nio/SSLChannelContext.java | 70 +++++--- .../security/transport/nio/SSLDriver.java | 163 ++++++++---------- .../transport/nio/SSLOutboundBuffer.java | 68 ++++++++ .../nio/SecurityNioHttpServerTransport.java | 5 +- .../transport/nio/SecurityNioTransport.java | 5 +- .../transport/nio/SSLChannelContextTests.java | 125 ++++++++------ .../transport/nio/SSLDriverTests.java | 85 ++++----- 22 files changed, 480 insertions(+), 329 deletions(-) create mode 100644 libs/nio/src/main/java/org/elasticsearch/nio/Page.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLOutboundBuffer.java diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/FlushOperation.java b/libs/nio/src/main/java/org/elasticsearch/nio/FlushOperation.java index 7a1696483db06..de0318a941af6 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/FlushOperation.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/FlushOperation.java @@ -25,6 +25,8 @@ public class FlushOperation { + private static final ByteBuffer[] EMPTY_ARRAY = new ByteBuffer[0]; + private final BiConsumer listener; private final ByteBuffer[] buffers; private final int[] offsets; @@ -61,19 +63,38 @@ public void incrementIndex(int delta) { } public ByteBuffer[] getBuffersToWrite() { + return getBuffersToWrite(length); + } + + public ByteBuffer[] getBuffersToWrite(int maxBytes) { final int index = Arrays.binarySearch(offsets, internalIndex); - int offsetIndex = index < 0 ? (-(index + 1)) - 1 : index; + final int offsetIndex = index < 0 ? (-(index + 1)) - 1 : index; + final int finalIndex = Arrays.binarySearch(offsets, Math.min(internalIndex + maxBytes, length)); + final int finalOffsetIndex = finalIndex < 0 ? (-(finalIndex + 1)) - 1 : finalIndex; - ByteBuffer[] postIndexBuffers = new ByteBuffer[buffers.length - offsetIndex]; + int nBuffers = (finalOffsetIndex - offsetIndex) + 1; + int firstBufferPosition = internalIndex - offsets[offsetIndex]; ByteBuffer firstBuffer = buffers[offsetIndex].duplicate(); - firstBuffer.position(internalIndex - offsets[offsetIndex]); + firstBuffer.position(firstBufferPosition); + if (nBuffers == 1 && firstBuffer.remaining() == 0) { + return EMPTY_ARRAY; + } + + ByteBuffer[] postIndexBuffers = new ByteBuffer[nBuffers]; postIndexBuffers[0] = firstBuffer; + int finalOffset = offsetIndex + nBuffers; + int nBytes = firstBuffer.remaining(); int j = 1; - for (int i = (offsetIndex + 1); i < buffers.length; ++i) { - postIndexBuffers[j++] = buffers[i].duplicate(); + for (int i = (offsetIndex + 1); i < finalOffset; ++i) { + ByteBuffer buffer = buffers[i].duplicate(); + nBytes += buffer.remaining(); + postIndexBuffers[j++] = buffer; } + int excessBytes = Math.max(0, nBytes - maxBytes); + ByteBuffer lastBuffer = postIndexBuffers[postIndexBuffers.length - 1]; + lastBuffer.limit(lastBuffer.limit() - excessBytes); return postIndexBuffers; } } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java b/libs/nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java index 61c997603ff97..4855e0cbade9c 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java @@ -27,7 +27,7 @@ public class FlushReadyWrite extends FlushOperation implements WriteOperation { private final SocketChannelContext channelContext; private final ByteBuffer[] buffers; - FlushReadyWrite(SocketChannelContext channelContext, ByteBuffer[] buffers, BiConsumer listener) { + public FlushReadyWrite(SocketChannelContext channelContext, ByteBuffer[] buffers, BiConsumer listener) { super(buffers, listener); this.channelContext = channelContext; this.buffers = buffers; diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/InboundChannelBuffer.java b/libs/nio/src/main/java/org/elasticsearch/nio/InboundChannelBuffer.java index f7e6fbb768728..2dfd53d27e109 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/InboundChannelBuffer.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/InboundChannelBuffer.java @@ -19,7 +19,6 @@ package org.elasticsearch.nio; -import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.nio.utils.ExceptionsHelper; import java.nio.ByteBuffer; @@ -140,11 +139,11 @@ public ByteBuffer[] sliceBuffersTo(long to) { ByteBuffer[] buffers = new ByteBuffer[pageCount]; Iterator pageIterator = pages.iterator(); - ByteBuffer firstBuffer = pageIterator.next().byteBuffer.duplicate(); + ByteBuffer firstBuffer = pageIterator.next().byteBuffer().duplicate(); firstBuffer.position(firstBuffer.position() + offset); buffers[0] = firstBuffer; for (int i = 1; i < buffers.length; i++) { - buffers[i] = pageIterator.next().byteBuffer.duplicate(); + buffers[i] = pageIterator.next().byteBuffer().duplicate(); } if (finalLimit != 0) { buffers[buffers.length - 1].limit(finalLimit); @@ -180,14 +179,14 @@ public Page[] sliceAndRetainPagesTo(long to) { Page[] pages = new Page[pageCount]; Iterator pageIterator = this.pages.iterator(); Page firstPage = pageIterator.next().duplicate(); - ByteBuffer firstBuffer = firstPage.byteBuffer; + ByteBuffer firstBuffer = firstPage.byteBuffer(); firstBuffer.position(firstBuffer.position() + offset); pages[0] = firstPage; for (int i = 1; i < pages.length; i++) { pages[i] = pageIterator.next().duplicate(); } if (finalLimit != 0) { - pages[pages.length - 1].byteBuffer.limit(finalLimit); + pages[pages.length - 1].byteBuffer().limit(finalLimit); } return pages; @@ -217,9 +216,9 @@ public ByteBuffer[] sliceBuffersFrom(long from) { ByteBuffer[] buffers = new ByteBuffer[pages.size() - pageIndex]; Iterator pageIterator = pages.descendingIterator(); for (int i = buffers.length - 1; i > 0; --i) { - buffers[i] = pageIterator.next().byteBuffer.duplicate(); + buffers[i] = pageIterator.next().byteBuffer().duplicate(); } - ByteBuffer firstPostIndexBuffer = pageIterator.next().byteBuffer.duplicate(); + ByteBuffer firstPostIndexBuffer = pageIterator.next().byteBuffer().duplicate(); firstPostIndexBuffer.position(firstPostIndexBuffer.position() + indexInPage); buffers[0] = firstPostIndexBuffer; @@ -268,53 +267,4 @@ private int pageIndex(long index) { private int indexInPage(long index) { return (int) (index & PAGE_MASK); } - - public static class Page implements AutoCloseable { - - private final ByteBuffer byteBuffer; - // This is reference counted as some implementations want to retain the byte pages by calling - // sliceAndRetainPagesTo. With reference counting we can increment the reference count, return the - // pages, and safely close them when this channel buffer is done with them. The reference count - // would be 1 at that point, meaning that the pages will remain until the implementation closes - // theirs. - private final RefCountedCloseable refCountedCloseable; - - public Page(ByteBuffer byteBuffer, Runnable closeable) { - this(byteBuffer, new RefCountedCloseable(closeable)); - } - - private Page(ByteBuffer byteBuffer, RefCountedCloseable refCountedCloseable) { - this.byteBuffer = byteBuffer; - this.refCountedCloseable = refCountedCloseable; - } - - private Page duplicate() { - refCountedCloseable.incRef(); - return new Page(byteBuffer.duplicate(), refCountedCloseable); - } - - public ByteBuffer getByteBuffer() { - return byteBuffer; - } - - @Override - public void close() { - refCountedCloseable.decRef(); - } - - private static class RefCountedCloseable extends AbstractRefCounted { - - private final Runnable closeable; - - private RefCountedCloseable(Runnable closeable) { - super("byte array page"); - this.closeable = closeable; - } - - @Override - protected void closeInternal() { - closeable.run(); - } - } - } } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/Page.java b/libs/nio/src/main/java/org/elasticsearch/nio/Page.java new file mode 100644 index 0000000000000..b60c1c0127919 --- /dev/null +++ b/libs/nio/src/main/java/org/elasticsearch/nio/Page.java @@ -0,0 +1,89 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.nio; + +import org.elasticsearch.common.util.concurrent.AbstractRefCounted; + +import java.io.Closeable; +import java.nio.ByteBuffer; + +public class Page implements Closeable { + + private final ByteBuffer byteBuffer; + // This is reference counted as some implementations want to retain the byte pages by calling + // duplicate. With reference counting we can increment the reference count, return a new page, + // and safely close the pages independently. The closeable will not be called until each page is + // released. + private final RefCountedCloseable refCountedCloseable; + + public Page(ByteBuffer byteBuffer) { + this(byteBuffer, () -> {}); + } + + public Page(ByteBuffer byteBuffer, Runnable closeable) { + this(byteBuffer, new RefCountedCloseable(closeable)); + } + + private Page(ByteBuffer byteBuffer, RefCountedCloseable refCountedCloseable) { + this.byteBuffer = byteBuffer; + this.refCountedCloseable = refCountedCloseable; + } + + /** + * Duplicates this page and increments the reference count. The new page must be closed independently + * of the original page. + * + * @return the new page + */ + public Page duplicate() { + refCountedCloseable.incRef(); + return new Page(byteBuffer.duplicate(), refCountedCloseable); + } + + /** + * Returns the {@link ByteBuffer} for this page. Modifications to the limits, positions, etc of the + * buffer will also mutate this page. Call {@link ByteBuffer#duplicate()} to avoid mutating the page. + * + * @return the byte buffer + */ + public ByteBuffer byteBuffer() { + return byteBuffer; + } + + @Override + public void close() { + refCountedCloseable.decRef(); + } + + private static class RefCountedCloseable extends AbstractRefCounted { + + private final Runnable closeable; + + private RefCountedCloseable(Runnable closeable) { + super("byte array page"); + this.closeable = closeable; + } + + @Override + protected void closeInternal() { + closeable.run(); + } + } +} diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java index 816f4adc8cbb1..1444422f7a7f6 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java @@ -325,7 +325,7 @@ protected int flushToChannel(FlushOperation flushOperation) throws IOException { ioBuffer.clear(); ioBuffer.limit(Math.min(WRITE_LIMIT, ioBuffer.limit())); int j = 0; - ByteBuffer[] buffers = flushOperation.getBuffersToWrite(); + ByteBuffer[] buffers = flushOperation.getBuffersToWrite(WRITE_LIMIT); while (j < buffers.length && ioBuffer.remaining() > 0) { ByteBuffer buffer = buffers[j++]; copyBytes(buffer, ioBuffer); diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java index 0591abdd69a97..c98e7dc8dfb29 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java @@ -31,6 +31,7 @@ import java.util.function.Consumer; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -168,7 +169,7 @@ public void testQueuedWriteIsFlushedInFlushCall() throws Exception { assertTrue(context.readyForFlush()); - when(flushOperation.getBuffersToWrite()).thenReturn(buffers); + when(flushOperation.getBuffersToWrite(anyInt())).thenReturn(buffers); when(flushOperation.isFullyFlushed()).thenReturn(false, true); when(flushOperation.getListener()).thenReturn(listener); context.flushChannel(); @@ -187,7 +188,7 @@ public void testPartialFlush() throws IOException { assertTrue(context.readyForFlush()); when(flushOperation.isFullyFlushed()).thenReturn(false); - when(flushOperation.getBuffersToWrite()).thenReturn(new ByteBuffer[] {ByteBuffer.allocate(3)}); + when(flushOperation.getBuffersToWrite(anyInt())).thenReturn(new ByteBuffer[] {ByteBuffer.allocate(3)}); context.flushChannel(); verify(listener, times(0)).accept(null, null); @@ -201,8 +202,8 @@ public void testMultipleWritesPartialFlushes() throws IOException { BiConsumer listener2 = mock(BiConsumer.class); FlushReadyWrite flushOperation1 = mock(FlushReadyWrite.class); FlushReadyWrite flushOperation2 = mock(FlushReadyWrite.class); - when(flushOperation1.getBuffersToWrite()).thenReturn(new ByteBuffer[] {ByteBuffer.allocate(3)}); - when(flushOperation2.getBuffersToWrite()).thenReturn(new ByteBuffer[] {ByteBuffer.allocate(3)}); + when(flushOperation1.getBuffersToWrite(anyInt())).thenReturn(new ByteBuffer[] {ByteBuffer.allocate(3)}); + when(flushOperation2.getBuffersToWrite(anyInt())).thenReturn(new ByteBuffer[] {ByteBuffer.allocate(3)}); when(flushOperation1.getListener()).thenReturn(listener); when(flushOperation2.getListener()).thenReturn(listener2); @@ -237,7 +238,7 @@ public void testWhenIOExceptionThrownListenerIsCalled() throws IOException { assertTrue(context.readyForFlush()); IOException exception = new IOException(); - when(flushOperation.getBuffersToWrite()).thenReturn(buffers); + when(flushOperation.getBuffersToWrite(anyInt())).thenReturn(buffers); when(rawChannel.write(any(ByteBuffer.class))).thenThrow(exception); when(flushOperation.getListener()).thenReturn(listener); expectThrows(IOException.class, () -> context.flushChannel()); @@ -252,7 +253,7 @@ public void testWriteIOExceptionMeansChannelReadyToClose() throws IOException { context.queueWriteOperation(flushOperation); IOException exception = new IOException(); - when(flushOperation.getBuffersToWrite()).thenReturn(buffers); + when(flushOperation.getBuffersToWrite(anyInt())).thenReturn(buffers); when(rawChannel.write(any(ByteBuffer.class))).thenThrow(exception); assertFalse(context.selectorShouldClose()); diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java index 4f2a320ad583d..73dba34cc30f7 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java @@ -65,29 +65,45 @@ public void testMultipleFlushesWithCompositeBuffer() throws IOException { ByteBuffer[] byteBuffers = writeOp.getBuffersToWrite(); assertEquals(3, byteBuffers.length); assertEquals(5, byteBuffers[0].remaining()); + ByteBuffer[] byteBuffersWithLimit = writeOp.getBuffersToWrite(10); + assertEquals(2, byteBuffersWithLimit.length); + assertEquals(5, byteBuffersWithLimit[0].remaining()); + assertEquals(5, byteBuffersWithLimit[1].remaining()); writeOp.incrementIndex(5); assertFalse(writeOp.isFullyFlushed()); byteBuffers = writeOp.getBuffersToWrite(); assertEquals(2, byteBuffers.length); assertEquals(15, byteBuffers[0].remaining()); + assertEquals(3, byteBuffers[1].remaining()); + byteBuffersWithLimit = writeOp.getBuffersToWrite(10); + assertEquals(1, byteBuffersWithLimit.length); + assertEquals(10, byteBuffersWithLimit[0].remaining()); writeOp.incrementIndex(2); assertFalse(writeOp.isFullyFlushed()); byteBuffers = writeOp.getBuffersToWrite(); assertEquals(2, byteBuffers.length); assertEquals(13, byteBuffers[0].remaining()); + assertEquals(3, byteBuffers[1].remaining()); + byteBuffersWithLimit = writeOp.getBuffersToWrite(10); + assertEquals(1, byteBuffersWithLimit.length); + assertEquals(10, byteBuffersWithLimit[0].remaining()); writeOp.incrementIndex(15); assertFalse(writeOp.isFullyFlushed()); byteBuffers = writeOp.getBuffersToWrite(); assertEquals(1, byteBuffers.length); assertEquals(1, byteBuffers[0].remaining()); + byteBuffersWithLimit = writeOp.getBuffersToWrite(10); + assertEquals(1, byteBuffersWithLimit.length); + assertEquals(1, byteBuffersWithLimit[0].remaining()); writeOp.incrementIndex(1); assertTrue(writeOp.isFullyFlushed()); byteBuffers = writeOp.getBuffersToWrite(); - assertEquals(1, byteBuffers.length); - assertEquals(0, byteBuffers[0].remaining()); + assertEquals(0, byteBuffers.length); + byteBuffersWithLimit = writeOp.getBuffersToWrite(10); + assertEquals(0, byteBuffersWithLimit.length); } } diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/InboundChannelBufferTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/InboundChannelBufferTests.java index 8917bec39f17e..f558043095372 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/InboundChannelBufferTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/InboundChannelBufferTests.java @@ -30,8 +30,8 @@ public class InboundChannelBufferTests extends ESTestCase { private static final int PAGE_SIZE = PageCacheRecycler.PAGE_SIZE_IN_BYTES; - private final Supplier defaultPageSupplier = () -> - new InboundChannelBuffer.Page(ByteBuffer.allocate(PageCacheRecycler.BYTE_PAGE_SIZE), () -> { + private final Supplier defaultPageSupplier = () -> + new Page(ByteBuffer.allocate(PageCacheRecycler.BYTE_PAGE_SIZE), () -> { }); public void testNewBufferNoPages() { @@ -126,10 +126,10 @@ public void testIncrementIndexWithOffset() { public void testReleaseClosesPages() { ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - Supplier supplier = () -> { + Supplier supplier = () -> { AtomicBoolean atomicBoolean = new AtomicBoolean(); queue.add(atomicBoolean); - return new InboundChannelBuffer.Page(ByteBuffer.allocate(PAGE_SIZE), () -> atomicBoolean.set(true)); + return new Page(ByteBuffer.allocate(PAGE_SIZE), () -> atomicBoolean.set(true)); }; InboundChannelBuffer channelBuffer = new InboundChannelBuffer(supplier); channelBuffer.ensureCapacity(PAGE_SIZE * 4); @@ -153,10 +153,10 @@ public void testReleaseClosesPages() { public void testClose() { ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - Supplier supplier = () -> { + Supplier supplier = () -> { AtomicBoolean atomicBoolean = new AtomicBoolean(); queue.add(atomicBoolean); - return new InboundChannelBuffer.Page(ByteBuffer.allocate(PAGE_SIZE), () -> atomicBoolean.set(true)); + return new Page(ByteBuffer.allocate(PAGE_SIZE), () -> atomicBoolean.set(true)); }; InboundChannelBuffer channelBuffer = new InboundChannelBuffer(supplier); channelBuffer.ensureCapacity(PAGE_SIZE * 4); @@ -178,10 +178,10 @@ public void testClose() { public void testCloseRetainedPages() { ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - Supplier supplier = () -> { + Supplier supplier = () -> { AtomicBoolean atomicBoolean = new AtomicBoolean(); queue.add(atomicBoolean); - return new InboundChannelBuffer.Page(ByteBuffer.allocate(PAGE_SIZE), () -> atomicBoolean.set(true)); + return new Page(ByteBuffer.allocate(PAGE_SIZE), () -> atomicBoolean.set(true)); }; InboundChannelBuffer channelBuffer = new InboundChannelBuffer(supplier); channelBuffer.ensureCapacity(PAGE_SIZE * 4); @@ -192,7 +192,7 @@ public void testCloseRetainedPages() { assertFalse(closedRef.get()); } - InboundChannelBuffer.Page[] pages = channelBuffer.sliceAndRetainPagesTo(PAGE_SIZE * 2); + Page[] pages = channelBuffer.sliceAndRetainPagesTo(PAGE_SIZE * 2); pages[1].close(); diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java index 345c5197c76b8..baf7abac79d1b 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java @@ -285,7 +285,7 @@ public void testCloseClosesChannelBuffer() throws IOException { when(channel.getRawChannel()).thenReturn(realChannel); when(channel.isOpen()).thenReturn(true); Runnable closer = mock(Runnable.class); - Supplier pageSupplier = () -> new InboundChannelBuffer.Page(ByteBuffer.allocate(1 << 14), closer); + Supplier pageSupplier = () -> new Page(ByteBuffer.allocate(1 << 14), closer); InboundChannelBuffer buffer = new InboundChannelBuffer(pageSupplier); buffer.ensureCapacity(1); TestSocketChannelContext context = new TestSocketChannelContext(channel, selector, exceptionHandler, readWriteHandler, buffer); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyAdaptor.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyAdaptor.java index c221fdf1378d7..96db559e60333 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyAdaptor.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyAdaptor.java @@ -29,7 +29,7 @@ import io.netty.channel.embedded.EmbeddedChannel; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.nio.FlushOperation; -import org.elasticsearch.nio.InboundChannelBuffer; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.WriteOperation; import java.nio.ByteBuffer; @@ -97,7 +97,7 @@ public int read(ByteBuffer[] buffers) { return byteBuf.readerIndex() - initialReaderIndex; } - public int read(InboundChannelBuffer.Page[] pages) { + public int read(Page[] pages) { ByteBuf byteBuf = PagedByteBuf.byteBufFromPages(pages); int readableBytes = byteBuf.readableBytes(); nettyChannel.writeInbound(byteBuf); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java index a5f274c7ccd34..57936ff70c628 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java @@ -43,6 +43,7 @@ import org.elasticsearch.nio.NioGroup; import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ServerChannelContext; import org.elasticsearch.nio.SocketChannelContext; import org.elasticsearch.rest.RestUtils; @@ -205,9 +206,9 @@ private HttpChannelFactory() { @Override public NioHttpChannel createChannel(NioSelector selector, SocketChannel channel) throws IOException { NioHttpChannel httpChannel = new NioHttpChannel(channel); - java.util.function.Supplier pageSupplier = () -> { + java.util.function.Supplier pageSupplier = () -> { Recycler.V bytes = pageCacheRecycler.bytePage(false); - return new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes.v()), bytes::close); + return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; HttpReadWriteHandler httpReadWritePipeline = new HttpReadWriteHandler(httpChannel,NioHttpServerTransport.this, handlingSettings, corsConfig); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/PagedByteBuf.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/PagedByteBuf.java index 40f3aeecfbc94..359926d43f9a7 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/PagedByteBuf.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/PagedByteBuf.java @@ -24,7 +24,7 @@ import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.buffer.UnpooledHeapByteBuf; -import org.elasticsearch.nio.InboundChannelBuffer; +import org.elasticsearch.nio.Page; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -39,7 +39,7 @@ private PagedByteBuf(byte[] array, Runnable releasable) { this.releasable = releasable; } - static ByteBuf byteBufFromPages(InboundChannelBuffer.Page[] pages) { + static ByteBuf byteBufFromPages(Page[] pages) { int componentCount = pages.length; if (componentCount == 0) { return Unpooled.EMPTY_BUFFER; @@ -48,15 +48,15 @@ static ByteBuf byteBufFromPages(InboundChannelBuffer.Page[] pages) { } else { int maxComponents = Math.max(16, componentCount); final List components = new ArrayList<>(componentCount); - for (InboundChannelBuffer.Page page : pages) { + for (Page page : pages) { components.add(byteBufFromPage(page)); } return new CompositeByteBuf(UnpooledByteBufAllocator.DEFAULT, false, maxComponents, components); } } - private static ByteBuf byteBufFromPage(InboundChannelBuffer.Page page) { - ByteBuffer buffer = page.getByteBuffer(); + private static ByteBuf byteBufFromPage(Page page) { + ByteBuffer buffer = page.byteBuffer(); assert buffer.isDirect() == false && buffer.hasArray() : "Must be a heap buffer with an array"; int offset = buffer.arrayOffset() + buffer.position(); PagedByteBuf newByteBuf = new PagedByteBuf(buffer.array(), page::close); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java index 17ab3a5bf3d8c..30b4b4913128d 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/transport/nio/NioTransport.java @@ -36,6 +36,7 @@ import org.elasticsearch.nio.NioGroup; import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ServerChannelContext; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TcpTransport; @@ -157,9 +158,9 @@ private TcpChannelFactoryImpl(ProfileSettings profileSettings, boolean isClient) @Override public NioTcpChannel createChannel(NioSelector selector, SocketChannel channel) { NioTcpChannel nioChannel = new NioTcpChannel(isClient == false, profileName, channel); - Supplier pageSupplier = () -> { + Supplier pageSupplier = () -> { Recycler.V bytes = pageCacheRecycler.bytePage(false); - return new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes.v()), bytes::close); + return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; TcpReadWriteHandler readWriteHandler = new TcpReadWriteHandler(nioChannel, NioTransport.this); Consumer exceptionHandler = (e) -> onException(nioChannel, e); diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/PagedByteBufTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/PagedByteBufTests.java index 15bd18ecf6959..df4bf3274b3bc 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/PagedByteBufTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/PagedByteBufTests.java @@ -20,7 +20,7 @@ package org.elasticsearch.http.nio; import io.netty.buffer.ByteBuf; -import org.elasticsearch.nio.InboundChannelBuffer; +import org.elasticsearch.nio.Page; import org.elasticsearch.test.ESTestCase; import java.nio.ByteBuffer; @@ -32,12 +32,12 @@ public class PagedByteBufTests extends ESTestCase { public void testReleasingPage() { AtomicInteger integer = new AtomicInteger(0); int pageCount = randomInt(10) + 1; - ArrayList pages = new ArrayList<>(); + ArrayList pages = new ArrayList<>(); for (int i = 0; i < pageCount; ++i) { - pages.add(new InboundChannelBuffer.Page(ByteBuffer.allocate(10), integer::incrementAndGet)); + pages.add(new Page(ByteBuffer.allocate(10), integer::incrementAndGet)); } - ByteBuf byteBuf = PagedByteBuf.byteBufFromPages(pages.toArray(new InboundChannelBuffer.Page[0])); + ByteBuf byteBuf = PagedByteBuf.byteBufFromPages(pages.toArray(new Page[0])); assertEquals(0, integer.get()); byteBuf.retain(); @@ -62,9 +62,9 @@ public void testBytesAreUsed() { bytes2[i - 10] = (byte) i; } - InboundChannelBuffer.Page[] pages = new InboundChannelBuffer.Page[2]; - pages[0] = new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes1), () -> {}); - pages[1] = new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes2), () -> {}); + Page[] pages = new Page[2]; + pages[0] = new Page(ByteBuffer.wrap(bytes1), () -> {}); + pages[1] = new Page(ByteBuffer.wrap(bytes2), () -> {}); ByteBuf byteBuf = PagedByteBuf.byteBufFromPages(pages); assertEquals(20, byteBuf.readableBytes()); @@ -73,13 +73,13 @@ public void testBytesAreUsed() { assertEquals((byte) i, byteBuf.getByte(i)); } - InboundChannelBuffer.Page[] pages2 = new InboundChannelBuffer.Page[2]; + Page[] pages2 = new Page[2]; ByteBuffer firstBuffer = ByteBuffer.wrap(bytes1); firstBuffer.position(2); ByteBuffer secondBuffer = ByteBuffer.wrap(bytes2); secondBuffer.limit(8); - pages2[0] = new InboundChannelBuffer.Page(firstBuffer, () -> {}); - pages2[1] = new InboundChannelBuffer.Page(secondBuffer, () -> {}); + pages2[0] = new Page(firstBuffer, () -> {}); + pages2[1] = new Page(secondBuffer, () -> {}); ByteBuf byteBuf2 = PagedByteBuf.byteBufFromPages(pages2); assertEquals(16, byteBuf2.readableBytes()); diff --git a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java index db9b1cfe74a71..66ff33213f605 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/nio/MockNioTransport.java @@ -41,6 +41,7 @@ import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioServerSocketChannel; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ServerChannelContext; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectionProfile; @@ -191,9 +192,9 @@ private MockTcpChannelFactory(boolean isClient, ProfileSettings profileSettings, @Override public MockSocketChannel createChannel(NioSelector selector, SocketChannel channel) throws IOException { MockSocketChannel nioChannel = new MockSocketChannel(isClient == false, profileName, channel); - Supplier pageSupplier = () -> { + Supplier pageSupplier = () -> { Recycler.V bytes = pageCacheRecycler.bytePage(false); - return new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes.v()), bytes::close); + return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; MockTcpReadWriteHandler readWriteHandler = new MockTcpReadWriteHandler(nioChannel, MockNioTransport.this); BytesChannelContext context = new BytesChannelContext(nioChannel, selector, (e) -> exceptionCaught(nioChannel, e), diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java index b5d5db2166c1f..2c00dd7092950 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java @@ -10,6 +10,7 @@ import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ReadWriteHandler; import org.elasticsearch.nio.SocketChannelContext; import org.elasticsearch.nio.NioSelector; @@ -17,6 +18,8 @@ import javax.net.ssl.SSLEngine; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -34,6 +37,8 @@ public final class SSLChannelContext extends SocketChannelContext { private static final Runnable DEFAULT_TIMEOUT_CANCELLER = () -> {}; private final SSLDriver sslDriver; + private final SSLOutboundBuffer outboundBuffer; + private FlushOperation encryptedFlush; private Runnable closeTimeoutCanceller = DEFAULT_TIMEOUT_CANCELLER; SSLChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, SSLDriver sslDriver, @@ -46,6 +51,8 @@ public final class SSLChannelContext extends SocketChannelContext { Predicate allowChannelPredicate) { super(channel, selector, exceptionHandler, readWriteHandler, channelBuffer, allowChannelPredicate); this.sslDriver = sslDriver; + // TODO: When the bytes are actually recycled, we need to test that they are released on context close + this.outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); } @Override @@ -72,34 +79,32 @@ public void flushChannel() throws IOException { return; } // If there is currently data in the outbound write buffer, flush the buffer. - if (sslDriver.hasFlushPending()) { + if (pendingChannelFlush()) { // If the data is not completely flushed, exit. We cannot produce new write data until the // existing data has been fully flushed. - flushToChannel(sslDriver.getNetworkWriteBuffer()); - if (sslDriver.hasFlushPending()) { + flushEncryptedOperation(); + if (pendingChannelFlush()) { return; } } // If the driver is ready for application writes, we can attempt to proceed with any queued writes. if (sslDriver.readyForApplicationWrites()) { - FlushOperation currentFlush; - while (sslDriver.hasFlushPending() == false && (currentFlush = getPendingFlush()) != null) { - // If the current operation has been fully consumed (encrypted) we now know that it has been - // sent (as we only get to this point if the write buffer has been fully flushed). - if (currentFlush.isFullyFlushed()) { + FlushOperation unencryptedFlush; + while (pendingChannelFlush() == false && (unencryptedFlush = getPendingFlush()) != null) { + if (unencryptedFlush.isFullyFlushed()) { currentFlushOperationComplete(); } else { try { // Attempt to encrypt application write data. The encrypted data ends up in the // outbound write buffer. - int bytesEncrypted = sslDriver.applicationWrite(currentFlush.getBuffersToWrite()); - if (bytesEncrypted == 0) { + sslDriver.write(unencryptedFlush, outboundBuffer); + if (outboundBuffer.hasEncryptedBytesToFlush() == false) { break; } - currentFlush.incrementIndex(bytesEncrypted); + encryptedFlush = outboundBuffer.buildNetworkFlushOperation(); // Flush the write buffer to the channel - flushToChannel(sslDriver.getNetworkWriteBuffer()); + flushEncryptedOperation(); } catch (IOException e) { currentFlushOperationFailed(e); throw e; @@ -109,23 +114,38 @@ public void flushChannel() throws IOException { } else { // We are not ready for application writes, check if the driver has non-application writes. We // only want to continue producing new writes if the outbound write buffer is fully flushed. - while (sslDriver.hasFlushPending() == false && sslDriver.needsNonApplicationWrite()) { - sslDriver.nonApplicationWrite(); + while (pendingChannelFlush() == false && sslDriver.needsNonApplicationWrite()) { + sslDriver.nonApplicationWrite(outboundBuffer); // If non-application writes were produced, flush the outbound write buffer. - if (sslDriver.hasFlushPending()) { - flushToChannel(sslDriver.getNetworkWriteBuffer()); + if (outboundBuffer.hasEncryptedBytesToFlush()) { + encryptedFlush = outboundBuffer.buildNetworkFlushOperation(); + flushEncryptedOperation(); } } } } + private void flushEncryptedOperation() throws IOException { + try { + flushToChannel(encryptedFlush); + if (encryptedFlush.isFullyFlushed()) { + getSelector().executeListener(encryptedFlush.getListener(), null); + encryptedFlush = null; + } + } catch (IOException e) { + getSelector().executeFailedListener(encryptedFlush.getListener(), e); + encryptedFlush = null; + throw e; + } + } + @Override public boolean readyForFlush() { getSelector().assertOnSelectorThread(); if (sslDriver.readyForApplicationWrites()) { - return sslDriver.hasFlushPending() || super.readyForFlush(); + return pendingChannelFlush() || super.readyForFlush(); } else { - return sslDriver.hasFlushPending() || sslDriver.needsNonApplicationWrite(); + return pendingChannelFlush() || sslDriver.needsNonApplicationWrite(); } } @@ -149,7 +169,7 @@ public int read() throws IOException { @Override public boolean selectorShouldClose() { - return closeNow() || sslDriver.isClosed(); + return closeNow() || (sslDriver.isClosed() && pendingChannelFlush() == false); } @Override @@ -170,7 +190,10 @@ public void closeFromSelector() throws IOException { getSelector().assertOnSelectorThread(); if (channel.isOpen()) { closeTimeoutCanceller.run(); - IOUtils.close(super::closeFromSelector, sslDriver::close); + if (encryptedFlush != null) { + getSelector().executeFailedListener(encryptedFlush.getListener(), new ClosedChannelException()); + } + IOUtils.close(super::closeFromSelector, outboundBuffer::close, sslDriver::close); } } @@ -184,9 +207,14 @@ private void channelCloseTimeout() { getSelector().queueChannelClose(channel); } + private boolean pendingChannelFlush() { + return encryptedFlush != null; + } + private static class CloseNotifyOperation implements WriteOperation { - private static final BiConsumer LISTENER = (v, t) -> {}; + private static final BiConsumer LISTENER = (v, t) -> { + }; private static final Object WRITE_OBJECT = new Object(); private final SocketChannelContext channelContext; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java index 93978bcc6a359..4dbf1d1f03fdf 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.security.transport.nio; +import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.utils.ExceptionsHelper; @@ -29,19 +30,17 @@ * the buffer passed as an argument. Otherwise, it will be consumed internally and advance the SSL/TLS close * or handshake process. * - * Producing writes for a channel is more complicated. If there is existing data in the outbound write buffer - * as indicated by {@link #hasFlushPending()}, that data must be written to the channel before more outbound - * data can be produced. If no flushes are pending, {@link #needsNonApplicationWrite()} can be called to - * determine if this driver needs to produce more data to advance the handshake or close process. If that - * method returns true, {@link #nonApplicationWrite()} should be called (and the data produced then flushed - * to the channel) until no further non-application writes are needed. + * Producing writes for a channel is more complicated. The method {@link #needsNonApplicationWrite()} can be + * called to determine if this driver needs to produce more data to advance the handshake or close process. + * If that method returns true, {@link #nonApplicationWrite(SSLOutboundBuffer)} should be called (and the + * data produced then flushed to the channel) until no further non-application writes are needed. * * If no non-application writes are needed, {@link #readyForApplicationWrites()} can be called to determine * if the driver is ready to consume application data. (Note: It is possible that * {@link #readyForApplicationWrites()} and {@link #needsNonApplicationWrite()} can both return false if the * driver is waiting on non-application data from the peer.) If the driver indicates it is ready for - * application writes, {@link #applicationWrite(ByteBuffer[])} can be called. This method will encrypt - * application data and place it in the write buffer for flushing to a channel. + * application writes, {@link #write(FlushOperation, SSLOutboundBuffer)} can be called. This method will + * encrypt flush operation application data and place it in the outbound buffer for flushing to a channel. * * If you are ready to close the channel {@link #initiateClose()} should be called. After that is called, the * driver will start producing non-application writes related to notifying the peer connection that this @@ -50,23 +49,23 @@ */ public class SSLDriver implements AutoCloseable { - private static final ByteBuffer[] EMPTY_BUFFER_ARRAY = new ByteBuffer[0]; + private static final ByteBuffer[] EMPTY_BUFFERS = {ByteBuffer.allocate(0)}; + private static final FlushOperation EMPTY_FLUSH_OPERATION = new FlushOperation(EMPTY_BUFFERS, (r, t) -> {}); private final SSLEngine engine; private final boolean isClientMode; // This should only be accessed by the network thread associated with this channel, so nothing needs to // be volatile. private Mode currentMode = new HandshakeMode(); - private ByteBuffer networkWriteBuffer; private ByteBuffer networkReadBuffer; + private int packetSize; public SSLDriver(SSLEngine engine, boolean isClientMode) { this.engine = engine; this.isClientMode = isClientMode; SSLSession session = engine.getSession(); - this.networkReadBuffer = ByteBuffer.allocate(session.getPacketBufferSize()); - this.networkWriteBuffer = ByteBuffer.allocate(session.getPacketBufferSize()); - this.networkWriteBuffer.position(this.networkWriteBuffer.limit()); + packetSize = session.getPacketBufferSize(); + this.networkReadBuffer = ByteBuffer.allocate(packetSize); } public void init() throws SSLException { @@ -100,18 +99,10 @@ public SSLEngine getSSLEngine() { return engine; } - public boolean hasFlushPending() { - return networkWriteBuffer.hasRemaining(); - } - public boolean isHandshaking() { return currentMode.isHandshake(); } - public ByteBuffer getNetworkWriteBuffer() { - return networkWriteBuffer; - } - public ByteBuffer getNetworkReadBuffer() { return networkReadBuffer; } @@ -134,15 +125,14 @@ public boolean needsNonApplicationWrite() { return currentMode.needsNonApplicationWrite(); } - public int applicationWrite(ByteBuffer[] buffers) throws SSLException { - assert readyForApplicationWrites() : "Should not be called if driver is not ready for application writes"; - return currentMode.write(buffers); + public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { + return currentMode.write(applicationBytes, outboundBuffer); } - public void nonApplicationWrite() throws SSLException { + public void nonApplicationWrite(SSLOutboundBuffer outboundBuffer) throws SSLException { assert currentMode.isApplication() == false : "Should not be called if driver is in application mode"; if (currentMode.isApplication() == false) { - currentMode.write(EMPTY_BUFFER_ARRAY); + currentMode.write(EMPTY_FLUSH_OPERATION, outboundBuffer); } else { throw new AssertionError("Attempted to non-application write from invalid mode: " + currentMode.modeName()); } @@ -205,45 +195,36 @@ private SSLEngineResult unwrap(InboundChannelBuffer buffer) throws SSLException } } - private SSLEngineResult wrap(ByteBuffer[] buffers) throws SSLException { - assert hasFlushPending() == false : "Should never called with pending writes"; + private SSLEngineResult wrap(SSLOutboundBuffer outboundBuffer) throws SSLException { + return wrap(outboundBuffer, EMPTY_FLUSH_OPERATION); + } - networkWriteBuffer.clear(); + private SSLEngineResult wrap(SSLOutboundBuffer outboundBuffer, FlushOperation applicationBytes) throws SSLException { + ByteBuffer[] buffers = applicationBytes.getBuffersToWrite(engine.getSession().getApplicationBufferSize()); while (true) { SSLEngineResult result; + ByteBuffer networkBuffer = outboundBuffer.nextWriteBuffer(packetSize); try { - if (buffers.length == 1) { - result = engine.wrap(buffers[0], networkWriteBuffer); - } else { - result = engine.wrap(buffers, networkWriteBuffer); - } + result = engine.wrap(buffers, networkBuffer); } catch (SSLException e) { - networkWriteBuffer.position(networkWriteBuffer.limit()); + outboundBuffer.incrementEncryptedBytes(0); throw e; } + outboundBuffer.incrementEncryptedBytes(result.bytesProduced()); + applicationBytes.incrementIndex(result.bytesConsumed()); switch (result.getStatus()) { case OK: - networkWriteBuffer.flip(); return result; case BUFFER_UNDERFLOW: throw new IllegalStateException("Should not receive BUFFER_UNDERFLOW on WRAP"); case BUFFER_OVERFLOW: - // There is not enough space in the network buffer for an entire SSL packet. Expand the - // buffer if it's smaller than the current session packet size. Otherwise return and wait - // for existing data to be flushed. - int currentCapacity = networkWriteBuffer.capacity(); - ensureNetworkWriteBufferSize(); - if (currentCapacity == networkWriteBuffer.capacity()) { - return result; - } + packetSize = engine.getSession().getPacketBufferSize(); + // There is not enough space in the network buffer for an entire SSL packet. We will + // allocate a buffer with the correct packet size the next time through the loop. break; case CLOSED: - if (result.bytesProduced() > 0) { - networkWriteBuffer.flip(); - } else { - assert false : "WRAP during close processing should produce close message."; - } + assert result.bytesProduced() > 0 : "WRAP during close processing should produce close message."; return result; default: throw new IllegalStateException("Unexpected WRAP result: " + result.getStatus()); @@ -265,23 +246,12 @@ private void ensureApplicationBufferSize(InboundChannelBuffer applicationBuffer) } } - private void ensureNetworkWriteBufferSize() { - networkWriteBuffer = ensureNetBufferSize(networkWriteBuffer); - } - private void ensureNetworkReadBufferSize() { - networkReadBuffer = ensureNetBufferSize(networkReadBuffer); - } - - private ByteBuffer ensureNetBufferSize(ByteBuffer current) { - int networkPacketSize = engine.getSession().getPacketBufferSize(); - if (current.capacity() < networkPacketSize) { - ByteBuffer newBuffer = ByteBuffer.allocate(networkPacketSize); - current.flip(); - newBuffer.put(current); - return newBuffer; - } else { - return current; + packetSize = engine.getSession().getPacketBufferSize(); + if (networkReadBuffer.capacity() < packetSize) { + ByteBuffer newBuffer = ByteBuffer.allocate(packetSize); + networkReadBuffer.flip(); + newBuffer.put(networkReadBuffer); } } @@ -306,7 +276,7 @@ private interface Mode { void read(InboundChannelBuffer buffer) throws SSLException; - int write(ByteBuffer[] buffers) throws SSLException; + int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException; boolean needsNonApplicationWrite(); @@ -329,7 +299,7 @@ private void startHandshake() throws SSLException { if (handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_UNWRAP && handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_WRAP) { try { - handshake(); + handshake(null); } catch (SSLException e) { closingInternal(); throw e; @@ -337,7 +307,7 @@ private void startHandshake() throws SSLException { } } - private void handshake() throws SSLException { + private void handshake(SSLOutboundBuffer outboundBuffer) throws SSLException { boolean continueHandshaking = true; while (continueHandshaking) { switch (handshakeStatus) { @@ -346,11 +316,13 @@ private void handshake() throws SSLException { continueHandshaking = false; break; case NEED_WRAP: - if (hasFlushPending() == false) { - handshakeStatus = wrap(EMPTY_BUFFER_ARRAY).getHandshakeStatus(); - } - // If we need NEED_TASK we should run the tasks immediately - if (handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_TASK) { + if (outboundBuffer != null) { + handshakeStatus = wrap(outboundBuffer).getHandshakeStatus(); + // If we need NEED_TASK we should run the tasks immediately + if (handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_TASK) { + continueHandshaking = false; + } + } else { continueHandshaking = false; } break; @@ -379,7 +351,7 @@ public void read(InboundChannelBuffer buffer) throws SSLException { try { SSLEngineResult result = unwrap(buffer); handshakeStatus = result.getHandshakeStatus(); - handshake(); + handshake(null); // If we are done handshaking we should exit the handshake read continueUnwrap = result.bytesConsumed() > 0 && currentMode.isHandshake(); } catch (SSLException e) { @@ -390,9 +362,9 @@ public void read(InboundChannelBuffer buffer) throws SSLException { } @Override - public int write(ByteBuffer[] buffers) throws SSLException { + public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { try { - handshake(); + handshake(outboundBuffer); } catch (SSLException e) { closingInternal(); throw e; @@ -445,8 +417,7 @@ private void maybeFinishHandshake() { String message = "Expected to be in handshaking/closed mode. Instead in application mode."; throw new AssertionError(message); } - } else if (hasFlushPending() == false) { - // We only acknowledge that we are done handshaking if there are no bytes that need to be written + } else { if (currentMode.isHandshake()) { currentMode = new ApplicationMode(); } else { @@ -473,10 +444,17 @@ public void read(InboundChannelBuffer buffer) throws SSLException { } @Override - public int write(ByteBuffer[] buffers) throws SSLException { - SSLEngineResult result = wrap(buffers); - maybeRenegotiation(result.getHandshakeStatus()); - return result.bytesConsumed(); + public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { + boolean continueWrap = true; + int totalBytesProduced = 0; + while (continueWrap && applicationBytes.isFullyFlushed() == false) { + SSLEngineResult result = wrap(outboundBuffer, applicationBytes); + int bytesProduced = result.bytesProduced(); + totalBytesProduced += bytesProduced; + boolean renegotiationRequested = maybeRenegotiation(result.getHandshakeStatus()); + continueWrap = bytesProduced > 0 && renegotiationRequested == false; + } + return totalBytesProduced; } private boolean maybeRenegotiation(SSLEngineResult.HandshakeStatus newStatus) throws SSLException { @@ -560,18 +538,19 @@ public void read(InboundChannelBuffer buffer) throws SSLException { } @Override - public int write(ByteBuffer[] buffers) throws SSLException { - if (hasFlushPending() == false && engine.isOutboundDone()) { - needToSendClose = false; - // Close inbound if it is still open and we have decided not to wait for response. - if (needToReceiveClose == false && engine.isInboundDone() == false) { - closeInboundAndSwallowPeerDidNotCloseException(); + public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { + int bytesProduced = 0; + if (engine.isOutboundDone() == false) { + bytesProduced += wrap(outboundBuffer).bytesProduced(); + if (engine.isOutboundDone()) { + needToSendClose = false; + // Close inbound if it is still open and we have decided not to wait for response. + if (needToReceiveClose == false && engine.isInboundDone() == false) { + closeInboundAndSwallowPeerDidNotCloseException(); + } } - } else { - wrap(EMPTY_BUFFER_ARRAY); - assert hasFlushPending() : "Should have produced close message"; } - return 0; + return bytesProduced; } @Override diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLOutboundBuffer.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLOutboundBuffer.java new file mode 100644 index 0000000000000..2cd28f7d7dc32 --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLOutboundBuffer.java @@ -0,0 +1,68 @@ +/* + * 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.security.transport.nio; + +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.nio.FlushOperation; +import org.elasticsearch.nio.Page; + +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.function.IntFunction; + +public class SSLOutboundBuffer implements AutoCloseable { + + private final ArrayDeque pages; + private final IntFunction pageSupplier; + + private Page currentPage; + + SSLOutboundBuffer(IntFunction pageSupplier) { + this.pages = new ArrayDeque<>(); + this.pageSupplier = pageSupplier; + } + + void incrementEncryptedBytes(int encryptedBytesProduced) { + if (encryptedBytesProduced != 0) { + currentPage.byteBuffer().limit(encryptedBytesProduced); + pages.addLast(currentPage); + } + currentPage = null; + } + + ByteBuffer nextWriteBuffer(int networkBufferSize) { + if (currentPage != null) { + // If there is an existing page, close it as it wasn't large enough to accommodate the SSLEngine. + currentPage.close(); + } + + Page newPage = pageSupplier.apply(networkBufferSize); + currentPage = newPage; + return newPage.byteBuffer().duplicate(); + } + + FlushOperation buildNetworkFlushOperation() { + int pageCount = pages.size(); + ByteBuffer[] byteBuffers = new ByteBuffer[pageCount]; + Page[] pagesToClose = new Page[pageCount]; + for (int i = 0; i < pageCount; ++i) { + Page page = pages.removeFirst(); + pagesToClose[i] = page; + byteBuffers[i] = page.byteBuffer(); + } + + return new FlushOperation(byteBuffers, (r, e) -> IOUtils.closeWhileHandlingException(pagesToClose)); + } + + boolean hasEncryptedBytesToFlush() { + return pages.isEmpty() == false; + } + + @Override + public void close() { + IOUtils.closeWhileHandlingException(pages); + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java index 9e0da2518835d..8ecba16fa460d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java @@ -22,6 +22,7 @@ import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ServerChannelContext; import org.elasticsearch.nio.SocketChannelContext; import org.elasticsearch.threadpool.ThreadPool; @@ -92,9 +93,9 @@ private SecurityHttpChannelFactory() { @Override public NioHttpChannel createChannel(NioSelector selector, SocketChannel channel) throws IOException { NioHttpChannel httpChannel = new NioHttpChannel(channel); - Supplier pageSupplier = () -> { + Supplier pageSupplier = () -> { Recycler.V bytes = pageCacheRecycler.bytePage(false); - return new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes.v()), bytes::close); + return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; HttpReadWriteHandler httpHandler = new HttpReadWriteHandler(httpChannel,SecurityNioHttpServerTransport.this, handlingSettings, corsConfig); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java index dbffeaec58e50..903fec52e9e9b 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java @@ -21,6 +21,7 @@ import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ServerChannelContext; import org.elasticsearch.nio.SocketChannelContext; import org.elasticsearch.threadpool.ThreadPool; @@ -155,9 +156,9 @@ private SecurityTcpChannelFactory(RawChannelFactory rawChannelFactory, String pr @Override public NioTcpChannel createChannel(NioSelector selector, SocketChannel channel) throws IOException { NioTcpChannel nioChannel = new NioTcpChannel(isClient == false, profileName, channel); - Supplier pageSupplier = () -> { + Supplier pageSupplier = () -> { Recycler.V bytes = pageCacheRecycler.bytePage(false); - return new InboundChannelBuffer.Page(ByteBuffer.wrap(bytes.v()), bytes::close); + return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; TcpReadWriteHandler readWriteHandler = new TcpReadWriteHandler(nioChannel, SecurityNioTransport.this); InboundChannelBuffer buffer = new InboundChannelBuffer(pageSupplier); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java index 0870124022850..893af2140b9b0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java @@ -8,6 +8,7 @@ import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.nio.BytesWriteHandler; +import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.FlushReadyWrite; import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.NioSelector; @@ -28,6 +29,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -49,7 +51,6 @@ public class SSLChannelContextTests extends ESTestCase { private Consumer exceptionHandler; private SSLDriver sslDriver; private ByteBuffer readBuffer = ByteBuffer.allocate(1 << 14); - private ByteBuffer writeBuffer = ByteBuffer.allocate(1 << 14); private int messageLength; @Before @@ -73,7 +74,6 @@ public void init() { when(selector.isOnCurrentThread()).thenReturn(true); when(selector.getTaskScheduler()).thenReturn(nioTimer); when(sslDriver.getNetworkReadBuffer()).thenReturn(readBuffer); - when(sslDriver.getNetworkWriteBuffer()).thenReturn(writeBuffer); ByteBuffer buffer = ByteBuffer.allocate(1 << 14); when(selector.getIoBuffer()).thenAnswer(invocationOnMock -> { buffer.clear(); @@ -85,7 +85,7 @@ public void testSuccessfulRead() throws IOException { byte[] bytes = createMessage(messageLength); when(rawChannel.read(any(ByteBuffer.class))).thenReturn(bytes.length); - doAnswer(getAnswerForBytes(bytes)).when(sslDriver).read(channelBuffer); + doAnswer(getReadAnswerForBytes(bytes)).when(sslDriver).read(channelBuffer); when(readConsumer.apply(channelBuffer)).thenReturn(messageLength, 0); @@ -100,7 +100,7 @@ public void testMultipleReadsConsumed() throws IOException { byte[] bytes = createMessage(messageLength * 2); when(rawChannel.read(any(ByteBuffer.class))).thenReturn(bytes.length); - doAnswer(getAnswerForBytes(bytes)).when(sslDriver).read(channelBuffer); + doAnswer(getReadAnswerForBytes(bytes)).when(sslDriver).read(channelBuffer); when(readConsumer.apply(channelBuffer)).thenReturn(messageLength, messageLength, 0); @@ -115,7 +115,7 @@ public void testPartialRead() throws IOException { byte[] bytes = createMessage(messageLength); when(rawChannel.read(any(ByteBuffer.class))).thenReturn(bytes.length); - doAnswer(getAnswerForBytes(bytes)).when(sslDriver).read(channelBuffer); + doAnswer(getReadAnswerForBytes(bytes)).when(sslDriver).read(channelBuffer); when(readConsumer.apply(channelBuffer)).thenReturn(0); @@ -173,7 +173,6 @@ public void testSSLDriverClosedOnClose() throws IOException { public void testQueuedWritesAreIgnoredWhenNotReadyForAppWrites() { when(sslDriver.readyForApplicationWrites()).thenReturn(false); - when(sslDriver.hasFlushPending()).thenReturn(false); when(sslDriver.needsNonApplicationWrite()).thenReturn(false); context.queueWriteOperation(mock(FlushReadyWrite.class)); @@ -181,25 +180,25 @@ public void testQueuedWritesAreIgnoredWhenNotReadyForAppWrites() { assertFalse(context.readyForFlush()); } - public void testPendingFlushMeansWriteInterested() { - when(sslDriver.readyForApplicationWrites()).thenReturn(randomBoolean()); - when(sslDriver.hasFlushPending()).thenReturn(true); - when(sslDriver.needsNonApplicationWrite()).thenReturn(false); + public void testPendingEncryptedFlushMeansWriteInterested() throws Exception { + when(sslDriver.readyForApplicationWrites()).thenReturn(false); + when(sslDriver.needsNonApplicationWrite()).thenReturn(true, false); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + // Call will put bytes in buffer to flush + context.flushChannel(); assertTrue(context.readyForFlush()); } public void testNeedsNonAppWritesMeansWriteInterested() { when(sslDriver.readyForApplicationWrites()).thenReturn(false); - when(sslDriver.hasFlushPending()).thenReturn(false); when(sslDriver.needsNonApplicationWrite()).thenReturn(true); assertTrue(context.readyForFlush()); } - public void testNotWritesInterestInAppMode() { + public void testNoNonAppWriteInterestInAppMode() { when(sslDriver.readyForApplicationWrites()).thenReturn(true); - when(sslDriver.hasFlushPending()).thenReturn(false); assertFalse(context.readyForFlush()); @@ -207,66 +206,68 @@ public void testNotWritesInterestInAppMode() { } public void testFirstFlushMustFinishForWriteToContinue() throws Exception { - when(sslDriver.hasFlushPending()).thenReturn(true, true); when(sslDriver.readyForApplicationWrites()).thenReturn(false); + when(sslDriver.needsNonApplicationWrite()).thenReturn(true); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + // First call will put bytes in buffer to flush + context.flushChannel(); + assertTrue(context.readyForFlush()); + // Second call will will not continue generating non-app bytes because they still need to be flushed context.flushChannel(); + assertTrue(context.readyForFlush()); - verify(sslDriver, times(0)).nonApplicationWrite(); + verify(sslDriver, times(1)).nonApplicationWrite(any(SSLOutboundBuffer.class)); } public void testNonAppWrites() throws Exception { - when(sslDriver.hasFlushPending()).thenReturn(false, false, true, false, true); when(sslDriver.needsNonApplicationWrite()).thenReturn(true, true, false); when(sslDriver.readyForApplicationWrites()).thenReturn(false); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(1); context.flushChannel(); - verify(sslDriver, times(2)).nonApplicationWrite(); + verify(sslDriver, times(2)).nonApplicationWrite(any(SSLOutboundBuffer.class)); verify(rawChannel, times(2)).write(same(selector.getIoBuffer())); } public void testNonAppWritesStopIfBufferNotFullyFlushed() throws Exception { - when(sslDriver.hasFlushPending()).thenReturn(false, false, true, true); - when(sslDriver.needsNonApplicationWrite()).thenReturn(true, true, true, true); + when(sslDriver.needsNonApplicationWrite()).thenReturn(true); when(sslDriver.readyForApplicationWrites()).thenReturn(false); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(0); context.flushChannel(); - verify(sslDriver, times(1)).nonApplicationWrite(); + verify(sslDriver, times(1)).nonApplicationWrite(any(SSLOutboundBuffer.class)); verify(rawChannel, times(1)).write(same(selector.getIoBuffer())); } public void testQueuedWriteIsFlushedInFlushCall() throws Exception { ByteBuffer[] buffers = {ByteBuffer.allocate(10)}; - FlushReadyWrite flushOperation = mock(FlushReadyWrite.class); + FlushReadyWrite flushOperation = new FlushReadyWrite(context, buffers, listener); context.queueWriteOperation(flushOperation); - when(flushOperation.getBuffersToWrite()).thenReturn(buffers); - when(flushOperation.getListener()).thenReturn(listener); - when(sslDriver.hasFlushPending()).thenReturn(false, false, false, false); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - when(sslDriver.applicationWrite(buffers)).thenReturn(10); - when(flushOperation.isFullyFlushed()).thenReturn(false,true); + doAnswer(getWriteAnswer(10, true)).when(sslDriver).write(eq(flushOperation), any(SSLOutboundBuffer.class)); + + when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(10); context.flushChannel(); - verify(flushOperation).incrementIndex(10); verify(rawChannel, times(1)).write(same(selector.getIoBuffer())); verify(selector).executeListener(listener, null); assertFalse(context.readyForFlush()); } public void testPartialFlush() throws IOException { - ByteBuffer[] buffers = {ByteBuffer.allocate(10)}; - FlushReadyWrite flushOperation = mock(FlushReadyWrite.class); + ByteBuffer[] buffers = {ByteBuffer.allocate(5)}; + FlushReadyWrite flushOperation = new FlushReadyWrite(context, buffers, listener); context.queueWriteOperation(flushOperation); - when(flushOperation.getBuffersToWrite()).thenReturn(buffers); - when(flushOperation.getListener()).thenReturn(listener); - when(sslDriver.hasFlushPending()).thenReturn(false, false, true); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - when(sslDriver.applicationWrite(buffers)).thenReturn(5); - when(flushOperation.isFullyFlushed()).thenReturn(false, false); + doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(eq(flushOperation), any(SSLOutboundBuffer.class)); + when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(4); context.flushChannel(); verify(rawChannel, times(1)).write(same(selector.getIoBuffer())); @@ -279,24 +280,16 @@ public void testMultipleWritesPartialFlushes() throws IOException { BiConsumer listener2 = mock(BiConsumer.class); ByteBuffer[] buffers1 = {ByteBuffer.allocate(10)}; ByteBuffer[] buffers2 = {ByteBuffer.allocate(5)}; - FlushReadyWrite flushOperation1 = mock(FlushReadyWrite.class); - FlushReadyWrite flushOperation2 = mock(FlushReadyWrite.class); - when(flushOperation1.getBuffersToWrite()).thenReturn(buffers1); - when(flushOperation2.getBuffersToWrite()).thenReturn(buffers2); - when(flushOperation1.getListener()).thenReturn(listener); - when(flushOperation2.getListener()).thenReturn(listener2); + FlushReadyWrite flushOperation1 = new FlushReadyWrite(context, buffers1, listener); + FlushReadyWrite flushOperation2 = new FlushReadyWrite(context, buffers2, listener2); context.queueWriteOperation(flushOperation1); context.queueWriteOperation(flushOperation2); - when(sslDriver.hasFlushPending()).thenReturn(false, false, false, false, false, true); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - when(sslDriver.applicationWrite(buffers1)).thenReturn(5, 5); - when(sslDriver.applicationWrite(buffers2)).thenReturn(3); - when(flushOperation1.isFullyFlushed()).thenReturn(false, false, true); - when(flushOperation2.isFullyFlushed()).thenReturn(false); + doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(any(FlushOperation.class), any(SSLOutboundBuffer.class)); + when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(5, 5, 2); context.flushChannel(); - verify(flushOperation1, times(2)).incrementIndex(5); verify(rawChannel, times(3)).write(same(selector.getIoBuffer())); verify(selector).executeListener(listener, null); verify(selector, times(0)).executeListener(listener2, null); @@ -304,29 +297,27 @@ public void testMultipleWritesPartialFlushes() throws IOException { } public void testWhenIOExceptionThrownListenerIsCalled() throws IOException { - ByteBuffer[] buffers = {ByteBuffer.allocate(10)}; - FlushReadyWrite flushOperation = mock(FlushReadyWrite.class); + ByteBuffer[] buffers = {ByteBuffer.allocate(5)}; + FlushReadyWrite flushOperation = new FlushReadyWrite(context, buffers, listener); context.queueWriteOperation(flushOperation); IOException exception = new IOException(); - when(flushOperation.getBuffersToWrite()).thenReturn(buffers); - when(flushOperation.getListener()).thenReturn(listener); - when(sslDriver.hasFlushPending()).thenReturn(false, false); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - when(sslDriver.applicationWrite(buffers)).thenReturn(5); + doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(eq(flushOperation), any(SSLOutboundBuffer.class)); when(rawChannel.write(any(ByteBuffer.class))).thenThrow(exception); - when(flushOperation.isFullyFlushed()).thenReturn(false); expectThrows(IOException.class, () -> context.flushChannel()); - verify(flushOperation).incrementIndex(5); verify(selector).executeFailedListener(listener, exception); assertFalse(context.readyForFlush()); } public void testWriteIOExceptionMeansChannelReadyToClose() throws Exception { - when(sslDriver.hasFlushPending()).thenReturn(true); - when(sslDriver.needsNonApplicationWrite()).thenReturn(true); when(sslDriver.readyForApplicationWrites()).thenReturn(false); + when(sslDriver.needsNonApplicationWrite()).thenReturn(true); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + + context.flushChannel(); + when(rawChannel.write(any(ByteBuffer.class))).thenThrow(new IOException()); assertFalse(context.selectorShouldClose()); @@ -413,7 +404,27 @@ public void testRegisterInitiatesDriver() throws IOException { } } - private Answer getAnswerForBytes(byte[] bytes) { + private Answer getWriteAnswer(int bytesToEncrypt, boolean isApp) { + return invocationOnMock -> { + SSLOutboundBuffer outboundBuffer; + if (isApp) { + outboundBuffer = (SSLOutboundBuffer) invocationOnMock.getArguments()[1]; + } else { + outboundBuffer = (SSLOutboundBuffer) invocationOnMock.getArguments()[0]; + } + ByteBuffer byteBuffer = outboundBuffer.nextWriteBuffer(bytesToEncrypt + 1); + for (int i = 0; i < bytesToEncrypt; ++i) { + byteBuffer.put((byte) i); + } + outboundBuffer.incrementEncryptedBytes(bytesToEncrypt); + if (isApp) { + ((FlushOperation) invocationOnMock.getArguments()[0]).incrementIndex(bytesToEncrypt); + } + return bytesToEncrypt; + }; + } + + private Answer getReadAnswerForBytes(byte[] bytes) { return invocationOnMock -> { InboundChannelBuffer buffer = (InboundChannelBuffer) invocationOnMock.getArguments()[0]; buffer.ensureCapacity(buffer.getIndex() + bytes.length); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java index b1d39ddc6ac9f..4b86d3223b061 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java @@ -6,7 +6,9 @@ package org.elasticsearch.xpack.security.transport.nio; import org.elasticsearch.bootstrap.JavaVersion; +import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; +import org.elasticsearch.nio.Page; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.ssl.CertParsingUtils; import org.elasticsearch.xpack.core.ssl.PemUtils; @@ -28,8 +30,7 @@ public class SSLDriverTests extends ESTestCase { - private final Supplier pageSupplier = - () -> new InboundChannelBuffer.Page(ByteBuffer.allocate(1 << 14), () -> {}); + private final Supplier pageSupplier = () -> new Page(ByteBuffer.allocate(1 << 14), () -> {}); private InboundChannelBuffer serverBuffer = new InboundChannelBuffer(pageSupplier); private InboundChannelBuffer clientBuffer = new InboundChannelBuffer(pageSupplier); private InboundChannelBuffer genericBuffer = new InboundChannelBuffer(pageSupplier); @@ -141,10 +142,6 @@ public void testHandshakeFailureBecauseProtocolMismatch() throws Exception { boolean expectedMessage = oldExpected.equals(sslException.getMessage()) || jdk11Expected.equals(sslException.getMessage()); assertTrue("Unexpected exception message: " + sslException.getMessage(), expectedMessage); - // In JDK11 we need an non-application write - if (serverDriver.needsNonApplicationWrite()) { - serverDriver.nonApplicationWrite(); - } // Prior to JDK11 we still need to send a close alert if (serverDriver.isClosed() == false) { failedCloseAlert(serverDriver, clientDriver, Arrays.asList("Received fatal alert: protocol_version", @@ -166,10 +163,7 @@ public void testHandshakeFailureBecauseNoCiphers() throws Exception { SSLDriver serverDriver = getDriver(serverEngine, false); expectThrows(SSLException.class, () -> handshake(clientDriver, serverDriver)); - // In JDK11 we need an non-application write - if (serverDriver.needsNonApplicationWrite()) { - serverDriver.nonApplicationWrite(); - } + // Prior to JDK11 we still need to send a close alert if (serverDriver.isClosed() == false) { List messages = Arrays.asList("Received fatal alert: handshake_failure", @@ -192,8 +186,6 @@ public void testCloseDuringHandshakeJDK11() throws Exception { sendHandshakeMessages(clientDriver, serverDriver); sendHandshakeMessages(serverDriver, clientDriver); - sendData(clientDriver, serverDriver); - assertTrue(clientDriver.isHandshaking()); assertTrue(serverDriver.isHandshaking()); @@ -227,8 +219,6 @@ public void testCloseDuringHandshakePreJDK11() throws Exception { sendHandshakeMessages(clientDriver, serverDriver); sendHandshakeMessages(serverDriver, clientDriver); - sendData(clientDriver, serverDriver); - assertTrue(clientDriver.isHandshaking()); assertTrue(serverDriver.isHandshaking()); @@ -306,12 +296,12 @@ private void normalClose(SSLDriver sendDriver, SSLDriver receiveDriver) throws I } private void sendNonApplicationWrites(SSLDriver sendDriver, SSLDriver receiveDriver) throws SSLException { - while (sendDriver.needsNonApplicationWrite() || sendDriver.hasFlushPending()) { - if (sendDriver.hasFlushPending() == false) { - sendDriver.nonApplicationWrite(); - } - if (sendDriver.hasFlushPending()) { - sendData(sendDriver, receiveDriver, true); + SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); + while (sendDriver.needsNonApplicationWrite() || outboundBuffer.hasEncryptedBytesToFlush()) { + if (outboundBuffer.hasEncryptedBytesToFlush()) { + sendData(outboundBuffer.buildNetworkFlushOperation(), receiveDriver); + } else { + sendDriver.nonApplicationWrite(outboundBuffer); } } } @@ -326,7 +316,7 @@ private void handshake(SSLDriver clientDriver, SSLDriver serverDriver, boolean i serverDriver.init(); } - assertTrue(clientDriver.needsNonApplicationWrite() || clientDriver.hasFlushPending()); + assertTrue(clientDriver.needsNonApplicationWrite()); assertFalse(serverDriver.needsNonApplicationWrite()); sendHandshakeMessages(clientDriver, serverDriver); @@ -350,58 +340,51 @@ private void handshake(SSLDriver clientDriver, SSLDriver serverDriver, boolean i } private void sendHandshakeMessages(SSLDriver sendDriver, SSLDriver receiveDriver) throws IOException { - assertTrue(sendDriver.needsNonApplicationWrite() || sendDriver.hasFlushPending()); + assertTrue(sendDriver.needsNonApplicationWrite()); - while (sendDriver.needsNonApplicationWrite() || sendDriver.hasFlushPending()) { - if (sendDriver.hasFlushPending() == false) { - sendDriver.nonApplicationWrite(); - } - if (sendDriver.isHandshaking()) { - assertTrue(sendDriver.hasFlushPending()); - sendData(sendDriver, receiveDriver); - assertFalse(sendDriver.hasFlushPending()); + SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); + + while (sendDriver.needsNonApplicationWrite() || outboundBuffer.hasEncryptedBytesToFlush()) { + if (outboundBuffer.hasEncryptedBytesToFlush()) { + sendData(outboundBuffer.buildNetworkFlushOperation(), receiveDriver); receiveDriver.read(genericBuffer); + } else { + sendDriver.nonApplicationWrite(outboundBuffer); } } if (receiveDriver.isHandshaking()) { - assertTrue(receiveDriver.needsNonApplicationWrite() || receiveDriver.hasFlushPending()); + assertTrue(receiveDriver.needsNonApplicationWrite()); } } private void sendAppData(SSLDriver sendDriver, SSLDriver receiveDriver, ByteBuffer[] message) throws IOException { - assertFalse(sendDriver.needsNonApplicationWrite()); int bytesToEncrypt = Arrays.stream(message).mapToInt(Buffer::remaining).sum(); + SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); + FlushOperation flushOperation = new FlushOperation(message, (r, l) -> {}); int bytesEncrypted = 0; while (bytesToEncrypt > bytesEncrypted) { - bytesEncrypted += sendDriver.applicationWrite(message); - sendData(sendDriver, receiveDriver); + bytesEncrypted += sendDriver.write(flushOperation, outboundBuffer); + sendData(outboundBuffer.buildNetworkFlushOperation(), receiveDriver); } } - private void sendData(SSLDriver sendDriver, SSLDriver receiveDriver) { - sendData(sendDriver, receiveDriver, randomBoolean()); - } - - private void sendData(SSLDriver sendDriver, SSLDriver receiveDriver, boolean partial) { - ByteBuffer writeBuffer = sendDriver.getNetworkWriteBuffer(); + private void sendData(FlushOperation flushOperation, SSLDriver receiveDriver) { ByteBuffer readBuffer = receiveDriver.getNetworkReadBuffer(); - if (partial) { - int initialLimit = writeBuffer.limit(); - int bytesToWrite = writeBuffer.remaining() / (randomInt(2) + 2); - writeBuffer.limit(writeBuffer.position() + bytesToWrite); - readBuffer.put(writeBuffer); - writeBuffer.limit(initialLimit); - assertTrue(sendDriver.hasFlushPending()); - readBuffer.put(writeBuffer); - assertFalse(sendDriver.hasFlushPending()); + ByteBuffer[] writeBuffers = flushOperation.getBuffersToWrite(); + int bytesToEncrypt = Arrays.stream(writeBuffers).mapToInt(Buffer::remaining).sum(); + assert bytesToEncrypt < readBuffer.capacity() : "Flush operation must be less that read buffer"; + assert writeBuffers.length > 0 : "No write buffers"; - } else { + for (ByteBuffer writeBuffer : writeBuffers) { + int written = writeBuffer.remaining(); readBuffer.put(writeBuffer); - assertFalse(sendDriver.hasFlushPending()); + flushOperation.incrementIndex(written); } + + assertTrue(flushOperation.isFullyFlushed()); } private SSLDriver getDriver(SSLEngine engine, boolean isClient) { From 29fefcfbea4fe5e25da4a3fa0746556f5e9c6128 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 25 Apr 2019 17:50:29 -0400 Subject: [PATCH 196/260] Mute testDriverConfigurationWithSSLInURL Tracked at #41557 --- .../org/elasticsearch/xpack/sql/jdbc/JdbcConfigurationTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcConfigurationTests.java b/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcConfigurationTests.java index d919a5819329e..acaf6917862c6 100644 --- a/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcConfigurationTests.java +++ b/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcConfigurationTests.java @@ -252,6 +252,7 @@ public void testSSLPropertiesOverride() throws Exception { assertSslConfig(props, JdbcConfiguration.create("jdbc:es://test?" + sslUrlProps.toString(), props, 0).sslConfig()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41557") public void testDriverConfigurationWithSSLInURL() { Map urlPropMap = sslProperties(); From 380172b49c970cc88c578884d30678e3e0da5bfd Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Fri, 26 Apr 2019 15:42:52 +1000 Subject: [PATCH 197/260] Remove Version.V_6_x_x constants use in security (#41185) This removes some use of the v6 constants in various parts of security. Mostly this is BWC testing code, which is no longer needed as ES8 will not need to maintain compatibility with ES6. Relates: #41164 --- .../hlrc/HasPrivilegesResponseTests.java | 25 ------- .../security/action/role/PutRoleRequest.java | 13 ++-- .../action/user/AuthenticateResponse.java | 16 +---- .../user/GetUserPrivilegesResponse.java | 11 +--- .../action/user/HasPrivilegesRequest.java | 9 +-- .../action/user/HasPrivilegesResponse.java | 25 ++----- .../core/security/authc/Authentication.java | 15 ++--- .../core/security/authz/RoleDescriptor.java | 26 ++------ .../action/role/PutRoleRequestTests.java | 66 +------------------ .../user/HasPrivilegesRequestTests.java | 15 +---- .../xpack/security/Security.java | 12 ---- .../xpack/security/authc/ApiKeyService.java | 6 -- .../xpack/security/authc/UserToken.java | 10 +-- .../xpack/security/SecurityTests.java | 13 ---- .../security/authz/RoleDescriptorTests.java | 4 +- 15 files changed, 33 insertions(+), 233 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/hlrc/HasPrivilegesResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/hlrc/HasPrivilegesResponseTests.java index bb748a71f42dc..94e326e10555e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/hlrc/HasPrivilegesResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/hlrc/HasPrivilegesResponseTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.client.security.hlrc; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.security.HasPrivilegesResponse; import org.elasticsearch.common.bytes.BytesReference; @@ -32,9 +31,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.client.AbstractHlrcStreamableXContentTestCase; -import org.elasticsearch.test.VersionUtils; import org.elasticsearch.xpack.core.security.authz.permission.ResourcePrivileges; -import org.hamcrest.Matchers; import org.junit.Assert; import java.io.IOException; @@ -55,28 +52,6 @@ public class HasPrivilegesResponseTests extends AbstractHlrcStreamableXContentTe org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse, HasPrivilegesResponse> { - public void testSerializationV64OrV65() throws IOException { - final org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse original = randomResponse(); - final Version version = VersionUtils.randomVersionBetween(LuceneTestCase.random(), Version.V_6_4_0, Version.V_6_5_1); - final org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse copy = serializeAndDeserialize(original, version); - - Assert.assertThat(copy.isCompleteMatch(), equalTo(original.isCompleteMatch())); - Assert.assertThat(copy.getClusterPrivileges().entrySet(), Matchers.emptyIterable()); - Assert.assertThat(copy.getIndexPrivileges(), equalTo(original.getIndexPrivileges())); - Assert.assertThat(copy.getApplicationPrivileges(), equalTo(original.getApplicationPrivileges())); - } - - public void testSerializationV63() throws IOException { - final org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse original = randomResponse(); - final org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse copy = - serializeAndDeserialize(original, Version.V_6_3_0); - - Assert.assertThat(copy.isCompleteMatch(), equalTo(original.isCompleteMatch())); - Assert.assertThat(copy.getClusterPrivileges().entrySet(), Matchers.emptyIterable()); - Assert.assertThat(copy.getIndexPrivileges(), equalTo(original.getIndexPrivileges())); - Assert.assertThat(copy.getApplicationPrivileges(), equalTo(Collections.emptyMap())); - } - public void testToXContent() throws Exception { final org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse response = new org.elasticsearch.xpack.core.security.action.user.HasPrivilegesResponse("daredevil", diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequest.java index 3f9de8f1e68b3..e19d9cebb64c1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequest.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.core.security.action.role; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.WriteRequest; @@ -168,10 +167,8 @@ public void readFrom(StreamInput in) throws IOException { for (int i = 0; i < indicesSize; i++) { indicesPrivileges.add(new RoleDescriptor.IndicesPrivileges(in)); } - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - applicationPrivileges = in.readList(RoleDescriptor.ApplicationResourcePrivileges::new); - conditionalClusterPrivileges = ConditionalClusterPrivileges.readArray(in); - } + applicationPrivileges = in.readList(RoleDescriptor.ApplicationResourcePrivileges::new); + conditionalClusterPrivileges = ConditionalClusterPrivileges.readArray(in); runAs = in.readStringArray(); refreshPolicy = RefreshPolicy.readFrom(in); metadata = in.readMap(); @@ -186,10 +183,8 @@ public void writeTo(StreamOutput out) throws IOException { for (RoleDescriptor.IndicesPrivileges index : indicesPrivileges) { index.writeTo(out); } - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeList(applicationPrivileges); - ConditionalClusterPrivileges.writeArray(out, this.conditionalClusterPrivileges); - } + out.writeList(applicationPrivileges); + ConditionalClusterPrivileges.writeArray(out, this.conditionalClusterPrivileges); out.writeStringArray(runAs); refreshPolicy.writeTo(out); out.writeMap(metadata); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/AuthenticateResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/AuthenticateResponse.java index 06a4df019c326..27e119ef5a9d8 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/AuthenticateResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/AuthenticateResponse.java @@ -5,12 +5,10 @@ */ package org.elasticsearch.xpack.core.security.action.user; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.xpack.core.security.authc.Authentication; -import org.elasticsearch.xpack.core.security.user.User; import java.io.IOException; @@ -31,23 +29,13 @@ public Authentication authentication() { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - if (out.getVersion().before(Version.V_6_6_0)) { - User.writeTo(authentication.getUser(), out); - } else { - authentication.writeTo(out); - } + authentication.writeTo(out); } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - if (in.getVersion().before(Version.V_6_6_0)) { - final User user = User.readFrom(in); - final Authentication.RealmRef unknownRealm = new Authentication.RealmRef("__unknown", "__unknown", "__unknown"); - authentication = new Authentication(user, unknownRealm, unknownRealm); - } else { - authentication = new Authentication(in); - } + authentication = new Authentication(in); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java index b6d84d766c328..7c47b700cc0b5 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.core.security.action.user; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; @@ -145,11 +144,7 @@ public Indices(StreamInput in) throws IOException { return new FieldPermissionsDefinition.FieldGrantExcludeGroup(grant, exclude); })); queries = Collections.unmodifiableSet(in.readSet(StreamInput::readBytesReference)); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - this.allowRestrictedIndices = in.readBoolean(); - } else { - this.allowRestrictedIndices = false; - } + this.allowRestrictedIndices = in.readBoolean(); } public Set getIndices() { @@ -254,9 +249,7 @@ public void writeTo(StreamOutput out) throws IOException { output.writeOptionalStringArray(fields.getExcludedFields()); }); out.writeCollection(queries, StreamOutput::writeBytesReference); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeBoolean(allowRestrictedIndices); - } + out.writeBoolean(allowRestrictedIndices); } } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequest.java index 64a69abd5ca1e..93ac7ff45dd66 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequest.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.core.security.action.user; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.common.io.stream.StreamInput; @@ -109,9 +108,7 @@ public void readFrom(StreamInput in) throws IOException { for (int i = 0; i < indexSize; i++) { indexPrivileges[i] = new RoleDescriptor.IndicesPrivileges(in); } - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - applicationPrivileges = in.readArray(ApplicationResourcePrivileges::new, ApplicationResourcePrivileges[]::new); - } + applicationPrivileges = in.readArray(ApplicationResourcePrivileges::new, ApplicationResourcePrivileges[]::new); } @Override @@ -123,9 +120,7 @@ public void writeTo(StreamOutput out) throws IOException { for (RoleDescriptor.IndicesPrivileges priv : indexPrivileges) { priv.writeTo(out); } - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeArray(ApplicationResourcePrivileges::write, applicationPrivileges); - } + out.writeArray(ApplicationResourcePrivileges::write, applicationPrivileges); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesResponse.java index 74984556dc1a0..f67219061bb85 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesResponse.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.core.security.action.user; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -103,16 +102,10 @@ public int hashCode() { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); completeMatch = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_6_6_0 )) { - cluster = in.readMap(StreamInput::readString, StreamInput::readBoolean); - } + cluster = in.readMap(StreamInput::readString, StreamInput::readBoolean); index = readResourcePrivileges(in); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - application = in.readMap(StreamInput::readString, HasPrivilegesResponse::readResourcePrivileges); - } - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - username = in.readString(); - } + application = in.readMap(StreamInput::readString, HasPrivilegesResponse::readResourcePrivileges); + username = in.readString(); } private static Set readResourcePrivileges(StreamInput in) throws IOException { @@ -130,16 +123,10 @@ private static Set readResourcePrivileges(StreamInput in) th public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeBoolean(completeMatch); - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeMap(cluster, StreamOutput::writeString, StreamOutput::writeBoolean); - } + out.writeMap(cluster, StreamOutput::writeString, StreamOutput::writeBoolean); writeResourcePrivileges(out, index); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeMap(application, StreamOutput::writeString, HasPrivilegesResponse::writeResourcePrivileges); - } - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeString(username); - } + out.writeMap(application, StreamOutput::writeString, HasPrivilegesResponse::writeResourcePrivileges); + out.writeString(username); } private static void writeResourcePrivileges(StreamOutput out, Set privileges) throws IOException { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Authentication.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Authentication.java index de75d90eca51f..17c3e05a772ce 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Authentication.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/Authentication.java @@ -60,13 +60,8 @@ public Authentication(StreamInput in) throws IOException { this.lookedUpBy = null; } this.version = in.getVersion(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - type = AuthenticationType.values()[in.readVInt()]; - metadata = in.readMap(); - } else { - type = AuthenticationType.REALM; - metadata = Collections.emptyMap(); - } + type = AuthenticationType.values()[in.readVInt()]; + metadata = in.readMap(); } public User getUser() { @@ -165,10 +160,8 @@ public void writeTo(StreamOutput out) throws IOException { } else { out.writeBoolean(false); } - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeVInt(type.ordinal()); - out.writeMap(metadata); - } + out.writeVInt(type.ordinal()); + out.writeMap(metadata); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptor.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptor.java index 5705d7bf35723..15304ff85dbd9 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptor.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptor.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.core.security.authz; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; @@ -122,13 +121,8 @@ public RoleDescriptor(StreamInput in) throws IOException { this.metadata = in.readMap(); this.transientMetadata = in.readMap(); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - this.applicationPrivileges = in.readArray(ApplicationResourcePrivileges::new, ApplicationResourcePrivileges[]::new); - this.conditionalClusterPrivileges = ConditionalClusterPrivileges.readArray(in); - } else { - this.applicationPrivileges = ApplicationResourcePrivileges.NONE; - this.conditionalClusterPrivileges = ConditionalClusterPrivileges.EMPTY_ARRAY; - } + this.applicationPrivileges = in.readArray(ApplicationResourcePrivileges::new, ApplicationResourcePrivileges[]::new); + this.conditionalClusterPrivileges = ConditionalClusterPrivileges.readArray(in); } public String getName() { @@ -264,10 +258,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(runAs); out.writeMap(metadata); out.writeMap(transientMetadata); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeArray(ApplicationResourcePrivileges::write, applicationPrivileges); - ConditionalClusterPrivileges.writeArray(out, getConditionalClusterPrivileges()); - } + out.writeArray(ApplicationResourcePrivileges::write, applicationPrivileges); + ConditionalClusterPrivileges.writeArray(out, getConditionalClusterPrivileges()); } public static RoleDescriptor parse(String name, BytesReference source, boolean allow2xFormat, XContentType xContentType) @@ -608,11 +600,7 @@ public IndicesPrivileges(StreamInput in) throws IOException { this.deniedFields = in.readOptionalStringArray(); this.privileges = in.readStringArray(); this.query = in.readOptionalBytesReference(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - allowRestrictedIndices = in.readBoolean(); - } else { - allowRestrictedIndices = false; - } + this.allowRestrictedIndices = in.readBoolean(); } @Override @@ -622,9 +610,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalStringArray(deniedFields); out.writeStringArray(privileges); out.writeOptionalBytesReference(query); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeBoolean(allowRestrictedIndices); - } + out.writeBoolean(allowRestrictedIndices); } public static Builder builder() { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequestTests.java index f642f3ab919d2..7ca9f4da74ab3 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/role/PutRoleRequestTests.java @@ -20,7 +20,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; import org.elasticsearch.xpack.core.XPackClientPlugin; -import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor.ApplicationResourcePrivileges; import org.elasticsearch.xpack.core.security.authz.privilege.ConditionalClusterPrivileges; @@ -31,11 +30,9 @@ import java.util.Map; import java.util.function.Supplier; -import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; -import static org.hamcrest.Matchers.iterableWithSize; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -60,7 +57,7 @@ public void testSerialization() throws IOException { final BytesStreamOutput out = new BytesStreamOutput(); if (randomBoolean()) { - final Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_7_0, Version.CURRENT); + final Version version = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); logger.info("Serializing with version {}", version); out.setVersion(version); } @@ -75,67 +72,6 @@ public void testSerialization() throws IOException { assertThat(copy.roleDescriptor(), equalTo(original.roleDescriptor())); } - public void testSerializationBetweenV64AndV66() throws IOException { - final PutRoleRequest original = buildRandomRequest(); - - final BytesStreamOutput out = new BytesStreamOutput(); - final Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_4_0, Version.V_6_6_0); - out.setVersion(version); - original.writeTo(out); - - final PutRoleRequest copy = new PutRoleRequest(); - final NamedWriteableRegistry registry = new NamedWriteableRegistry(new XPackClientPlugin(Settings.EMPTY).getNamedWriteables()); - StreamInput in = new NamedWriteableAwareStreamInput(ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), registry); - in.setVersion(version); - copy.readFrom(in); - - assertThat(copy.name(), equalTo(original.name())); - assertThat(copy.cluster(), equalTo(original.cluster())); - assertIndicesSerializedRestricted(copy.indices(), original.indices()); - assertThat(copy.runAs(), equalTo(original.runAs())); - assertThat(copy.metadata(), equalTo(original.metadata())); - assertThat(copy.getRefreshPolicy(), equalTo(original.getRefreshPolicy())); - - assertThat(copy.applicationPrivileges(), equalTo(original.applicationPrivileges())); - assertThat(copy.conditionalClusterPrivileges(), equalTo(original.conditionalClusterPrivileges())); - } - - public void testSerializationV60AndV32() throws IOException { - final PutRoleRequest original = buildRandomRequest(); - - final BytesStreamOutput out = new BytesStreamOutput(); - final Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, Version.V_6_3_2); - out.setVersion(version); - original.writeTo(out); - - final PutRoleRequest copy = new PutRoleRequest(); - final StreamInput in = out.bytes().streamInput(); - in.setVersion(version); - copy.readFrom(in); - - assertThat(copy.name(), equalTo(original.name())); - assertThat(copy.cluster(), equalTo(original.cluster())); - assertIndicesSerializedRestricted(copy.indices(), original.indices()); - assertThat(copy.runAs(), equalTo(original.runAs())); - assertThat(copy.metadata(), equalTo(original.metadata())); - assertThat(copy.getRefreshPolicy(), equalTo(original.getRefreshPolicy())); - - assertThat(copy.applicationPrivileges(), iterableWithSize(0)); - assertThat(copy.conditionalClusterPrivileges(), arrayWithSize(0)); - } - - private void assertIndicesSerializedRestricted(RoleDescriptor.IndicesPrivileges[] copy, RoleDescriptor.IndicesPrivileges[] original) { - assertThat(copy.length, equalTo(original.length)); - for (int i = 0; i < copy.length; i++) { - assertThat(copy[i].allowRestrictedIndices(), equalTo(false)); - assertThat(copy[i].getIndices(), equalTo(original[i].getIndices())); - assertThat(copy[i].getPrivileges(), equalTo(original[i].getPrivileges())); - assertThat(copy[i].getDeniedFields(), equalTo(original[i].getDeniedFields())); - assertThat(copy[i].getGrantedFields(), equalTo(original[i].getGrantedFields())); - assertThat(copy[i].getQuery(), equalTo(original[i].getQuery())); - } - } - private void assertSuccessfulValidation(PutRoleRequest request) { final ActionRequestValidationException exception = request.validate(); assertThat(exception, nullValue()); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequestTests.java index a6706542e9613..6dd1d8a25f088 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/HasPrivilegesRequestTests.java @@ -25,13 +25,12 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; public class HasPrivilegesRequestTests extends ESTestCase { - public void testSerializationV64OrLater() throws IOException { + public void testSerializationCurrentVersion() throws IOException { final HasPrivilegesRequest original = randomRequest(); - final Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_4_0, Version.CURRENT); + final Version version = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); final HasPrivilegesRequest copy = serializeAndDeserialize(original, version); assertThat(copy.username(), equalTo(original.username())); @@ -40,16 +39,6 @@ public void testSerializationV64OrLater() throws IOException { assertThat(copy.applicationPrivileges(), equalTo(original.applicationPrivileges())); } - public void testSerializationV63() throws IOException { - final HasPrivilegesRequest original = randomRequest(); - final HasPrivilegesRequest copy = serializeAndDeserialize(original, Version.V_6_3_0); - - assertThat(copy.username(), equalTo(original.username())); - assertThat(copy.clusterPrivileges(), equalTo(original.clusterPrivileges())); - assertThat(copy.indexPrivileges(), equalTo(original.indexPrivileges())); - assertThat(copy.applicationPrivileges(), nullValue()); - } - public void testValidateNullPrivileges() { final HasPrivilegesRequest request = new HasPrivilegesRequest(); final ActionRequestValidationException exception = request.validate(); 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 8cc970ca77e4f..ff92122626630 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 @@ -985,7 +985,6 @@ public BiConsumer getJoinValidator() { return new ValidateTLSOnJoin(XPackSettings.TRANSPORT_SSL_ENABLED.get(settings), DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings)) .andThen(new ValidateUpgradedSecurityIndex()) - .andThen(new ValidateLicenseCanBeDeserialized()) .andThen(new ValidateLicenseForFIPS(XPackSettings.FIPS_MODE_ENABLED.get(settings))); } return null; @@ -1023,17 +1022,6 @@ public void accept(DiscoveryNode node, ClusterState state) { } } - static final class ValidateLicenseCanBeDeserialized implements BiConsumer { - @Override - public void accept(DiscoveryNode node, ClusterState state) { - License license = LicenseService.getLicense(state.metaData()); - if (license != null && license.version() >= License.VERSION_CRYPTO_ALGORITHMS && node.getVersion().before(Version.V_6_4_0)) { - throw new IllegalStateException("node " + node + " is on version [" + node.getVersion() + - "] that cannot deserialize the license format [" + license.version() + "], upgrade node to at least 6.4.0"); - } - } - } - static final class ValidateLicenseForFIPS implements BiConsumer { private final boolean inFipsMode; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java index 980a39a186637..1297df19f259e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java @@ -233,12 +233,6 @@ private void createApiKeyAndIndexIt(Authentication authentication, CreateApiKeyR final Instant expiration = getApiKeyExpiration(created, request); final SecureString apiKey = UUIDs.randomBase64UUIDSecureString(); final Version version = clusterService.state().nodes().getMinNodeVersion(); - if (version.before(Version.V_6_7_0)) { - logger.warn( - "nodes prior to the minimum supported version for api keys {} exist in the cluster;" - + " these nodes will not be able to use api keys", - Version.V_6_7_0); - } final char[] keyHash = hasher.hash(apiKey); try (XContentBuilder builder = XContentFactory.jsonBuilder()) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/UserToken.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/UserToken.java index fe8b3823120f4..2bcf0849084bc 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/UserToken.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/UserToken.java @@ -73,11 +73,7 @@ public final class UserToken implements Writeable, ToXContentObject { this.id = input.readString(); this.authentication = new Authentication(input); this.expirationTime = Instant.ofEpochSecond(input.readLong(), input.readInt()); - if (version.before(Version.V_6_2_0)) { - this.metadata = Collections.emptyMap(); - } else { - this.metadata = input.readMap(); - } + this.metadata = input.readMap(); } @Override @@ -86,9 +82,7 @@ public void writeTo(StreamOutput out) throws IOException { authentication.writeTo(out); out.writeLong(expirationTime.getEpochSecond()); out.writeInt(expirationTime.getNano()); - if (out.getVersion().onOrAfter(Version.V_6_2_0)) { - out.writeMap(metadata); - } + out.writeMap(metadata); } /** diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index cc573fd9247f9..08f1149572faa 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -271,19 +271,6 @@ public void testTLSJoinValidator() throws Exception { } } - public void testJoinValidatorForLicenseDeserialization() throws Exception { - DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), - VersionUtils.randomVersionBetween(random(), null, Version.V_6_3_0)); - MetaData.Builder builder = MetaData.builder(); - License license = TestUtils.generateSignedLicense(null, - randomIntBetween(License.VERSION_CRYPTO_ALGORITHMS, License.VERSION_CURRENT), -1, TimeValue.timeValueHours(24)); - TestUtils.putLicense(builder, license); - ClusterState state = ClusterState.builder(ClusterName.DEFAULT).metaData(builder.build()).build(); - IllegalStateException e = expectThrows(IllegalStateException.class, - () -> new Security.ValidateLicenseCanBeDeserialized().accept(node, state)); - assertThat(e.getMessage(), containsString("cannot deserialize the license format")); - } - public void testJoinValidatorForFIPSLicense() throws Exception { DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), VersionUtils.randomVersionBetween(random(), null, Version.CURRENT)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RoleDescriptorTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RoleDescriptorTests.java index 0f4b53e984186..a2d828cf92284 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RoleDescriptorTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RoleDescriptorTests.java @@ -213,8 +213,8 @@ public void testParse() throws Exception { assertThat(ex.getMessage(), containsString("not_supported")); } - public void testSerialization() throws Exception { - final Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_4_0, null); + public void testSerializationForCurrentVersion() throws Exception { + final Version version = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); logger.info("Testing serialization with version {}", version); BytesStreamOutput output = new BytesStreamOutput(); output.setVersion(version); From 1038f23510e81b0edffb12d0dfd6a13179682bce Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Fri, 26 Apr 2019 08:56:34 +0200 Subject: [PATCH 198/260] Update community client and integration docs (#41513) Added integrations for a couple of frameworks. Removed community clients where the last commit was more than three years ago. Also added the official go client link and removed the official groovy client, as it is outdated. --- docs/community-clients/index.asciidoc | 94 ++------------------------- docs/plugins/integrations.asciidoc | 30 +++++++++ 2 files changed, 34 insertions(+), 90 deletions(-) diff --git a/docs/community-clients/index.asciidoc b/docs/community-clients/index.asciidoc index 58a6e625aa8b3..def32890a62ab 100644 --- a/docs/community-clients/index.asciidoc +++ b/docs/community-clients/index.asciidoc @@ -13,14 +13,12 @@ a number of clients that have been contributed by the community for various lang * <> * <> * <> -* <> * <> * <> * <> * <> * <> * <> -* <> * <> * <> * <> @@ -55,20 +53,12 @@ a number of clients that have been contributed by the community for various lang * https://www.forgebox.io/view/cbelasticsearch[cbElasticSearch] Native ColdFusion (CFML) support for the ColdBox MVC Platform which provides you with a fluent search interface for Elasticsearch, in addition to a CacheBox Cache provider and a Logbox Appender for logging. -The following project appears to be abandoned: - -* https://github.com/jasonfill/ColdFusion-ElasticSearch-Client[ColdFusion-Elasticsearch-Client] - ColdFusion client for Elasticsearch - [[erlang]] == Erlang * http://github.com/tsloughter/erlastic_search[erlastic_search]: Erlang client using HTTP. -* https://github.com/dieswaytoofast/erlasticsearch[erlasticsearch]: - Erlang client using Thrift. - * https://github.com/datahogs/tirexs[Tirexs]: An https://github.com/elixir-lang/elixir[Elixir] based API/DSL, inspired by http://github.com/karmi/tire[Tire]. Ready to use in pure Erlang @@ -78,12 +68,11 @@ The following project appears to be abandoned: [[go]] == Go +Also see the {client}/go-api/current/index.html[official Elasticsearch Go client]. + * https://github.com/mattbaird/elastigo[elastigo]: Go client. -* https://github.com/belogik/goes[goes]: - Go lib. - * https://github.com/olivere/elastic[elastic]: Elasticsearch client for Google Go. @@ -91,11 +80,6 @@ The following project appears to be abandoned: Golang lib for Elasticsearch client. -[[groovy]] -== Groovy - -See the {client}/groovy-api/current/index.html[official Elasticsearch Groovy client]. - [[haskell]] == Haskell * https://github.com/bitemyapp/bloodhound[bloodhound]: @@ -117,19 +101,6 @@ Also see the {client}/java-api/current/index.html[official Elasticsearch Java cl Also see the {client}/javascript-api/current/index.html[official Elasticsearch JavaScript client]. -* https://github.com/fullscale/elastic.js[Elastic.js]: - A JavaScript implementation of the Elasticsearch Query DSL and Core API. - -* https://github.com/printercu/elastics[elastics]: Simple tiny client that just works - -* https://github.com/roundscope/ember-data-elasticsearch-kit[ember-data-elasticsearch-kit]: - An ember-data kit for both pushing and querying objects to Elasticsearch cluster - -The following project appears to be abandoned: - -* https://github.com/ramv/node-elastical[node-elastical]: - Node.js client for the Elasticsearch REST API - [[kotlin]] == Kotlin @@ -150,17 +121,6 @@ The following project appears to be abandoned: Also see the {client}/net-api/current/index.html[official Elasticsearch .NET client]. -* https://github.com/Yegoroff/PlainElastic.Net[PlainElastic.Net]: - .NET client. - -[[ocaml]] -== OCaml - -The following project appears to be abandoned: - -* https://github.com/tovbinm/ocaml-elasticsearch[ocaml-elasticsearch]: - OCaml client for Elasticsearch - [[perl]] == Perl @@ -186,26 +146,6 @@ Also see the {client}/php-api/current/index.html[official Elasticsearch PHP clie Also see the {client}/python-api/current/index.html[official Elasticsearch Python client]. -* http://github.com/rhec/pyelasticsearch[pyelasticsearch]: - Python client. - -* https://github.com/eriky/ESClient[ESClient]: - A lightweight and easy to use Python client for Elasticsearch. - -* https://github.com/mozilla/elasticutils/[elasticutils]: - A friendly chainable Elasticsearch interface for Python. - -* http://github.com/aparo/pyes[pyes]: - Python client. - -The following projects appear to be abandoned: - -* https://github.com/humangeo/rawes[rawes]: - Python low level client. - -* http://intridea.github.io/surfiki-refine-elasticsearch/[Surfiki Refine]: - Python Map-Reduce engine targeting Elasticsearch indices. - [[r]] == R @@ -218,19 +158,11 @@ The following projects appear to be abandoned: * https://github.com/UptakeOpenSource/uptasticsearch[uptasticsearch]: An R client tailored to data science workflows. -The following projects appear to be abandoned: - -* https://github.com/Tomesch/elasticsearch[elasticsearch] - R client for Elasticsearch - [[ruby]] == Ruby Also see the {client}/ruby-api/current/index.html[official Elasticsearch Ruby client]. -* https://github.com/PoseBiz/stretcher[stretcher]: - Ruby client. - * https://github.com/printercu/elastics-rb[elastics]: Tiny client with built-in zero-downtime migrations and ActiveRecord integration. @@ -243,14 +175,6 @@ Also see the {client}/ruby-api/current/index.html[official Elasticsearch Ruby cl * https://github.com/artsy/estella[Estella]: Make your Ruby models searchable -The following projects appear to be abandoned: - -* https://github.com/wireframe/elastic_searchable/[elastic_searchable]: - Ruby client + Rails integration. - -* https://github.com/ddnexus/flex[Flex]: - Ruby Client. - [[rust]] == Rust @@ -275,15 +199,6 @@ The following projects appear to be abandoned: * https://github.com/SumoLogic/elasticsearch-client[elasticsearch-client]: Scala DSL that uses the REST API. Akka and AWS helpers included. -The following projects appear to be abandoned: - -* https://github.com/scalastuff/esclient[esclient]: - Thin Scala client. - -* https://github.com/bsadeh/scalastic[scalastic]: - Scala client. - - [[smalltalk]] == Smalltalk @@ -293,9 +208,8 @@ The following projects appear to be abandoned: * http://ss3.gemstone.com/ss/Elasticsearch.html[Elasticsearch] - Smalltalk client for Elasticsearch - [[vertx]] == Vert.x -* https://github.com/goodow/realtime-search[realtime-search]: - Elasticsearch module for Vert.x +* https://github.com/reactiverse/elasticsearch-client[elasticsearch-client]: + An Elasticsearch client for Eclipse Vert.x diff --git a/docs/plugins/integrations.asciidoc b/docs/plugins/integrations.asciidoc index a107527cc8acf..5bb4c3260ac9d 100644 --- a/docs/plugins/integrations.asciidoc +++ b/docs/plugins/integrations.asciidoc @@ -72,6 +72,9 @@ releases 2.0 and later do not support rivers. * https://github.com/dadoonet/fscrawler[FS Crawler]: The File System (FS) crawler allows to index documents (PDF, Open Office...) from your local file system and over SSH. (by David Pilato) +* https://github.com/senacor/elasticsearch-evolution[Elasticsearch Evolution]: + A library to migrate elasticsearch mappings. + [float] [[deployment]] === Deployment @@ -132,6 +135,27 @@ releases 2.0 and later do not support rivers. * https://github.com/twitter/storehaus[Twitter Storehaus]: Thin asynchronous Scala client for Storehaus. +* https://zeebe.io[Zeebe]: + An Elasticsearch exporter acts as a bridge between Zeebe and Elasticsearch + +* https://pulsar.apache.org/docs/en/io-elasticsearch[Apache Pulsar]: + The Elasticsearch Sink Connector is used to pull messages from Pulsar topics + and persist the messages to a index. + +* https://micronaut-projects.github.io/micronaut-elasticsearch/latest/guide/index.html[Micronaut Elasticsearch Integration]: + Integration of Micronaut with Elasticsearch + +* https://docs.streampipes.org/docs/user-guide-introduction[StreamPipes]: + StreamPipes is a framework that enables users to work with data streams allowing to store data in Elasticsearch. + +* https://metamodel.apache.org/[Apache MetaModel]: + Providing a common interface for discovery, exploration of metadata and querying of different types of data sources. + +* https://jooby.org/doc/elasticsearch/[Jooby Framework]: + Scalable, fast and modular micro web framework for Java. + +* https://micrometer.io[Micrometer]: + Vendor-neutral application metrics facade. Think SLF4j, but for metrics. [float] [[hadoop-integrations]] @@ -144,6 +168,12 @@ releases 2.0 and later do not support rivers. search and analytics natively integrated with Hadoop. Supports Map/Reduce, Cascading, Apache Hive, Apache Pig, Apache Spark and Apache Storm. +[float] +==== Supported by the community: + +* https://github.com/criteo/garmadon[Garmadon]: + Garmadon is a solution for Hadoop Cluster realtime introspection. + [float] [[monitoring-integrations]] From de222731f8ec17270a8139c2a18be0f7c02c24a7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 26 Apr 2019 09:48:11 +0200 Subject: [PATCH 199/260] Remove search analyzers from DocumentFieldMappers (#41484) These references seem to be unused except for tests and should be removed to keep the places we store analyzers limited. --- .../index/mapper/DocumentFieldMappers.java | 17 +---------------- .../index/mapper/DocumentFieldMapperTests.java | 4 ---- 2 files changed, 1 insertion(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java index f70c003846495..664ea467d9e5f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java @@ -34,8 +34,6 @@ public final class DocumentFieldMappers implements Iterable { private final Map fieldMappers; private final FieldNameAnalyzer indexAnalyzer; - private final FieldNameAnalyzer searchAnalyzer; - private final FieldNameAnalyzer searchQuoteAnalyzer; private static void put(Map analyzers, String key, Analyzer value, Analyzer defaultValue) { if (value == null) { @@ -67,8 +65,6 @@ public DocumentFieldMappers(Collection mappers, this.fieldMappers = Collections.unmodifiableMap(fieldMappers); this.indexAnalyzer = new FieldNameAnalyzer(indexAnalyzers); - this.searchAnalyzer = new FieldNameAnalyzer(searchAnalyzers); - this.searchQuoteAnalyzer = new FieldNameAnalyzer(searchQuoteAnalyzers); } /** @@ -89,18 +85,7 @@ public Analyzer indexAnalyzer() { return this.indexAnalyzer; } - /** - * A smart analyzer used for searching that takes into account specific analyzers configured - * per {@link FieldMapper}. - */ - public Analyzer searchAnalyzer() { - return this.searchAnalyzer; - } - - public Analyzer searchQuoteAnalyzer() { - return this.searchQuoteAnalyzer; - } - + @Override public Iterator iterator() { return fieldMappers.values().iterator(); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java index 4373f2210a7c7..8327b7d4f2e13 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java @@ -147,12 +147,8 @@ public void testAnalyzers() throws IOException { defaultSearchQuote); assertAnalyzes(documentFieldMappers.indexAnalyzer(), "field1", "index"); - assertAnalyzes(documentFieldMappers.searchAnalyzer(), "field1", "search"); - assertAnalyzes(documentFieldMappers.searchQuoteAnalyzer(), "field1", "search_quote"); assertAnalyzes(documentFieldMappers.indexAnalyzer(), "field2", "default_index"); - assertAnalyzes(documentFieldMappers.searchAnalyzer(), "field2", "default_search"); - assertAnalyzes(documentFieldMappers.searchQuoteAnalyzer(), "field2", "default_search_quote"); } private void assertAnalyzes(Analyzer analyzer, String field, String output) throws IOException { From 501c2a7ec4293d033c4d477ee71ac99419396dad Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 26 Apr 2019 10:18:48 +0200 Subject: [PATCH 200/260] Fix search_as_you_type's sub-fields to pick their names from the full path of the root field (#41541) The subfields of the search_as_you_type are prefixed with the name of their root field. However they should used the full path of the root field rather than just the name since these fields can appear in a multi-`fields` definition or under an object field. Since this field type is not released yet, this should be considered as a non-issue. --- .../mapper/SearchAsYouTypeFieldMapper.java | 7 +-- .../SearchAsYouTypeFieldMapperTests.java | 54 +++++++++++++++++++ 2 files changed, 58 insertions(+), 3 deletions(-) diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapper.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapper.java index 867e975e9f51c..9a5c389dd8e79 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapper.java @@ -160,8 +160,9 @@ public SearchAsYouTypeFieldMapper build(Mapper.BuilderContext context) { final NamedAnalyzer searchQuoteAnalyzer = fieldType().searchQuoteAnalyzer(); // set up the prefix field - final String prefixFieldName = name() + PREFIX_FIELD_SUFFIX; - final PrefixFieldType prefixFieldType = new PrefixFieldType(name(), prefixFieldName, Defaults.MIN_GRAM, Defaults.MAX_GRAM); + final String fullName = buildFullName(context); + final String prefixFieldName = fullName + PREFIX_FIELD_SUFFIX; + final PrefixFieldType prefixFieldType = new PrefixFieldType(fullName, prefixFieldName, Defaults.MIN_GRAM, Defaults.MAX_GRAM); prefixFieldType.setIndexOptions(fieldType().indexOptions()); // wrap the root field's index analyzer with shingles and edge ngrams final SearchAsYouTypeAnalyzer prefixIndexWrapper = @@ -180,7 +181,7 @@ public SearchAsYouTypeFieldMapper build(Mapper.BuilderContext context) { for (int i = 0; i < shingleFieldMappers.length; i++) { final int shingleSize = i + 2; final ShingleFieldType shingleFieldType = new ShingleFieldType(fieldType(), shingleSize); - shingleFieldType.setName(getShingleFieldName(name(), shingleSize)); + shingleFieldType.setName(getShingleFieldName(buildFullName(context), shingleSize)); // wrap the root field's index, search, and search quote analyzers with shingles final SearchAsYouTypeAnalyzer shingleIndexWrapper = SearchAsYouTypeAnalyzer.withShingle(indexAnalyzer.analyzer(), shingleSize); diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapperTests.java index 4622b34ea1514..40bd6d436c148 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/SearchAsYouTypeFieldMapperTests.java @@ -235,6 +235,60 @@ public void testSimpleMerge() throws IOException { } } + public void testMultiFields() throws IOException { + for (int shingleSize = 2; shingleSize < 4; shingleSize++) { + final XContentBuilder mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("a_field") + .field("type", "text") + .startObject("fields") + .startObject("suggest") + .field("type", "search_as_you_type") + .field("max_shingle_size", shingleSize) + .endObject() + .endObject() + .endObject() + .endObject() + .endObject(); + + final String index = "foo_" + shingleSize; + final String path = "a_field.suggest"; + List fields = new ArrayList<>(); + fields.add(path); + final MapperService mapperService = + createIndex(index, Settings.EMPTY, "_doc", mapping).mapperService(); + FieldType fieldType = mapperService.fullName(path + "._index_prefix"); + assertThat(fieldType, instanceOf(PrefixFieldType.class)); + PrefixFieldType prefixFieldType = (PrefixFieldType) fieldType; + assertEquals(path, prefixFieldType.parentField); + for (int i = 2; i < shingleSize; i++) { + String name = path + "._" + i + "gram"; + fields.add(name); + fieldType = mapperService.fullName(name); + assertThat(fieldType, instanceOf(ShingleFieldType.class)); + ShingleFieldType ft = (ShingleFieldType) fieldType; + assertEquals(i, ft.shingleSize); + assertTrue(prefixFieldType == ft.prefixFieldType); + } + + ParsedDocument doc = mapperService.documentMapper() + .parse(new SourceToParse("test", "_doc", "1", + BytesReference.bytes( + XContentFactory.jsonBuilder() + .startObject() + .field("a_field", "new york city") + .endObject() + ), XContentType.JSON) + ); + for (String field : fields) { + IndexableField[] indexFields = doc.rootDoc().getFields(field); + assertEquals(1, indexFields.length); + assertEquals("new york city", indexFields[0].stringValue()); + } + } + } + public void testIndexOptions() throws IOException { final String mapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() From 35d3c8f31a48a2f57cac104e5b777de64e67e3da Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 26 Apr 2019 11:29:22 +0200 Subject: [PATCH 201/260] Reenable bwc Tests in master (#41540) --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 932086423aa9b..583569ef7cd45 100644 --- a/build.gradle +++ b/build.gradle @@ -162,8 +162,8 @@ task verifyVersions { * after the backport of the backcompat code is complete. */ -boolean bwc_tests_enabled = false -final String bwc_tests_disabled_issue = "https://github.com/elastic/elasticsearch/pull/41426" /* place a PR link here when committing bwc changes */ +boolean bwc_tests_enabled = true +final String bwc_tests_disabled_issue = "" /* place a PR link here when committing bwc changes */ if (bwc_tests_enabled == false) { if (bwc_tests_disabled_issue.isEmpty()) { throw new GradleException("bwc_tests_disabled_issue must be set when bwc_tests_enabled == false") From d4636a18cb0b36b224becdf4077eed1b8442d7fd Mon Sep 17 00:00:00 2001 From: David Kyle Date: Fri, 26 Apr 2019 12:01:21 +0100 Subject: [PATCH 202/260] [Ml-Dataframe] Update URLs in Data frame client java doc (#41539) --- .../elasticsearch/client/DataFrameClient.java | 43 ++++++++++++------- .../client/RestHighLevelClient.java | 3 +- 2 files changed, 30 insertions(+), 16 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java index b758968f0a98a..8d7d3bc2ed497 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java @@ -49,7 +49,8 @@ public final class DataFrameClient { * Creates a new Data Frame Transform *

    * For additional info - * see Data Frame PUT transform documentation + * see + * Create data frame transform documentation * * @param request The PutDataFrameTransformRequest containing the * {@link org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig}. @@ -69,7 +70,8 @@ public AcknowledgedResponse putDataFrameTransform(PutDataFrameTransformRequest r * Creates a new Data Frame Transform asynchronously and notifies listener on completion *

    * For additional info - * see Data Frame PUT transform documentation + * see + * Create data frame transform documentation * * @param request The PutDataFrameTransformRequest containing the * {@link org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig}. @@ -90,7 +92,8 @@ public void putDataFrameTransformAsync(PutDataFrameTransformRequest request, Req * Get the running statistics of a Data Frame Transform *

    * For additional info - * see Get Data Frame transform stats documentation + * see + * Get data frame transform stats documentation * * @param request Specifies the which transforms to get the stats for * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -110,7 +113,8 @@ public GetDataFrameTransformStatsResponse getDataFrameTransformStats(GetDataFram * Get the running statistics of a Data Frame Transform asynchronously and notifies listener on completion *

    * For additional info - * see Get Data Frame transform stats documentation + * see + * Get data frame transform stats documentation * * @param request Specifies the which transforms to get the stats for * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -130,7 +134,8 @@ public void getDataFrameTransformStatsAsync(GetDataFrameTransformStatsRequest re * Delete a data frame transform *

    * For additional info - * see Data Frame delete transform documentation + * see + * Delete data frame transform documentation * * @param request The delete data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -150,7 +155,8 @@ public AcknowledgedResponse deleteDataFrameTransform(DeleteDataFrameTransformReq * Delete a data frame transform asynchronously and notifies listener on completion *

    * For additional info - * see Data Frame delete transform documentation + * see + * Delete data frame transform documentation * * @param request The delete data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -170,7 +176,8 @@ public void deleteDataFrameTransformAsync(DeleteDataFrameTransformRequest reques * Preview the result of a data frame transform *

    * For additional info - * see Preview Data Frame transform documentation + * see + * Preview data frame transform documentation * * @param request The preview data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -189,8 +196,8 @@ public PreviewDataFrameTransformResponse previewDataFrameTransform(PreviewDataFr /** * Preview the result of a data frame transform asynchronously and notifies listener on completion *

    - * For additional info - * see Preview Data Frame transform documentation + * see + * Preview data frame transform documentation * * @param request The preview data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -210,7 +217,8 @@ public void previewDataFrameTransformAsync(PreviewDataFrameTransformRequest requ * Start a data frame transform *

    * For additional info - * see Start Data Frame transform documentation + * see + * Start data frame transform documentation * * @param request The start data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -230,7 +238,8 @@ public StartDataFrameTransformResponse startDataFrameTransform(StartDataFrameTra * Start a data frame transform asynchronously and notifies listener on completion *

    * For additional info - * see Start Data Frame transform documentation + * see + * Start data frame transform documentation * * @param request The start data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -250,7 +259,8 @@ public void startDataFrameTransformAsync(StartDataFrameTransformRequest request, * Stop a data frame transform *

    * For additional info - * see Stop Data Frame transform documentation + * see + * Stop data frame transform documentation * * @param request The stop data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -270,7 +280,8 @@ public StopDataFrameTransformResponse stopDataFrameTransform(StopDataFrameTransf * Stop a data frame transform asynchronously and notifies listener on completion *

    * For additional info - * see Stop Data Frame transform documentation + * see + * Stop data frame transform documentation * * @param request The stop data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -290,7 +301,8 @@ public void stopDataFrameTransformAsync(StopDataFrameTransformRequest request, R * Get one or more data frame transform configurations *

    * For additional info - * see Get Data Frame transform documentation + * see + * Get data frame transform documentation * * @param request The get data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized @@ -310,7 +322,8 @@ public GetDataFrameTransformResponse getDataFrameTransform(GetDataFrameTransform * Get one or more data frame transform configurations asynchronously and notifies listener on completion *

    * For additional info - * see Get Data Frame transform documentation + * see + * Get data frame transform documentation * * @param request The get data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java index 5727967b5d4e4..35abf5e85180f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java @@ -472,7 +472,8 @@ public SecurityClient security() { * are shipped with the Elastic Stack distribution of Elasticsearch. All of * these APIs will 404 if run against the OSS distribution of Elasticsearch. *

    - * See the Data Frame APIs on elastic.co for more information. + * See the + * Data Frame APIs on elastic.co for more information. * * @return the client wrapper for making Data Frame API calls */ From 29701b0229782c4d11e8e7f8b441c5f73b3b022c Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Fri, 26 Apr 2019 08:50:27 -0400 Subject: [PATCH 203/260] Remove experimental label froms script_score query (#41572) --- docs/reference/query-dsl/script-score-query.asciidoc | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/reference/query-dsl/script-score-query.asciidoc b/docs/reference/query-dsl/script-score-query.asciidoc index edab8a5fc45f5..e6418d408d7f2 100644 --- a/docs/reference/query-dsl/script-score-query.asciidoc +++ b/docs/reference/query-dsl/script-score-query.asciidoc @@ -1,8 +1,6 @@ [[query-dsl-script-score-query]] === Script Score Query -experimental[] - The `script_score` allows you to modify the score of documents that are retrieved by a query. This can be useful if, for example, a score function is computationally expensive and it is sufficient to compute From df777fee214d0dda1bbc585806ff8a0012ccf5b3 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 26 Apr 2019 08:19:51 -0500 Subject: [PATCH 204/260] [ML] data frame, verify primary shards are active for configs index before task start (#41551) --- ...FrameTransformPersistentTasksExecutor.java | 35 ++++++++ ...TransformPersistentTasksExecutorTests.java | 88 +++++++++++++++++++ 2 files changed, 123 insertions(+) create mode 100644 x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutorTests.java 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 708585a8dc309..01999eff64c61 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 @@ -11,7 +11,11 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.common.Nullable; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.persistent.PersistentTaskState; @@ -32,9 +36,12 @@ import org.elasticsearch.xpack.dataframe.DataFrame; import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService; 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.transforms.pivot.SchemaUtil; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -67,6 +74,34 @@ public DataFrameTransformPersistentTasksExecutor(Client client, this.threadPool = threadPool; } + @Override + public PersistentTasksCustomMetaData.Assignment getAssignment(DataFrameTransform params, ClusterState clusterState) { + List unavailableIndices = verifyIndicesPrimaryShardsAreActive(clusterState); + if (unavailableIndices.size() != 0) { + String reason = "Not starting data frame transform [" + params.getId() + "], " + + "because not all primary shards are active for the following indices [" + + String.join(",", unavailableIndices) + "]"; + logger.debug(reason); + return new PersistentTasksCustomMetaData.Assignment(null, reason); + } + return super.getAssignment(params, clusterState); + } + + static List verifyIndicesPrimaryShardsAreActive(ClusterState clusterState) { + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + String[] indices = resolver.concreteIndexNames(clusterState, + IndicesOptions.lenientExpandOpen(), + DataFrameInternalIndex.INDEX_TEMPLATE_PATTERN + "*"); + List unavailableIndices = new ArrayList<>(indices.length); + for (String index : indices) { + IndexRoutingTable routingTable = clusterState.getRoutingTable().index(index); + if (routingTable == null || routingTable.allPrimaryShardsActive() == false) { + unavailableIndices.add(index); + } + } + return unavailableIndices; + } + @Override protected void nodeOperation(AllocatedPersistentTask task, @Nullable DataFrameTransform params, PersistentTaskState state) { final String transformId = params.getId(); diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutorTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutorTests.java new file mode 100644 index 0000000000000..9e29fef1b5a03 --- /dev/null +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/DataFrameTransformPersistentTasksExecutorTests.java @@ -0,0 +1,88 @@ +/* + * 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.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex; + +import java.util.ArrayList; +import java.util.List; + +public class DataFrameTransformPersistentTasksExecutorTests extends ESTestCase { + + public void testVerifyIndicesPrimaryShardsAreActive() { + MetaData.Builder metaData = MetaData.builder(); + RoutingTable.Builder routingTable = RoutingTable.builder(); + addIndices(metaData, routingTable); + + ClusterState.Builder csBuilder = ClusterState.builder(new ClusterName("_name")); + csBuilder.routingTable(routingTable.build()); + csBuilder.metaData(metaData); + + ClusterState cs = csBuilder.build(); + assertEquals(0, DataFrameTransformPersistentTasksExecutor.verifyIndicesPrimaryShardsAreActive(cs).size()); + + metaData = new MetaData.Builder(cs.metaData()); + routingTable = new RoutingTable.Builder(cs.routingTable()); + String indexToRemove = DataFrameInternalIndex.INDEX_NAME; + if (randomBoolean()) { + routingTable.remove(indexToRemove); + } else { + Index index = new Index(indexToRemove, "_uuid"); + ShardId shardId = new ShardId(index, 0); + ShardRouting shardRouting = ShardRouting.newUnassigned(shardId, true, RecoverySource.EmptyStoreRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "")); + shardRouting = shardRouting.initialize("node_id", null, 0L); + routingTable.add(IndexRoutingTable.builder(index) + .addIndexShard(new IndexShardRoutingTable.Builder(shardId).addShard(shardRouting).build())); + } + + csBuilder.routingTable(routingTable.build()); + csBuilder.metaData(metaData); + List result = DataFrameTransformPersistentTasksExecutor.verifyIndicesPrimaryShardsAreActive(csBuilder.build()); + assertEquals(1, result.size()); + assertEquals(indexToRemove, result.get(0)); + } + + private void addIndices(MetaData.Builder metaData, RoutingTable.Builder routingTable) { + List indices = new ArrayList<>(); + indices.add(DataFrameInternalIndex.AUDIT_INDEX); + indices.add(DataFrameInternalIndex.INDEX_NAME); + for (String indexName : indices) { + IndexMetaData.Builder indexMetaData = IndexMetaData.builder(indexName); + indexMetaData.settings(Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + ); + metaData.put(indexMetaData); + Index index = new Index(indexName, "_uuid"); + ShardId shardId = new ShardId(index, 0); + ShardRouting shardRouting = ShardRouting.newUnassigned(shardId, true, RecoverySource.EmptyStoreRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "")); + shardRouting = shardRouting.initialize("node_id", null, 0L); + shardRouting = shardRouting.moveToStarted(); + routingTable.add(IndexRoutingTable.builder(index) + .addIndexShard(new IndexShardRoutingTable.Builder(shardId).addShard(shardRouting).build())); + } + } + +} From d922d4d44a49d5868b3146748834a6666fd292fd Mon Sep 17 00:00:00 2001 From: Dan Hermann Date: Fri, 26 Apr 2019 08:54:49 -0500 Subject: [PATCH 205/260] Restricts naming for repositories (#41008) Applies the same naming restrictions to repositories as to snapshots except that leading underscores and uppercase characters are permitted. Fixes #40817. --- .../repositories/RepositoriesService.java | 16 ++++++++++++++++ .../repositories/RepositoriesServiceTests.java | 15 +++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index bb20b2085d456..88f1051ca2769 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -35,6 +35,7 @@ import org.elasticsearch.cluster.metadata.RepositoryMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -94,6 +95,7 @@ public RepositoriesService(Settings settings, ClusterService clusterService, Tra */ public void registerRepository(final PutRepositoryRequest request, final ActionListener listener) { final RepositoryMetaData newRepositoryMetaData = new RepositoryMetaData(request.name(), request.type(), request.settings()); + validate(request.name()); final ActionListener registrationListener; if (request.verify()) { @@ -418,6 +420,20 @@ private Repository createRepository(RepositoryMetaData repositoryMetaData, Map repositoriesService.registerRepository(request, null)); + } + private static class TestRepository implements Repository { private static final String TYPE = "internal"; From 0225af44a03dba8c7416e95f778f887c38f7d95d Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 26 Apr 2019 10:23:30 -0400 Subject: [PATCH 206/260] [DOCS] Clarify Recovery Settings for Shard Relocation (#40329) * Clarify that peer recovery settings apply to shard relocation * Fix awkward wording of 1st sentence * [DOCS] Remove snapshot recovery reference. Call out link to [[cat-recovery]]. Separate expert settings. --- .../modules/indices/recovery.asciidoc | 67 ++++++++++--------- 1 file changed, 37 insertions(+), 30 deletions(-) diff --git a/docs/reference/modules/indices/recovery.asciidoc b/docs/reference/modules/indices/recovery.asciidoc index d9e85c27105c8..613d0885d1db7 100644 --- a/docs/reference/modules/indices/recovery.asciidoc +++ b/docs/reference/modules/indices/recovery.asciidoc @@ -1,33 +1,40 @@ [[recovery]] === Indices Recovery -<> is the process used to build a new copy of a -shard on a node by copying data from the primary. {es} uses this peer recovery -process to rebuild shard copies that were lost if a node has failed, and uses -the same process when migrating a shard copy between nodes to rebalance the -cluster or to honor any changes to the <>. - -The following _expert_ setting can be set to manage the resources consumed by -peer recoveries: - -`indices.recovery.max_bytes_per_sec`:: - Limits the total inbound and outbound peer recovery traffic on each node. - Since this limit applies on each node, but there may be many nodes - performing peer recoveries concurrently, the total amount of peer recovery - traffic within a cluster may be much higher than this limit. If you set - this limit too high then there is a risk that ongoing peer recoveries will - consume an excess of bandwidth (or other resources) which could destabilize - the cluster. Defaults to `40mb`. - -`indices.recovery.max_concurrent_file_chunks`:: - Controls the number of file chunk requests that can be sent in parallel per recovery. - As multiple recoveries are already running in parallel (controlled by - cluster.routing.allocation.node_concurrent_recoveries), increasing this expert-level - setting might only help in situations where peer recovery of a single shard is not - reaching the total inbound and outbound peer recovery traffic as configured by - indices.recovery.max_bytes_per_sec, but is CPU-bound instead, typically when using - transport-level security or compression. Defaults to `2`. - -This setting can be dynamically updated on a live cluster with the -<> API. +Peer recovery syncs data from a primary shard to a new or +existing shard copy. + +Peer recovery automatically occurs when {es}: + +* Recreates a shard lost during node failure +* Relocates a shard to another node due to a cluster rebalance or changes to the +<> + +You can view a list of in-progress and completed recoveries using the +<>. + +[float] +==== Peer recovery settings + +`indices.recovery.max_bytes_per_sec` (<>):: +Limits total inbound and outbound recovery traffic for each node. +Defaults to `40mb`. ++ +This limit applies to nodes only. If multiple nodes in a cluster perform +recoveries at the same time, the cluster's total recovery traffic may exceed +this limit. ++ +If this limit is too high, ongoing recoveries may consume an excess +of bandwidth and other resources, which can destabilize the cluster. + +[float] +==== Expert peer recovery settings +You can use the following _expert_ setting to manage resources for peer +recoveries. + +`indices.recovery.max_concurrent_file_chunks` (<>, Expert):: +Number of file chunk requests sent in parallel for each recovery. Defaults to +`2`. ++ +You can increase the value of this setting when the recovery of a single shard +is not reaching the traffic limit set by `indices.recovery.max_bytes_per_sec`. \ No newline at end of file From 3816a1be1de6bcd26879fba667e202626aa933be Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 26 Apr 2019 11:20:34 -0400 Subject: [PATCH 207/260] [DOCS] Fix ID for Asciidoctor migration (#41577) --- docs/reference/setup/stopping.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/setup/stopping.asciidoc b/docs/reference/setup/stopping.asciidoc index 4f632ec06f1dd..c9f718aa088c3 100644 --- a/docs/reference/setup/stopping.asciidoc +++ b/docs/reference/setup/stopping.asciidoc @@ -36,8 +36,8 @@ $ cat /tmp/elasticsearch-pid && echo $ kill -SIGTERM 15516 -------------------------------------------------- -[[fatal-errors] [float] +[[fatal-errors]] === Stopping on Fatal Errors During the life of the Elasticsearch virtual machine, certain fatal errors could arise that put the From 14af0b4ba810f1d938ea3bb8c5da596763f03ae4 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 26 Apr 2019 18:34:03 +0300 Subject: [PATCH 208/260] Testclsuters: convert plugins qa projects (#41496) Add testclusters support for files in keystore and convert qa subprojects within plugins. --- .../gradle/plugin/PluginBuildPlugin.groovy | 1 + .../gradle/test/RestIntegTestTask.groovy | 1 - .../elasticsearch/gradle/FileSupplier.java | 7 ++++ .../testclusters/ElasticsearchCluster.java | 11 ++++++ .../testclusters/ElasticsearchNode.java | 38 ++++++++++++++++--- .../TestClusterConfiguration.java | 5 +++ plugins/build.gradle | 5 ++- .../discovery-ec2/qa/amazon-ec2/build.gradle | 37 +++++++----------- plugins/discovery-gce/qa/gce/build.gradle | 36 ++++++------------ .../qa/microsoft-azure-storage/build.gradle | 19 +++++----- .../qa/google-cloud-storage/build.gradle | 20 +++++----- x-pack/qa/security-migrate-tests/build.gradle | 1 + 12 files changed, 107 insertions(+), 74 deletions(-) create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/FileSupplier.java diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy index 9d303fcb78dc0..58ef5b6d323d4 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy @@ -89,6 +89,7 @@ class PluginBuildPlugin extends BuildPlugin { project.extensions.getByType(PluginPropertiesExtension).extendedPlugins.each { pluginName -> // Auto add dependent modules to the test cluster if (project.findProject(":modules:${pluginName}") != null) { + project.integTest.dependsOn(project.project(":modules:${pluginName}").tasks.bundlePlugin) project.testClusters.integTest.module( project.file(project.project(":modules:${pluginName}").tasks.bundlePlugin.archiveFile) ) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy index df26a5a07cb21..eec46f9a522a7 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/RestIntegTestTask.groovy @@ -120,7 +120,6 @@ class RestIntegTestTask extends DefaultTask { if (usesTestclusters == true) { ElasticsearchCluster cluster = project.testClusters."${name}" nonInputProperties.systemProperty('tests.rest.cluster', "${-> cluster.allHttpSocketURI.join(",") }") - nonInputProperties.systemProperty('tests.config.dir', "${-> cluster.singleNode().getConfigDir() }") nonInputProperties.systemProperty('tests.cluster', "${-> cluster.transportPortURI }") } else { // we pass all nodes to the rest cluster to allow the clients to round-robin between them diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/FileSupplier.java b/buildSrc/src/main/java/org/elasticsearch/gradle/FileSupplier.java new file mode 100644 index 0000000000000..d28afe0c41dd0 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/FileSupplier.java @@ -0,0 +1,7 @@ +package org.elasticsearch.gradle; + +import java.io.File; +import java.util.function.Supplier; + +public interface FileSupplier extends Supplier { +} diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java index 3bc750248ae37..48569ecd8b21d 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchCluster.java @@ -20,6 +20,7 @@ import org.elasticsearch.GradleServicesAdapter; import org.elasticsearch.gradle.Distribution; +import org.elasticsearch.gradle.FileSupplier; import org.elasticsearch.gradle.Version; import org.gradle.api.NamedDomainObjectContainer; import org.gradle.api.Project; @@ -143,6 +144,16 @@ public void keystore(String key, Supplier valueSupplier) { nodes.all(each -> each.keystore(key, valueSupplier)); } + @Override + public void keystore(String key, File value) { + nodes.all(each -> each.keystore(key, value)); + } + + @Override + public void keystore(String key, FileSupplier valueSupplier) { + nodes.all(each -> each.keystore(key, valueSupplier)); + } + @Override public void setting(String key, String value) { nodes.all(each -> each.setting(key, value)); 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 5a3e0d599f173..b7ba4377a1a8c 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java @@ -20,6 +20,7 @@ import org.elasticsearch.GradleServicesAdapter; import org.elasticsearch.gradle.Distribution; +import org.elasticsearch.gradle.FileSupplier; import org.elasticsearch.gradle.OS; import org.elasticsearch.gradle.Version; import org.gradle.api.logging.Logger; @@ -63,7 +64,8 @@ public class ElasticsearchNode implements TestClusterConfiguration { private static final int NODE_UP_TIMEOUT = 60; private static final TimeUnit NODE_UP_TIMEOUT_UNIT = TimeUnit.SECONDS; private static final List OVERRIDABLE_SETTINGS = Arrays.asList( - "path.repo" + "path.repo", + "discovery.seed_providers" ); private final String path; @@ -79,6 +81,7 @@ public class ElasticsearchNode implements TestClusterConfiguration { private final List modules = new ArrayList<>(); private final Map> settings = new LinkedHashMap<>(); private final Map> keystoreSettings = new LinkedHashMap<>(); + private final Map keystoreFiles = new LinkedHashMap<>(); private final Map> systemProperties = new LinkedHashMap<>(); private final Map> environment = new LinkedHashMap<>(); private final Map extraConfigFiles = new HashMap<>(); @@ -171,6 +174,19 @@ public void keystore(String key, Supplier valueSupplier) { addSupplier("Keystore", keystoreSettings, key, valueSupplier); } + @Override + public void keystore(String key, File value) { + requireNonNull(value, "keystore value was null when configuring test cluster`" + this + "`"); + keystore(key, () -> value); + } + + @Override + public void keystore(String key, FileSupplier valueSupplier) { + requireNonNull(key, "Keystore" + " key was null when configuring test cluster `" + this + "`"); + requireNonNull(valueSupplier, "Keystore" + " value supplier was null when configuring test cluster `" + this + "`"); + keystoreFiles.put(key, valueSupplier); + } + @Override public void setting(String key, String value) { addSupplier("Settings", settings, key, value); @@ -281,12 +297,22 @@ public synchronized void start() { "install", "--batch", plugin.toString()) ); - if (keystoreSettings.isEmpty() == false) { - checkSuppliers("Keystore", keystoreSettings); + if (keystoreSettings.isEmpty() == false || keystoreFiles.isEmpty() == false) { runElaticsearchBinScript("elasticsearch-keystore", "create"); - keystoreSettings.forEach((key, value) -> { - runElaticsearchBinScriptWithInput(value.get().toString(), "elasticsearch-keystore", "add", "-x", key); - }); + + checkSuppliers("Keystore", keystoreSettings); + keystoreSettings.forEach((key, value) -> + runElaticsearchBinScriptWithInput(value.get().toString(), "elasticsearch-keystore", "add", "-x", key) + ); + + for (Map.Entry entry : keystoreFiles.entrySet()) { + File file = entry.getValue().get(); + requireNonNull(file, "supplied keystoreFile was null when configuring " + this); + if (file.exists() == false) { + throw new TestClustersException("supplied keystore file " + file + " does not exist, require for " + this); + } + runElaticsearchBinScript("elasticsearch-keystore", "add-file", entry.getKey(), file.getAbsolutePath()); + } } installModules(); diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java index 4f7abf3d68a8a..39f9683ff4863 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testclusters/TestClusterConfiguration.java @@ -19,6 +19,7 @@ package org.elasticsearch.gradle.testclusters; import org.elasticsearch.gradle.Distribution; +import org.elasticsearch.gradle.FileSupplier; import org.gradle.api.logging.Logging; import org.slf4j.Logger; @@ -47,6 +48,10 @@ public interface TestClusterConfiguration { void keystore(String key, Supplier valueSupplier); + void keystore(String key, File value); + + void keystore(String key, FileSupplier valueSupplier); + void setting(String key, String value); void setting(String key, Supplier valueSupplier); diff --git a/plugins/build.gradle b/plugins/build.gradle index 89a4fe7384c9c..2aee6cafffaf4 100644 --- a/plugins/build.gradle +++ b/plugins/build.gradle @@ -17,10 +17,13 @@ * under the License. */ +subprojects { + apply plugin: 'elasticsearch.testclusters' +} + // only configure immediate children of plugins dir configure(subprojects.findAll { it.parent.path == project.path }) { group = 'org.elasticsearch.plugin' - apply plugin: 'elasticsearch.testclusters' apply plugin: 'elasticsearch.esplugin' esplugin { diff --git a/plugins/discovery-ec2/qa/amazon-ec2/build.gradle b/plugins/discovery-ec2/qa/amazon-ec2/build.gradle index b5a6d1bb7d41c..209ab3278398d 100644 --- a/plugins/discovery-ec2/qa/amazon-ec2/build.gradle +++ b/plugins/discovery-ec2/qa/amazon-ec2/build.gradle @@ -29,14 +29,13 @@ dependencies { } final int ec2NumberOfNodes = 3 -File ec2DiscoveryFile = new File(project.buildDir, 'generated-resources/nodes.uri') /** A task to start the AmazonEC2Fixture which emulates an EC2 service **/ task ec2Fixture(type: AntFixture) { dependsOn compileTestJava env 'CLASSPATH', "${ -> project.sourceSets.test.runtimeClasspath.asPath }" executable = new File(project.runtimeJavaHome, 'bin/java') - args 'org.elasticsearch.discovery.ec2.AmazonEC2Fixture', baseDir, ec2DiscoveryFile.absolutePath + args 'org.elasticsearch.discovery.ec2.AmazonEC2Fixture', baseDir, "${buildDir}/testclusters/integTest-1/config/unicast_hosts.txt" } Map expansions = [ @@ -48,28 +47,20 @@ processTestResources { MavenFilteringHack.filter(it, expansions) } -integTestCluster { - dependsOn ec2Fixture - numNodes = ec2NumberOfNodes - plugin ':plugins:discovery-ec2' - keystoreSetting 'discovery.ec2.access_key', 'ec2_integration_test_access_key' - keystoreSetting 'discovery.ec2.secret_key', 'ec2_integration_test_secret_key' +integTest { + dependsOn ec2Fixture, project(':plugins:discovery-ec2').bundlePlugin +} + +testClusters.integTest { + numberOfNodes = ec2NumberOfNodes + plugin file(project(':plugins:discovery-ec2').bundlePlugin.archiveFile) + + keystore 'discovery.ec2.access_key', 'ec2_integration_test_access_key' + keystore 'discovery.ec2.secret_key', 'ec2_integration_test_secret_key' + setting 'discovery.seed_providers', 'ec2' setting 'network.host', '_ec2_' - setting 'discovery.ec2.endpoint', "http://${-> ec2Fixture.addressAndPort}" - systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", "http://${-> ec2Fixture.addressAndPort}" + setting 'discovery.ec2.endpoint', { "http://${ec2Fixture.addressAndPort}" } - unicastTransportUri = { seedNode, node, ant -> return null } - - waitCondition = { node, ant -> - ec2DiscoveryFile.parentFile.mkdirs() - ec2DiscoveryFile.setText(integTest.nodes.collect { n -> "${n.transportUri()}" }.join('\n'), 'UTF-8') - - File tmpFile = new File(node.cwd, 'wait.success') - ant.get(src: "http://${node.httpUri()}/", - dest: tmpFile.toString(), - ignoreerrors: true, - retries: 10) - return tmpFile.exists() - } + systemProperty "com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", { "http://${ec2Fixture.addressAndPort}" } } diff --git a/plugins/discovery-gce/qa/gce/build.gradle b/plugins/discovery-gce/qa/gce/build.gradle index c02aecc27ed53..16a65c762f6f7 100644 --- a/plugins/discovery-gce/qa/gce/build.gradle +++ b/plugins/discovery-gce/qa/gce/build.gradle @@ -24,7 +24,6 @@ apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' final int gceNumberOfNodes = 3 -File gceDiscoveryFile = new File(project.buildDir, 'generated-resources/nodes.uri') dependencies { testCompile project(path: ':plugins:discovery-gce', configuration: 'runtime') @@ -35,7 +34,7 @@ task gceFixture(type: AntFixture) { dependsOn compileTestJava env 'CLASSPATH', "${ -> project.sourceSets.test.runtimeClasspath.asPath }" executable = new File(project.runtimeJavaHome, 'bin/java') - args 'org.elasticsearch.cloud.gce.GCEFixture', baseDir, gceDiscoveryFile.getAbsolutePath() + args 'org.elasticsearch.cloud.gce.GCEFixture', baseDir, "${buildDir}/testclusters/integTest-1/config/unicast_hosts.txt" } Map expansions = [ @@ -47,34 +46,21 @@ processTestResources { MavenFilteringHack.filter(it, expansions) } -integTestCluster { - dependsOn gceFixture - numNodes = gceNumberOfNodes - plugin ':plugins:discovery-gce' - setting 'discovery.seed_providers', 'gce' +integTest { + dependsOn gceFixture, project(':plugins:discovery-gce').bundlePlugin +} +testClusters.integTest { + numberOfNodes = gceNumberOfNodes + plugin file(project(':plugins:discovery-gce').bundlePlugin.archiveFile) // use gce fixture for Auth calls instead of http://metadata.google.internal - integTestCluster.environment 'GCE_METADATA_HOST', "http://${-> gceFixture.addressAndPort}" - + environment 'GCE_METADATA_HOST', { "http://${gceFixture.addressAndPort}" } // allows to configure hidden settings (`cloud.gce.host` and `cloud.gce.root_url`) systemProperty 'es.allow_reroute_gce_settings', 'true' + setting 'discovery.seed_providers', 'gce' // use gce fixture for metadata server calls instead of http://metadata.google.internal - setting 'cloud.gce.host', "http://${-> gceFixture.addressAndPort}" + setting 'cloud.gce.host', { "http://${gceFixture.addressAndPort}" } // use gce fixture for API calls instead of https://www.googleapis.com - setting 'cloud.gce.root_url', "http://${-> gceFixture.addressAndPort}" - - unicastTransportUri = { seedNode, node, ant -> return null } - - waitCondition = { node, ant -> - gceDiscoveryFile.parentFile.mkdirs() - gceDiscoveryFile.setText(integTest.nodes.collect { n -> "${n.transportUri()}" }.join('\n'), 'UTF-8') - - File tmpFile = new File(node.cwd, 'wait.success') - ant.get(src: "http://${node.httpUri()}/", - dest: tmpFile.toString(), - ignoreerrors: true, - retries: 10) - return tmpFile.exists() - } + setting 'cloud.gce.root_url', { "http://${gceFixture.addressAndPort}" } } diff --git a/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle b/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle index d9658d4d2f9e2..8d8aaebac9f3f 100644 --- a/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle +++ b/plugins/repository-azure/qa/microsoft-azure-storage/build.gradle @@ -23,10 +23,6 @@ import org.elasticsearch.gradle.test.AntFixture apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' -integTestCluster { - plugin ':plugins:repository-azure' -} - boolean useFixture = false String azureAccount = System.getenv("azure_storage_account") @@ -60,16 +56,21 @@ processTestResources { MavenFilteringHack.filter(it, expansions) } -integTestCluster { - keystoreSetting 'azure.client.integration_test.account', azureAccount - keystoreSetting 'azure.client.integration_test.key', azureKey +integTest { + dependsOn project(':plugins:repository-azure').bundlePlugin +} + +testClusters.integTest { + plugin file(project(':plugins:repository-azure').bundlePlugin.archiveFile) + keystore 'azure.client.integration_test.account', azureAccount + keystore 'azure.client.integration_test.key', azureKey if (useFixture) { - dependsOn azureStorageFixture + tasks.integTest.dependsOn azureStorageFixture // Use a closure on the string to delay evaluation until tests are executed. The endpoint_suffix is used // in a hacky way to change the protocol and endpoint. We must fix that. setting 'azure.client.integration_test.endpoint_suffix', - "ignored;DefaultEndpointsProtocol=http;BlobEndpoint=http://${ -> azureStorageFixture.addressAndPort }" + { "ignored;DefaultEndpointsProtocol=http;BlobEndpoint=http://${azureStorageFixture.addressAndPort }" } } else { println "Using an external service to test the repository-azure plugin" } diff --git a/plugins/repository-gcs/qa/google-cloud-storage/build.gradle b/plugins/repository-gcs/qa/google-cloud-storage/build.gradle index 0a610123a6fcc..4f201a812f2b2 100644 --- a/plugins/repository-gcs/qa/google-cloud-storage/build.gradle +++ b/plugins/repository-gcs/qa/google-cloud-storage/build.gradle @@ -26,10 +26,6 @@ import java.security.KeyPairGenerator apply plugin: 'elasticsearch.standalone-rest-test' apply plugin: 'elasticsearch.rest-test' -integTestCluster { - plugin ':plugins:repository-gcs' -} - boolean useFixture = false String gcsServiceAccount = System.getenv("google_storage_service_account") @@ -87,14 +83,20 @@ processTestResources { MavenFilteringHack.filter(it, expansions) } -integTestCluster { - keystoreFile 'gcs.client.integration_test.credentials_file', "${serviceAccountFile.absolutePath}" +integTest { + dependsOn project(':plugins:repository-gcs').bundlePlugin +} + +testClusters.integTest { + plugin file(project(':plugins:repository-gcs').bundlePlugin.archiveFile) + + keystore 'gcs.client.integration_test.credentials_file', serviceAccountFile if (useFixture) { - dependsOn createServiceAccountFile, googleCloudStorageFixture + tasks.integTest.dependsOn createServiceAccountFile, googleCloudStorageFixture /* Use a closure on the string to delay evaluation until tests are executed */ - setting 'gcs.client.integration_test.endpoint', "http://${ -> googleCloudStorageFixture.addressAndPort }" - setting 'gcs.client.integration_test.token_uri', "http://${ -> googleCloudStorageFixture.addressAndPort }/o/oauth2/token" + setting 'gcs.client.integration_test.endpoint', { "http://${googleCloudStorageFixture.addressAndPort}" } + setting 'gcs.client.integration_test.token_uri', { "http://${googleCloudStorageFixture.addressAndPort}/o/oauth2/token" } } else { println "Using an external service to test the repository-gcs plugin" } diff --git a/x-pack/qa/security-migrate-tests/build.gradle b/x-pack/qa/security-migrate-tests/build.gradle index 88006a38bd5e7..1851f0e21b027 100644 --- a/x-pack/qa/security-migrate-tests/build.gradle +++ b/x-pack/qa/security-migrate-tests/build.gradle @@ -30,6 +30,7 @@ integTestCluster { retries: 10) return tmpFile.exists() } + // TODO: systemProperty('tests.cluster', "${-> cluster.transportPortURI }") when migerating to testclusters } testingConventions { From b7f57c2e62b3116c76b41571b2455e29eb25bd7a Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 26 Apr 2019 11:35:07 -0400 Subject: [PATCH 209/260] [DOCS] Escape commas in experimental[] for Asciidoctor migration (#41578) --- docs/reference/search/rank-eval.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/search/rank-eval.asciidoc b/docs/reference/search/rank-eval.asciidoc index cf23121f53b33..29176f15f8cd4 100644 --- a/docs/reference/search/rank-eval.asciidoc +++ b/docs/reference/search/rank-eval.asciidoc @@ -1,7 +1,7 @@ [[search-rank-eval]] == Ranking Evaluation API -experimental[The ranking evaluation API is experimental and may be changed or removed completely in a future release, as well as change in non-backwards compatible ways on minor versions updates. Elastic will take a best effort approach to fix any issues, but experimental features are not subject to the support SLA of official GA features.] +experimental["The ranking evaluation API is experimental and may be changed or removed completely in a future release, as well as change in non-backwards compatible ways on minor versions updates. Elastic will take a best effort approach to fix any issues, but experimental features are not subject to the support SLA of official GA features."] The ranking evaluation API allows to evaluate the quality of ranked search results over a set of typical search queries. Given this set of queries and a From 4ab9e88cf7654da994d3f3ecb791e8c3e1ead949 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 26 Apr 2019 11:41:19 -0400 Subject: [PATCH 210/260] [DOCS] Fix delimited blocks for Asciidoctor migration (#41583) --- docs/reference/upgrade/cluster_restart.asciidoc | 3 +++ docs/reference/upgrade/upgrade-node.asciidoc | 2 -- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/reference/upgrade/cluster_restart.asciidoc b/docs/reference/upgrade/cluster_restart.asciidoc index 20786857993a2..1865f005dccc8 100644 --- a/docs/reference/upgrade/cluster_restart.asciidoc +++ b/docs/reference/upgrade/cluster_restart.asciidoc @@ -46,6 +46,9 @@ include::remove-xpack.asciidoc[] + -- include::upgrade-node.asciidoc[] +-- ++ +-- include::set-paths-tip.asciidoc[] -- diff --git a/docs/reference/upgrade/upgrade-node.asciidoc b/docs/reference/upgrade/upgrade-node.asciidoc index 12a2e692662b8..c445c03a38abb 100644 --- a/docs/reference/upgrade/upgrade-node.asciidoc +++ b/docs/reference/upgrade/upgrade-node.asciidoc @@ -18,12 +18,10 @@ To upgrade using a zip or compressed tarball: data directory. If you are not using an external `data` directory, copy your old data directory over to the new installation. + + --- IMPORTANT: If you use {monitor-features}, re-use the data directory when you upgrade {es}. Monitoring identifies unique {es} nodes by using the persistent UUID, which is stored in the data directory. --- .. Set `path.logs` in `config/elasticsearch.yml` to point to the location where you want to store your logs. If you do not specify this setting, From 32be7009895981d70c900afc10cc7075f8fcfc80 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 26 Apr 2019 12:12:57 -0400 Subject: [PATCH 211/260] [DOCS] Add space to fix Asciidoctor output (#41579) --- x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec index 2a9804dfd5d56..c181023db03eb 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec @@ -607,7 +607,7 @@ M |57 groupByAndAggExpression // tag::groupByAndAggExpression schema::g:s|salary:i -SELECT gender AS g, ROUND((MIN(salary) / 100)) AS salary FROM emp GROUP BY gender; +SELECT gender AS g, ROUND( (MIN(salary) / 100) ) AS salary FROM emp GROUP BY gender; g | salary ---------------+--------------- From ce59d45fcaf63a61b89e1864698f1c3e960b01fd Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Fri, 26 Apr 2019 12:16:26 -0400 Subject: [PATCH 212/260] [DOCS] Fix important block for Asciidoctor migration (#41582) From f48ddd53fd66cfef3032a17023cd9e2f88702af9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 26 Apr 2019 12:32:48 -0400 Subject: [PATCH 213/260] Use Java 11 collections conveniences everywhere (#41399) This commit replaces all applicable uses with Java 11 collections convenience methods. --- .../ml/filestructurefinder/FileStructure.java | 11 +- .../security/GetPrivilegesResponse.java | 3 +- .../client/security/GetUsersResponse.java | 49 +- .../CompositeRoleMapperExpression.java | 4 +- .../fields/FieldRoleMapperExpression.java | 4 +- .../client/security/user/User.java | 21 +- .../privileges/AbstractIndicesPrivileges.java | 31 +- .../user/privileges/ApplicationPrivilege.java | 21 +- .../ApplicationResourcePrivileges.java | 23 +- .../user/privileges/GlobalPrivileges.java | 3 +- .../user/privileges/IndicesPrivileges.java | 24 +- .../privileges/UserIndicesPrivileges.java | 16 +- .../SecurityRequestConvertersTests.java | 4 +- .../SecurityDocumentationIT.java | 9 +- .../security/AuthenticateResponseTests.java | 7 +- .../security/GetPrivilegesResponseTests.java | 3 +- .../security/GetUsersResponseTests.java | 44 +- .../security/HasPrivilegesRequestTests.java | 13 +- .../security/PutPrivilegesRequestTests.java | 22 +- .../ApplicationResourcePrivilegesTests.java | 6 +- .../client/PreBuiltTransportClient.java | 11 +- .../ConstructingObjectParserTests.java | 6 +- .../ingest/common/IngestCommonPlugin.java | 61 +- .../ingest/common/ScriptProcessor.java | 12 +- .../common/ScriptProcessorFactoryTests.java | 14 +- .../mustache/CustomMustacheFactory.java | 19 +- .../mustache/CustomMustacheFactoryTests.java | 24 +- .../org/elasticsearch/painless/Locals.java | 6 +- .../action/PainlessExecuteAction.java | 15 +- .../plugin/store/smb/SMBStorePlugin.java | 9 +- .../node/tasks/list/ListTasksResponse.java | 2 +- .../cluster/node/tasks/list/TaskGroup.java | 4 +- .../cluster/remote/RemoteInfoResponse.java | 6 +- .../rollover/TransportRolloverAction.java | 21 +- .../support/tasks/BaseTasksResponse.java | 4 +- .../bootstrap/SystemCallFilter.java | 9 +- .../TransportClientNodesService.java | 8 +- .../elasticsearch/cluster/DiffableUtils.java | 5 +- .../cluster/block/ClusterBlocks.java | 4 +- .../coordination/CoordinationMetaData.java | 4 +- .../cluster/metadata/IndexMetaData.java | 2 +- .../metadata/IndexNameExpressionResolver.java | 8 +- .../metadata/RepositoriesMetaData.java | 3 +- .../routing/IndexShardRoutingTable.java | 2 +- .../routing/allocation/RoutingAllocation.java | 3 +- .../common/collect/MapBuilder.java | 5 +- .../inject/internal/BindingBuilder.java | 6 +- .../inject/internal/InstanceBindingImpl.java | 5 +- .../internal/ProviderInstanceBindingImpl.java | 5 +- .../common/io/stream/StreamOutput.java | 312 +++++----- .../common/lucene/ShardCoreKeyMap.java | 2 +- .../settings/AbstractScopedSettings.java | 2 +- .../common/settings/ClusterSettings.java | 562 +++++++++--------- .../common/settings/IndexScopedSettings.java | 245 ++++---- .../common/settings/Setting.java | 3 +- .../common/settings/Settings.java | 3 +- .../common/settings/SettingsFilter.java | 4 +- .../elasticsearch/common/time/DateUtils.java | 235 ++++---- .../common/util/concurrent/ThreadContext.java | 3 +- .../elasticsearch/env/NodeEnvironment.java | 4 +- .../gateway/AsyncShardFetch.java | 3 +- .../gateway/DanglingIndicesState.java | 3 +- .../index/CompositeIndexEventListener.java | 4 +- .../index/analysis/Analysis.java | 76 ++- .../plain/PagedBytesAtomicFieldData.java | 11 +- .../index/mapper/DocumentMapperParser.java | 21 +- .../index/query/QueryShardContext.java | 4 +- .../index/similarity/SimilarityProviders.java | 89 +-- .../flush/ShardsSyncedFlushResult.java | 3 +- .../indices/recovery/RecoveryState.java | 4 +- .../ingest/ConditionalProcessor.java | 22 +- .../admin/cluster/RestNodesStatsAction.java | 34 +- .../rest/action/cat/AbstractCatAction.java | 6 +- .../script/AbstractSortScript.java | 25 +- .../script/AggregationScript.java | 27 +- .../elasticsearch/script/ClassPermission.java | 34 +- .../org/elasticsearch/script/FieldScript.java | 18 +- .../org/elasticsearch/script/ScoreScript.java | 20 +- .../script/ScriptedMetricAggContexts.java | 28 +- .../script/TermsSetQueryScript.java | 27 +- .../elasticsearch/script/UpdateScript.java | 13 +- .../DateHistogramAggregationBuilder.java | 42 +- .../search/lookup/LeafSearchLookup.java | 28 +- .../search/sort/SortBuilder.java | 18 +- .../transport/RemoteClusterConnection.java | 2 +- .../transport/TransportService.java | 23 +- .../tasks/RecordingTaskManagerListener.java | 2 +- .../cluster/node/tasks/TestTaskPlugin.java | 8 +- .../node/tasks/TransportTasksActionTests.java | 2 +- .../reroute/ClusterRerouteRequestTests.java | 28 +- .../shrink/TransportResizeActionTests.java | 12 +- ...icsearchUncaughtExceptionHandlerTests.java | 26 +- .../MetaDataCreateIndexServiceTests.java | 8 +- .../structure/RoutingIteratorTests.java | 12 +- .../index/mapper/ExternalMapperPlugin.java | 20 +- .../elasticsearch/index/store/StoreTests.java | 5 +- .../PersistentTasksClusterServiceTests.java | 11 +- .../persistent/TestPersistentTasksPlugin.java | 3 +- .../aggregations/AggregationsTests.java | 122 ++-- .../SharedClusterSnapshotRestoreIT.java | 22 +- .../elasticsearch/test/ESIntegTestCase.java | 6 +- .../test/rest/yaml/Features.java | 11 +- .../rest/yaml/section/ExecutableSection.java | 7 +- .../xpack/ccr/action/ShardFollowTask.java | 4 +- .../action/TransportResumeFollowAction.java | 174 +++--- .../action/TransportXPackUsageAction.java | 2 +- .../GetDataFrameTransformsStatsAction.java | 3 +- .../ml/filestructurefinder/FileStructure.java | 11 +- .../privilege/DeletePrivilegesResponse.java | 3 +- .../privilege/PutPrivilegesRequest.java | 3 +- .../oidc/OpenIdConnectRealmSettings.java | 6 +- .../permission/ApplicationPermission.java | 3 +- .../ApplicationPrivilegeDescriptor.java | 4 +- .../xpack/core/ssl/SSLService.java | 4 +- .../support/xcontent/WatcherParams.java | 3 +- .../DeprecationInfoActionResponseTests.java | 15 +- .../xpack/deprecation/Deprecation.java | 9 +- .../xpack/deprecation/DeprecationChecks.java | 14 +- .../xpack/ml/MachineLearning.java | 24 +- .../FileStructureFinderManager.java | 270 +++++++-- .../FileStructureOverrides.java | 4 +- .../ml/job/persistence/JobConfigProvider.java | 8 +- .../ml/utils/TypedChainTaskExecutor.java | 6 +- .../xpack/monitoring/Monitoring.java | 2 +- .../rest/action/RestMonitoringBulkAction.java | 11 +- .../rollup/rest/RestRollupSearchAction.java | 7 +- .../xpack/security/Security.java | 36 +- .../xpack/security/authc/InternalRealms.java | 47 +- .../kerberos/KerberosTicketValidator.java | 42 +- .../ldap/UserAttributeGroupsResolver.java | 2 +- .../security/authc/support/DnRoleMapper.java | 33 +- .../authc/support/UserRoleMapper.java | 5 +- .../authz/store/CompositeRolesStore.java | 2 +- .../audit/logfile/LoggingAuditTrailTests.java | 77 +-- .../xpack/sql/querydsl/query/MatchQuery.java | 28 +- .../sql/querydsl/query/MultiMatchQuery.java | 39 +- .../sql/querydsl/query/QueryStringQuery.java | 58 +- .../elasticsearch/xpack/watcher/Watcher.java | 26 +- .../common/http/HttpRequestTemplate.java | 5 +- .../watcher/test/WatcherMockScriptPlugin.java | 52 +- .../xpack/watcher/watch/WatchTests.java | 10 +- .../security/authc/kerberos/SpnegoClient.java | 40 +- 142 files changed, 1948 insertions(+), 2010 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FileStructure.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FileStructure.java index 215c19a0b3b78..9e3e2162a69bc 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FileStructure.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ml/filestructurefinder/FileStructure.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -148,22 +147,20 @@ private FileStructure(int numLinesAnalyzed, int numMessagesAnalyzed, String samp this.format = Objects.requireNonNull(format); this.multilineStartPattern = multilineStartPattern; this.excludeLinesPattern = excludeLinesPattern; - this.columnNames = (columnNames == null) ? null : Collections.unmodifiableList(new ArrayList<>(columnNames)); + this.columnNames = (columnNames == null) ? null : List.copyOf(columnNames); this.hasHeaderRow = hasHeaderRow; this.delimiter = delimiter; this.quote = quote; this.shouldTrimFields = shouldTrimFields; this.grokPattern = grokPattern; this.timestampField = timestampField; - this.jodaTimestampFormats = - (jodaTimestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(jodaTimestampFormats)); - this.javaTimestampFormats = - (javaTimestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(javaTimestampFormats)); + this.jodaTimestampFormats = (jodaTimestampFormats == null) ? null : List.copyOf(jodaTimestampFormats); + this.javaTimestampFormats = (javaTimestampFormats == null) ? null : List.copyOf(javaTimestampFormats); this.needClientTimezone = needClientTimezone; this.mappings = Collections.unmodifiableSortedMap(new TreeMap<>(mappings)); this.ingestPipeline = (ingestPipeline == null) ? null : Collections.unmodifiableMap(new LinkedHashMap<>(ingestPipeline)); this.fieldStats = Collections.unmodifiableSortedMap(new TreeMap<>(fieldStats)); - this.explanation = (explanation == null) ? null : Collections.unmodifiableList(new ArrayList<>(explanation)); + this.explanation = (explanation == null) ? null : List.copyOf(explanation); } public int getNumLinesAnalyzed() { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetPrivilegesResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetPrivilegesResponse.java index 182f14ef00c51..168ca17c926c5 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetPrivilegesResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetPrivilegesResponse.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Objects; @@ -44,7 +43,7 @@ public Set getPrivileges() { } public GetPrivilegesResponse(Collection privileges) { - this.privileges = Collections.unmodifiableSet(new HashSet<>(privileges)); + this.privileges = Set.copyOf(privileges); } public static GetPrivilegesResponse fromXContent(XContentParser parser) throws IOException { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetUsersResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetUsersResponse.java index 39d4a25a64207..d4e325749a847 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetUsersResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/GetUsersResponse.java @@ -27,12 +27,14 @@ import org.elasticsearch.common.xcontent.XContentParserUtils; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -42,26 +44,27 @@ * Returns a List of {@link User} objects */ public class GetUsersResponse { - private final Set users; - private final Set enabledUsers; - public GetUsersResponse(Set users, Set enabledUsers) { - this.users = Collections.unmodifiableSet(users); - this.enabledUsers = Collections.unmodifiableSet(enabledUsers); + private final Map users; + private final Map enabledUsers; + + GetUsersResponse(final Map users, final Map enabledUsers) { + this.users = Map.copyOf(users); + this.enabledUsers = Map.copyOf(enabledUsers); } - public Set getUsers() { - return users; + public List getUsers() { + return List.copyOf(users.values()); } - public Set getEnabledUsers() { - return enabledUsers; + public List getEnabledUsers() { + return List.copyOf(enabledUsers.values()); } public static GetUsersResponse fromXContent(XContentParser parser) throws IOException { XContentParserUtils.ensureExpectedToken(Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); - final Set users = new HashSet<>(); - final Set enabledUsers = new HashSet<>(); + final List users = new ArrayList<>(); + final List enabledUsers = new ArrayList<>(); Token token; while ((token = parser.nextToken()) != Token.END_OBJECT) { XContentParserUtils.ensureExpectedToken(Token.FIELD_NAME, token, parser::getTokenLocation); @@ -71,7 +74,11 @@ public static GetUsersResponse fromXContent(XContentParser parser) throws IOExce enabledUsers.add(parsedUser.user); } } - return new GetUsersResponse(users, enabledUsers); + return new GetUsersResponse(toMap(users), toMap(enabledUsers)); + } + + static Map toMap(final Collection users) { + return users.stream().collect(Collectors.toUnmodifiableMap(User::getUsername, Function.identity())); } @Override @@ -99,7 +106,7 @@ public int hashCode() { (constructorObjects) -> { int i = 0; final String username = (String) constructorObjects[i++]; - final Collection roles = (Collection) constructorObjects[i++]; + final List roles = (List) constructorObjects[i++]; final Map metadata = (Map) constructorObjects[i++]; final Boolean enabled = (Boolean) constructorObjects[i++]; final String fullName = (String) constructorObjects[i++]; @@ -120,13 +127,13 @@ protected static final class ParsedUser { protected User user; protected boolean enabled; - public ParsedUser(String username, Collection roles, Map metadata, Boolean enabled, + public ParsedUser(String username, List roles, Map metadata, Boolean enabled, @Nullable String fullName, @Nullable String email) { - String checkedUsername = username = Objects.requireNonNull(username, "`username` is required, cannot be null"); - Collection checkedRoles = Collections.unmodifiableSet(new HashSet<>( - Objects.requireNonNull(roles, "`roles` is required, cannot be null. Pass an empty Collection instead."))); - Map checkedMetadata = Collections - .unmodifiableMap(Objects.requireNonNull(metadata, "`metadata` is required, cannot be null. Pass an empty map instead.")); + String checkedUsername = Objects.requireNonNull(username, "`username` is required, cannot be null"); + List checkedRoles = + List.copyOf(Objects.requireNonNull(roles, "`roles` is required, cannot be null. Pass an empty list instead.")); + Map checkedMetadata = Collections.unmodifiableMap( + Objects.requireNonNull(metadata, "`metadata` is required, cannot be null. Pass an empty map instead.")); this.user = new User(checkedUsername, checkedRoles, checkedMetadata, fullName, email); this.enabled = enabled; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/expressions/CompositeRoleMapperExpression.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/expressions/CompositeRoleMapperExpression.java index 3373f325bfcdb..b44abab37ec71 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/expressions/CompositeRoleMapperExpression.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/expressions/CompositeRoleMapperExpression.java @@ -23,8 +23,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -53,7 +51,7 @@ public abstract class CompositeRoleMapperExpression implements RoleMapperExpress assert name != null : "field name cannot be null"; assert elements != null : "at least one field expression is required"; this.name = name; - this.elements = Collections.unmodifiableList(Arrays.asList(elements)); + this.elements = List.of(elements); } public String getName() { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/fields/FieldRoleMapperExpression.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/fields/FieldRoleMapperExpression.java index 4111b9fa2a2c3..1a6bded342191 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/fields/FieldRoleMapperExpression.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/support/expressiondsl/fields/FieldRoleMapperExpression.java @@ -23,8 +23,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -53,7 +51,7 @@ public FieldRoleMapperExpression(final String field, final Object... values) { throw new IllegalArgumentException("null or empty values for field (" + field + ")"); } this.field = field; - this.values = Collections.unmodifiableList(Arrays.asList(values)); + this.values = List.of(values); } public String getField() { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/User.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/User.java index 4ac8f54c4741b..92ee4b51fe181 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/User.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/User.java @@ -22,12 +22,10 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; -import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; /** * A user to be utilized with security APIs. @@ -36,7 +34,7 @@ public final class User { private final String username; - private final Set roles; + private final List roles; private final Map metadata; @Nullable private final String fullName; @Nullable private final String email; @@ -50,13 +48,12 @@ public final class User { * @param fullName the full name of the user that may be used for display purposes * @param email the email address of the user */ - public User(String username, Collection roles, Map metadata, @Nullable String fullName, + public User(String username, List roles, Map metadata, @Nullable String fullName, @Nullable String email) { - this.username = username = Objects.requireNonNull(username, "`username` is required, cannot be null"); - this.roles = Collections.unmodifiableSet(new HashSet<>( - Objects.requireNonNull(roles, "`roles` is required, cannot be null. Pass an empty Collection instead."))); - this.metadata = Collections - .unmodifiableMap(Objects.requireNonNull(metadata, "`metadata` is required, cannot be null. Pass an empty map instead.")); + this.username = Objects.requireNonNull(username, "`username` is required, cannot be null"); + this.roles = List.copyOf(Objects.requireNonNull(roles, "`roles` is required, cannot be null. Pass an empty list instead.")); + this.metadata = Collections.unmodifiableMap( + Objects.requireNonNull(metadata, "`metadata` is required, cannot be null. Pass an empty map instead.")); this.fullName = fullName; this.email = email; } @@ -67,7 +64,7 @@ public User(String username, Collection roles, Map metad * @param username the username, also known as the principal, unique for in the scope of a realm * @param roles the roles that this user is assigned */ - public User(String username, Collection roles) { + public User(String username, List roles) { this(username, roles, Collections.emptyMap(), null, null); } @@ -84,7 +81,7 @@ public String getUsername() { * identified by their unique names and each represents as * set of permissions. Can never be {@code null}. */ - public Set getRoles() { + public List getRoles() { return this.roles; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/AbstractIndicesPrivileges.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/AbstractIndicesPrivileges.java index 4187ca7638d4e..99af2b20877b6 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/AbstractIndicesPrivileges.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/AbstractIndicesPrivileges.java @@ -31,9 +31,8 @@ import java.io.UncheckedIOException; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.List; import java.util.Objects; -import java.util.Set; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -44,8 +43,8 @@ public abstract class AbstractIndicesPrivileges { static final ParseField FIELD_PERMISSIONS = new ParseField("field_security"); static final ParseField QUERY = new ParseField("query"); - protected final Set indices; - protected final Set privileges; + protected final List indices; + protected final List privileges; protected final boolean allowRestrictedIndices; AbstractIndicesPrivileges(Collection indices, Collection privileges, boolean allowRestrictedIndices) { @@ -55,15 +54,15 @@ public abstract class AbstractIndicesPrivileges { if (null == privileges || privileges.isEmpty()) { throw new IllegalArgumentException("indices privileges must define at least one privilege"); } - this.indices = Collections.unmodifiableSet(new HashSet<>(indices)); - this.privileges = Collections.unmodifiableSet(new HashSet<>(privileges)); + this.indices = List.copyOf(indices); + this.privileges = List.copyOf(privileges); this.allowRestrictedIndices = allowRestrictedIndices; } /** * The indices names covered by the privileges. */ - public Set getIndices() { + public List getIndices() { return this.indices; } @@ -72,7 +71,7 @@ public Set getIndices() { * such privileges, but the {@code String} datatype allows for flexibility in defining * finer grained privileges. */ - public Set getPrivileges() { + public List getPrivileges() { return this.privileges; } @@ -106,8 +105,8 @@ public static class FieldSecurity implements ToXContentObject { @SuppressWarnings("unchecked") private static FieldSecurity buildObjectFromParserArgs(Object[] args) { return new FieldSecurity( - (Collection) args[0], - (Collection) args[1] + (List) args[0], + (List) args[1] ); } @@ -121,15 +120,15 @@ static FieldSecurity parse(XContentParser parser, Void context) throws IOExcepti } // null or singleton '*' means all fields are granted, empty means no fields are granted - private final Set grantedFields; + private final List grantedFields; // null or empty means no fields are denied - private final Set deniedFields; + private final List deniedFields; FieldSecurity(Collection grantedFields, Collection deniedFields) { // unspecified granted fields means no restriction - this.grantedFields = grantedFields == null ? null : Collections.unmodifiableSet(new HashSet<>(grantedFields)); + this.grantedFields = grantedFields == null ? null : List.copyOf(grantedFields); // unspecified denied fields means no restriction - this.deniedFields = deniedFields == null ? null : Collections.unmodifiableSet(new HashSet<>(deniedFields)); + this.deniedFields = deniedFields == null ? null : List.copyOf(deniedFields); } /** @@ -137,7 +136,7 @@ static FieldSecurity parse(XContentParser parser, Void context) throws IOExcepti * all the document's fields are granted access to. Can also be empty, in which * case no fields are granted access to. */ - public Set getGrantedFields() { + public List getGrantedFields() { return grantedFields; } @@ -145,7 +144,7 @@ public Set getGrantedFields() { * The document fields that cannot be accessed or queried. Can be null or empty, * in which case no fields are denied. */ - public Set getDeniedFields() { + public List getDeniedFields() { return deniedFields; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationPrivilege.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationPrivilege.java index fc21dfc0b2816..ec78cca6b5c55 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationPrivilege.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationPrivilege.java @@ -29,14 +29,11 @@ import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; - import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -55,10 +52,10 @@ public final class ApplicationPrivilege implements ToXContentObject { private final String application; private final String name; - private final Set actions; + private final List actions; private final Map metadata; - public ApplicationPrivilege(String application, String name, Collection actions, @Nullable Map metadata) { + public ApplicationPrivilege(String application, String name, List actions, @Nullable Map metadata) { if (Strings.isNullOrEmpty(application)) { throw new IllegalArgumentException("application name must be provided"); } else { @@ -72,12 +69,12 @@ public ApplicationPrivilege(String application, String name, Collection if (actions == null || actions.isEmpty()) { throw new IllegalArgumentException("actions must be provided"); } else { - this.actions = Collections.unmodifiableSet(new HashSet<>(actions)); + this.actions = List.copyOf(actions); } if (metadata == null || metadata.isEmpty()) { this.metadata = Collections.emptyMap(); } else { - this.metadata = Collections.unmodifiableMap(metadata); + this.metadata = Map.copyOf(metadata); } } @@ -89,7 +86,7 @@ public String getName() { return name; } - public Set getActions() { + public List getActions() { return actions; } @@ -100,7 +97,7 @@ public Map getMetadata() { @SuppressWarnings("unchecked") public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "application_privilege", - true, args -> new ApplicationPrivilege((String) args[0], (String) args[1], (Collection) args[2], + true, args -> new ApplicationPrivilege((String) args[0], (String) args[1], (List) args[2], (Map) args[3])); static { @@ -137,7 +134,7 @@ public static Builder builder() { public static final class Builder { private String applicationName = null; private String privilegeName = null; - private Collection actions = null; + private List actions = null; private Map metadata = null; private Builder() { @@ -158,7 +155,7 @@ public Builder actions(String... actions) { return this; } - public Builder actions(Collection actions) { + public Builder actions(List actions) { this.actions = Objects.requireNonNull(actions); return this; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivileges.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivileges.java index 8846e259e26b3..a296a80ad49b4 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivileges.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivileges.java @@ -29,11 +29,8 @@ import org.elasticsearch.common.xcontent.XContentType; import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; +import java.util.List; import java.util.Objects; -import java.util.Set; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; @@ -57,8 +54,8 @@ public final class ApplicationResourcePrivileges implements ToXContentObject { // end up being implicitly set to null in that request. int i = 0; final String application = (String) constructorObjects[i++]; - final Collection privileges = (Collection) constructorObjects[i++]; - final Collection resources = (Collection) constructorObjects[i]; + final List privileges = (List) constructorObjects[i++]; + final List resources = (List) constructorObjects[i]; return new ApplicationResourcePrivileges(application, privileges, resources); }); @@ -69,8 +66,8 @@ public final class ApplicationResourcePrivileges implements ToXContentObject { } private final String application; - private final Set privileges; - private final Set resources; + private final List privileges; + private final List resources; /** * Constructs privileges for resources under an application scope. @@ -85,7 +82,7 @@ public final class ApplicationResourcePrivileges implements ToXContentObject { * The resources names. Cannot be null or empty. Resource identifiers * are completely under the clients control. */ - public ApplicationResourcePrivileges(String application, Collection privileges, Collection resources) { + public ApplicationResourcePrivileges(String application, List privileges, List resources) { if (Strings.isNullOrEmpty(application)) { throw new IllegalArgumentException("application privileges must have an application name"); } @@ -96,19 +93,19 @@ public ApplicationResourcePrivileges(String application, Collection priv throw new IllegalArgumentException("application privileges must refer to at least one resource"); } this.application = application; - this.privileges = Collections.unmodifiableSet(new HashSet<>(privileges)); - this.resources = Collections.unmodifiableSet(new HashSet<>(resources)); + this.privileges = List.copyOf(privileges); + this.resources = List.copyOf(resources); } public String getApplication() { return application; } - public Set getResources() { + public List getResources() { return this.resources; } - public Set getPrivileges() { + public List getPrivileges() { return this.privileges; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/GlobalPrivileges.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/GlobalPrivileges.java index c1bd5550eca1a..81768e894b6ce 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/GlobalPrivileges.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/GlobalPrivileges.java @@ -28,7 +28,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -82,7 +81,7 @@ public GlobalPrivileges(Collection privilege throw new IllegalArgumentException("Privileges cannot be empty or null"); } // duplicates are just ignored - this.privileges = Collections.unmodifiableSet(new HashSet<>(Objects.requireNonNull(privileges))); + this.privileges = Set.copyOf(Objects.requireNonNull(privileges)); this.privilegesByCategoryMap = Collections .unmodifiableMap(this.privileges.stream().collect(Collectors.groupingBy(GlobalOperationPrivilege::getCategory))); for (final Map.Entry> privilegesByCategory : privilegesByCategoryMap.entrySet()) { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/IndicesPrivileges.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/IndicesPrivileges.java index 26c303304cd66..2e845906696da 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/IndicesPrivileges.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/IndicesPrivileges.java @@ -29,7 +29,7 @@ import java.io.IOException; import java.util.Arrays; -import java.util.Collection; +import java.util.List; import java.util.Objects; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; @@ -48,8 +48,8 @@ public final class IndicesPrivileges extends AbstractIndicesPrivileges implement static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("indices_privileges", false, constructorObjects -> { int i = 0; - final Collection indices = (Collection) constructorObjects[i++]; - final Collection privileges = (Collection) constructorObjects[i++]; + final List indices = (List) constructorObjects[i++]; + final List privileges = (List) constructorObjects[i++]; final boolean allowRestrictedIndices = (Boolean) constructorObjects[i++]; final FieldSecurity fields = (FieldSecurity) constructorObjects[i++]; final String query = (String) constructorObjects[i]; @@ -68,7 +68,7 @@ public final class IndicesPrivileges extends AbstractIndicesPrivileges implement // missing query means all documents, i.e. no restrictions private final @Nullable String query; - private IndicesPrivileges(Collection indices, Collection privileges, boolean allowRestrictedIndices, + private IndicesPrivileges(List indices, List privileges, boolean allowRestrictedIndices, @Nullable FieldSecurity fieldSecurity, @Nullable String query) { super(indices, privileges, allowRestrictedIndices); this.fieldSecurity = fieldSecurity; @@ -165,13 +165,13 @@ public static Builder builder() { public static final class Builder { private @Nullable - Collection indices = null; + List indices = null; private @Nullable - Collection privileges = null; + List privileges = null; private @Nullable - Collection grantedFields = null; + List grantedFields = null; private @Nullable - Collection deniedFields = null; + List deniedFields = null; private @Nullable String query = null; boolean allowRestrictedIndices = false; @@ -183,7 +183,7 @@ public Builder indices(String... indices) { return indices(Arrays.asList(Objects.requireNonNull(indices, "indices required"))); } - public Builder indices(Collection indices) { + public Builder indices(List indices) { this.indices = Objects.requireNonNull(indices, "indices required"); return this; } @@ -192,7 +192,7 @@ public Builder privileges(String... privileges) { return privileges(Arrays.asList(Objects.requireNonNull(privileges, "privileges required"))); } - public Builder privileges(Collection privileges) { + public Builder privileges(List privileges) { this.privileges = Objects.requireNonNull(privileges, "privileges required"); return this; } @@ -205,7 +205,7 @@ public Builder grantedFields(@Nullable String... grantedFields) { return grantedFields(Arrays.asList(grantedFields)); } - public Builder grantedFields(@Nullable Collection grantedFields) { + public Builder grantedFields(@Nullable List grantedFields) { this.grantedFields = grantedFields; return this; } @@ -218,7 +218,7 @@ public Builder deniedFields(@Nullable String... deniedFields) { return deniedFields(Arrays.asList(deniedFields)); } - public Builder deniedFields(@Nullable Collection deniedFields) { + public Builder deniedFields(@Nullable List deniedFields) { this.deniedFields = deniedFields; return this; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/UserIndicesPrivileges.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/UserIndicesPrivileges.java index 9878bb0071740..474030f9689d5 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/UserIndicesPrivileges.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/security/user/privileges/UserIndicesPrivileges.java @@ -26,10 +26,8 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.Set; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -43,8 +41,8 @@ */ public class UserIndicesPrivileges extends AbstractIndicesPrivileges { - private final Set fieldSecurity; - private final Set query; + private final List fieldSecurity; + private final List query; private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "user_indices_privilege", true, UserIndicesPrivileges::buildObjectFromParserArgs); @@ -73,17 +71,17 @@ public static UserIndicesPrivileges fromXContent(XContentParser parser) throws I } public UserIndicesPrivileges(Collection indices, Collection privileges, boolean allowRestrictedIndices, - Collection fieldSecurity, Collection query) { + List fieldSecurity, List query) { super(indices, privileges, allowRestrictedIndices); - this.fieldSecurity = fieldSecurity == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(fieldSecurity)); - this.query = query == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(query)); + this.fieldSecurity = fieldSecurity == null ? Collections.emptyList() : List.copyOf(fieldSecurity); + this.query = query == null ? Collections.emptyList() : List.copyOf(query); } - public Set getFieldSecurity() { + public List getFieldSecurity() { return fieldSecurity; } - public Set getQueries() { + public List getQueries() { return query; } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java index 1176cabcc3d9c..99350fc29db8a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SecurityRequestConvertersTests.java @@ -55,7 +55,6 @@ import org.elasticsearch.client.security.user.privileges.Role.IndexPrivilegeName; import org.elasticsearch.common.Strings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -353,11 +352,12 @@ public void testPutPrivileges() throws Exception { int noOfApplicationPrivileges = randomIntBetween(2, 4); final List privileges = new ArrayList<>(); for (int count = 0; count < noOfApplicationPrivileges; count++) { + final String[] actions = generateRandomStringArray(3, 5, false, false); privileges.add(ApplicationPrivilege.builder() .application(randomAlphaOfLength(4)) .privilege(randomAlphaOfLengthBetween(3, 5)) - .actions(Sets.newHashSet(generateRandomStringArray(3, 5, false, false))) .metadata(Collections.singletonMap("k1", "v1")) + .actions(actions == null ? Collections.emptyList() : List.of(actions)) .build()); } final RefreshPolicy refreshPolicy = randomFrom(RefreshPolicy.values()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java index b095ca5a9a0db..f9a1c5c6571eb 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SecurityDocumentationIT.java @@ -97,12 +97,14 @@ import javax.crypto.SecretKeyFactory; import javax.crypto.spec.PBEKeySpec; + import java.io.IOException; import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -158,6 +160,7 @@ public void testGetUsers() throws Exception { List users = new ArrayList<>(3); users.addAll(response.getUsers()); + users.sort(Comparator.comparing(User::getUsername)); assertNotNull(response); assertThat(users.size(), equalTo(3)); assertThat(users.get(0).getUsername(), equalTo(usernames[0])); @@ -1635,13 +1638,13 @@ public void testPutPrivileges() throws Exception { privileges.add(ApplicationPrivilege.builder() .application("app01") .privilege("all") - .actions(Sets.newHashSet("action:login")) + .actions(List.of("action:login")) .metadata(Collections.singletonMap("k1", "v1")) .build()); privileges.add(ApplicationPrivilege.builder() .application("app01") .privilege("write") - .actions(Sets.newHashSet("action:write")) + .actions(List.of("action:write")) .build()); final PutPrivilegesRequest putPrivilegesRequest = new PutPrivilegesRequest(privileges, RefreshPolicy.IMMEDIATE); // end::put-privileges-request @@ -1664,7 +1667,7 @@ public void testPutPrivileges() throws Exception { privileges.add(ApplicationPrivilege.builder() .application("app01") .privilege("all") - .actions(Sets.newHashSet("action:login")) + .actions(List.of("action:login")) .metadata(Collections.singletonMap("k1", "v1")) .build()); final PutPrivilegesRequest putPrivilegesRequest = new PutPrivilegesRequest(privileges, RefreshPolicy.IMMEDIATE); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/AuthenticateResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/AuthenticateResponseTests.java index e348cb4d46c7e..7ca84283c501c 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/AuthenticateResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/AuthenticateResponseTests.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -49,8 +48,8 @@ public void testFromXContent() throws IOException { } public void testEqualsAndHashCode() { - final AuthenticateResponse reponse = createTestInstance(); - EqualsHashCodeTestUtils.checkEqualsAndHashCode(reponse, this::copy, + final AuthenticateResponse response = createTestInstance(); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(response, this::copy, this::mutate); } @@ -123,7 +122,7 @@ private AuthenticateResponse mutate(AuthenticateResponse response) { originalUser.getMetadata(), originalUser.getFullName(), originalUser.getEmail()), response.enabled(), response.getAuthenticationRealm(), response.getLookupRealm()); case 2: - final Collection wrongRoles = new ArrayList<>(originalUser.getRoles()); + final List wrongRoles = new ArrayList<>(originalUser.getRoles()); wrongRoles.add(randomAlphaOfLengthBetween(1, 4)); return new AuthenticateResponse(new User(originalUser.getUsername(), wrongRoles, originalUser.getMetadata(), originalUser.getFullName(), originalUser.getEmail()), response.enabled(), response.getAuthenticationRealm(), diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetPrivilegesResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetPrivilegesResponseTests.java index 74211892a09e8..bf55e224095fe 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetPrivilegesResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetPrivilegesResponseTests.java @@ -31,6 +31,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -139,7 +140,7 @@ public void testEqualsHashCode() { private static GetPrivilegesResponse mutateTestItem(GetPrivilegesResponse original) { if (randomBoolean()) { Set originalPrivileges = original.getPrivileges(); - List privileges = new ArrayList<>(); + Set privileges = new HashSet<>(); privileges.addAll(originalPrivileges); privileges.add(new ApplicationPrivilege("testapp", "all", Arrays.asList("action:login", "data:read/*", "manage:*"), null)); return new GetPrivilegesResponse(privileges); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetUsersResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetUsersResponseTests.java index 69db584287ceb..1c30760ee53a3 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetUsersResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/GetUsersResponseTests.java @@ -31,13 +31,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Predicate; -import java.util.stream.Collectors; +import static org.elasticsearch.client.security.GetUsersResponse.toMap; import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; /** tests the Response for getting users from the security HLRC */ @@ -91,8 +89,8 @@ private XContentBuilder toXContent(GetUsersResponse response, XContentBuilder bu } private static GetUsersResponse createTestInstance() { - final Set users = new HashSet<>(); - final Set enabledUsers = new HashSet<>(); + final List users = new ArrayList<>(); + final List enabledUsers = new ArrayList<>(); Map metadata = new HashMap<>(); metadata.put(randomAlphaOfLengthBetween(1, 5), randomInt()); @@ -109,12 +107,12 @@ private static GetUsersResponse createTestInstance() { Arrays.asList(new String[] {randomAlphaOfLength(5), randomAlphaOfLength(5)}), metadata2, randomAlphaOfLength(10), null); users.add(user2); - return new GetUsersResponse(users, enabledUsers); + return new GetUsersResponse(toMap(users), toMap(enabledUsers)); } public void testEqualsHashCode() { - final Set users = new HashSet<>(); - final Set enabledUsers = new HashSet<>(); + final List users = new ArrayList<>(); + final List enabledUsers = new ArrayList<>(); Map metadata = new HashMap<>(); metadata.put("intelligence", 1); final User user1 = new User("testUser1", Arrays.asList(new String[] {"admin", "other_role1"}), @@ -125,41 +123,43 @@ public void testEqualsHashCode() { metadata2.put("intelligence", 9); metadata2.put("specialty", "geo"); final User user2 = new User("testUser2", Arrays.asList(new String[] {"admin"}), - metadata, "Test User 2", "testuser2@example.com"); + metadata2, "Test User 2", "testuser2@example.com"); users.add(user2); enabledUsers.add(user2); - final GetUsersResponse getUsersResponse = new GetUsersResponse(users, enabledUsers); + final GetUsersResponse getUsersResponse = new GetUsersResponse(toMap(users), toMap(enabledUsers)); assertNotNull(getUsersResponse); - EqualsHashCodeTestUtils.checkEqualsAndHashCode(getUsersResponse, (original) -> { - return new GetUsersResponse(original.getUsers(), original.getEnabledUsers()); - }); - EqualsHashCodeTestUtils.checkEqualsAndHashCode(getUsersResponse, (original) -> { - return new GetUsersResponse(original.getUsers(), original.getEnabledUsers()); - }, GetUsersResponseTests::mutateTestItem); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + getUsersResponse, + (original) -> new GetUsersResponse(toMap(original.getUsers()), toMap(original.getEnabledUsers()))); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + getUsersResponse, + (original) -> new GetUsersResponse(toMap(original.getUsers()), toMap(original.getEnabledUsers())), + GetUsersResponseTests::mutateTestItem); } private static GetUsersResponse mutateTestItem(GetUsersResponse original) { if (randomBoolean()) { - final Set users = new HashSet<>(); - final Set enabledUsers = new HashSet<>(); + final List users = new ArrayList<>(); + final List enabledUsers = new ArrayList<>(); Map metadata = new HashMap<>(); metadata.put("intelligence", 1); final User user1 = new User("testUser1", Arrays.asList(new String[] {"admin", "other_role1"}), metadata, "Test User 1", null); users.add(user1); enabledUsers.add(user1); - return new GetUsersResponse(users, enabledUsers); + return new GetUsersResponse(toMap(users), toMap(enabledUsers)); } Map metadata = new HashMap<>(); metadata.put("intelligence", 5); // change intelligence final User user1 = new User("testUser1", Arrays.asList(new String[] {"admin", "other_role1"}), metadata, "Test User 1", null); - Set newUsers = original.getUsers().stream().collect(Collectors.toSet()); - Set enabledUsers = original.getEnabledUsers().stream().collect(Collectors.toSet()); + List newUsers = new ArrayList<>(original.getUsers()); + List enabledUsers = new ArrayList<>(original.getEnabledUsers()); newUsers.clear(); enabledUsers.clear(); newUsers.add(user1); enabledUsers.add(user1); - return new GetUsersResponse(newUsers, enabledUsers); + return new GetUsersResponse(toMap(newUsers), toMap(enabledUsers)); } + } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/HasPrivilegesRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/HasPrivilegesRequestTests.java index fa1245925d7ea..da8cf92bdc14e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/HasPrivilegesRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/HasPrivilegesRequestTests.java @@ -31,7 +31,9 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -87,12 +89,13 @@ public void testEqualsAndHashCode() { .privileges(generateRandomStringArray(3, 8, false, false)) .allowRestrictedIndices(randomBoolean()) .build())); + final String[] privileges = generateRandomStringArray(3, 8, false, false); + final String[] resources = generateRandomStringArray(2, 6, false, false); final Set application = Sets.newHashSet(randomArray(1, 5, ApplicationResourcePrivileges[]::new, - () -> new ApplicationResourcePrivileges( - randomAlphaOfLengthBetween(5, 12), - Sets.newHashSet(generateRandomStringArray(3, 8, false, false)), - Sets.newHashSet(generateRandomStringArray(2, 6, false, false)) - ))); + () -> new ApplicationResourcePrivileges( + randomAlphaOfLengthBetween(5, 12), + privileges == null ? Collections.emptyList() : List.of(privileges), + resources == null ? Collections.emptyList() : List.of(resources)))); final HasPrivilegesRequest request = new HasPrivilegesRequest(cluster, indices, application); EqualsHashCodeTestUtils.checkEqualsAndHashCode(request, this::copy, this::mutate); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/PutPrivilegesRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/PutPrivilegesRequestTests.java index 8c60382eeec38..6c6c9f7ea8709 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/PutPrivilegesRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/PutPrivilegesRequestTests.java @@ -45,7 +45,7 @@ public void testConstructor() { Arrays.asList(Collections.singletonList(ApplicationPrivilege.builder() .application("app01") .privilege("all") - .actions(Sets.newHashSet("action:login", "action:logout")) + .actions(List.of("action:login", "action:logout")) .metadata(Collections.singletonMap("k1", "v1")) .build()), null, Collections.emptyList())); @@ -69,8 +69,8 @@ public void testToXContent() throws IOException { + " \"application\" : \"app01\",\n" + " \"name\" : \"all\",\n" + " \"actions\" : [\n" - + " \"action:logout\",\n" - + " \"action:login\"\n" + + " \"action:login\",\n" + + " \"action:logout\"\n" + " ],\n" + " \"metadata\" : {\n" + " \"k1\" : \"v1\"\n" @@ -88,8 +88,8 @@ public void testToXContent() throws IOException { + " \"application\" : \"app02\",\n" + " \"name\" : \"all\",\n" + " \"actions\" : [\n" - + " \"action:logout\",\n" - + " \"action:login\"\n" + + " \"action:login\",\n" + + " \"action:logout\"\n" + " ],\n" + " \"metadata\" : {\n" + " \"k2\" : \"v2\"\n" @@ -101,18 +101,18 @@ public void testToXContent() throws IOException { privileges.add(ApplicationPrivilege.builder() .application("app01") .privilege("all") - .actions(Sets.newHashSet("action:login", "action:logout")) + .actions(List.of("action:login", "action:logout")) .metadata(Collections.singletonMap("k1", "v1")) .build()); privileges.add(ApplicationPrivilege.builder() .application("app01") .privilege("read") - .actions(Sets.newHashSet("data:read")) + .actions(List.of("data:read")) .build()); privileges.add(ApplicationPrivilege.builder() .application("app02") .privilege("all") - .actions(Sets.newHashSet("action:login", "action:logout")) + .actions(List.of("action:login", "action:logout")) .metadata(Collections.singletonMap("k2", "v2")) .build()); final RefreshPolicy refreshPolicy = randomFrom(RefreshPolicy.values()); @@ -126,13 +126,13 @@ public void testEqualsHashCode() { privileges.add(ApplicationPrivilege.builder() .application(randomAlphaOfLength(5)) .privilege(randomAlphaOfLength(3)) - .actions(Sets.newHashSet(randomAlphaOfLength(5), randomAlphaOfLength(5))) + .actions(List.of(randomAlphaOfLength(5), randomAlphaOfLength(5))) .metadata(Collections.singletonMap(randomAlphaOfLength(3), randomAlphaOfLength(3))) .build()); privileges.add(ApplicationPrivilege.builder() .application(randomAlphaOfLength(5)) .privilege(randomAlphaOfLength(3)) - .actions(Sets.newHashSet(randomAlphaOfLength(5), randomAlphaOfLength(5))) + .actions(List.of(randomAlphaOfLength(5), randomAlphaOfLength(5))) .metadata(Collections.singletonMap(randomAlphaOfLength(3), randomAlphaOfLength(3))) .build()); final RefreshPolicy refreshPolicy = randomFrom(RefreshPolicy.values()); @@ -156,7 +156,7 @@ private static PutPrivilegesRequest mutateTestItem(PutPrivilegesRequest original privileges.add(ApplicationPrivilege.builder() .application(randomAlphaOfLength(5)) .privilege(randomAlphaOfLength(3)) - .actions(Sets.newHashSet(randomAlphaOfLength(6))) + .actions(List.of(randomAlphaOfLength(6))) .build()); return new PutPrivilegesRequest(privileges, original.getRefreshPolicy()); case 1: diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivilegesTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivilegesTests.java index 29845441a2c05..2a0d89500ae89 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivilegesTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/security/user/privileges/ApplicationResourcePrivilegesTests.java @@ -24,8 +24,8 @@ import java.io.IOException; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; +import java.util.List; import static org.hamcrest.Matchers.is; @@ -62,7 +62,7 @@ public void testEmptyApplicationName() { } public void testEmptyPrivileges() { - final Collection emptyPrivileges = randomBoolean() ? Collections.emptyList() : null; + final List emptyPrivileges = randomBoolean() ? Collections.emptyList() : null; final IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new ApplicationResourcePrivileges(randomAlphaOfLengthBetween(1, 8), emptyPrivileges, @@ -71,7 +71,7 @@ public void testEmptyPrivileges() { } public void testEmptyResources() { - final Collection emptyResources = randomBoolean() ? Collections.emptyList() : null; + final List emptyResources = randomBoolean() ? Collections.emptyList() : null; final IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new ApplicationResourcePrivileges(randomAlphaOfLengthBetween(1, 8), Arrays.asList(randomArray(1, 8, size -> new String[size], () -> randomAlphaOfLengthBetween(1, 8))), diff --git a/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java b/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java index 7bde7fbc06f6b..4a78c38d06cc6 100644 --- a/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java +++ b/client/transport/src/main/java/org/elasticsearch/transport/client/PreBuiltTransportClient.java @@ -21,7 +21,6 @@ import io.netty.util.ThreadDeathWatcher; import io.netty.util.concurrent.GlobalEventExecutor; - import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.network.NetworkModule; @@ -35,7 +34,7 @@ import java.util.Arrays; import java.util.Collection; -import java.util.Collections; +import java.util.List; import java.util.concurrent.TimeUnit; /** @@ -87,13 +86,7 @@ private static void setSystemPropertyIfUnset(final String key, final String valu } private static final Collection> PRE_INSTALLED_PLUGINS = - Collections.unmodifiableList( - Arrays.asList( - Netty4Plugin.class, - ReindexPlugin.class, - PercolatorPlugin.class, - MustachePlugin.class, - ParentJoinPlugin.class)); + List.of(Netty4Plugin.class, ReindexPlugin.class, PercolatorPlugin.class, MustachePlugin.class, ParentJoinPlugin.class); /** * Creates a new transport client with pre-installed plugins. diff --git a/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java b/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java index 5fc3e612c18f7..00e34183d2f78 100644 --- a/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java +++ b/libs/x-content/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java @@ -29,10 +29,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.Arrays; import java.util.List; -import static java.util.Collections.unmodifiableList; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; import static org.hamcrest.Matchers.anyOf; @@ -378,8 +376,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public static final ConstructingObjectParser PARSER_VEGETABLE_OPTIONAL = buildParser(true, false); public static final ConstructingObjectParser PARSER_ALL_OPTIONAL = buildParser(false, false); - public static final List> ALL_PARSERS = unmodifiableList( - Arrays.asList(PARSER, PARSER_VEGETABLE_OPTIONAL, PARSER_ALL_OPTIONAL)); + public static final List> ALL_PARSERS = + List.of(PARSER, PARSER_VEGETABLE_OPTIONAL, PARSER_ALL_OPTIONAL); public static final ConstructingObjectParser PARSER_INT_CONTEXT = buildContextParser(); diff --git a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/IngestCommonPlugin.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/IngestCommonPlugin.java index a839e147c77ad..c3b1328d0b276 100644 --- a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/IngestCommonPlugin.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/IngestCommonPlugin.java @@ -42,11 +42,12 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; +import static java.util.Map.entry; + public class IngestCommonPlugin extends Plugin implements ActionPlugin, IngestPlugin { static final Map GROK_PATTERNS = Grok.getBuiltinPatterns(); @@ -60,39 +61,39 @@ public IngestCommonPlugin() { @Override public Map getProcessors(Processor.Parameters parameters) { - Map processors = new HashMap<>(); - processors.put(DateProcessor.TYPE, new DateProcessor.Factory(parameters.scriptService)); - processors.put(SetProcessor.TYPE, new SetProcessor.Factory(parameters.scriptService)); - processors.put(AppendProcessor.TYPE, new AppendProcessor.Factory(parameters.scriptService)); - processors.put(RenameProcessor.TYPE, new RenameProcessor.Factory(parameters.scriptService)); - processors.put(RemoveProcessor.TYPE, new RemoveProcessor.Factory(parameters.scriptService)); - processors.put(SplitProcessor.TYPE, new SplitProcessor.Factory()); - processors.put(JoinProcessor.TYPE, new JoinProcessor.Factory()); - processors.put(UppercaseProcessor.TYPE, new UppercaseProcessor.Factory()); - processors.put(LowercaseProcessor.TYPE, new LowercaseProcessor.Factory()); - processors.put(TrimProcessor.TYPE, new TrimProcessor.Factory()); - processors.put(ConvertProcessor.TYPE, new ConvertProcessor.Factory()); - processors.put(GsubProcessor.TYPE, new GsubProcessor.Factory()); - processors.put(FailProcessor.TYPE, new FailProcessor.Factory(parameters.scriptService)); - processors.put(ForEachProcessor.TYPE, new ForEachProcessor.Factory(parameters.scriptService)); - processors.put(DateIndexNameProcessor.TYPE, new DateIndexNameProcessor.Factory(parameters.scriptService)); - processors.put(SortProcessor.TYPE, new SortProcessor.Factory()); - processors.put(GrokProcessor.TYPE, new GrokProcessor.Factory(GROK_PATTERNS, createGrokThreadWatchdog(parameters))); - processors.put(ScriptProcessor.TYPE, new ScriptProcessor.Factory(parameters.scriptService)); - processors.put(DotExpanderProcessor.TYPE, new DotExpanderProcessor.Factory()); - processors.put(JsonProcessor.TYPE, new JsonProcessor.Factory()); - processors.put(KeyValueProcessor.TYPE, new KeyValueProcessor.Factory()); - processors.put(URLDecodeProcessor.TYPE, new URLDecodeProcessor.Factory()); - processors.put(BytesProcessor.TYPE, new BytesProcessor.Factory()); - processors.put(PipelineProcessor.TYPE, new PipelineProcessor.Factory(parameters.ingestService)); - processors.put(DissectProcessor.TYPE, new DissectProcessor.Factory()); - processors.put(DropProcessor.TYPE, new DropProcessor.Factory()); - return Collections.unmodifiableMap(processors); + return Map.ofEntries( + entry(DateProcessor.TYPE, new DateProcessor.Factory(parameters.scriptService)), + entry(SetProcessor.TYPE, new SetProcessor.Factory(parameters.scriptService)), + entry(AppendProcessor.TYPE, new AppendProcessor.Factory(parameters.scriptService)), + entry(RenameProcessor.TYPE, new RenameProcessor.Factory(parameters.scriptService)), + entry(RemoveProcessor.TYPE, new RemoveProcessor.Factory(parameters.scriptService)), + entry(SplitProcessor.TYPE, new SplitProcessor.Factory()), + entry(JoinProcessor.TYPE, new JoinProcessor.Factory()), + entry(UppercaseProcessor.TYPE, new UppercaseProcessor.Factory()), + entry(LowercaseProcessor.TYPE, new LowercaseProcessor.Factory()), + entry(TrimProcessor.TYPE, new TrimProcessor.Factory()), + entry(ConvertProcessor.TYPE, new ConvertProcessor.Factory()), + entry(GsubProcessor.TYPE, new GsubProcessor.Factory()), + entry(FailProcessor.TYPE, new FailProcessor.Factory(parameters.scriptService)), + entry(ForEachProcessor.TYPE, new ForEachProcessor.Factory(parameters.scriptService)), + entry(DateIndexNameProcessor.TYPE, new DateIndexNameProcessor.Factory(parameters.scriptService)), + entry(SortProcessor.TYPE, new SortProcessor.Factory()), + entry(GrokProcessor.TYPE, new GrokProcessor.Factory(GROK_PATTERNS, createGrokThreadWatchdog(parameters))), + entry(ScriptProcessor.TYPE, new ScriptProcessor.Factory(parameters.scriptService)), + entry(DotExpanderProcessor.TYPE, new DotExpanderProcessor.Factory()), + entry(JsonProcessor.TYPE, new JsonProcessor.Factory()), + entry(KeyValueProcessor.TYPE, new KeyValueProcessor.Factory()), + entry(URLDecodeProcessor.TYPE, new URLDecodeProcessor.Factory()), + entry(BytesProcessor.TYPE, new BytesProcessor.Factory()), + entry(PipelineProcessor.TYPE, new PipelineProcessor.Factory(parameters.ingestService)), + entry(DissectProcessor.TYPE, new DissectProcessor.Factory()), + entry(DropProcessor.TYPE, new DropProcessor.Factory())); } @Override public List> getActions() { - return Arrays.asList(new ActionHandler<>(GrokProcessorGetAction.INSTANCE, GrokProcessorGetAction.TransportAction.class)); + return Collections.singletonList( + new ActionHandler<>(GrokProcessorGetAction.INSTANCE, GrokProcessorGetAction.TransportAction.class)); } @Override diff --git a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/ScriptProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/ScriptProcessor.java index fcc6c68014207..ee8d41c37c2f3 100644 --- a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/ScriptProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/ScriptProcessor.java @@ -39,7 +39,6 @@ import java.io.InputStream; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import static org.elasticsearch.ingest.ConfigurationUtils.newConfigurationException; @@ -49,15 +48,8 @@ */ public final class ScriptProcessor extends AbstractProcessor { - private static final Map DEPRECATIONS; - static { - Map deprecations = new HashMap<>(); - deprecations.put( - "_type", - "[types removal] Looking up doc types [_type] in scripts is deprecated." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + private static final Map DEPRECATIONS = + Collections.singletonMap("_type", "[types removal] Looking up doc types [_type] in scripts is deprecated."); public static final String TYPE = "script"; diff --git a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/ScriptProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/ScriptProcessorFactoryTests.java index d7f8e8838bb8c..9559c150df68a 100644 --- a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/ScriptProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/ScriptProcessorFactoryTests.java @@ -41,13 +41,9 @@ public class ScriptProcessorFactoryTests extends ESTestCase { private ScriptProcessor.Factory factory; - private static final Map ingestScriptParamToType; - static { - Map map = new HashMap<>(); - map.put("id", "stored"); - map.put("source", "inline"); - ingestScriptParamToType = Collections.unmodifiableMap(map); - } + private static final Map INGEST_SCRIPT_PARAM_TO_TYPE = Map.of( + "id", "stored", + "source", "inline"); @Before public void init() { @@ -60,7 +56,7 @@ public void testFactoryValidationWithDefaultLang() throws Exception { configMap.put(randomType, "foo"); ScriptProcessor processor = factory.create(null, randomAlphaOfLength(10), configMap); assertThat(processor.getScript().getLang(), equalTo(randomType.equals("id") ? null : Script.DEFAULT_SCRIPT_LANG)); - assertThat(processor.getScript().getType().toString(), equalTo(ingestScriptParamToType.get(randomType))); + assertThat(processor.getScript().getType().toString(), equalTo(INGEST_SCRIPT_PARAM_TO_TYPE.get(randomType))); assertThat(processor.getScript().getParams(), equalTo(Collections.emptyMap())); } @@ -72,7 +68,7 @@ public void testFactoryValidationWithParams() throws Exception { configMap.put("params", randomParams); ScriptProcessor processor = factory.create(null, randomAlphaOfLength(10), configMap); assertThat(processor.getScript().getLang(), equalTo(randomType.equals("id") ? null : Script.DEFAULT_SCRIPT_LANG)); - assertThat(processor.getScript().getType().toString(), equalTo(ingestScriptParamToType.get(randomType))); + assertThat(processor.getScript().getType().toString(), equalTo(INGEST_SCRIPT_PARAM_TO_TYPE.get(randomType))); assertThat(processor.getScript().getParams(), equalTo(randomParams)); } diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/CustomMustacheFactory.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/CustomMustacheFactory.java index 04835bed116d4..9750f3918f0d6 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/CustomMustacheFactory.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/CustomMustacheFactory.java @@ -30,7 +30,7 @@ import com.github.mustachejava.codes.DefaultMustache; import com.github.mustachejava.codes.IterableCode; import com.github.mustachejava.codes.WriteCode; - +import org.apache.lucene.search.highlight.DefaultEncoder; import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; @@ -41,7 +41,6 @@ import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -60,15 +59,11 @@ public class CustomMustacheFactory extends DefaultMustacheFactory { private static final String DEFAULT_MIME_TYPE = JSON_MIME_TYPE; - private static final Map> ENCODERS; - static { - Map> encoders = new HashMap<>(); - encoders.put(JSON_MIME_TYPE_WITH_CHARSET, JsonEscapeEncoder::new); - encoders.put(JSON_MIME_TYPE, JsonEscapeEncoder::new); - encoders.put(PLAIN_TEXT_MIME_TYPE, DefaultEncoder::new); - encoders.put(X_WWW_FORM_URLENCODED_MIME_TYPE, UrlEncoder::new); - ENCODERS = Collections.unmodifiableMap(encoders); - } + private static final Map> ENCODERS = Map.of( + JSON_MIME_TYPE_WITH_CHARSET, JsonEscapeEncoder::new, + JSON_MIME_TYPE, JsonEscapeEncoder::new, + PLAIN_TEXT_MIME_TYPE, DefaultEncoder::new, + X_WWW_FORM_URLENCODED_MIME_TYPE, UrlEncoder::new); private final Encoder encoder; @@ -92,7 +87,7 @@ public void encode(String value, Writer writer) { } static Encoder createEncoder(String mimeType) { - Supplier supplier = ENCODERS.get(mimeType); + final Supplier supplier = ENCODERS.get(mimeType); if (supplier == null) { throw new IllegalArgumentException("No encoder found for MIME type [" + mimeType + "]"); } diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java index 28c536d788e6a..f5465f256c225 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/script/mustache/CustomMustacheFactoryTests.java @@ -37,14 +37,22 @@ public class CustomMustacheFactoryTests extends ESTestCase { public void testCreateEncoder() { - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> CustomMustacheFactory.createEncoder(null)); - assertThat(e.getMessage(), equalTo("No encoder found for MIME type [null]")); - - e = expectThrows(IllegalArgumentException.class, () -> CustomMustacheFactory.createEncoder("")); - assertThat(e.getMessage(), equalTo("No encoder found for MIME type []")); - - e = expectThrows(IllegalArgumentException.class, () -> CustomMustacheFactory.createEncoder("test")); - assertThat(e.getMessage(), equalTo("No encoder found for MIME type [test]")); + { + final IllegalArgumentException e = + expectThrows(IllegalArgumentException.class, () -> CustomMustacheFactory.createEncoder("non-existent")); + assertThat(e.getMessage(), equalTo("No encoder found for MIME type [non-existent]")); + } + + { + final IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> CustomMustacheFactory.createEncoder("")); + assertThat(e.getMessage(), equalTo("No encoder found for MIME type []")); + } + + { + final IllegalArgumentException e = + expectThrows(IllegalArgumentException.class, () -> CustomMustacheFactory.createEncoder("test")); + assertThat(e.getMessage(), equalTo("No encoder found for MIME type [test]")); + } assertThat(CustomMustacheFactory.createEncoder(CustomMustacheFactory.JSON_MIME_TYPE_WITH_CHARSET), instanceOf(CustomMustacheFactory.JsonEscapeEncoder.class)); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Locals.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Locals.java index 85cdfa9f915b4..121111f8c317c 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Locals.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Locals.java @@ -24,11 +24,9 @@ import org.elasticsearch.painless.lookup.PainlessLookupUtility; import java.lang.invoke.MethodType; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -71,9 +69,7 @@ public LocalMethod(String name, Class returnType, List> typeParamete public static final String THIS = "#this"; /** Set of reserved keywords. */ - public static final Set KEYWORDS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - THIS, LOOP - ))); + public static final Set KEYWORDS = Set.of(THIS, LOOP); /** Creates a new local variable scope (e.g. loop) inside the current scope */ public static Locals newLocalScope(Locals currentScope) { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java index e8d93b8ef772e..217b5fc76588a 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java @@ -83,8 +83,6 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -120,15 +118,10 @@ public static class Request extends SingleShardRequest implements ToXCo PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(), ContextSetup::parse, CONTEXT_SETUP_FIELD); } - static final Map> SUPPORTED_CONTEXTS; - - static { - Map> supportedContexts = new HashMap<>(); - supportedContexts.put("painless_test", PainlessTestScript.CONTEXT); - supportedContexts.put("filter", FilterScript.CONTEXT); - supportedContexts.put("score", ScoreScript.CONTEXT); - SUPPORTED_CONTEXTS = Collections.unmodifiableMap(supportedContexts); - } + static final Map> SUPPORTED_CONTEXTS = Map.of( + "painless_test", PainlessTestScript.CONTEXT, + "filter", FilterScript.CONTEXT, + "score", ScoreScript.CONTEXT); static ScriptContext fromScriptContextName(String name) { ScriptContext scriptContext = SUPPORTED_CONTEXTS.get(name); diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/plugin/store/smb/SMBStorePlugin.java b/plugins/store-smb/src/main/java/org/elasticsearch/plugin/store/smb/SMBStorePlugin.java index a2b0f46345231..111100a2f1580 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/plugin/store/smb/SMBStorePlugin.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/plugin/store/smb/SMBStorePlugin.java @@ -26,8 +26,6 @@ import org.elasticsearch.plugins.IndexStorePlugin; import org.elasticsearch.plugins.Plugin; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.function.Function; @@ -35,10 +33,9 @@ public class SMBStorePlugin extends Plugin implements IndexStorePlugin { @Override public Map> getIndexStoreFactories() { - final Map> indexStoreFactories = new HashMap<>(2); - indexStoreFactories.put("smb_mmap_fs", SmbMmapFsIndexStore::new); - indexStoreFactories.put("smb_simple_fs", SmbSimpleFsIndexStore::new); - return Collections.unmodifiableMap(indexStoreFactories); + return Map.of( + "smb_mmap_fs", SmbMmapFsIndexStore::new, + "smb_simple_fs", SmbSimpleFsIndexStore::new); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java index 166a59f235902..557338ed0f77e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java @@ -61,7 +61,7 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContentOb public ListTasksResponse(List tasks, List taskFailures, List nodeFailures) { super(taskFailures, nodeFailures); - this.tasks = tasks == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(tasks)); + this.tasks = tasks == null ? Collections.emptyList() : List.copyOf(tasks); } public ListTasksResponse(StreamInput in) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskGroup.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskGroup.java index c0a0930aaaf34..78ffadd791e63 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskGroup.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TaskGroup.java @@ -19,14 +19,12 @@ package org.elasticsearch.action.admin.cluster.node.tasks.list; -import org.elasticsearch.common.xcontent.ToXContent.Params; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.TaskInfo; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -42,7 +40,7 @@ public class TaskGroup implements ToXContentObject { public TaskGroup(TaskInfo task, List childTasks) { this.task = task; - this.childTasks = Collections.unmodifiableList(new ArrayList<>(childTasks)); + this.childTasks = List.copyOf(childTasks); } public static Builder builder(TaskInfo taskInfo) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoResponse.java index b8c2c29a0c8f9..fbb94069342b8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoResponse.java @@ -20,16 +20,14 @@ package org.elasticsearch.action.admin.cluster.remote; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.transport.RemoteConnectionInfo; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.transport.RemoteConnectionInfo; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; public final class RemoteInfoResponse extends ActionResponse implements ToXContentObject { @@ -40,7 +38,7 @@ public final class RemoteInfoResponse extends ActionResponse implements ToXConte } RemoteInfoResponse(Collection infos) { - this.infos = Collections.unmodifiableList(new ArrayList<>(infos)); + this.infos = List.copyOf(infos); } public List getInfos() { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java index edd59f8b18ca1..c36d6092a7e49 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverAction.java @@ -50,7 +50,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Locale; @@ -58,8 +57,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -import static java.util.Collections.unmodifiableList; - /** * Main class to swap the index pointed to by an alias, given some conditions */ @@ -200,24 +197,22 @@ public void onFailure(Exception e) { static IndicesAliasesClusterStateUpdateRequest prepareRolloverAliasesUpdateRequest(String oldIndex, String newIndex, RolloverRequest request) { - List actions = unmodifiableList(Arrays.asList( - new AliasAction.Add(newIndex, request.getAlias(), null, null, null, null), - new AliasAction.Remove(oldIndex, request.getAlias()))); - final IndicesAliasesClusterStateUpdateRequest updateRequest = new IndicesAliasesClusterStateUpdateRequest(actions) + final List actions = List.of( + new AliasAction.Add(newIndex, request.getAlias(), null, null, null, null), + new AliasAction.Remove(oldIndex, request.getAlias())); + return new IndicesAliasesClusterStateUpdateRequest(actions) .ackTimeout(request.ackTimeout()) .masterNodeTimeout(request.masterNodeTimeout()); - return updateRequest; } static IndicesAliasesClusterStateUpdateRequest prepareRolloverAliasesWriteIndexUpdateRequest(String oldIndex, String newIndex, RolloverRequest request) { - List actions = unmodifiableList(Arrays.asList( - new AliasAction.Add(newIndex, request.getAlias(), null, null, null, true), - new AliasAction.Add(oldIndex, request.getAlias(), null, null, null, false))); - final IndicesAliasesClusterStateUpdateRequest updateRequest = new IndicesAliasesClusterStateUpdateRequest(actions) + final List actions = List.of( + new AliasAction.Add(newIndex, request.getAlias(), null, null, null, true), + new AliasAction.Add(oldIndex, request.getAlias(), null, null, null, false)); + return new IndicesAliasesClusterStateUpdateRequest(actions) .ackTimeout(request.ackTimeout()) .masterNodeTimeout(request.masterNodeTimeout()); - return updateRequest; } diff --git a/server/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java b/server/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java index d859959f2e661..8c1980bae8af7 100644 --- a/server/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java +++ b/server/src/main/java/org/elasticsearch/action/support/tasks/BaseTasksResponse.java @@ -51,8 +51,8 @@ public class BaseTasksResponse extends ActionResponse { private List nodeFailures; public BaseTasksResponse(List taskFailures, List nodeFailures) { - this.taskFailures = taskFailures == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(taskFailures)); - this.nodeFailures = nodeFailures == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(nodeFailures)); + this.taskFailures = taskFailures == null ? Collections.emptyList() : List.copyOf(taskFailures); + this.nodeFailures = nodeFailures == null ? Collections.emptyList() : List.copyOf(nodeFailures); } public BaseTasksResponse(StreamInput in) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/bootstrap/SystemCallFilter.java b/server/src/main/java/org/elasticsearch/bootstrap/SystemCallFilter.java index 59f8bd5daf77b..f1536fe944e7f 100644 --- a/server/src/main/java/org/elasticsearch/bootstrap/SystemCallFilter.java +++ b/server/src/main/java/org/elasticsearch/bootstrap/SystemCallFilter.java @@ -26,7 +26,6 @@ import com.sun.jna.Pointer; import com.sun.jna.Structure; import com.sun.jna.ptr.PointerByReference; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.Constants; @@ -39,7 +38,6 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -240,10 +238,9 @@ static class Arch { /** supported architectures map keyed by os.arch */ private static final Map ARCHITECTURES; static { - Map m = new HashMap<>(); - m.put("amd64", new Arch(0xC000003E, 0x3FFFFFFF, 57, 58, 59, 322, 317)); - m.put("aarch64", new Arch(0xC00000B7, 0xFFFFFFFF, 1079, 1071, 221, 281, 277)); - ARCHITECTURES = Collections.unmodifiableMap(m); + ARCHITECTURES = Map.of( + "amd64", new Arch(0xC000003E, 0x3FFFFFFF, 57, 58, 59, 322, 317), + "aarch64", new Arch(0xC00000B7, 0xFFFFFFFF, 1079, 1071, 221, 281, 277)); } /** invokes prctl() from linux libc library */ diff --git a/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java index d442069b57f93..9d28ea0e5bc16 100644 --- a/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java +++ b/server/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java @@ -23,8 +23,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; @@ -37,11 +35,13 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; @@ -383,7 +383,7 @@ List establishNodeConnections(Set nodes) { } } - return Collections.unmodifiableList(new ArrayList<>(nodes)); + return List.copyOf(nodes); } } @@ -566,7 +566,7 @@ public void handleException(TransportException e) { } nodes = establishNodeConnections(newNodes); - filteredNodes = Collections.unmodifiableList(new ArrayList<>(newFilteredNodes)); + filteredNodes = List.copyOf(newFilteredNodes); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/DiffableUtils.java b/server/src/main/java/org/elasticsearch/cluster/DiffableUtils.java index 725da675952f2..112856f7490d0 100644 --- a/server/src/main/java/org/elasticsearch/cluster/DiffableUtils.java +++ b/server/src/main/java/org/elasticsearch/cluster/DiffableUtils.java @@ -32,10 +32,7 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -720,7 +717,7 @@ public void write(Set value, StreamOutput out) throws IOException { @Override public Set read(StreamInput in, K key) throws IOException { - return Collections.unmodifiableSet(new HashSet<>(Arrays.asList(in.readStringArray()))); + return Set.of(in.readStringArray()); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java index c46bc291e7397..3b39d283befde 100644 --- a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -445,9 +445,9 @@ public ClusterBlocks build() { // We copy the block sets here in case of the builder is modified after build is called ImmutableOpenMap.Builder> indicesBuilder = ImmutableOpenMap.builder(indices.size()); for (Map.Entry> entry : indices.entrySet()) { - indicesBuilder.put(entry.getKey(), unmodifiableSet(new HashSet<>(entry.getValue()))); + indicesBuilder.put(entry.getKey(), Set.copyOf(entry.getValue())); } - return new ClusterBlocks(unmodifiableSet(new HashSet<>(global)), indicesBuilder.build()); + return new ClusterBlocks(Set.copyOf(global), indicesBuilder.build()); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java index b63cb07feff99..244ed105257bd 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java @@ -94,7 +94,7 @@ public CoordinationMetaData(long term, VotingConfiguration lastCommittedConfigur this.term = term; this.lastCommittedConfiguration = lastCommittedConfiguration; this.lastAcceptedConfiguration = lastAcceptedConfiguration; - this.votingConfigExclusions = Collections.unmodifiableSet(new HashSet<>(votingConfigExclusions)); + this.votingConfigExclusions = Set.copyOf(votingConfigExclusions); } public CoordinationMetaData(StreamInput in) throws IOException { @@ -331,7 +331,7 @@ public static class VotingConfiguration implements Writeable, ToXContentFragment private final Set nodeIds; public VotingConfiguration(Set nodeIds) { - this.nodeIds = Collections.unmodifiableSet(new HashSet<>(nodeIds)); + this.nodeIds = Set.copyOf(nodeIds); } public VotingConfiguration(StreamInput in) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 661cbb1bf8180..5af1dc9b508ac 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -1137,7 +1137,7 @@ public IndexMetaData build() { ImmutableOpenIntMap.Builder> filledInSyncAllocationIds = ImmutableOpenIntMap.builder(); for (int i = 0; i < numberOfShards; i++) { if (inSyncAllocationIds.containsKey(i)) { - filledInSyncAllocationIds.put(i, Collections.unmodifiableSet(new HashSet<>(inSyncAllocationIds.get(i)))); + filledInSyncAllocationIds.put(i, Set.copyOf(inSyncAllocationIds.get(i))); } else { filledInSyncAllocationIds.put(i, Collections.emptySet()); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 19c6d31ccc82a..ee3c2370b46f2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -57,14 +57,10 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import static java.util.Collections.unmodifiableList; - public class IndexNameExpressionResolver { private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); - private final List expressionResolvers = unmodifiableList(Arrays.asList( - dateMathExpressionResolver, - new WildcardExpressionResolver())); + private final List expressionResolvers = List.of(dateMathExpressionResolver, new WildcardExpressionResolver()); /** * Same as {@link #concreteIndexNames(ClusterState, IndicesOptions, String...)}, but the index expressions and options @@ -336,7 +332,7 @@ public Set resolveExpressions(ClusterState state, String... expressions) for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); } - return Collections.unmodifiableSet(new HashSet<>(resolvedExpressions)); + return Set.copyOf(resolvedExpressions); } /** diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java index 1c618c1ef88fd..4f182b6ca381e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java @@ -33,7 +33,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; import java.util.List; @@ -114,7 +113,7 @@ public RepositoriesMetaData(StreamInput in) throws IOException { for (int i = 0; i < repository.length; i++) { repository[i] = new RepositoryMetaData(in); } - this.repositories = Collections.unmodifiableList(Arrays.asList(repository)); + this.repositories = List.of(repository); } public static NamedDiff readDiffFrom(StreamInput in) throws IOException { 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 c35b7d810ca0e..572cf496114aa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -691,7 +691,7 @@ public Builder removeShard(ShardRouting shardEntry) { public IndexShardRoutingTable build() { // don't allow more than one shard copy with same id to be allocated to same node assert distinctNodes(shards) : "more than one shard with same id assigned to same node (shards: " + shards + ")"; - return new IndexShardRoutingTable(shardId, Collections.unmodifiableList(new ArrayList<>(shards))); + return new IndexShardRoutingTable(shardId, List.copyOf(shards)); } static boolean distinctNodes(List shards) { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java index 059b13c5a1bd4..d49869d030eb7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java @@ -40,7 +40,6 @@ import java.util.Set; import static java.util.Collections.emptySet; -import static java.util.Collections.unmodifiableSet; /** * The {@link RoutingAllocation} keep the state of the current allocation @@ -215,7 +214,7 @@ public Set getIgnoreNodes(ShardId shardId) { if (ignore == null) { return emptySet(); } - return unmodifiableSet(new HashSet<>(ignore)); + return Set.copyOf(ignore); } /** diff --git a/server/src/main/java/org/elasticsearch/common/collect/MapBuilder.java b/server/src/main/java/org/elasticsearch/common/collect/MapBuilder.java index a840ee037f80e..abf27b662bb7c 100644 --- a/server/src/main/java/org/elasticsearch/common/collect/MapBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/collect/MapBuilder.java @@ -22,8 +22,7 @@ import java.util.HashMap; import java.util.Map; -import static java.util.Collections.unmodifiableMap; - +// TODO: replace with usages of Map.of and Map.ofEntries public class MapBuilder { public static MapBuilder newMapBuilder() { @@ -86,6 +85,6 @@ public Map map() { */ public Map immutableMap() { // TODO: follow the directions in the Javadoc for this method - return unmodifiableMap(new HashMap<>(map)); + return Map.copyOf(map); } } diff --git a/server/src/main/java/org/elasticsearch/common/inject/internal/BindingBuilder.java b/server/src/main/java/org/elasticsearch/common/inject/internal/BindingBuilder.java index 5aa84dea58f4d..35ad7e0e8181b 100644 --- a/server/src/main/java/org/elasticsearch/common/inject/internal/BindingBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/inject/internal/BindingBuilder.java @@ -27,13 +27,11 @@ import org.elasticsearch.common.inject.spi.Message; import java.lang.annotation.Annotation; -import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Set; import static java.util.Collections.emptySet; -import static java.util.Collections.unmodifiableSet; /** * Bind a non-constant key. @@ -92,7 +90,7 @@ public void toInstance(T instance) { for (Message message : e.getErrorMessages()) { binder.addError(message); } - injectionPoints = unmodifiableSet(new HashSet(e.getPartialValue())); + injectionPoints = Set.copyOf(e.getPartialValue()); } } else { binder.addError(BINDING_TO_NULL); @@ -117,7 +115,7 @@ public BindingBuilder toProvider(Provider provider) { for (Message message : e.getErrorMessages()) { binder.addError(message); } - injectionPoints = unmodifiableSet(new HashSet(e.getPartialValue())); + injectionPoints = Set.copyOf(e.getPartialValue()); } BindingImpl base = getBinding(); diff --git a/server/src/main/java/org/elasticsearch/common/inject/internal/InstanceBindingImpl.java b/server/src/main/java/org/elasticsearch/common/inject/internal/InstanceBindingImpl.java index 7b74e6483a3ab..d3c45d8c3df4e 100644 --- a/server/src/main/java/org/elasticsearch/common/inject/internal/InstanceBindingImpl.java +++ b/server/src/main/java/org/elasticsearch/common/inject/internal/InstanceBindingImpl.java @@ -27,11 +27,8 @@ import org.elasticsearch.common.inject.spi.InstanceBinding; import org.elasticsearch.common.inject.util.Providers; -import java.util.HashSet; import java.util.Set; -import static java.util.Collections.unmodifiableSet; - public class InstanceBindingImpl extends BindingImpl implements InstanceBinding { final T instance; @@ -78,7 +75,7 @@ public Set getInjectionPoints() { @Override public Set> getDependencies() { return instance instanceof HasDependencies - ? unmodifiableSet(new HashSet<>((((HasDependencies) instance).getDependencies()))) + ? Set.copyOf(((HasDependencies) instance).getDependencies()) : Dependency.forInjectionPoints(injectionPoints); } diff --git a/server/src/main/java/org/elasticsearch/common/inject/internal/ProviderInstanceBindingImpl.java b/server/src/main/java/org/elasticsearch/common/inject/internal/ProviderInstanceBindingImpl.java index e95d8fe491b69..cab8a8b8088c8 100644 --- a/server/src/main/java/org/elasticsearch/common/inject/internal/ProviderInstanceBindingImpl.java +++ b/server/src/main/java/org/elasticsearch/common/inject/internal/ProviderInstanceBindingImpl.java @@ -26,11 +26,8 @@ import org.elasticsearch.common.inject.spi.InjectionPoint; import org.elasticsearch.common.inject.spi.ProviderInstanceBinding; -import java.util.HashSet; import java.util.Set; -import static java.util.Collections.unmodifiableSet; - public final class ProviderInstanceBindingImpl extends BindingImpl implements ProviderInstanceBinding { @@ -71,7 +68,7 @@ public Set getInjectionPoints() { @Override public Set> getDependencies() { return providerInstance instanceof HasDependencies - ? unmodifiableSet(new HashSet<>((((HasDependencies) providerInstance).getDependencies()))) + ? Set.copyOf(((HasDependencies) providerInstance).getDependencies()) : Dependency.forInjectionPoints(injectionPoints); } diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java index 1c9dfd7ea4433..431e00fcf063a 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java @@ -55,16 +55,14 @@ import java.nio.file.FileSystemLoopException; import java.nio.file.NoSuchFileException; import java.nio.file.NotDirectoryException; -import java.time.ZoneId; import java.time.Instant; +import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Collection; import java.util.Collections; import java.util.Date; import java.util.EnumMap; import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -73,6 +71,8 @@ import java.util.concurrent.TimeUnit; import java.util.function.IntFunction; +import static java.util.Map.entry; + /** * A stream from another node to this node. Technically, it can also be streamed from a byte array but that is mostly for testing. * @@ -140,7 +140,7 @@ public boolean hasFeature(final String feature) { */ public void setFeatures(final Set features) { assert this.features.isEmpty() : this.features; - this.features = Collections.unmodifiableSet(new HashSet<>(features)); + this.features = Set.copyOf(features); } public long position() throws IOException { @@ -594,137 +594,179 @@ public final void writeOptionalInstant(@Nullable Instant instant) throws IOExcep } } - private static final Map, Writer> WRITERS; - - static { - Map, Writer> writers = new HashMap<>(); - writers.put(String.class, (o, v) -> { - o.writeByte((byte) 0); - o.writeString((String) v); - }); - writers.put(Integer.class, (o, v) -> { - o.writeByte((byte) 1); - o.writeInt((Integer) v); - }); - writers.put(Long.class, (o, v) -> { - o.writeByte((byte) 2); - o.writeLong((Long) v); - }); - writers.put(Float.class, (o, v) -> { - o.writeByte((byte) 3); - o.writeFloat((float) v); - }); - writers.put(Double.class, (o, v) -> { - o.writeByte((byte) 4); - o.writeDouble((double) v); - }); - writers.put(Boolean.class, (o, v) -> { - o.writeByte((byte) 5); - o.writeBoolean((boolean) v); - }); - writers.put(byte[].class, (o, v) -> { - o.writeByte((byte) 6); - final byte[] bytes = (byte[]) v; - o.writeVInt(bytes.length); - o.writeBytes(bytes); - }); - writers.put(List.class, (o, v) -> { - o.writeByte((byte) 7); - final List list = (List) v; - o.writeVInt(list.size()); - for (Object item : list) { - o.writeGenericValue(item); - } - }); - writers.put(Object[].class, (o, v) -> { - o.writeByte((byte) 8); - final Object[] list = (Object[]) v; - o.writeVInt(list.length); - for (Object item : list) { - o.writeGenericValue(item); - } - }); - writers.put(Map.class, (o, v) -> { - if (v instanceof LinkedHashMap) { - o.writeByte((byte) 9); - } else { - o.writeByte((byte) 10); - } - @SuppressWarnings("unchecked") - final Map map = (Map) v; - o.writeVInt(map.size()); - for (Map.Entry entry : map.entrySet()) { - o.writeString(entry.getKey()); - o.writeGenericValue(entry.getValue()); - } - }); - writers.put(Byte.class, (o, v) -> { - o.writeByte((byte) 11); - o.writeByte((Byte) v); - }); - writers.put(Date.class, (o, v) -> { - o.writeByte((byte) 12); - o.writeLong(((Date) v).getTime()); - }); - writers.put(ReadableInstant.class, (o, v) -> { - o.writeByte((byte) 13); - final ReadableInstant instant = (ReadableInstant) v; - o.writeString(instant.getZone().getID()); - o.writeLong(instant.getMillis()); - }); - writers.put(BytesReference.class, (o, v) -> { - o.writeByte((byte) 14); - o.writeBytesReference((BytesReference) v); - }); - writers.put(Text.class, (o, v) -> { - o.writeByte((byte) 15); - o.writeText((Text) v); - }); - writers.put(Short.class, (o, v) -> { - o.writeByte((byte) 16); - o.writeShort((Short) v); - }); - writers.put(int[].class, (o, v) -> { - o.writeByte((byte) 17); - o.writeIntArray((int[]) v); - }); - writers.put(long[].class, (o, v) -> { - o.writeByte((byte) 18); - o.writeLongArray((long[]) v); - }); - writers.put(float[].class, (o, v) -> { - o.writeByte((byte) 19); - o.writeFloatArray((float[]) v); - }); - writers.put(double[].class, (o, v) -> { - o.writeByte((byte) 20); - o.writeDoubleArray((double[]) v); - }); - writers.put(BytesRef.class, (o, v) -> { - o.writeByte((byte) 21); - o.writeBytesRef((BytesRef) v); - }); - writers.put(GeoPoint.class, (o, v) -> { - o.writeByte((byte) 22); - o.writeGeoPoint((GeoPoint) v); - }); - writers.put(ZonedDateTime.class, (o, v) -> { - o.writeByte((byte) 23); - final ZonedDateTime zonedDateTime = (ZonedDateTime) v; - o.writeString(zonedDateTime.getZone().getId()); - o.writeLong(zonedDateTime.toInstant().toEpochMilli()); - }); - writers.put(JodaCompatibleZonedDateTime.class, (o, v) -> { - // write the joda compatibility datetime as joda datetime - o.writeByte((byte) 13); - final JodaCompatibleZonedDateTime zonedDateTime = (JodaCompatibleZonedDateTime) v; - String zoneId = zonedDateTime.getZonedDateTime().getZone().getId(); - // joda does not understand "Z" for utc, so we must special case - o.writeString(zoneId.equals("Z") ? DateTimeZone.UTC.getID() : zoneId); - o.writeLong(zonedDateTime.toInstant().toEpochMilli()); - }); - WRITERS = Collections.unmodifiableMap(writers); - } + private static final Map, Writer> WRITERS = Map.ofEntries( + entry( + String.class, + (o, v) -> { + o.writeByte((byte) 0); + o.writeString((String) v); + }), + entry( + Integer.class, + (o, v) -> { + o.writeByte((byte) 1); + o.writeInt((Integer) v); + }), + entry( + Long.class, + (o, v) -> { + o.writeByte((byte) 2); + o.writeLong((Long) v); + }), + entry( + Float.class, + (o, v) -> { + o.writeByte((byte) 3); + o.writeFloat((float) v); + }), + entry( + Double.class, + (o, v) -> { + o.writeByte((byte) 4); + o.writeDouble((double) v); + }), + entry( + Boolean.class, + (o, v) -> { + o.writeByte((byte) 5); + o.writeBoolean((boolean) v); + }), + entry( + byte[].class, + (o, v) -> { + o.writeByte((byte) 6); + final byte[] bytes = (byte[]) v; + o.writeVInt(bytes.length); + o.writeBytes(bytes); + }), + entry( + List.class, + (o, v) -> { + o.writeByte((byte) 7); + final List list = (List) v; + o.writeVInt(list.size()); + for (Object item : list) { + o.writeGenericValue(item); + } + }), + entry( + Object[].class, + (o, v) -> { + o.writeByte((byte) 8); + final Object[] list = (Object[]) v; + o.writeVInt(list.length); + for (Object item : list) { + o.writeGenericValue(item); + } + }), + entry( + Map.class, + (o, v) -> { + if (v instanceof LinkedHashMap) { + o.writeByte((byte) 9); + } else { + o.writeByte((byte) 10); + } + @SuppressWarnings("unchecked") final Map map = (Map) v; + o.writeVInt(map.size()); + for (Map.Entry entry : map.entrySet()) { + o.writeString(entry.getKey()); + o.writeGenericValue(entry.getValue()); + } + }), + entry( + Byte.class, + (o, v) -> { + o.writeByte((byte) 11); + o.writeByte((Byte) v); + }), + entry( + Date.class, + (o, v) -> { + o.writeByte((byte) 12); + o.writeLong(((Date) v).getTime()); + }), + entry( + ReadableInstant.class, + (o, v) -> { + o.writeByte((byte) 13); + final ReadableInstant instant = (ReadableInstant) v; + o.writeString(instant.getZone().getID()); + o.writeLong(instant.getMillis()); + }), + entry( + BytesReference.class, + (o, v) -> { + o.writeByte((byte) 14); + o.writeBytesReference((BytesReference) v); + }), + entry( + Text.class, + (o, v) -> { + o.writeByte((byte) 15); + o.writeText((Text) v); + }), + entry( + Short.class, + (o, v) -> { + o.writeByte((byte) 16); + o.writeShort((Short) v); + }), + entry( + int[].class, + (o, v) -> { + o.writeByte((byte) 17); + o.writeIntArray((int[]) v); + }), + entry( + long[].class, + (o, v) -> { + o.writeByte((byte) 18); + o.writeLongArray((long[]) v); + }), + entry( + float[].class, + (o, v) -> { + o.writeByte((byte) 19); + o.writeFloatArray((float[]) v); + }), + entry( + double[].class, + (o, v) -> { + o.writeByte((byte) 20); + o.writeDoubleArray((double[]) v); + }), + entry( + BytesRef.class, + (o, v) -> { + o.writeByte((byte) 21); + o.writeBytesRef((BytesRef) v); + }), + entry( + GeoPoint.class, + (o, v) -> { + o.writeByte((byte) 22); + o.writeGeoPoint((GeoPoint) v); + }), + entry( + ZonedDateTime.class, + (o, v) -> { + o.writeByte((byte) 23); + final ZonedDateTime zonedDateTime = (ZonedDateTime) v; + o.writeString(zonedDateTime.getZone().getId()); + o.writeLong(zonedDateTime.toInstant().toEpochMilli()); + }), + entry( + JodaCompatibleZonedDateTime.class, + (o, v) -> { + // write the joda compatibility datetime as joda datetime + o.writeByte((byte) 13); + final JodaCompatibleZonedDateTime zonedDateTime = (JodaCompatibleZonedDateTime) v; + String zoneId = zonedDateTime.getZonedDateTime().getZone().getId(); + // joda does not understand "Z" for utc, so we must special case + o.writeString(zoneId.equals("Z") ? DateTimeZone.UTC.getID() : zoneId); + o.writeLong(zonedDateTime.toInstant().toEpochMilli()); + })); /** * Notice: when serialization a map, the stream out map with the stream in map maybe have the diff --git a/server/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java b/server/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java index 93b7907812bb8..3ea9a9525dc1f 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/ShardCoreKeyMap.java @@ -141,7 +141,7 @@ public synchronized Set getCoreKeysForIndex(String index) { return Collections.emptySet(); } // we have to copy otherwise we risk ConcurrentModificationException - return Collections.unmodifiableSet(new HashSet<>(objects)); + return Set.copyOf(objects); } /** diff --git a/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java index 9d3e278e889f6..998e4e123a975 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java @@ -116,7 +116,7 @@ protected AbstractScopedSettings(Settings nodeSettings, Settings scopeSettings, this.scope = other.scope; complexMatchers = other.complexMatchers; keySettings = other.keySettings; - settingUpgraders = Collections.unmodifiableMap(new HashMap<>(other.settingUpgraders)); + settingUpgraders = Map.copyOf(other.settingUpgraders); settingUpdaters.addAll(other.settingUpdaters); } diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 82ae117c3e24b..6b50c0f1c112c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -104,9 +104,7 @@ import org.elasticsearch.transport.TransportSettings; import org.elasticsearch.watcher.ResourceWatcherService; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.function.Predicate; @@ -176,289 +174,287 @@ public void apply(Settings value, Settings current, Settings previous) { } } - public static Set> BUILT_IN_CLUSTER_SETTINGS = Collections.unmodifiableSet(new HashSet<>( - Arrays.asList(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING, - TransportClient.CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL, // TODO these transport client settings are kind - // of odd here and should only be valid if we are a transport client - TransportClient.CLIENT_TRANSPORT_PING_TIMEOUT, - TransportClient.CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME, - TransportClient.CLIENT_TRANSPORT_SNIFF, - AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING, - BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, - BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, - BalancedShardsAllocator.THRESHOLD_SETTING, - ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, - ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING, - EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, - EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, - FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING, - FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING, - FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING, - FsRepository.REPOSITORIES_CHUNK_SIZE_SETTING, - FsRepository.REPOSITORIES_LOCATION_SETTING, - IndicesQueryCache.INDICES_CACHE_QUERY_SIZE_SETTING, - IndicesQueryCache.INDICES_CACHE_QUERY_COUNT_SETTING, - IndicesQueryCache.INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING, - MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING, - MetaData.SETTING_READ_ONLY_SETTING, - MetaData.SETTING_READ_ONLY_ALLOW_DELETE_SETTING, - MetaData.SETTING_CLUSTER_MAX_SHARDS_PER_NODE, - RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, - RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, - RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, - RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, - RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING, - RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING, - RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING, - ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING, - ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, - ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, - ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING, - DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, - DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, - DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING, - DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, - DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, - DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, - SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING, - InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, - InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, - DestructiveOperations.REQUIRES_NAME_SETTING, - NoMasterBlockService.NO_MASTER_BLOCK_SETTING, - GatewayService.EXPECTED_DATA_NODES_SETTING, - GatewayService.EXPECTED_MASTER_NODES_SETTING, - GatewayService.EXPECTED_NODES_SETTING, - GatewayService.RECOVER_AFTER_DATA_NODES_SETTING, - GatewayService.RECOVER_AFTER_MASTER_NODES_SETTING, - GatewayService.RECOVER_AFTER_NODES_SETTING, - GatewayService.RECOVER_AFTER_TIME_SETTING, - NetworkModule.HTTP_DEFAULT_TYPE_SETTING, - NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING, - NetworkModule.HTTP_TYPE_SETTING, - NetworkModule.TRANSPORT_TYPE_SETTING, - HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS, - HttpTransportSettings.SETTING_CORS_ENABLED, - HttpTransportSettings.SETTING_CORS_MAX_AGE, - HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED, - HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN, - HttpTransportSettings.SETTING_HTTP_HOST, - HttpTransportSettings.SETTING_HTTP_PUBLISH_HOST, - HttpTransportSettings.SETTING_HTTP_BIND_HOST, - HttpTransportSettings.SETTING_HTTP_PORT, - HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT, - HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS, - HttpTransportSettings.SETTING_HTTP_COMPRESSION, - HttpTransportSettings.SETTING_HTTP_COMPRESSION_LEVEL, - HttpTransportSettings.SETTING_CORS_ALLOW_METHODS, - HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS, - HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED, - HttpTransportSettings.SETTING_HTTP_CONTENT_TYPE_REQUIRED, - HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH, - HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE, - HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE, - HttpTransportSettings.SETTING_HTTP_MAX_WARNING_HEADER_COUNT, - HttpTransportSettings.SETTING_HTTP_MAX_WARNING_HEADER_SIZE, - HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH, - HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT, - HttpTransportSettings.SETTING_HTTP_RESET_COOKIES, - HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY, - HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE, - HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS, - HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE, - HttpTransportSettings.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE, - HierarchyCircuitBreakerService.USE_REAL_MEMORY_USAGE_SETTING, - HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, - HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, - HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, - HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING, - HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_OVERHEAD_SETTING, - HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, - HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, - HierarchyCircuitBreakerService.ACCOUNTING_CIRCUIT_BREAKER_LIMIT_SETTING, - HierarchyCircuitBreakerService.ACCOUNTING_CIRCUIT_BREAKER_OVERHEAD_SETTING, - IndexModule.NODE_STORE_ALLOW_MMAP, - ClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, - ClusterService.USER_DEFINED_META_DATA, - SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING, - SearchService.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS, - TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, - RemoteClusterAware.REMOTE_CLUSTERS_SEEDS, - RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_SEEDS, - RemoteClusterAware.REMOTE_CLUSTERS_PROXY, - RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_PROXY, - RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE, - RemoteClusterService.SEARCH_REMOTE_CLUSTER_SKIP_UNAVAILABLE, - RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER, - RemoteClusterService.SEARCH_REMOTE_CONNECTIONS_PER_CLUSTER, - RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, - RemoteClusterService.SEARCH_REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, - RemoteClusterService.REMOTE_NODE_ATTRIBUTE, - RemoteClusterService.SEARCH_REMOTE_NODE_ATTRIBUTE, - RemoteClusterService.ENABLE_REMOTE_CLUSTERS, - RemoteClusterService.SEARCH_ENABLE_REMOTE_CLUSTERS, - RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE, - RemoteClusterService.REMOTE_CLUSTER_COMPRESS, - TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING, - ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING, - NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING, - HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING, - HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING, - TransportSettings.HOST, - TransportSettings.PUBLISH_HOST, - TransportSettings.PUBLISH_HOST_PROFILE, - TransportSettings.BIND_HOST, - TransportSettings.BIND_HOST_PROFILE, - TransportSettings.PORT, - TransportSettings.PORT_PROFILE, - TransportSettings.PUBLISH_PORT, - TransportSettings.PUBLISH_PORT_PROFILE, - TransportSettings.TRANSPORT_COMPRESS, - TransportSettings.PING_SCHEDULE, - TransportSettings.CONNECT_TIMEOUT, - TransportSettings.DEFAULT_FEATURES_SETTING, - TransportSettings.TCP_NO_DELAY, - TransportSettings.TCP_NO_DELAY_PROFILE, - TransportSettings.TCP_KEEP_ALIVE, - TransportSettings.TCP_KEEP_ALIVE_PROFILE, - TransportSettings.TCP_REUSE_ADDRESS, - TransportSettings.TCP_REUSE_ADDRESS_PROFILE, - TransportSettings.TCP_SEND_BUFFER_SIZE, - TransportSettings.TCP_SEND_BUFFER_SIZE_PROFILE, - TransportSettings.TCP_RECEIVE_BUFFER_SIZE, - TransportSettings.TCP_RECEIVE_BUFFER_SIZE_PROFILE, - TransportSettings.CONNECTIONS_PER_NODE_RECOVERY, - TransportSettings.CONNECTIONS_PER_NODE_BULK, - TransportSettings.CONNECTIONS_PER_NODE_REG, - TransportSettings.CONNECTIONS_PER_NODE_STATE, - TransportSettings.CONNECTIONS_PER_NODE_PING, - TransportSettings.TRACE_LOG_EXCLUDE_SETTING, - TransportSettings.TRACE_LOG_INCLUDE_SETTING, - NetworkService.NETWORK_SERVER, - NetworkService.GLOBAL_NETWORK_HOST_SETTING, - NetworkService.GLOBAL_NETWORK_BIND_HOST_SETTING, - NetworkService.GLOBAL_NETWORK_PUBLISH_HOST_SETTING, - NetworkService.TCP_NO_DELAY, - NetworkService.TCP_KEEP_ALIVE, - NetworkService.TCP_REUSE_ADDRESS, - NetworkService.TCP_SEND_BUFFER_SIZE, - NetworkService.TCP_RECEIVE_BUFFER_SIZE, - IndexSettings.QUERY_STRING_ANALYZE_WILDCARD, - IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD, - ScriptService.SCRIPT_CACHE_SIZE_SETTING, - ScriptService.SCRIPT_CACHE_EXPIRE_SETTING, - ScriptService.SCRIPT_MAX_SIZE_IN_BYTES, - ScriptService.SCRIPT_MAX_COMPILATIONS_RATE, - ScriptService.TYPES_ALLOWED_SETTING, - ScriptService.CONTEXTS_ALLOWED_SETTING, - IndicesService.INDICES_CACHE_CLEAN_INTERVAL_SETTING, - IndicesFieldDataCache.INDICES_FIELDDATA_CACHE_SIZE_KEY, - IndicesRequestCache.INDICES_CACHE_QUERY_SIZE, - IndicesRequestCache.INDICES_CACHE_QUERY_EXPIRE, - HunspellService.HUNSPELL_LAZY_LOAD, - HunspellService.HUNSPELL_IGNORE_CASE, - HunspellService.HUNSPELL_DICTIONARY_OPTIONS, - IndicesStore.INDICES_STORE_DELETE_SHARD_TIMEOUT, - Environment.PATH_DATA_SETTING, - Environment.PATH_HOME_SETTING, - Environment.PATH_LOGS_SETTING, - Environment.PATH_REPO_SETTING, - Environment.PATH_SHARED_DATA_SETTING, - Environment.PIDFILE_SETTING, - NodeEnvironment.NODE_ID_SEED_SETTING, - Node.INITIAL_STATE_TIMEOUT_SETTING, - DiscoveryModule.DISCOVERY_TYPE_SETTING, - DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, - SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, - SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, - SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, - SearchService.DEFAULT_KEEPALIVE_SETTING, - SearchService.KEEPALIVE_INTERVAL_SETTING, - SearchService.MAX_KEEPALIVE_SETTING, - MultiBucketConsumerService.MAX_BUCKET_SETTING, - SearchService.LOW_LEVEL_CANCELLATION_SETTING, - SearchService.MAX_OPEN_SCROLL_CONTEXT, - Node.WRITE_PORTS_FILE_SETTING, - Node.NODE_NAME_SETTING, - Node.NODE_DATA_SETTING, - Node.NODE_MASTER_SETTING, - Node.NODE_INGEST_SETTING, - Node.NODE_ATTRIBUTES, - Node.NODE_LOCAL_STORAGE_SETTING, - AutoCreateIndex.AUTO_CREATE_INDEX_SETTING, - BaseRestHandler.MULTI_ALLOW_EXPLICIT_INDEX, - ClusterName.CLUSTER_NAME_SETTING, - Client.CLIENT_TYPE_SETTING_S, - ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING, - EsExecutors.PROCESSORS_SETTING, - ThreadContext.DEFAULT_HEADERS_SETTING, - Loggers.LOG_DEFAULT_LEVEL_SETTING, - Loggers.LOG_LEVEL_SETTING, - NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING, - NodeEnvironment.ENABLE_LUCENE_SEGMENT_INFOS_TRACE_SETTING, - OsService.REFRESH_INTERVAL_SETTING, - ProcessService.REFRESH_INTERVAL_SETTING, - JvmService.REFRESH_INTERVAL_SETTING, - FsService.REFRESH_INTERVAL_SETTING, - JvmGcMonitorService.ENABLED_SETTING, - JvmGcMonitorService.REFRESH_INTERVAL_SETTING, - JvmGcMonitorService.GC_SETTING, - JvmGcMonitorService.GC_OVERHEAD_WARN_SETTING, - JvmGcMonitorService.GC_OVERHEAD_INFO_SETTING, - JvmGcMonitorService.GC_OVERHEAD_DEBUG_SETTING, - PageCacheRecycler.LIMIT_HEAP_SETTING, - PageCacheRecycler.WEIGHT_BYTES_SETTING, - PageCacheRecycler.WEIGHT_INT_SETTING, - PageCacheRecycler.WEIGHT_LONG_SETTING, - PageCacheRecycler.WEIGHT_OBJECTS_SETTING, - PageCacheRecycler.TYPE_SETTING, - PluginsService.MANDATORY_SETTING, - BootstrapSettings.SECURITY_FILTER_BAD_DEFAULTS_SETTING, - BootstrapSettings.MEMORY_LOCK_SETTING, - BootstrapSettings.SYSTEM_CALL_FILTER_SETTING, - BootstrapSettings.CTRLHANDLER_SETTING, - KeyStoreWrapper.SEED_SETTING, - IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, - IndexingMemoryController.MIN_INDEX_BUFFER_SIZE_SETTING, - IndexingMemoryController.MAX_INDEX_BUFFER_SIZE_SETTING, - IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, - IndexingMemoryController.SHARD_MEMORY_INTERVAL_TIME_SETTING, - ResourceWatcherService.ENABLED, - ResourceWatcherService.RELOAD_INTERVAL_HIGH, - ResourceWatcherService.RELOAD_INTERVAL_MEDIUM, - ResourceWatcherService.RELOAD_INTERVAL_LOW, - SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING, - ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING, - FastVectorHighlighter.SETTING_TV_HIGHLIGHT_MULTI_VALUE, - Node.BREAKER_TYPE_KEY, - OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, - IndexGraveyard.SETTING_MAX_TOMBSTONES, - PersistentTasksClusterService.CLUSTER_TASKS_ALLOCATION_RECHECK_INTERVAL_SETTING, - EnableAssignmentDecider.CLUSTER_TASKS_ALLOCATION_ENABLE_SETTING, - PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING, - PeerFinder.DISCOVERY_REQUEST_PEERS_TIMEOUT_SETTING, - ClusterFormationFailureHelper.DISCOVERY_CLUSTER_FORMATION_WARNING_TIMEOUT_SETTING, - ElectionSchedulerFactory.ELECTION_INITIAL_TIMEOUT_SETTING, - ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING, - ElectionSchedulerFactory.ELECTION_MAX_TIMEOUT_SETTING, - ElectionSchedulerFactory.ELECTION_DURATION_SETTING, - Coordinator.PUBLISH_TIMEOUT_SETTING, - JoinHelper.JOIN_TIMEOUT_SETTING, - FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING, - FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING, - FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING, - LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING, - LeaderChecker.LEADER_CHECK_INTERVAL_SETTING, - LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING, - Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION, - TransportAddVotingConfigExclusionsAction.MAXIMUM_VOTING_CONFIG_EXCLUSIONS_SETTING, - ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING, - ClusterBootstrapService.UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING, - LagDetector.CLUSTER_FOLLOWER_LAG_TIMEOUT_SETTING - ))); + public static Set> BUILT_IN_CLUSTER_SETTINGS = Set.of( + AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING, + // TODO these transport client settings are kind of odd here and should only be valid if we are a transport client + TransportClient.CLIENT_TRANSPORT_NODES_SAMPLER_INTERVAL, + TransportClient.CLIENT_TRANSPORT_PING_TIMEOUT, + TransportClient.CLIENT_TRANSPORT_IGNORE_CLUSTER_NAME, + TransportClient.CLIENT_TRANSPORT_SNIFF, + AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING, + BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, + BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, + BalancedShardsAllocator.THRESHOLD_SETTING, + ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, + ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING, + EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, + EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, + FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING, + FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING, + FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING, + FsRepository.REPOSITORIES_CHUNK_SIZE_SETTING, + FsRepository.REPOSITORIES_LOCATION_SETTING, + IndicesQueryCache.INDICES_CACHE_QUERY_SIZE_SETTING, + IndicesQueryCache.INDICES_CACHE_QUERY_COUNT_SETTING, + IndicesQueryCache.INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING, + MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING, + MetaData.SETTING_READ_ONLY_SETTING, + MetaData.SETTING_READ_ONLY_ALLOW_DELETE_SETTING, + MetaData.SETTING_CLUSTER_MAX_SHARDS_PER_NODE, + RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, + RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, + RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, + RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, + RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING, + RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING, + RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING, + ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING, + ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, + ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, + ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, + SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING, + InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, + InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, + DestructiveOperations.REQUIRES_NAME_SETTING, + NoMasterBlockService.NO_MASTER_BLOCK_SETTING, + GatewayService.EXPECTED_DATA_NODES_SETTING, + GatewayService.EXPECTED_MASTER_NODES_SETTING, + GatewayService.EXPECTED_NODES_SETTING, + GatewayService.RECOVER_AFTER_DATA_NODES_SETTING, + GatewayService.RECOVER_AFTER_MASTER_NODES_SETTING, + GatewayService.RECOVER_AFTER_NODES_SETTING, + GatewayService.RECOVER_AFTER_TIME_SETTING, + NetworkModule.HTTP_DEFAULT_TYPE_SETTING, + NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING, + NetworkModule.HTTP_TYPE_SETTING, + NetworkModule.TRANSPORT_TYPE_SETTING, + HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS, + HttpTransportSettings.SETTING_CORS_ENABLED, + HttpTransportSettings.SETTING_CORS_MAX_AGE, + HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN, + HttpTransportSettings.SETTING_HTTP_HOST, + HttpTransportSettings.SETTING_HTTP_PUBLISH_HOST, + HttpTransportSettings.SETTING_HTTP_BIND_HOST, + HttpTransportSettings.SETTING_HTTP_PORT, + HttpTransportSettings.SETTING_HTTP_PUBLISH_PORT, + HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS, + HttpTransportSettings.SETTING_HTTP_COMPRESSION, + HttpTransportSettings.SETTING_HTTP_COMPRESSION_LEVEL, + HttpTransportSettings.SETTING_CORS_ALLOW_METHODS, + HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS, + HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED, + HttpTransportSettings.SETTING_HTTP_CONTENT_TYPE_REQUIRED, + HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH, + HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE, + HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE, + HttpTransportSettings.SETTING_HTTP_MAX_WARNING_HEADER_COUNT, + HttpTransportSettings.SETTING_HTTP_MAX_WARNING_HEADER_SIZE, + HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH, + HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT, + HttpTransportSettings.SETTING_HTTP_RESET_COOKIES, + HttpTransportSettings.SETTING_HTTP_TCP_NO_DELAY, + HttpTransportSettings.SETTING_HTTP_TCP_KEEP_ALIVE, + HttpTransportSettings.SETTING_HTTP_TCP_REUSE_ADDRESS, + HttpTransportSettings.SETTING_HTTP_TCP_SEND_BUFFER_SIZE, + HttpTransportSettings.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE, + HierarchyCircuitBreakerService.USE_REAL_MEMORY_USAGE_SETTING, + HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, + HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, + HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, + HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING, + HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_OVERHEAD_SETTING, + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, + HierarchyCircuitBreakerService.ACCOUNTING_CIRCUIT_BREAKER_LIMIT_SETTING, + HierarchyCircuitBreakerService.ACCOUNTING_CIRCUIT_BREAKER_OVERHEAD_SETTING, + IndexModule.NODE_STORE_ALLOW_MMAP, + ClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, + ClusterService.USER_DEFINED_META_DATA, + SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING, + SearchService.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS, + TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, + RemoteClusterAware.REMOTE_CLUSTERS_SEEDS, + RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_SEEDS, + RemoteClusterAware.REMOTE_CLUSTERS_PROXY, + RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_PROXY, + RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE, + RemoteClusterService.SEARCH_REMOTE_CLUSTER_SKIP_UNAVAILABLE, + RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER, + RemoteClusterService.SEARCH_REMOTE_CONNECTIONS_PER_CLUSTER, + RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, + RemoteClusterService.SEARCH_REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, + RemoteClusterService.REMOTE_NODE_ATTRIBUTE, + RemoteClusterService.SEARCH_REMOTE_NODE_ATTRIBUTE, + RemoteClusterService.ENABLE_REMOTE_CLUSTERS, + RemoteClusterService.SEARCH_ENABLE_REMOTE_CLUSTERS, + RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE, + RemoteClusterService.REMOTE_CLUSTER_COMPRESS, + TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING, + ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING, + NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING, + HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING, + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING, + TransportSettings.HOST, + TransportSettings.PUBLISH_HOST, + TransportSettings.PUBLISH_HOST_PROFILE, + TransportSettings.BIND_HOST, + TransportSettings.BIND_HOST_PROFILE, + TransportSettings.PORT, + TransportSettings.PORT_PROFILE, + TransportSettings.PUBLISH_PORT, + TransportSettings.PUBLISH_PORT_PROFILE, + TransportSettings.TRANSPORT_COMPRESS, + TransportSettings.PING_SCHEDULE, + TransportSettings.CONNECT_TIMEOUT, + TransportSettings.DEFAULT_FEATURES_SETTING, + TransportSettings.TCP_NO_DELAY, + TransportSettings.TCP_NO_DELAY_PROFILE, + TransportSettings.TCP_KEEP_ALIVE, + TransportSettings.TCP_KEEP_ALIVE_PROFILE, + TransportSettings.TCP_REUSE_ADDRESS, + TransportSettings.TCP_REUSE_ADDRESS_PROFILE, + TransportSettings.TCP_SEND_BUFFER_SIZE, + TransportSettings.TCP_SEND_BUFFER_SIZE_PROFILE, + TransportSettings.TCP_RECEIVE_BUFFER_SIZE, + TransportSettings.TCP_RECEIVE_BUFFER_SIZE_PROFILE, + TransportSettings.CONNECTIONS_PER_NODE_RECOVERY, + TransportSettings.CONNECTIONS_PER_NODE_BULK, + TransportSettings.CONNECTIONS_PER_NODE_REG, + TransportSettings.CONNECTIONS_PER_NODE_STATE, + TransportSettings.CONNECTIONS_PER_NODE_PING, + TransportSettings.TRACE_LOG_EXCLUDE_SETTING, + TransportSettings.TRACE_LOG_INCLUDE_SETTING, + NetworkService.NETWORK_SERVER, + NetworkService.GLOBAL_NETWORK_HOST_SETTING, + NetworkService.GLOBAL_NETWORK_BIND_HOST_SETTING, + NetworkService.GLOBAL_NETWORK_PUBLISH_HOST_SETTING, + NetworkService.TCP_NO_DELAY, + NetworkService.TCP_KEEP_ALIVE, + NetworkService.TCP_REUSE_ADDRESS, + NetworkService.TCP_SEND_BUFFER_SIZE, + NetworkService.TCP_RECEIVE_BUFFER_SIZE, + IndexSettings.QUERY_STRING_ANALYZE_WILDCARD, + IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD, + ScriptService.SCRIPT_CACHE_SIZE_SETTING, + ScriptService.SCRIPT_CACHE_EXPIRE_SETTING, + ScriptService.SCRIPT_MAX_SIZE_IN_BYTES, + ScriptService.SCRIPT_MAX_COMPILATIONS_RATE, + ScriptService.TYPES_ALLOWED_SETTING, + ScriptService.CONTEXTS_ALLOWED_SETTING, + IndicesService.INDICES_CACHE_CLEAN_INTERVAL_SETTING, + IndicesFieldDataCache.INDICES_FIELDDATA_CACHE_SIZE_KEY, + IndicesRequestCache.INDICES_CACHE_QUERY_SIZE, + IndicesRequestCache.INDICES_CACHE_QUERY_EXPIRE, + HunspellService.HUNSPELL_LAZY_LOAD, + HunspellService.HUNSPELL_IGNORE_CASE, + HunspellService.HUNSPELL_DICTIONARY_OPTIONS, + IndicesStore.INDICES_STORE_DELETE_SHARD_TIMEOUT, + Environment.PATH_DATA_SETTING, + Environment.PATH_HOME_SETTING, + Environment.PATH_LOGS_SETTING, + Environment.PATH_REPO_SETTING, + Environment.PATH_SHARED_DATA_SETTING, + Environment.PIDFILE_SETTING, + NodeEnvironment.NODE_ID_SEED_SETTING, + Node.INITIAL_STATE_TIMEOUT_SETTING, + DiscoveryModule.DISCOVERY_TYPE_SETTING, + DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING, + SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING, + SeedHostsResolver.DISCOVERY_SEED_RESOLVER_MAX_CONCURRENT_RESOLVERS_SETTING, + SeedHostsResolver.DISCOVERY_SEED_RESOLVER_TIMEOUT_SETTING, + SearchService.DEFAULT_KEEPALIVE_SETTING, + SearchService.KEEPALIVE_INTERVAL_SETTING, + SearchService.MAX_KEEPALIVE_SETTING, + MultiBucketConsumerService.MAX_BUCKET_SETTING, + SearchService.LOW_LEVEL_CANCELLATION_SETTING, + SearchService.MAX_OPEN_SCROLL_CONTEXT, + Node.WRITE_PORTS_FILE_SETTING, + Node.NODE_NAME_SETTING, + Node.NODE_DATA_SETTING, + Node.NODE_MASTER_SETTING, + Node.NODE_INGEST_SETTING, + Node.NODE_ATTRIBUTES, + Node.NODE_LOCAL_STORAGE_SETTING, + AutoCreateIndex.AUTO_CREATE_INDEX_SETTING, + BaseRestHandler.MULTI_ALLOW_EXPLICIT_INDEX, + ClusterName.CLUSTER_NAME_SETTING, + Client.CLIENT_TYPE_SETTING_S, + ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING, + EsExecutors.PROCESSORS_SETTING, + ThreadContext.DEFAULT_HEADERS_SETTING, + Loggers.LOG_DEFAULT_LEVEL_SETTING, + Loggers.LOG_LEVEL_SETTING, + NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING, + NodeEnvironment.ENABLE_LUCENE_SEGMENT_INFOS_TRACE_SETTING, + OsService.REFRESH_INTERVAL_SETTING, + ProcessService.REFRESH_INTERVAL_SETTING, + JvmService.REFRESH_INTERVAL_SETTING, + FsService.REFRESH_INTERVAL_SETTING, + JvmGcMonitorService.ENABLED_SETTING, + JvmGcMonitorService.REFRESH_INTERVAL_SETTING, + JvmGcMonitorService.GC_SETTING, + JvmGcMonitorService.GC_OVERHEAD_WARN_SETTING, + JvmGcMonitorService.GC_OVERHEAD_INFO_SETTING, + JvmGcMonitorService.GC_OVERHEAD_DEBUG_SETTING, + PageCacheRecycler.LIMIT_HEAP_SETTING, + PageCacheRecycler.WEIGHT_BYTES_SETTING, + PageCacheRecycler.WEIGHT_INT_SETTING, + PageCacheRecycler.WEIGHT_LONG_SETTING, + PageCacheRecycler.WEIGHT_OBJECTS_SETTING, + PageCacheRecycler.TYPE_SETTING, + PluginsService.MANDATORY_SETTING, + BootstrapSettings.SECURITY_FILTER_BAD_DEFAULTS_SETTING, + BootstrapSettings.MEMORY_LOCK_SETTING, + BootstrapSettings.SYSTEM_CALL_FILTER_SETTING, + BootstrapSettings.CTRLHANDLER_SETTING, + KeyStoreWrapper.SEED_SETTING, + IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, + IndexingMemoryController.MIN_INDEX_BUFFER_SIZE_SETTING, + IndexingMemoryController.MAX_INDEX_BUFFER_SIZE_SETTING, + IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, + IndexingMemoryController.SHARD_MEMORY_INTERVAL_TIME_SETTING, + ResourceWatcherService.ENABLED, + ResourceWatcherService.RELOAD_INTERVAL_HIGH, + ResourceWatcherService.RELOAD_INTERVAL_MEDIUM, + ResourceWatcherService.RELOAD_INTERVAL_LOW, + SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING, + ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING, + FastVectorHighlighter.SETTING_TV_HIGHLIGHT_MULTI_VALUE, + Node.BREAKER_TYPE_KEY, + OperationRouting.USE_ADAPTIVE_REPLICA_SELECTION_SETTING, + IndexGraveyard.SETTING_MAX_TOMBSTONES, + PersistentTasksClusterService.CLUSTER_TASKS_ALLOCATION_RECHECK_INTERVAL_SETTING, + EnableAssignmentDecider.CLUSTER_TASKS_ALLOCATION_ENABLE_SETTING, + PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING, + PeerFinder.DISCOVERY_REQUEST_PEERS_TIMEOUT_SETTING, + ClusterFormationFailureHelper.DISCOVERY_CLUSTER_FORMATION_WARNING_TIMEOUT_SETTING, + ElectionSchedulerFactory.ELECTION_INITIAL_TIMEOUT_SETTING, + ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING, + ElectionSchedulerFactory.ELECTION_MAX_TIMEOUT_SETTING, + ElectionSchedulerFactory.ELECTION_DURATION_SETTING, + Coordinator.PUBLISH_TIMEOUT_SETTING, + JoinHelper.JOIN_TIMEOUT_SETTING, + FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING, + FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING, + FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING, + LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING, + LeaderChecker.LEADER_CHECK_INTERVAL_SETTING, + LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING, + Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION, + TransportAddVotingConfigExclusionsAction.MAXIMUM_VOTING_CONFIG_EXCLUSIONS_SETTING, + ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING, + ClusterBootstrapService.UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING, + LagDetector.CLUSTER_FOLLOWER_LAG_TIMEOUT_SETTING); - public static List> BUILT_IN_SETTING_UPGRADERS = Collections.unmodifiableList(Arrays.asList( + static List> BUILT_IN_SETTING_UPGRADERS = List.of( RemoteClusterAware.SEARCH_REMOTE_CLUSTER_SEEDS_UPGRADER, RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_PROXY_UPGRADER, - RemoteClusterService.SEARCH_REMOTE_CLUSTER_SKIP_UNAVAILABLE_UPGRADER)); + RemoteClusterService.SEARCH_REMOTE_CLUSTER_SKIP_UNAVAILABLE_UPGRADER); } diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index bbb824a497998..2acbbec3f8171 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -42,9 +42,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesRequestCache; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.function.Predicate; @@ -57,128 +55,129 @@ public final class IndexScopedSettings extends AbstractScopedSettings { public static final Predicate INDEX_SETTINGS_KEY_PREDICATE = (s) -> s.startsWith(IndexMetaData.INDEX_SETTING_PREFIX); - public static final Set> BUILT_IN_INDEX_SETTINGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY, - MergeSchedulerConfig.AUTO_THROTTLE_SETTING, - MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING, - MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING, - IndexMetaData.SETTING_INDEX_VERSION_CREATED, - IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING, - IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING, - IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING, - IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING, - IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING, - IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING, - IndexMetaData.INDEX_ROUTING_PARTITION_SIZE_SETTING, - IndexMetaData.INDEX_NUMBER_OF_ROUTING_SHARDS_SETTING, - IndexMetaData.INDEX_READ_ONLY_SETTING, - IndexMetaData.INDEX_BLOCKS_READ_SETTING, - IndexMetaData.INDEX_BLOCKS_WRITE_SETTING, - IndexMetaData.INDEX_BLOCKS_METADATA_SETTING, - IndexMetaData.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING, - IndexMetaData.INDEX_PRIORITY_SETTING, - IndexMetaData.INDEX_DATA_PATH_SETTING, - IndexMetaData.INDEX_FORMAT_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE_SETTING, - SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG_SETTING, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO_SETTING, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE_SETTING, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL_SETTING, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING, - IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG_SETTING, - MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_DELETES_PCT_ALLOWED_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING, - MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING, - IndexSortConfig.INDEX_SORT_FIELD_SETTING, - IndexSortConfig.INDEX_SORT_ORDER_SETTING, - IndexSortConfig.INDEX_SORT_MISSING_SETTING, - IndexSortConfig.INDEX_SORT_MODE_SETTING, - IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING, - IndexSettings.INDEX_WARMER_ENABLED_SETTING, - IndexSettings.INDEX_REFRESH_INTERVAL_SETTING, - IndexSettings.MAX_RESULT_WINDOW_SETTING, - IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING, - IndexSettings.MAX_TOKEN_COUNT_SETTING, - IndexSettings.MAX_DOCVALUE_FIELDS_SEARCH_SETTING, - IndexSettings.MAX_SCRIPT_FIELDS_SETTING, - IndexSettings.MAX_NGRAM_DIFF_SETTING, - IndexSettings.MAX_SHINGLE_DIFF_SETTING, - IndexSettings.MAX_RESCORE_WINDOW_SETTING, - IndexSettings.MAX_ADJACENCY_MATRIX_FILTERS_SETTING, - IndexSettings.MAX_ANALYZED_OFFSET_SETTING, - IndexSettings.MAX_TERMS_COUNT_SETTING, - IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING, - IndexSettings.DEFAULT_FIELD_SETTING, - IndexSettings.QUERY_STRING_LENIENT_SETTING, - IndexSettings.ALLOW_UNMAPPED, - IndexSettings.INDEX_CHECK_ON_STARTUP, - IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD, - IndexSettings.MAX_SLICES_PER_SCROLL, - IndexSettings.MAX_REGEX_LENGTH_SETTING, - ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, - IndexSettings.INDEX_GC_DELETES_SETTING, - IndexSettings.INDEX_SOFT_DELETES_SETTING, - IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, - IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING, - IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, - UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, - EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, - EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING, - IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, - IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING, - IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING, - IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING, - IndexSettings.INDEX_SEARCH_IDLE_AFTER, - IndexSettings.INDEX_SEARCH_THROTTLED, - IndexFieldDataService.INDEX_FIELDDATA_CACHE_KEY, - FieldMapper.IGNORE_MALFORMED_SETTING, - FieldMapper.COERCE_SETTING, - Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING, - MapperService.INDEX_MAPPER_DYNAMIC_SETTING, - MapperService.INDEX_MAPPING_NESTED_FIELDS_LIMIT_SETTING, - MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING, - MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING, - MapperService.INDEX_MAPPING_DEPTH_LIMIT_SETTING, - MapperService.INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING, - BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING, - IndexModule.INDEX_STORE_TYPE_SETTING, - IndexModule.INDEX_STORE_PRE_LOAD_SETTING, - IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING, - FsDirectoryService.INDEX_LOCK_FACTOR_SETTING, - Store.FORCE_RAM_TERM_DICT, - EngineConfig.INDEX_CODEC_SETTING, - IndexMetaData.SETTING_WAIT_FOR_ACTIVE_SHARDS, - IndexSettings.DEFAULT_PIPELINE, - MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING, + public static final Set> BUILT_IN_INDEX_SETTINGS = Set.of( + MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY, + MergeSchedulerConfig.AUTO_THROTTLE_SETTING, + MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING, + MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING, + IndexMetaData.SETTING_INDEX_VERSION_CREATED, + IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING, + IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING, + IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING, + IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING, + IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING, + IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING, + IndexMetaData.INDEX_ROUTING_PARTITION_SIZE_SETTING, + IndexMetaData.INDEX_NUMBER_OF_ROUTING_SHARDS_SETTING, + IndexMetaData.INDEX_READ_ONLY_SETTING, + IndexMetaData.INDEX_BLOCKS_READ_SETTING, + IndexMetaData.INDEX_BLOCKS_WRITE_SETTING, + IndexMetaData.INDEX_BLOCKS_METADATA_SETTING, + IndexMetaData.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING, + IndexMetaData.INDEX_PRIORITY_SETTING, + IndexMetaData.INDEX_DATA_PATH_SETTING, + IndexMetaData.INDEX_FORMAT_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG_SETTING, + MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_DELETES_PCT_ALLOWED_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING, + IndexSortConfig.INDEX_SORT_FIELD_SETTING, + IndexSortConfig.INDEX_SORT_ORDER_SETTING, + IndexSortConfig.INDEX_SORT_MISSING_SETTING, + IndexSortConfig.INDEX_SORT_MODE_SETTING, + IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING, + IndexSettings.INDEX_WARMER_ENABLED_SETTING, + IndexSettings.INDEX_REFRESH_INTERVAL_SETTING, + IndexSettings.MAX_RESULT_WINDOW_SETTING, + IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING, + IndexSettings.MAX_TOKEN_COUNT_SETTING, + IndexSettings.MAX_DOCVALUE_FIELDS_SEARCH_SETTING, + IndexSettings.MAX_SCRIPT_FIELDS_SETTING, + IndexSettings.MAX_NGRAM_DIFF_SETTING, + IndexSettings.MAX_SHINGLE_DIFF_SETTING, + IndexSettings.MAX_RESCORE_WINDOW_SETTING, + IndexSettings.MAX_ADJACENCY_MATRIX_FILTERS_SETTING, + IndexSettings.MAX_ANALYZED_OFFSET_SETTING, + IndexSettings.MAX_TERMS_COUNT_SETTING, + IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING, + IndexSettings.DEFAULT_FIELD_SETTING, + IndexSettings.QUERY_STRING_LENIENT_SETTING, + IndexSettings.ALLOW_UNMAPPED, + IndexSettings.INDEX_CHECK_ON_STARTUP, + IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD, + IndexSettings.MAX_SLICES_PER_SCROLL, + IndexSettings.MAX_REGEX_LENGTH_SETTING, + ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, + IndexSettings.INDEX_GC_DELETES_SETTING, + IndexSettings.INDEX_SOFT_DELETES_SETTING, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING, + IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, + UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, + EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, + EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING, + IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, + IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING, + IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING, + IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING, + IndexSettings.INDEX_SEARCH_IDLE_AFTER, + IndexSettings.INDEX_SEARCH_THROTTLED, + IndexFieldDataService.INDEX_FIELDDATA_CACHE_KEY, + FieldMapper.IGNORE_MALFORMED_SETTING, + FieldMapper.COERCE_SETTING, + Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING, + MapperService.INDEX_MAPPER_DYNAMIC_SETTING, + MapperService.INDEX_MAPPING_NESTED_FIELDS_LIMIT_SETTING, + MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING, + MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING, + MapperService.INDEX_MAPPING_DEPTH_LIMIT_SETTING, + MapperService.INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING, + BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING, + IndexModule.INDEX_STORE_TYPE_SETTING, + IndexModule.INDEX_STORE_PRE_LOAD_SETTING, + IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING, + FsDirectoryService.INDEX_LOCK_FACTOR_SETTING, + Store.FORCE_RAM_TERM_DICT, + EngineConfig.INDEX_CODEC_SETTING, + IndexMetaData.SETTING_WAIT_FOR_ACTIVE_SHARDS, + IndexSettings.DEFAULT_PIPELINE, + MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING, - // validate that built-in similarities don't get redefined - Setting.groupSetting("index.similarity.", (s) -> { - Map groups = s.getAsGroups(); - for (String key : SimilarityService.BUILT_IN.keySet()) { - if (groups.containsKey(key)) { - throw new IllegalArgumentException("illegal value for [index.similarity." + key + - "] cannot redefine built-in similarity"); - } - } - }, Property.IndexScope), // this allows similarity settings to be passed - Setting.groupSetting("index.analysis.", Property.IndexScope) // this allows analysis settings to be passed - - ))); + // validate that built-in similarities don't get redefined + Setting.groupSetting( + "index.similarity.", + (s) -> { + Map groups = s.getAsGroups(); + for (String key : SimilarityService.BUILT_IN.keySet()) { + if (groups.containsKey(key)) { + throw new IllegalArgumentException("illegal value for [index.similarity." + key + + "] cannot redefine built-in similarity"); + } + } + }, + Property.IndexScope), // this allows similarity settings to be passed + Setting.groupSetting("index.analysis.", Property.IndexScope)); // this allows analysis settings to be passed public static final IndexScopedSettings DEFAULT_SCOPED_SETTINGS = new IndexScopedSettings(Settings.EMPTY, BUILT_IN_INDEX_SETTINGS); diff --git a/server/src/main/java/org/elasticsearch/common/settings/Setting.java b/server/src/main/java/org/elasticsearch/common/settings/Setting.java index 9c3762f857e4a..514cfd3ce4ca8 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Setting.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Setting.java @@ -45,7 +45,6 @@ import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; -import java.util.HashSet; import java.util.IdentityHashMap; import java.util.Iterator; import java.util.List; @@ -650,7 +649,7 @@ public AffixSetting(AffixKey key, Setting delegate, Function(Arrays.asList(dependencies))); + this.dependencies = Set.of(dependencies); } boolean isGroupSetting() { diff --git a/server/src/main/java/org/elasticsearch/common/settings/Settings.java b/server/src/main/java/org/elasticsearch/common/settings/Settings.java index 3a7d719105afe..5789abf76d8b0 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Settings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Settings.java @@ -57,7 +57,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.ListIterator; @@ -674,7 +673,7 @@ private static void validateValue(String key, Object currentValue, XContentParse public static final Set FORMAT_PARAMS = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("settings_filter", "flat_settings"))); + Set.of("settings_filter", "flat_settings"); /** * Returns {@code true} if this settings object contains no settings diff --git a/server/src/main/java/org/elasticsearch/common/settings/SettingsFilter.java b/server/src/main/java/org/elasticsearch/common/settings/SettingsFilter.java index 549713fd15261..588905f54d969 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/SettingsFilter.java +++ b/server/src/main/java/org/elasticsearch/common/settings/SettingsFilter.java @@ -26,8 +26,6 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Set; @@ -50,7 +48,7 @@ public SettingsFilter(Collection patterns) { throw new IllegalArgumentException("invalid pattern: " + pattern); } } - this.patterns = Collections.unmodifiableSet(new HashSet<>(patterns)); + this.patterns = Set.copyOf(patterns); patternString = Strings.collectionToDelimitedString(patterns, ","); } diff --git a/server/src/main/java/org/elasticsearch/common/time/DateUtils.java b/server/src/main/java/org/elasticsearch/common/time/DateUtils.java index e6bf6a65105b7..729cee7a6852c 100644 --- a/server/src/main/java/org/elasticsearch/common/time/DateUtils.java +++ b/server/src/main/java/org/elasticsearch/common/time/DateUtils.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Set; +import static java.util.Map.entry; import static org.elasticsearch.common.time.DateUtilsRounding.getMonthOfYear; import static org.elasticsearch.common.time.DateUtilsRounding.getTotalMillisByYearMonth; import static org.elasticsearch.common.time.DateUtilsRounding.getYear; @@ -67,125 +68,121 @@ public static DateTimeZone zoneIdToDateTimeZone(ZoneId zoneId) { } // Map of deprecated timezones and their recommended new counterpart - public static final Map DEPRECATED_LONG_TIMEZONES; - static { - Map tzs = new HashMap<>(); - tzs.put("Africa/Asmera","Africa/Nairobi"); - tzs.put("Africa/Timbuktu","Africa/Abidjan"); - tzs.put("America/Argentina/ComodRivadavia","America/Argentina/Catamarca"); - tzs.put("America/Atka","America/Adak"); - tzs.put("America/Buenos_Aires","America/Argentina/Buenos_Aires"); - tzs.put("America/Catamarca","America/Argentina/Catamarca"); - tzs.put("America/Coral_Harbour","America/Atikokan"); - tzs.put("America/Cordoba","America/Argentina/Cordoba"); - tzs.put("America/Ensenada","America/Tijuana"); - tzs.put("America/Fort_Wayne","America/Indiana/Indianapolis"); - tzs.put("America/Indianapolis","America/Indiana/Indianapolis"); - tzs.put("America/Jujuy","America/Argentina/Jujuy"); - tzs.put("America/Knox_IN","America/Indiana/Knox"); - tzs.put("America/Louisville","America/Kentucky/Louisville"); - tzs.put("America/Mendoza","America/Argentina/Mendoza"); - tzs.put("America/Montreal","America/Toronto"); - tzs.put("America/Porto_Acre","America/Rio_Branco"); - tzs.put("America/Rosario","America/Argentina/Cordoba"); - tzs.put("America/Santa_Isabel","America/Tijuana"); - tzs.put("America/Shiprock","America/Denver"); - tzs.put("America/Virgin","America/Port_of_Spain"); - tzs.put("Antarctica/South_Pole","Pacific/Auckland"); - tzs.put("Asia/Ashkhabad","Asia/Ashgabat"); - tzs.put("Asia/Calcutta","Asia/Kolkata"); - tzs.put("Asia/Chongqing","Asia/Shanghai"); - tzs.put("Asia/Chungking","Asia/Shanghai"); - tzs.put("Asia/Dacca","Asia/Dhaka"); - tzs.put("Asia/Harbin","Asia/Shanghai"); - tzs.put("Asia/Kashgar","Asia/Urumqi"); - tzs.put("Asia/Katmandu","Asia/Kathmandu"); - tzs.put("Asia/Macao","Asia/Macau"); - tzs.put("Asia/Rangoon","Asia/Yangon"); - tzs.put("Asia/Saigon","Asia/Ho_Chi_Minh"); - tzs.put("Asia/Tel_Aviv","Asia/Jerusalem"); - tzs.put("Asia/Thimbu","Asia/Thimphu"); - tzs.put("Asia/Ujung_Pandang","Asia/Makassar"); - tzs.put("Asia/Ulan_Bator","Asia/Ulaanbaatar"); - tzs.put("Atlantic/Faeroe","Atlantic/Faroe"); - tzs.put("Atlantic/Jan_Mayen","Europe/Oslo"); - tzs.put("Australia/ACT","Australia/Sydney"); - tzs.put("Australia/Canberra","Australia/Sydney"); - tzs.put("Australia/LHI","Australia/Lord_Howe"); - tzs.put("Australia/NSW","Australia/Sydney"); - tzs.put("Australia/North","Australia/Darwin"); - tzs.put("Australia/Queensland","Australia/Brisbane"); - tzs.put("Australia/South","Australia/Adelaide"); - tzs.put("Australia/Tasmania","Australia/Hobart"); - tzs.put("Australia/Victoria","Australia/Melbourne"); - tzs.put("Australia/West","Australia/Perth"); - tzs.put("Australia/Yancowinna","Australia/Broken_Hill"); - tzs.put("Brazil/Acre","America/Rio_Branco"); - tzs.put("Brazil/DeNoronha","America/Noronha"); - tzs.put("Brazil/East","America/Sao_Paulo"); - tzs.put("Brazil/West","America/Manaus"); - tzs.put("Canada/Atlantic","America/Halifax"); - tzs.put("Canada/Central","America/Winnipeg"); - tzs.put("Canada/East-Saskatchewan","America/Regina"); - tzs.put("Canada/Eastern","America/Toronto"); - tzs.put("Canada/Mountain","America/Edmonton"); - tzs.put("Canada/Newfoundland","America/St_Johns"); - tzs.put("Canada/Pacific","America/Vancouver"); - tzs.put("Canada/Yukon","America/Whitehorse"); - tzs.put("Chile/Continental","America/Santiago"); - tzs.put("Chile/EasterIsland","Pacific/Easter"); - tzs.put("Cuba","America/Havana"); - tzs.put("Egypt","Africa/Cairo"); - tzs.put("Eire","Europe/Dublin"); - tzs.put("Europe/Belfast","Europe/London"); - tzs.put("Europe/Tiraspol","Europe/Chisinau"); - tzs.put("GB","Europe/London"); - tzs.put("GB-Eire","Europe/London"); - tzs.put("Greenwich","Etc/GMT"); - tzs.put("Hongkong","Asia/Hong_Kong"); - tzs.put("Iceland","Atlantic/Reykjavik"); - tzs.put("Iran","Asia/Tehran"); - tzs.put("Israel","Asia/Jerusalem"); - tzs.put("Jamaica","America/Jamaica"); - tzs.put("Japan","Asia/Tokyo"); - tzs.put("Kwajalein","Pacific/Kwajalein"); - tzs.put("Libya","Africa/Tripoli"); - tzs.put("Mexico/BajaNorte","America/Tijuana"); - tzs.put("Mexico/BajaSur","America/Mazatlan"); - tzs.put("Mexico/General","America/Mexico_City"); - tzs.put("NZ","Pacific/Auckland"); - tzs.put("NZ-CHAT","Pacific/Chatham"); - tzs.put("Navajo","America/Denver"); - tzs.put("PRC","Asia/Shanghai"); - tzs.put("Pacific/Johnston","Pacific/Honolulu"); - tzs.put("Pacific/Ponape","Pacific/Pohnpei"); - tzs.put("Pacific/Samoa","Pacific/Pago_Pago"); - tzs.put("Pacific/Truk","Pacific/Chuuk"); - tzs.put("Pacific/Yap","Pacific/Chuuk"); - tzs.put("Poland","Europe/Warsaw"); - tzs.put("Portugal","Europe/Lisbon"); - tzs.put("ROC","Asia/Taipei"); - tzs.put("ROK","Asia/Seoul"); - tzs.put("Singapore","Asia/Singapore"); - tzs.put("Turkey","Europe/Istanbul"); - tzs.put("UCT","Etc/UCT"); - tzs.put("US/Alaska","America/Anchorage"); - tzs.put("US/Aleutian","America/Adak"); - tzs.put("US/Arizona","America/Phoenix"); - tzs.put("US/Central","America/Chicago"); - tzs.put("US/East-Indiana","America/Indiana/Indianapolis"); - tzs.put("US/Eastern","America/New_York"); - tzs.put("US/Hawaii","Pacific/Honolulu"); - tzs.put("US/Indiana-Starke","America/Indiana/Knox"); - tzs.put("US/Michigan","America/Detroit"); - tzs.put("US/Mountain","America/Denver"); - tzs.put("US/Pacific","America/Los_Angeles"); - tzs.put("US/Samoa","Pacific/Pago_Pago"); - tzs.put("Universal","Etc/UTC"); - tzs.put("W-SU","Europe/Moscow"); - tzs.put("Zulu","Etc/UTC"); - DEPRECATED_LONG_TIMEZONES = Collections.unmodifiableMap(tzs); - } + public static final Map DEPRECATED_LONG_TIMEZONES = Map.ofEntries( + entry("Africa/Asmera", "Africa/Nairobi"), + entry("Africa/Timbuktu", "Africa/Abidjan"), + entry("America/Argentina/ComodRivadavia", "America/Argentina/Catamarca"), + entry("America/Atka", "America/Adak"), + entry("America/Buenos_Aires", "America/Argentina/Buenos_Aires"), + entry("America/Catamarca", "America/Argentina/Catamarca"), + entry("America/Coral_Harbour", "America/Atikokan"), + entry("America/Cordoba", "America/Argentina/Cordoba"), + entry("America/Ensenada", "America/Tijuana"), + entry("America/Fort_Wayne", "America/Indiana/Indianapolis"), + entry("America/Indianapolis", "America/Indiana/Indianapolis"), + entry("America/Jujuy", "America/Argentina/Jujuy"), + entry("America/Knox_IN", "America/Indiana/Knox"), + entry("America/Louisville", "America/Kentucky/Louisville"), + entry("America/Mendoza", "America/Argentina/Mendoza"), + entry("America/Montreal", "America/Toronto"), + entry("America/Porto_Acre", "America/Rio_Branco"), + entry("America/Rosario", "America/Argentina/Cordoba"), + entry("America/Santa_Isabel", "America/Tijuana"), + entry("America/Shiprock", "America/Denver"), + entry("America/Virgin", "America/Port_of_Spain"), + entry("Antarctica/South_Pole", "Pacific/Auckland"), + entry("Asia/Ashkhabad", "Asia/Ashgabat"), + entry("Asia/Calcutta", "Asia/Kolkata"), + entry("Asia/Chongqing", "Asia/Shanghai"), + entry("Asia/Chungking", "Asia/Shanghai"), + entry("Asia/Dacca", "Asia/Dhaka"), + entry("Asia/Harbin", "Asia/Shanghai"), + entry("Asia/Kashgar", "Asia/Urumqi"), + entry("Asia/Katmandu", "Asia/Kathmandu"), + entry("Asia/Macao", "Asia/Macau"), + entry("Asia/Rangoon", "Asia/Yangon"), + entry("Asia/Saigon", "Asia/Ho_Chi_Minh"), + entry("Asia/Tel_Aviv", "Asia/Jerusalem"), + entry("Asia/Thimbu", "Asia/Thimphu"), + entry("Asia/Ujung_Pandang", "Asia/Makassar"), + entry("Asia/Ulan_Bator", "Asia/Ulaanbaatar"), + entry("Atlantic/Faeroe", "Atlantic/Faroe"), + entry("Atlantic/Jan_Mayen", "Europe/Oslo"), + entry("Australia/ACT", "Australia/Sydney"), + entry("Australia/Canberra", "Australia/Sydney"), + entry("Australia/LHI", "Australia/Lord_Howe"), + entry("Australia/NSW", "Australia/Sydney"), + entry("Australia/North", "Australia/Darwin"), + entry("Australia/Queensland", "Australia/Brisbane"), + entry("Australia/South", "Australia/Adelaide"), + entry("Australia/Tasmania", "Australia/Hobart"), + entry("Australia/Victoria", "Australia/Melbourne"), + entry("Australia/West", "Australia/Perth"), + entry("Australia/Yancowinna", "Australia/Broken_Hill"), + entry("Brazil/Acre", "America/Rio_Branco"), + entry("Brazil/DeNoronha", "America/Noronha"), + entry("Brazil/East", "America/Sao_Paulo"), + entry("Brazil/West", "America/Manaus"), + entry("Canada/Atlantic", "America/Halifax"), + entry("Canada/Central", "America/Winnipeg"), + entry("Canada/East-Saskatchewan", "America/Regina"), + entry("Canada/Eastern", "America/Toronto"), + entry("Canada/Mountain", "America/Edmonton"), + entry("Canada/Newfoundland", "America/St_Johns"), + entry("Canada/Pacific", "America/Vancouver"), + entry("Canada/Yukon", "America/Whitehorse"), + entry("Chile/Continental", "America/Santiago"), + entry("Chile/EasterIsland", "Pacific/Easter"), + entry("Cuba", "America/Havana"), + entry("Egypt", "Africa/Cairo"), + entry("Eire", "Europe/Dublin"), + entry("Europe/Belfast", "Europe/London"), + entry("Europe/Tiraspol", "Europe/Chisinau"), + entry("GB", "Europe/London"), + entry("GB-Eire", "Europe/London"), + entry("Greenwich", "Etc/GMT"), + entry("Hongkong", "Asia/Hong_Kong"), + entry("Iceland", "Atlantic/Reykjavik"), + entry("Iran", "Asia/Tehran"), + entry("Israel", "Asia/Jerusalem"), + entry("Jamaica", "America/Jamaica"), + entry("Japan", "Asia/Tokyo"), + entry("Kwajalein", "Pacific/Kwajalein"), + entry("Libya", "Africa/Tripoli"), + entry("Mexico/BajaNorte", "America/Tijuana"), + entry("Mexico/BajaSur", "America/Mazatlan"), + entry("Mexico/General", "America/Mexico_City"), + entry("NZ", "Pacific/Auckland"), + entry("NZ-CHAT", "Pacific/Chatham"), + entry("Navajo", "America/Denver"), + entry("PRC", "Asia/Shanghai"), + entry("Pacific/Johnston", "Pacific/Honolulu"), + entry("Pacific/Ponape", "Pacific/Pohnpei"), + entry("Pacific/Samoa", "Pacific/Pago_Pago"), + entry("Pacific/Truk", "Pacific/Chuuk"), + entry("Pacific/Yap", "Pacific/Chuuk"), + entry("Poland", "Europe/Warsaw"), + entry("Portugal", "Europe/Lisbon"), + entry("ROC", "Asia/Taipei"), + entry("ROK", "Asia/Seoul"), + entry("Singapore", "Asia/Singapore"), + entry("Turkey", "Europe/Istanbul"), + entry("UCT", "Etc/UCT"), + entry("US/Alaska", "America/Anchorage"), + entry("US/Aleutian", "America/Adak"), + entry("US/Arizona", "America/Phoenix"), + entry("US/Central", "America/Chicago"), + entry("US/East-Indiana", "America/Indiana/Indianapolis"), + entry("US/Eastern", "America/New_York"), + entry("US/Hawaii", "Pacific/Honolulu"), + entry("US/Indiana-Starke", "America/Indiana/Knox"), + entry("US/Michigan", "America/Detroit"), + entry("US/Mountain", "America/Denver"), + entry("US/Pacific", "America/Los_Angeles"), + entry("US/Samoa", "Pacific/Pago_Pago"), + entry("Universal", "Etc/UTC"), + entry("W-SU", "Europe/Moscow"), + entry("Zulu", "Etc/UTC")); public static ZoneId dateTimeZoneToZoneId(DateTimeZone timeZone) { if (timeZone == null) { diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java index 80a9a30032e17..b52bc2b199c80 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/ThreadContext.java @@ -34,7 +34,6 @@ import java.io.Closeable; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -277,7 +276,7 @@ public Map> getResponseHeaders() { HashMap> map = new HashMap<>(responseHeaders.size()); for (Map.Entry> entry : responseHeaders.entrySet()) { - map.put(entry.getKey(), Collections.unmodifiableList(new ArrayList<>(entry.getValue()))); + map.put(entry.getKey(), List.copyOf(entry.getValue())); } return Collections.unmodifiableMap(map); diff --git a/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java index f7e6f8e949b37..fc2f76d3436c0 100644 --- a/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/server/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -85,8 +85,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static java.util.Collections.unmodifiableSet; - /** * A component that holds all data paths for a single node. */ @@ -693,7 +691,7 @@ public interface ShardLocker { */ public Set lockedShards() { synchronized (shardLocks) { - return unmodifiableSet(new HashSet<>(shardLocks.keySet())); + return Set.copyOf(shardLocks.keySet()); } } diff --git a/server/src/main/java/org/elasticsearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/elasticsearch/gateway/AsyncShardFetch.java index 75654c077705f..d03f6abf7d9bd 100644 --- a/server/src/main/java/org/elasticsearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/elasticsearch/gateway/AsyncShardFetch.java @@ -46,7 +46,6 @@ import java.util.concurrent.atomic.AtomicLong; import static java.util.Collections.emptySet; -import static java.util.Collections.unmodifiableSet; /** * Allows to asynchronously fetch shard related data from other nodes for allocation, without blocking @@ -152,7 +151,7 @@ public synchronized FetchResult fetchData(DiscoveryNodes nodes, Set i } } } - Set allIgnoreNodes = unmodifiableSet(new HashSet<>(nodesToIgnore)); + Set allIgnoreNodes = Set.copyOf(nodesToIgnore); // clear the nodes to ignore, we had a successful run in fetching everything we can // we need to try them if another full run is needed nodesToIgnore.clear(); diff --git a/server/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java b/server/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java index 4d7949cdf4de8..9a1c79b476e1b 100644 --- a/server/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java +++ b/server/src/main/java/org/elasticsearch/gateway/DanglingIndicesState.java @@ -44,7 +44,6 @@ import java.util.stream.Collectors; import static java.util.Collections.emptyMap; -import static java.util.Collections.unmodifiableMap; /** * The dangling indices state is responsible for finding new dangling indices (indices that have @@ -88,7 +87,7 @@ public void processDanglingIndices(final MetaData metaData) { */ Map getDanglingIndices() { // This might be a good use case for CopyOnWriteHashMap - return unmodifiableMap(new HashMap<>(danglingIndices)); + return Map.copyOf(danglingIndices); } /** diff --git a/server/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java b/server/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java index d4e6362085238..e19cbe38c7e37 100644 --- a/server/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java +++ b/server/src/main/java/org/elasticsearch/index/CompositeIndexEventListener.java @@ -31,9 +31,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; /** @@ -50,7 +48,7 @@ final class CompositeIndexEventListener implements IndexEventListener { throw new IllegalArgumentException("listeners must be non-null"); } } - this.listeners = Collections.unmodifiableList(new ArrayList<>(listeners)); + this.listeners = List.copyOf(listeners); this.logger = Loggers.getLogger(getClass(), indexSettings.getIndex()); } diff --git a/server/src/main/java/org/elasticsearch/index/analysis/Analysis.java b/server/src/main/java/org/elasticsearch/index/analysis/Analysis.java index 09a87124110b3..303e7049306a7 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/Analysis.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/Analysis.java @@ -69,13 +69,12 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; -import static java.util.Collections.unmodifiableMap; +import static java.util.Map.entry; public class Analysis { @@ -113,45 +112,40 @@ public static CharArraySet parseStemExclusion(Settings settings, CharArraySet de } } - public static final Map> NAMED_STOP_WORDS; - static { - Map> namedStopWords = new HashMap<>(); - namedStopWords.put("_arabic_", ArabicAnalyzer.getDefaultStopSet()); - namedStopWords.put("_armenian_", ArmenianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_basque_", BasqueAnalyzer.getDefaultStopSet()); - namedStopWords.put("_bengali_", BengaliAnalyzer.getDefaultStopSet()); - namedStopWords.put("_brazilian_", BrazilianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_bulgarian_", BulgarianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_catalan_", CatalanAnalyzer.getDefaultStopSet()); - namedStopWords.put("_czech_", CzechAnalyzer.getDefaultStopSet()); - namedStopWords.put("_danish_", DanishAnalyzer.getDefaultStopSet()); - namedStopWords.put("_dutch_", DutchAnalyzer.getDefaultStopSet()); - namedStopWords.put("_english_", EnglishAnalyzer.getDefaultStopSet()); - namedStopWords.put("_finnish_", FinnishAnalyzer.getDefaultStopSet()); - namedStopWords.put("_french_", FrenchAnalyzer.getDefaultStopSet()); - namedStopWords.put("_galician_", GalicianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_german_", GermanAnalyzer.getDefaultStopSet()); - namedStopWords.put("_greek_", GreekAnalyzer.getDefaultStopSet()); - namedStopWords.put("_hindi_", HindiAnalyzer.getDefaultStopSet()); - namedStopWords.put("_hungarian_", HungarianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_indonesian_", IndonesianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_irish_", IrishAnalyzer.getDefaultStopSet()); - namedStopWords.put("_italian_", ItalianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_latvian_", LatvianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_lithuanian_", LithuanianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_norwegian_", NorwegianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_persian_", PersianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_portuguese_", PortugueseAnalyzer.getDefaultStopSet()); - namedStopWords.put("_romanian_", RomanianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_russian_", RussianAnalyzer.getDefaultStopSet()); - namedStopWords.put("_sorani_", SoraniAnalyzer.getDefaultStopSet()); - namedStopWords.put("_spanish_", SpanishAnalyzer.getDefaultStopSet()); - namedStopWords.put("_swedish_", SwedishAnalyzer.getDefaultStopSet()); - namedStopWords.put("_thai_", ThaiAnalyzer.getDefaultStopSet()); - namedStopWords.put("_turkish_", TurkishAnalyzer.getDefaultStopSet()); - - NAMED_STOP_WORDS = unmodifiableMap(namedStopWords); - } + private static final Map> NAMED_STOP_WORDS = Map.ofEntries( + entry("_arabic_", ArabicAnalyzer.getDefaultStopSet()), + entry("_armenian_", ArmenianAnalyzer.getDefaultStopSet()), + entry("_basque_", BasqueAnalyzer.getDefaultStopSet()), + entry("_bengali_", BengaliAnalyzer.getDefaultStopSet()), + entry("_brazilian_", BrazilianAnalyzer.getDefaultStopSet()), + entry("_bulgarian_", BulgarianAnalyzer.getDefaultStopSet()), + entry("_catalan_", CatalanAnalyzer.getDefaultStopSet()), + entry("_czech_", CzechAnalyzer.getDefaultStopSet()), + entry("_danish_", DanishAnalyzer.getDefaultStopSet()), + entry("_dutch_", DutchAnalyzer.getDefaultStopSet()), + entry("_english_", EnglishAnalyzer.getDefaultStopSet()), + entry("_finnish_", FinnishAnalyzer.getDefaultStopSet()), + entry("_french_", FrenchAnalyzer.getDefaultStopSet()), + entry("_galician_", GalicianAnalyzer.getDefaultStopSet()), + entry("_german_", GermanAnalyzer.getDefaultStopSet()), + entry("_greek_", GreekAnalyzer.getDefaultStopSet()), + entry("_hindi_", HindiAnalyzer.getDefaultStopSet()), + entry("_hungarian_", HungarianAnalyzer.getDefaultStopSet()), + entry("_indonesian_", IndonesianAnalyzer.getDefaultStopSet()), + entry("_irish_", IrishAnalyzer.getDefaultStopSet()), + entry("_italian_", ItalianAnalyzer.getDefaultStopSet()), + entry("_latvian_", LatvianAnalyzer.getDefaultStopSet()), + entry("_lithuanian_", LithuanianAnalyzer.getDefaultStopSet()), + entry("_norwegian_", NorwegianAnalyzer.getDefaultStopSet()), + entry("_persian_", PersianAnalyzer.getDefaultStopSet()), + entry("_portuguese_", PortugueseAnalyzer.getDefaultStopSet()), + entry("_romanian_", RomanianAnalyzer.getDefaultStopSet()), + entry("_russian_", RussianAnalyzer.getDefaultStopSet()), + entry("_sorani_", SoraniAnalyzer.getDefaultStopSet()), + entry("_spanish_", SpanishAnalyzer.getDefaultStopSet()), + entry("_swedish_", SwedishAnalyzer.getDefaultStopSet()), + entry("_thai_", ThaiAnalyzer.getDefaultStopSet()), + entry("_turkish_", TurkishAnalyzer.getDefaultStopSet())); public static CharArraySet parseWords(Environment env, Settings settings, String name, CharArraySet defaultWords, Map> namedWords, boolean ignoreCase) { diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java b/server/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java index c9e81ca4f2dc1..e90ba987b62ec 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java @@ -26,9 +26,7 @@ import org.apache.lucene.util.packed.PackedLongValues; import org.elasticsearch.index.fielddata.ordinals.Ordinals; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; public class PagedBytesAtomicFieldData extends AbstractAtomicOrdinalsFieldData { @@ -60,11 +58,10 @@ public long ramBytesUsed() { @Override public Collection getChildResources() { - List resources = new ArrayList<>(); - resources.add(Accountables.namedAccountable("ordinals", ordinals)); - resources.add(Accountables.namedAccountable("term bytes", bytes)); - resources.add(Accountables.namedAccountable("term offsets", termOrdToBytesOffset)); - return Collections.unmodifiableList(resources); + return List.of( + Accountables.namedAccountable("ordinals", ordinals), + Accountables.namedAccountable("term bytes", bytes), + Accountables.namedAccountable("term offsets", termOrdToBytesOffset)); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java index 31a0f900db54f..8028960ab4262 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java @@ -33,13 +33,12 @@ import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.mapper.MapperRegistry; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.function.Supplier; -import static java.util.Collections.unmodifiableMap; - public class DocumentMapperParser { final MapperService mapperService; @@ -128,9 +127,21 @@ private DocumentMapper parse(String type, Map mapping, String de Map meta = (Map) mapping.remove("_meta"); if (meta != null) { - // It may not be required to copy meta here to maintain immutability - // but the cost is pretty low here. - docBuilder.meta(unmodifiableMap(new HashMap<>(meta))); + /* + * It may not be required to copy meta here to maintain immutability but the cost is pretty low here. + * + * Note: this copy can not be replaced by Map#copyOf because we rely on consistent serialization order since we do byte-level + * checks on the mapping between what we receive from the master and what we have locally. As Map#copyOf is not necessarily + * the same underlying map implementation, we could end up with a different iteration order. For reference, see + * MapperService#assertSerializtion and GitHub issues #10302 and #10318. + * + * Do not change this to Map#copyOf or any other method of copying meta that could change the iteration order. + * + * TODO: + * - this should almost surely be a copy as a LinkedHashMap to have the ordering guarantees that we are relying on + * - investigate the above note about whether or not we really need to be copying here, the ideal outcome would be to not + */ + docBuilder.meta(Collections.unmodifiableMap(new HashMap<>(meta))); } checkNoRemainingFields(mapping, parserContext.indexVersionCreated(), "Root mapping definition has unsupported parameters: "); diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 2b5415115895d..0c33ee7102346 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -67,8 +67,6 @@ import java.util.function.BiFunction; import java.util.function.LongSupplier; -import static java.util.Collections.unmodifiableMap; - /** * Context object used to create lucene queries on the shard level. */ @@ -187,7 +185,7 @@ public void addNamedQuery(String name, Query query) { public Map copyNamedQueries() { // This might be a good use case for CopyOnWriteHashMap - return unmodifiableMap(new HashMap<>(namedQueries)); + return Map.copyOf(namedQueries); } /** diff --git a/server/src/main/java/org/elasticsearch/index/similarity/SimilarityProviders.java b/server/src/main/java/org/elasticsearch/index/similarity/SimilarityProviders.java index 04970a38bd99d..135a3afee1ea5 100644 --- a/server/src/main/java/org/elasticsearch/index/similarity/SimilarityProviders.java +++ b/server/src/main/java/org/elasticsearch/index/similarity/SimilarityProviders.java @@ -56,13 +56,10 @@ import org.elasticsearch.common.settings.Settings; import java.util.Arrays; -import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; -import static java.util.Collections.unmodifiableMap; - final class SimilarityProviders { private SimilarityProviders() {} // no instantiation @@ -70,61 +67,37 @@ private SimilarityProviders() {} // no instantiation private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(SimilarityProviders.class)); static final String DISCOUNT_OVERLAPS = "discount_overlaps"; - private static final Map BASIC_MODELS; - private static final Map LEGACY_BASIC_MODELS; - private static final Map AFTER_EFFECTS; - private static final Map LEGACY_AFTER_EFFECTS; - - static { - Map models = new HashMap<>(); - models.put("g", new BasicModelG()); - models.put("if", new BasicModelIF()); - models.put("in", new BasicModelIn()); - models.put("ine", new BasicModelIne()); - BASIC_MODELS = unmodifiableMap(models); - - Map legacyModels = new HashMap<>(); - // TODO: be and g and both based on the bose-einstein model. - // Is there a better replacement for d and p which use the binomial model? - legacyModels.put("be", "g"); - legacyModels.put("d", "ine"); - legacyModels.put("p", "ine"); - LEGACY_BASIC_MODELS = unmodifiableMap(legacyModels); - - Map effects = new HashMap<>(); - effects.put("b", new AfterEffectB()); - effects.put("l", new AfterEffectL()); - AFTER_EFFECTS = unmodifiableMap(effects); - - Map legacyEffects = new HashMap<>(); - // l is simpler than b, so this should be a better replacement for "no" - legacyEffects.put("no", "l"); - LEGACY_AFTER_EFFECTS = unmodifiableMap(legacyEffects); - } - - private static final Map INDEPENDENCE_MEASURES; - static { - Map measures = new HashMap<>(); - measures.put("standardized", new IndependenceStandardized()); - measures.put("saturated", new IndependenceSaturated()); - measures.put("chisquared", new IndependenceChiSquared()); - INDEPENDENCE_MEASURES = unmodifiableMap(measures); - } - - private static final Map DISTRIBUTIONS; - private static final Map LAMBDAS; - - static { - Map distributions = new HashMap<>(); - distributions.put("ll", new DistributionLL()); - distributions.put("spl", new DistributionSPL()); - DISTRIBUTIONS = unmodifiableMap(distributions); - - Map lamdas = new HashMap<>(); - lamdas.put("df", new LambdaDF()); - lamdas.put("ttf", new LambdaTTF()); - LAMBDAS = unmodifiableMap(lamdas); - } + private static final Map BASIC_MODELS = Map.of( + "g", new BasicModelG(), + "if", new BasicModelIF(), + "in", new BasicModelIn(), + "ine", new BasicModelIne()); + + // TODO: be and g and both based on the bose-einstein model. + // Is there a better replacement for d and p which use the binomial model? + private static final Map LEGACY_BASIC_MODELS = Map.of( + "be", "g", + "d", "ine", + "p", "ine"); + + private static final Map AFTER_EFFECTS = Map.of( + "b", new AfterEffectB(), + "l", new AfterEffectL()); + // l is simpler than b, so this should be a better replacement for "no" + private static final Map LEGACY_AFTER_EFFECTS = Map.of("no", "l"); + + private static final Map INDEPENDENCE_MEASURES = Map.of( + "standardized", new IndependenceStandardized(), + "saturated", new IndependenceSaturated(), + "chisquared", new IndependenceChiSquared()); + + private static final Map DISTRIBUTIONS = Map.of( + "ll", new DistributionLL(), + "spl", new DistributionSPL()); + + private static final Map LAMBDAS = Map.of( + "df", new LambdaDF(), + "ttf", new LambdaTTF()); /** * Parses the given Settings and creates the appropriate {@link BasicModel} diff --git a/server/src/main/java/org/elasticsearch/indices/flush/ShardsSyncedFlushResult.java b/server/src/main/java/org/elasticsearch/indices/flush/ShardsSyncedFlushResult.java index a2d9d5943261d..bd0451498c6ba 100644 --- a/server/src/main/java/org/elasticsearch/indices/flush/ShardsSyncedFlushResult.java +++ b/server/src/main/java/org/elasticsearch/indices/flush/ShardsSyncedFlushResult.java @@ -29,7 +29,6 @@ import java.util.Map; import static java.util.Collections.emptyMap; -import static java.util.Collections.unmodifiableMap; /** * Result for all copies of a shard @@ -68,7 +67,7 @@ public ShardsSyncedFlushResult(ShardId shardId, int totalShards, Map shardResponses) { this.failureReason = null; - this.shardResponses = unmodifiableMap(new HashMap<>(shardResponses)); + this.shardResponses = Map.copyOf(shardResponses); this.syncId = syncId; this.totalShards = totalShards; this.shardId = shardId; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index 4d27362af22b5..7393eccc44e86 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -37,8 +37,6 @@ import org.elasticsearch.index.shard.ShardId; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -713,7 +711,7 @@ public synchronized void writeTo(StreamOutput out) throws IOException { } public synchronized List fileDetails() { - return Collections.unmodifiableList(new ArrayList<>(fileDetails.values())); + return List.copyOf(fileDetails.values()); } public synchronized void reset() { diff --git a/server/src/main/java/org/elasticsearch/ingest/ConditionalProcessor.java b/server/src/main/java/org/elasticsearch/ingest/ConditionalProcessor.java index d5d489ec0e626..e5333485db52b 100644 --- a/server/src/main/java/org/elasticsearch/ingest/ConditionalProcessor.java +++ b/server/src/main/java/org/elasticsearch/ingest/ConditionalProcessor.java @@ -19,11 +19,15 @@ package org.elasticsearch.ingest; +import org.elasticsearch.script.DeprecationMap; +import org.elasticsearch.script.IngestConditionalScript; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptService; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.ListIterator; @@ -33,22 +37,10 @@ import java.util.function.LongSupplier; import java.util.stream.Collectors; -import org.elasticsearch.script.DeprecationMap; -import org.elasticsearch.script.IngestConditionalScript; -import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptService; - public class ConditionalProcessor extends AbstractProcessor { - private static final Map DEPRECATIONS; - static { - Map deprecations = new HashMap<>(); - deprecations.put( - "_type", - "[types removal] Looking up doc types [_type] in scripts is deprecated." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + private static final Map DEPRECATIONS = + Map.of("_type", "[types removal] Looking up doc types [_type] in scripts is deprecated."); static final String TYPE = "conditional"; diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestNodesStatsAction.java index 14c8655e48c18..e74c42f3c396d 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -39,6 +39,7 @@ import java.util.TreeSet; import java.util.function.Consumer; +import static java.util.Map.entry; import static org.elasticsearch.rest.RestRequest.Method.GET; public class RestNodesStatsAction extends BaseRestHandler { @@ -55,25 +56,20 @@ public RestNodesStatsAction(Settings settings, RestController controller) { controller.registerHandler(GET, "/_nodes/{nodeId}/stats/{metric}/{index_metric}", this); } - static final Map> METRICS; - - static { - final Map> metrics = new HashMap<>(); - metrics.put("os", r -> r.os(true)); - metrics.put("jvm", r -> r.jvm(true)); - metrics.put("thread_pool", r -> r.threadPool(true)); - metrics.put("fs", r -> r.fs(true)); - metrics.put("transport", r -> r.transport(true)); - metrics.put("http", r -> r.http(true)); - metrics.put("indices", r -> r.indices(true)); - metrics.put("process", r -> r.process(true)); - metrics.put("breaker", r -> r.breaker(true)); - metrics.put("script", r -> r.script(true)); - metrics.put("discovery", r -> r.discovery(true)); - metrics.put("ingest", r -> r.ingest(true)); - metrics.put("adaptive_selection", r -> r.adaptiveSelection(true)); - METRICS = Collections.unmodifiableMap(metrics); - } + static final Map> METRICS = Map.ofEntries( + entry("os", r -> r.os(true)), + entry("jvm", r -> r.jvm(true)), + entry("thread_pool", r -> r.threadPool(true)), + entry("fs", r -> r.fs(true)), + entry("transport", r -> r.transport(true)), + entry("http", r -> r.http(true)), + entry("indices", r -> r.indices(true)), + entry("process", r -> r.process(true)), + entry("breaker", r -> r.breaker(true)), + entry("script", r -> r.script(true)), + entry("discovery", r -> r.discovery(true)), + entry("ingest", r -> r.ingest(true)), + entry("adaptive_selection", r -> r.adaptiveSelection(true))); static final Map> FLAGS; diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java index 58dc861126bfe..69a659faf4dc0 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java @@ -30,9 +30,6 @@ import org.elasticsearch.rest.RestStatus; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; import java.util.Set; import static org.elasticsearch.rest.action.cat.RestTable.buildHelpWidths; @@ -76,8 +73,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC } } - static Set RESPONSE_PARAMS = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("format", "h", "v", "ts", "pri", "bytes", "size", "time", "s"))); + static Set RESPONSE_PARAMS = Set.of("format", "h", "v", "ts", "pri", "bytes", "size", "time", "s"); @Override protected Set responseParams() { diff --git a/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java b/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java index 13b109766af83..ff2cafcc37f50 100644 --- a/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java +++ b/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java @@ -18,10 +18,6 @@ */ package org.elasticsearch.script; -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorable; import org.elasticsearch.ElasticsearchException; @@ -30,24 +26,17 @@ import org.elasticsearch.search.lookup.LeafSearchLookup; import org.elasticsearch.search.lookup.SearchLookup; -abstract class AbstractSortScript implements ScorerAware { +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; - private static final Map DEPRECATIONS; +abstract class AbstractSortScript implements ScorerAware { - static { - Map deprecations = new HashMap<>(); - deprecations.put( + private static final Map DEPRECATIONS = Map.of( "doc", - "Accessing variable [doc] via [params.doc] from within a sort-script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( + "Accessing variable [doc] via [params.doc] from within a sort-script is deprecated in favor of directly accessing [doc].", "_doc", - "Accessing variable [doc] via [params._doc] from within a sort-script " + - "is deprecated in favor of directly accessing [doc]." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + "Accessing variable [doc] via [params._doc] from within a sort-script is deprecated in favor of directly accessing [doc]."); /** * The generic runtime parameters for the script. diff --git a/server/src/main/java/org/elasticsearch/script/AggregationScript.java b/server/src/main/java/org/elasticsearch/script/AggregationScript.java index f48d097355110..d6ef2e7b14be2 100644 --- a/server/src/main/java/org/elasticsearch/script/AggregationScript.java +++ b/server/src/main/java/org/elasticsearch/script/AggregationScript.java @@ -18,10 +18,6 @@ */ package org.elasticsearch.script; -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorable; import org.elasticsearch.ElasticsearchException; @@ -30,28 +26,23 @@ import org.elasticsearch.search.lookup.LeafSearchLookup; import org.elasticsearch.search.lookup.SearchLookup; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + public abstract class AggregationScript implements ScorerAware { public static final String[] PARAMETERS = {}; public static final ScriptContext CONTEXT = new ScriptContext<>("aggs", Factory.class); - private static final Map DEPRECATIONS; - - static { - Map deprecations = new HashMap<>(); - deprecations.put( + private static final Map DEPRECATIONS = Map.of( "doc", - "Accessing variable [doc] via [params.doc] from within an aggregation-script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( + "Accessing variable [doc] via [params.doc] from within an aggregation-script " + + "is deprecated in favor of directly accessing [doc].", "_doc", - "Accessing variable [doc] via [params._doc] from within an aggregation-script " + - "is deprecated in favor of directly accessing [doc]." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + "Accessing variable [doc] via [params._doc] from within an aggregation-script " + + "is deprecated in favor of directly accessing [doc]."); /** * The generic runtime parameters for the script. diff --git a/server/src/main/java/org/elasticsearch/script/ClassPermission.java b/server/src/main/java/org/elasticsearch/script/ClassPermission.java index f636a0190c47f..dfef473398a86 100644 --- a/server/src/main/java/org/elasticsearch/script/ClassPermission.java +++ b/server/src/main/java/org/elasticsearch/script/ClassPermission.java @@ -23,7 +23,6 @@ import java.security.Permission; import java.security.PermissionCollection; import java.util.Arrays; -import java.util.Collections; import java.util.Enumeration; import java.util.HashSet; import java.util.Set; @@ -75,30 +74,30 @@ */ public final class ClassPermission extends BasicPermission { public static final String STANDARD = "<>"; + // jdk classes /** Typical set of classes for scripting: basic data types, math, dates, and simple collections */ // this is the list from the old grovy sandbox impl (+ some things like String, Iterator, etc that were missing) - public static final Set STANDARD_CLASSES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - // jdk classes - java.lang.Boolean.class.getName(), - java.lang.Byte.class.getName(), - java.lang.Character.class.getName(), - java.lang.Double.class.getName(), - java.lang.Integer.class.getName(), - java.lang.Long.class.getName(), - java.lang.Math.class.getName(), - java.lang.Object.class.getName(), - java.lang.Short.class.getName(), - java.lang.String.class.getName(), + private static final Set STANDARD_CLASSES = Set.of( + Boolean.class.getName(), + Byte.class.getName(), + Character.class.getName(), + Double.class.getName(), + Integer.class.getName(), + Long.class.getName(), + Math.class.getName(), + Object.class.getName(), + Short.class.getName(), + String.class.getName(), java.math.BigDecimal.class.getName(), java.util.ArrayList.class.getName(), - java.util.Arrays.class.getName(), + Arrays.class.getName(), java.util.Date.class.getName(), java.util.HashMap.class.getName(), - java.util.HashSet.class.getName(), + HashSet.class.getName(), java.util.Iterator.class.getName(), java.util.List.class.getName(), java.util.Map.class.getName(), - java.util.Set.class.getName(), + Set.class.getName(), java.util.UUID.class.getName(), // joda-time org.joda.time.DateTime.class.getName(), @@ -106,8 +105,7 @@ public final class ClassPermission extends BasicPermission { org.joda.time.DateTimeZone.class.getName(), org.joda.time.Instant.class.getName(), org.joda.time.ReadableDateTime.class.getName(), - org.joda.time.ReadableInstant.class.getName() - ))); + org.joda.time.ReadableInstant.class.getName()); /** * Creates a new ClassPermission object. diff --git a/server/src/main/java/org/elasticsearch/script/FieldScript.java b/server/src/main/java/org/elasticsearch/script/FieldScript.java index 7707301ab4c1d..806c5b92cb9e3 100644 --- a/server/src/main/java/org/elasticsearch/script/FieldScript.java +++ b/server/src/main/java/org/elasticsearch/script/FieldScript.java @@ -25,7 +25,6 @@ import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -36,22 +35,11 @@ public abstract class FieldScript { public static final String[] PARAMETERS = {}; - private static final Map DEPRECATIONS; - - static { - Map deprecations = new HashMap<>(); - deprecations.put( + private static final Map DEPRECATIONS = Map.of( "doc", - "Accessing variable [doc] via [params.doc] from within a field script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( + "Accessing variable [doc] via [params.doc] from within a field script is deprecated in favor of directly accessing [doc].", "_doc", - "Accessing variable [doc] via [params._doc] from within a field script " + - "is deprecated in favor of directly accessing [doc]." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + "Accessing variable [doc] via [params._doc] from within a field script is deprecated in favor of directly accessing [doc]."); /** The generic runtime parameters for the script. */ private final Map params; diff --git a/server/src/main/java/org/elasticsearch/script/ScoreScript.java b/server/src/main/java/org/elasticsearch/script/ScoreScript.java index f31af4c008c74..f681a2e5f6092 100644 --- a/server/src/main/java/org/elasticsearch/script/ScoreScript.java +++ b/server/src/main/java/org/elasticsearch/script/ScoreScript.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.DoubleSupplier; @@ -36,21 +35,12 @@ */ public abstract class ScoreScript { - private static final Map DEPRECATIONS; - static { - Map deprecations = new HashMap<>(); - deprecations.put( + private static final Map DEPRECATIONS = Map.of( "doc", - "Accessing variable [doc] via [params.doc] from within a score script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( - "_doc", - "Accessing variable [doc] via [params._doc] from within a score script " + - "is deprecated in favor of directly accessing [doc]." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + "Accessing variable [doc] via [params.doc] from within a score script " + + "is deprecated in favor of directly accessing [doc].", + "_doc", "Accessing variable [doc] via [params._doc] from within a score script " + + "is deprecated in favor of directly accessing [doc]."); public static final String[] PARAMETERS = new String[]{}; diff --git a/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java b/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java index f1c3e996ec8be..53c1aa3da1bd4 100644 --- a/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java +++ b/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java @@ -27,7 +27,6 @@ import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -62,27 +61,14 @@ public interface Factory { } public abstract static class MapScript { - private static final Map DEPRECATIONS; - - static { - Map deprecations = new HashMap<>(); - deprecations.put( + private static final Map DEPRECATIONS = Map.of( "doc", - "Accessing variable [doc] via [params.doc] from within a scripted metric agg map script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( - "_doc", - "Accessing variable [doc] via [params._doc] from within a scripted metric agg map script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( - "_agg", - "Accessing variable [_agg] via [params._agg] from within a scripted metric agg map script " + - "is deprecated in favor of using [state]." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + "Accessing variable [doc] via [params.doc] from within a scripted metric agg map script " + + "is deprecated in favor of directly accessing [doc].", + "_doc", "Accessing variable [doc] via [params._doc] from within a scripted metric agg map script " + + "is deprecated in favor of directly accessing [doc].", + "_agg", "Accessing variable [_agg] via [params._agg] from within a scripted metric agg map script " + + "is deprecated in favor of using [state]."); private final Map params; private final Map state; diff --git a/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java b/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java index c651c3ce69041..b3d29ff50f278 100644 --- a/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java +++ b/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java @@ -18,37 +18,28 @@ */ package org.elasticsearch.script; -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.index.fielddata.ScriptDocValues; import org.elasticsearch.search.lookup.LeafSearchLookup; import org.elasticsearch.search.lookup.SearchLookup; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + public abstract class TermsSetQueryScript { public static final String[] PARAMETERS = {}; public static final ScriptContext CONTEXT = new ScriptContext<>("terms_set", Factory.class); - private static final Map DEPRECATIONS; - - static { - Map deprecations = new HashMap<>(); - deprecations.put( + private static final Map DEPRECATIONS = Map.of( "doc", - "Accessing variable [doc] via [params.doc] from within a terms-set-query-script " + - "is deprecated in favor of directly accessing [doc]." - ); - deprecations.put( + "Accessing variable [doc] via [params.doc] from within a terms-set-query-script " + + "is deprecated in favor of directly accessing [doc].", "_doc", - "Accessing variable [doc] via [params._doc] from within a terms-set-query-script " + - "is deprecated in favor of directly accessing [doc]." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + "Accessing variable [doc] via [params._doc] from within a terms-set-query-script " + + "is deprecated in favor of directly accessing [doc]."); /** * The generic runtime parameters for the script. diff --git a/server/src/main/java/org/elasticsearch/script/UpdateScript.java b/server/src/main/java/org/elasticsearch/script/UpdateScript.java index 765489b7e4449..ae0827ff83934 100644 --- a/server/src/main/java/org/elasticsearch/script/UpdateScript.java +++ b/server/src/main/java/org/elasticsearch/script/UpdateScript.java @@ -20,8 +20,6 @@ package org.elasticsearch.script; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; /** @@ -29,15 +27,8 @@ */ public abstract class UpdateScript { - private static final Map DEPRECATIONS; - static { - Map deprecations = new HashMap<>(); - deprecations.put( - "_type", - "[types removal] Looking up doc types [_type] in scripts is deprecated." - ); - DEPRECATIONS = Collections.unmodifiableMap(deprecations); - } + private static final Map DEPRECATIONS = + Map.of("_type", "[types removal] Looking up doc types [_type] in scripts is deprecated."); public static final String[] PARAMETERS = { }; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index d607f4bfaf204..dc5af1582f15a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -59,12 +59,11 @@ import java.time.ZoneId; import java.time.ZoneOffset; import java.time.zone.ZoneOffsetTransition; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import static java.util.Collections.unmodifiableMap; +import static java.util.Map.entry; /** * A builder for histograms on date fields. @@ -75,28 +74,23 @@ public class DateHistogramAggregationBuilder extends ValuesSourceAggregationBuil public static final String NAME = "date_histogram"; private static DateMathParser EPOCH_MILLIS_PARSER = DateFormatter.forPattern("epoch_millis").toDateMathParser(); - public static final Map DATE_FIELD_UNITS; - - static { - Map dateFieldUnits = new HashMap<>(); - dateFieldUnits.put("year", Rounding.DateTimeUnit.YEAR_OF_CENTURY); - dateFieldUnits.put("1y", Rounding.DateTimeUnit.YEAR_OF_CENTURY); - dateFieldUnits.put("quarter", Rounding.DateTimeUnit.QUARTER_OF_YEAR); - dateFieldUnits.put("1q", Rounding.DateTimeUnit.QUARTER_OF_YEAR); - dateFieldUnits.put("month", Rounding.DateTimeUnit.MONTH_OF_YEAR); - dateFieldUnits.put("1M", Rounding.DateTimeUnit.MONTH_OF_YEAR); - dateFieldUnits.put("week", Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR); - dateFieldUnits.put("1w", Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR); - dateFieldUnits.put("day", Rounding.DateTimeUnit.DAY_OF_MONTH); - dateFieldUnits.put("1d", Rounding.DateTimeUnit.DAY_OF_MONTH); - dateFieldUnits.put("hour", Rounding.DateTimeUnit.HOUR_OF_DAY); - dateFieldUnits.put("1h", Rounding.DateTimeUnit.HOUR_OF_DAY); - dateFieldUnits.put("minute", Rounding.DateTimeUnit.MINUTES_OF_HOUR); - dateFieldUnits.put("1m", Rounding.DateTimeUnit.MINUTES_OF_HOUR); - dateFieldUnits.put("second", Rounding.DateTimeUnit.SECOND_OF_MINUTE); - dateFieldUnits.put("1s", Rounding.DateTimeUnit.SECOND_OF_MINUTE); - DATE_FIELD_UNITS = unmodifiableMap(dateFieldUnits); - } + public static final Map DATE_FIELD_UNITS = Map.ofEntries( + entry("year", Rounding.DateTimeUnit.YEAR_OF_CENTURY), + entry("1y", Rounding.DateTimeUnit.YEAR_OF_CENTURY), + entry("quarter", Rounding.DateTimeUnit.QUARTER_OF_YEAR), + entry("1q", Rounding.DateTimeUnit.QUARTER_OF_YEAR), + entry("month", Rounding.DateTimeUnit.MONTH_OF_YEAR), + entry("1M", Rounding.DateTimeUnit.MONTH_OF_YEAR), + entry("week", Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR), + entry("1w", Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR), + entry("day", Rounding.DateTimeUnit.DAY_OF_MONTH), + entry("1d", Rounding.DateTimeUnit.DAY_OF_MONTH), + entry("hour", Rounding.DateTimeUnit.HOUR_OF_DAY), + entry("1h", Rounding.DateTimeUnit.HOUR_OF_DAY), + entry("minute", Rounding.DateTimeUnit.MINUTES_OF_HOUR), + entry("1m", Rounding.DateTimeUnit.MINUTES_OF_HOUR), + entry("second", Rounding.DateTimeUnit.SECOND_OF_MINUTE), + entry("1s", Rounding.DateTimeUnit.SECOND_OF_MINUTE)); private static final ObjectParser PARSER; static { diff --git a/server/src/main/java/org/elasticsearch/search/lookup/LeafSearchLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/LeafSearchLookup.java index e76206d6cba2a..a903593c85e67 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/LeafSearchLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/LeafSearchLookup.java @@ -21,35 +21,29 @@ import org.apache.lucene.index.LeafReaderContext; -import java.util.HashMap; import java.util.Map; -import static java.util.Collections.unmodifiableMap; - /** * Per-segment version of {@link SearchLookup}. */ public class LeafSearchLookup { - final LeafReaderContext ctx; - final LeafDocLookup docMap; - final SourceLookup sourceLookup; - final LeafFieldsLookup fieldsLookup; - final Map asMap; + private final LeafReaderContext ctx; + private final LeafDocLookup docMap; + private final SourceLookup sourceLookup; + private final LeafFieldsLookup fieldsLookup; + private final Map asMap; - public LeafSearchLookup(LeafReaderContext ctx, LeafDocLookup docMap, SourceLookup sourceLookup, - LeafFieldsLookup fieldsLookup) { + public LeafSearchLookup(LeafReaderContext ctx, LeafDocLookup docMap, SourceLookup sourceLookup, LeafFieldsLookup fieldsLookup) { this.ctx = ctx; this.docMap = docMap; this.sourceLookup = sourceLookup; this.fieldsLookup = fieldsLookup; - - Map asMap = new HashMap<>(4); - asMap.put("doc", docMap); - asMap.put("_doc", docMap); - asMap.put("_source", sourceLookup); - asMap.put("_fields", fieldsLookup); - this.asMap = unmodifiableMap(asMap); + this.asMap = Map.of( + "doc", docMap, + "_doc", docMap, + "_source", sourceLookup, + "_fields", fieldsLookup); } public Map asMap() { diff --git a/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java index abef6d8ebec33..b4358abee0728 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java @@ -40,13 +40,11 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; -import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.index.query.AbstractQueryBuilder.parseInnerQueryBuilder; public abstract class SortBuilder> implements NamedWriteable, ToXContentObject, Rewriteable> { @@ -58,16 +56,12 @@ public abstract class SortBuilder> implements NamedWrit public static final ParseField NESTED_FILTER_FIELD = new ParseField("nested_filter"); public static final ParseField NESTED_PATH_FIELD = new ParseField("nested_path"); - private static final Map> PARSERS; - static { - Map> parsers = new HashMap<>(); - parsers.put(ScriptSortBuilder.NAME, ScriptSortBuilder::fromXContent); - parsers.put(GeoDistanceSortBuilder.NAME, GeoDistanceSortBuilder::fromXContent); - parsers.put(GeoDistanceSortBuilder.ALTERNATIVE_NAME, GeoDistanceSortBuilder::fromXContent); - parsers.put(ScoreSortBuilder.NAME, ScoreSortBuilder::fromXContent); - // FieldSortBuilder gets involved if the user specifies a name that isn't one of these. - PARSERS = unmodifiableMap(parsers); - } + private static final Map> PARSERS = Map.of( + ScriptSortBuilder.NAME, ScriptSortBuilder::fromXContent, + GeoDistanceSortBuilder.NAME, GeoDistanceSortBuilder::fromXContent, + GeoDistanceSortBuilder.ALTERNATIVE_NAME, GeoDistanceSortBuilder::fromXContent, + // TODO: this can deadlock as it might access the ScoreSortBuilder (subclass) initializer from the SortBuilder initializer!!! + ScoreSortBuilder.NAME, ScoreSortBuilder::fromXContent); /** * Create a @link {@link SortFieldAndFormat} from this builder. diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java index f4a1b250e7f5e..6b6ce18cfefbb 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java @@ -154,7 +154,7 @@ synchronized void updateSeedNodes( final String proxyAddress, final List>> seedNodes, final ActionListener connectListener) { - this.seedNodes = Collections.unmodifiableList(new ArrayList<>(seedNodes)); + this.seedNodes = List.copyOf(seedNodes); this.proxyAddress = proxyAddress; connectHandler.connect(connectListener); } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportService.java b/server/src/main/java/org/elasticsearch/transport/TransportService.java index c8493edc97983..52a5a15e405b4 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportService.java @@ -56,9 +56,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.net.UnknownHostException; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -757,17 +755,16 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi /** * A set of all valid action prefixes. */ - public static final Set VALID_ACTION_PREFIXES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - "indices:admin", - "indices:monitor", - "indices:data/write", - "indices:data/read", - "indices:internal", - "cluster:admin", - "cluster:monitor", - "cluster:internal", - "internal:" - ))); + public static final Set VALID_ACTION_PREFIXES = Set.of( + "indices:admin", + "indices:monitor", + "indices:data/write", + "indices:data/read", + "indices:internal", + "cluster:admin", + "cluster:monitor", + "cluster:internal", + "internal:"); private void validateActionName(String actionName) { // TODO we should makes this a hard validation and throw an exception but we need a good way to add backwards layer diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/RecordingTaskManagerListener.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/RecordingTaskManagerListener.java index 5046a48b938f6..5050dac2a4e3f 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/RecordingTaskManagerListener.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/RecordingTaskManagerListener.java @@ -64,7 +64,7 @@ public void waitForTaskCompletion(Task task) { } public synchronized List> getEvents() { - return Collections.unmodifiableList(new ArrayList<>(events)); + return List.copyOf(events); } public synchronized List getRegistrationEvents() { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java index 18e5702ed6a77..d39ab823dc490 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TestTaskPlugin.java @@ -23,8 +23,8 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.nodes.BaseNodeRequest; @@ -49,8 +49,8 @@ import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.plugins.ActionPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.NetworkPlugin; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; @@ -59,10 +59,10 @@ import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.ArrayList; @@ -416,7 +416,7 @@ public static class UnblockTestTasksResponse extends BaseTasksResponse { public UnblockTestTasksResponse(List tasks, List taskFailures, List nodeFailures) { super(taskFailures, nodeFailures); - this.tasks = tasks == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(tasks)); + this.tasks = tasks == null ? Collections.emptyList() : List.copyOf(tasks); } public UnblockTestTasksResponse(StreamInput in) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java index ec8af36dbf27e..e9b940df3847d 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java @@ -205,7 +205,7 @@ static class TestTasksResponse extends BaseTasksResponse { TestTasksResponse(List tasks, List taskFailures, List nodeFailures) { super(taskFailures, nodeFailures); - this.tasks = tasks == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(tasks)); + this.tasks = tasks == null ? Collections.emptyList() : List.copyOf(tasks); } TestTasksResponse(StreamInput in) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteRequestTests.java index 8d1a306f26094..dfef08944de60 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteRequestTests.java @@ -44,13 +44,11 @@ import org.elasticsearch.test.rest.FakeRestRequest; import java.io.IOException; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; -import static java.util.Collections.unmodifiableList; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; /** @@ -58,18 +56,20 @@ */ public class ClusterRerouteRequestTests extends ESTestCase { private static final int ROUNDS = 30; - private final List> RANDOM_COMMAND_GENERATORS = unmodifiableList( - Arrays.> asList( - () -> new AllocateReplicaAllocationCommand(randomAlphaOfLengthBetween(2, 10), between(0, 1000), - randomAlphaOfLengthBetween(2, 10)), - () -> new AllocateEmptyPrimaryAllocationCommand(randomAlphaOfLengthBetween(2, 10), between(0, 1000), - randomAlphaOfLengthBetween(2, 10), randomBoolean()), - () -> new AllocateStalePrimaryAllocationCommand(randomAlphaOfLengthBetween(2, 10), between(0, 1000), - randomAlphaOfLengthBetween(2, 10), randomBoolean()), - () -> new CancelAllocationCommand(randomAlphaOfLengthBetween(2, 10), between(0, 1000), - randomAlphaOfLengthBetween(2, 10), randomBoolean()), - () -> new MoveAllocationCommand(randomAlphaOfLengthBetween(2, 10), between(0, 1000), - randomAlphaOfLengthBetween(2, 10), randomAlphaOfLengthBetween(2, 10)))); + private final List> RANDOM_COMMAND_GENERATORS = List.of( + () -> new AllocateReplicaAllocationCommand( + randomAlphaOfLengthBetween(2, 10), between(0, 1000), randomAlphaOfLengthBetween(2, 10)), + () -> new AllocateEmptyPrimaryAllocationCommand( + randomAlphaOfLengthBetween(2, 10), between(0, 1000), randomAlphaOfLengthBetween(2, 10), randomBoolean()), + () -> new AllocateStalePrimaryAllocationCommand( + randomAlphaOfLengthBetween(2, 10), between(0, 1000), randomAlphaOfLengthBetween(2, 10), randomBoolean()), + () -> new CancelAllocationCommand( + randomAlphaOfLengthBetween(2, 10), between(0, 1000), randomAlphaOfLengthBetween(2, 10), randomBoolean()), + () -> new MoveAllocationCommand( + randomAlphaOfLengthBetween(2, 10), + between(0, 1000), + randomAlphaOfLengthBetween(2, 10), + randomAlphaOfLengthBetween(2, 10))); private final NamedWriteableRegistry namedWriteableRegistry; public ClusterRerouteRequestTests() { diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java index 8f2db1281485d..580ea76429388 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportResizeActionTests.java @@ -42,9 +42,8 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.gateway.TestGatewayAllocator; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; +import java.util.Set; import static java.util.Collections.emptyMap; import static org.hamcrest.Matchers.equalTo; @@ -216,7 +215,12 @@ public void testShrinkIndexSettings() { } private DiscoveryNode newNode(String nodeId) { - return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), - Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA))), Version.CURRENT); + return new DiscoveryNode( + nodeId, + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA), + Version.CURRENT); } + } diff --git a/server/src/test/java/org/elasticsearch/bootstrap/ElasticsearchUncaughtExceptionHandlerTests.java b/server/src/test/java/org/elasticsearch/bootstrap/ElasticsearchUncaughtExceptionHandlerTests.java index a6e50170d7e80..df415d713d103 100644 --- a/server/src/test/java/org/elasticsearch/bootstrap/ElasticsearchUncaughtExceptionHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/bootstrap/ElasticsearchUncaughtExceptionHandlerTests.java @@ -20,13 +20,10 @@ package org.elasticsearch.bootstrap; import org.elasticsearch.test.ESTestCase; -import org.junit.Before; import java.io.IOError; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -36,19 +33,12 @@ public class ElasticsearchUncaughtExceptionHandlerTests extends ESTestCase { - private Map, Integer> expectedStatus; - - @Before - public void setUp() throws Exception { - super.setUp(); - Map, Integer> expectedStatus = new HashMap<>(); - expectedStatus.put(InternalError.class, 128); - expectedStatus.put(OutOfMemoryError.class, 127); - expectedStatus.put(StackOverflowError.class, 126); - expectedStatus.put(UnknownError.class, 125); - expectedStatus.put(IOError.class, 124); - this.expectedStatus = Collections.unmodifiableMap(expectedStatus); - } + private static Map, Integer> EXPECTED_STATUS = Map.of( + InternalError.class, 128, + OutOfMemoryError.class, 127, + StackOverflowError.class, 126, + UnknownError.class, 125, + IOError.class, 124); public void testUncaughtError() throws InterruptedException { final Error error = randomFrom( @@ -89,8 +79,8 @@ void onNonFatalUncaught(String threadName, Throwable t) { thread.join(); assertTrue(halt.get()); final int status; - if (expectedStatus.containsKey(error.getClass())) { - status = expectedStatus.get(error.getClass()); + if (EXPECTED_STATUS.containsKey(error.getClass())) { + status = EXPECTED_STATUS.get(error.getClass()); } else { status = 1; } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java index ec89e085f0784..e3ba62b3b79ab 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java @@ -50,7 +50,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; @@ -409,8 +408,11 @@ private void runPrepareResizeIndexSettingsTest( } private DiscoveryNode newNode(String nodeId) { - return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), - Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA))), Version.CURRENT); + return new DiscoveryNode( + nodeId, + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA), Version.CURRENT); } public void testValidateIndexName() throws Exception { 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 e9bae772f4ea0..59c11b2bc9442 100644 --- a/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/structure/RoutingIteratorTests.java @@ -43,12 +43,10 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.Map; import static java.util.Collections.singletonMap; -import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; @@ -250,15 +248,9 @@ public void testAttributePreferenceRouting() { ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING .getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build(); - Map node1Attributes = new HashMap<>(); - node1Attributes.put("rack_id", "rack_1"); - node1Attributes.put("zone", "zone1"); - Map node2Attributes = new HashMap<>(); - node2Attributes.put("rack_id", "rack_2"); - node2Attributes.put("zone", "zone2"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder() - .add(newNode("node1", unmodifiableMap(node1Attributes))) - .add(newNode("node2", unmodifiableMap(node2Attributes))) + .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"); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ExternalMapperPlugin.java b/server/src/test/java/org/elasticsearch/index/mapper/ExternalMapperPlugin.java index 0824e1e0afcc7..6f1a96992f5d6 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/ExternalMapperPlugin.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/ExternalMapperPlugin.java @@ -19,15 +19,12 @@ package org.elasticsearch.index.mapper; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.plugins.MapperPlugin; import org.elasticsearch.plugins.Plugin; +import java.util.Collections; +import java.util.Map; + public class ExternalMapperPlugin extends Plugin implements MapperPlugin { public static final String EXTERNAL = "external"; @@ -36,12 +33,11 @@ public class ExternalMapperPlugin extends Plugin implements MapperPlugin { @Override public Map getMappers() { - Map mappers = new HashMap<>(); - mappers.put(EXTERNAL, new ExternalMapper.TypeParser(EXTERNAL, "foo")); - mappers.put(EXTERNAL_BIS, new ExternalMapper.TypeParser(EXTERNAL_BIS, "bar")); - mappers.put(EXTERNAL_UPPER, new ExternalMapper.TypeParser(EXTERNAL_UPPER, "FOO BAR")); - mappers.put(FakeStringFieldMapper.CONTENT_TYPE, new FakeStringFieldMapper.TypeParser()); - return Collections.unmodifiableMap(mappers); + return Map.of( + EXTERNAL, new ExternalMapper.TypeParser(EXTERNAL, "foo"), + EXTERNAL_BIS, new ExternalMapper.TypeParser(EXTERNAL_BIS, "bar"), + EXTERNAL_UPPER, new ExternalMapper.TypeParser(EXTERNAL_UPPER, "FOO BAR"), + FakeStringFieldMapper.CONTENT_TYPE, new FakeStringFieldMapper.TypeParser()); } @Override diff --git a/server/src/test/java/org/elasticsearch/index/store/StoreTests.java b/server/src/test/java/org/elasticsearch/index/store/StoreTests.java index fc5870bc8e12c..9e8fae209dd81 100644 --- a/server/src/test/java/org/elasticsearch/index/store/StoreTests.java +++ b/server/src/test/java/org/elasticsearch/index/store/StoreTests.java @@ -825,10 +825,7 @@ protected Store.MetadataSnapshot createMetaDataSnapshot() { Map storeFileMetaDataMap = new HashMap<>(); storeFileMetaDataMap.put(storeFileMetaData1.name(), storeFileMetaData1); storeFileMetaDataMap.put(storeFileMetaData2.name(), storeFileMetaData2); - Map commitUserData = new HashMap<>(); - commitUserData.put("userdata_1", "test"); - commitUserData.put("userdata_2", "test"); - return new Store.MetadataSnapshot(unmodifiableMap(storeFileMetaDataMap), unmodifiableMap(commitUserData), 0); + return new Store.MetadataSnapshot(unmodifiableMap(storeFileMetaDataMap), Map.of("userdata_1", "test", "userdata_2", "test"), 0); } public void testUserDataRead() throws IOException { diff --git a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java index c3863e8ec08e4..f29953633a1bb 100644 --- a/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java +++ b/server/src/test/java/org/elasticsearch/persistent/PersistentTasksClusterServiceTests.java @@ -51,10 +51,8 @@ import org.junit.BeforeClass; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; @@ -791,8 +789,11 @@ private String addTask(PersistentTasksCustomMetaData.Builder tasks, String param } private DiscoveryNode newNode(String nodeId) { - return new DiscoveryNode(nodeId, buildNewFakeTransportAddress(), emptyMap(), - Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA))), + return new DiscoveryNode( + nodeId, + buildNewFakeTransportAddress(), + emptyMap(), + Set.of(DiscoveryNode.Role.MASTER, DiscoveryNode.Role.DATA), Version.CURRENT); } diff --git a/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java b/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java index 151129c0cc1fa..0b5c432d5978c 100644 --- a/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java +++ b/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java @@ -62,7 +62,6 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -500,7 +499,7 @@ public static class TestTasksResponse extends BaseTasksResponse { public TestTasksResponse(List tasks, List taskFailures, List nodeFailures) { super(taskFailures, nodeFailures); - this.tasks = tasks == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(tasks)); + this.tasks = tasks == null ? Collections.emptyList() : List.copyOf(tasks); } public TestTasksResponse(StreamInput in) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java index 012171ec25a0b..c8163539d4a55 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/AggregationsTests.java @@ -54,30 +54,30 @@ import org.elasticsearch.search.aggregations.bucket.terms.DoubleTermsTests; import org.elasticsearch.search.aggregations.bucket.terms.LongTermsTests; import org.elasticsearch.search.aggregations.bucket.terms.StringTermsTests; +import org.elasticsearch.search.aggregations.metrics.InternalAvgTests; +import org.elasticsearch.search.aggregations.metrics.InternalCardinalityTests; import org.elasticsearch.search.aggregations.metrics.InternalExtendedStatsTests; +import org.elasticsearch.search.aggregations.metrics.InternalGeoBoundsTests; +import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroidTests; +import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentilesRanksTests; +import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentilesTests; import org.elasticsearch.search.aggregations.metrics.InternalMaxTests; import org.elasticsearch.search.aggregations.metrics.InternalMedianAbsoluteDeviationTests; import org.elasticsearch.search.aggregations.metrics.InternalMinTests; +import org.elasticsearch.search.aggregations.metrics.InternalScriptedMetricTests; import org.elasticsearch.search.aggregations.metrics.InternalStatsBucketTests; import org.elasticsearch.search.aggregations.metrics.InternalStatsTests; import org.elasticsearch.search.aggregations.metrics.InternalSumTests; -import org.elasticsearch.search.aggregations.metrics.InternalAvgTests; -import org.elasticsearch.search.aggregations.metrics.InternalCardinalityTests; -import org.elasticsearch.search.aggregations.metrics.InternalGeoBoundsTests; -import org.elasticsearch.search.aggregations.metrics.InternalGeoCentroidTests; -import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentilesRanksTests; -import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentilesTests; import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentilesRanksTests; import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentilesTests; -import org.elasticsearch.search.aggregations.metrics.InternalScriptedMetricTests; import org.elasticsearch.search.aggregations.metrics.InternalTopHitsTests; import org.elasticsearch.search.aggregations.metrics.InternalValueCountTests; import org.elasticsearch.search.aggregations.metrics.InternalWeightedAvgTests; -import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValueTests; import org.elasticsearch.search.aggregations.pipeline.InternalBucketMetricValueTests; -import org.elasticsearch.search.aggregations.pipeline.InternalPercentilesBucketTests; -import org.elasticsearch.search.aggregations.pipeline.InternalExtendedStatsBucketTests; import org.elasticsearch.search.aggregations.pipeline.InternalDerivativeTests; +import org.elasticsearch.search.aggregations.pipeline.InternalExtendedStatsBucketTests; +import org.elasticsearch.search.aggregations.pipeline.InternalPercentilesBucketTests; +import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValueTests; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalAggregationTestCase; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; @@ -87,7 +87,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Set; import java.util.function.Predicate; @@ -103,59 +102,54 @@ */ public class AggregationsTests extends ESTestCase { - private static final List> aggsTests = getAggsTests(); - - private static List> getAggsTests() { - List> aggsTests = new ArrayList<>(); - aggsTests.add(new InternalCardinalityTests()); - aggsTests.add(new InternalTDigestPercentilesTests()); - aggsTests.add(new InternalTDigestPercentilesRanksTests()); - aggsTests.add(new InternalHDRPercentilesTests()); - aggsTests.add(new InternalHDRPercentilesRanksTests()); - aggsTests.add(new InternalPercentilesBucketTests()); - aggsTests.add(new InternalMinTests()); - aggsTests.add(new InternalMaxTests()); - aggsTests.add(new InternalAvgTests()); - aggsTests.add(new InternalWeightedAvgTests()); - aggsTests.add(new InternalSumTests()); - aggsTests.add(new InternalValueCountTests()); - aggsTests.add(new InternalSimpleValueTests()); - aggsTests.add(new InternalDerivativeTests()); - aggsTests.add(new InternalBucketMetricValueTests()); - aggsTests.add(new InternalStatsTests()); - aggsTests.add(new InternalStatsBucketTests()); - aggsTests.add(new InternalExtendedStatsTests()); - aggsTests.add(new InternalExtendedStatsBucketTests()); - aggsTests.add(new InternalGeoBoundsTests()); - aggsTests.add(new InternalGeoCentroidTests()); - aggsTests.add(new InternalHistogramTests()); - aggsTests.add(new InternalDateHistogramTests()); - aggsTests.add(new InternalAutoDateHistogramTests()); - aggsTests.add(new LongTermsTests()); - aggsTests.add(new DoubleTermsTests()); - aggsTests.add(new StringTermsTests()); - aggsTests.add(new InternalMissingTests()); - aggsTests.add(new InternalNestedTests()); - aggsTests.add(new InternalReverseNestedTests()); - aggsTests.add(new InternalGlobalTests()); - aggsTests.add(new InternalFilterTests()); - aggsTests.add(new InternalSamplerTests()); - aggsTests.add(new GeoHashGridTests()); - aggsTests.add(new GeoTileGridTests()); - aggsTests.add(new InternalRangeTests()); - aggsTests.add(new InternalDateRangeTests()); - aggsTests.add(new InternalGeoDistanceTests()); - aggsTests.add(new InternalFiltersTests()); - aggsTests.add(new InternalAdjacencyMatrixTests()); - aggsTests.add(new SignificantLongTermsTests()); - aggsTests.add(new SignificantStringTermsTests()); - aggsTests.add(new InternalScriptedMetricTests()); - aggsTests.add(new InternalBinaryRangeTests()); - aggsTests.add(new InternalTopHitsTests()); - aggsTests.add(new InternalCompositeTests()); - aggsTests.add(new InternalMedianAbsoluteDeviationTests()); - return Collections.unmodifiableList(aggsTests); - } + private static final List> aggsTests = List.of( + new InternalCardinalityTests(), + new InternalTDigestPercentilesTests(), + new InternalTDigestPercentilesRanksTests(), + new InternalHDRPercentilesTests(), + new InternalHDRPercentilesRanksTests(), + new InternalPercentilesBucketTests(), + new InternalMinTests(), + new InternalMaxTests(), + new InternalAvgTests(), + new InternalWeightedAvgTests(), + new InternalSumTests(), + new InternalValueCountTests(), + new InternalSimpleValueTests(), + new InternalDerivativeTests(), + new InternalBucketMetricValueTests(), + new InternalStatsTests(), + new InternalStatsBucketTests(), + new InternalExtendedStatsTests(), + new InternalExtendedStatsBucketTests(), + new InternalGeoBoundsTests(), + new InternalGeoCentroidTests(), + new InternalHistogramTests(), + new InternalDateHistogramTests(), + new InternalAutoDateHistogramTests(), + new LongTermsTests(), + new DoubleTermsTests(), + new StringTermsTests(), + new InternalMissingTests(), + new InternalNestedTests(), + new InternalReverseNestedTests(), + new InternalGlobalTests(), + new InternalFilterTests(), + new InternalSamplerTests(), + new GeoHashGridTests(), + new GeoTileGridTests(), + new InternalRangeTests(), + new InternalDateRangeTests(), + new InternalGeoDistanceTests(), + new InternalFiltersTests(), + new InternalAdjacencyMatrixTests(), + new SignificantLongTermsTests(), + new SignificantStringTermsTests(), + new InternalScriptedMetricTests(), + new InternalBinaryRangeTests(), + new InternalTopHitsTests(), + new InternalCompositeTests(), + new InternalMedianAbsoluteDeviationTests()); @Override protected NamedXContentRegistry xContentRegistry() { diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index e41898727d7e2..3a78b4786fc5c 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -2704,19 +2704,17 @@ public ClusterState execute(ClusterState currentState) { shards.put(new ShardId(idxName, "_na_", 0), new ShardSnapshotStatus("unknown-node", State.ABORTED, "aborted")); shards.put(new ShardId(idxName, "_na_", 1), new ShardSnapshotStatus("unknown-node", State.ABORTED, "aborted")); shards.put(new ShardId(idxName, "_na_", 2), new ShardSnapshotStatus("unknown-node", State.ABORTED, "aborted")); - List entries = new ArrayList<>(); - entries.add(new Entry(new Snapshot(repositoryName, - createSnapshotResponse.getSnapshotInfo().snapshotId()), - true, - false, - State.ABORTED, - Collections.singletonList(indexId), - System.currentTimeMillis(), - repositoryData.getGenId(), - shards.build())); return ClusterState.builder(currentState) - .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(Collections.unmodifiableList(entries))) - .build(); + .putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(List.of(new Entry( + new Snapshot(repositoryName, createSnapshotResponse.getSnapshotInfo().snapshotId()), + true, + false, + State.ABORTED, + Collections.singletonList(indexId), + System.currentTimeMillis(), + repositoryData.getGenId(), + shards.build())))) + .build(); } @Override diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index d657952260e81..134a91ac296c1 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -1175,12 +1175,10 @@ private boolean isTransportClient(final Client client) { } private static final Set SAFE_METADATA_CUSTOMS = - Collections.unmodifiableSet( - new HashSet<>(Arrays.asList(IndexGraveyard.TYPE, IngestMetadata.TYPE, RepositoriesMetaData.TYPE, ScriptMetaData.TYPE))); + Set.of(IndexGraveyard.TYPE, IngestMetadata.TYPE, RepositoriesMetaData.TYPE, ScriptMetaData.TYPE); private static final Set SAFE_CUSTOMS = - Collections.unmodifiableSet( - new HashSet<>(Arrays.asList(RestoreInProgress.TYPE, SnapshotDeletionsInProgress.TYPE, SnapshotsInProgress.TYPE))); + Set.of(RestoreInProgress.TYPE, SnapshotDeletionsInProgress.TYPE, SnapshotsInProgress.TYPE); /** * Remove any customs except for customs that we know all clients understand. diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java index bdcf426d118f3..c53627b0b15aa 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java @@ -19,12 +19,9 @@ package org.elasticsearch.test.rest.yaml; -import java.util.Arrays; -import java.util.List; - import org.elasticsearch.test.rest.ESRestTestCase; -import static java.util.Collections.unmodifiableList; +import java.util.List; /** * Allows to register additional features supported by the tests runner. @@ -35,7 +32,8 @@ * and the related skip sections can be removed from the tests as well. */ public final class Features { - private static final List SUPPORTED = unmodifiableList(Arrays.asList( + + private static final List SUPPORTED = List.of( "catch_unauthorized", "default_shards", "embedded_stash_key", @@ -48,8 +46,7 @@ public final class Features { "yaml", "contains", "transform_and_set", - "arbitrary_key" - )); + "arbitrary_key"); private Features() { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java index 135a60cca3431..19690b235281f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/section/ExecutableSection.java @@ -25,11 +25,8 @@ import org.elasticsearch.test.rest.yaml.ClientYamlTestExecutionContext; import java.io.IOException; -import java.util.Arrays; import java.util.List; -import static java.util.Collections.unmodifiableList; - /** * Represents a test fragment that can be executed (e.g. api call, assertion) */ @@ -37,7 +34,7 @@ public interface ExecutableSection { /** * Default list of {@link ExecutableSection}s available for tests. */ - List DEFAULT_EXECUTABLE_CONTEXTS = unmodifiableList(Arrays.asList( + List DEFAULT_EXECUTABLE_CONTEXTS = List.of( new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("do"), DoSection::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("set"), SetSection::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("transform_and_set"), TransformAndSetSection::parse), @@ -49,7 +46,7 @@ public interface ExecutableSection { new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("lt"), LessThanAssertion::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("lte"), LessThanOrEqualToAssertion::parse), new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("contains"), ContainsAssertion::parse), - new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("length"), LengthAssertion::parse))); + new NamedXContentRegistry.Entry(ExecutableSection.class, new ParseField("length"), LengthAssertion::parse)); /** * {@link NamedXContentRegistry} that parses the default list of diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java index 8852e81607c16..b00bf15782fa3 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTask.java @@ -21,9 +21,7 @@ import org.elasticsearch.xpack.core.ccr.action.ImmutableFollowParameters; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -34,7 +32,7 @@ public class ShardFollowTask extends ImmutableFollowParameters implements XPackP // list of headers that will be stored when a job is created public static final Set HEADER_FILTERS = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList("es-security-runas-user", "_xpack_security_authentication"))); + Set.of("es-security-runas-user", "_xpack_security_authentication"); private static final ParseField REMOTE_CLUSTER_FIELD = new ParseField("remote_cluster"); private static final ParseField FOLLOW_SHARD_INDEX_FIELD = new ParseField("follow_shard_index"); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java index cf35387c6ece4..ec71c93a33f30 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java @@ -50,8 +50,6 @@ import org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction; import java.io.IOException; -import java.util.Collections; -import java.util.HashSet; import java.util.Iterator; import java.util.Map; import java.util.Objects; @@ -351,98 +349,86 @@ static String[] extractLeaderShardHistoryUUIDs(Map ccrIndexMetaD * These dynamic settings don't affect how documents are indexed (affect index time text analysis) and / or * are inconvenient if they were replicated (e.g. changing number of replicas). */ - static final Set> NON_REPLICATED_SETTINGS; - - static { - final Set> nonReplicatedSettings = new HashSet<>(); - nonReplicatedSettings.add(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_READ_ONLY_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_BLOCKS_READ_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_BLOCKS_METADATA_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING); - nonReplicatedSettings.add(IndexMetaData.INDEX_PRIORITY_SETTING); - nonReplicatedSettings.add(IndexMetaData.SETTING_WAIT_FOR_ACTIVE_SHARDS); - - nonReplicatedSettings.add(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING); - nonReplicatedSettings.add(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING); - nonReplicatedSettings.add(ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING); - nonReplicatedSettings.add(MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY); - nonReplicatedSettings.add(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING); - - nonReplicatedSettings.add(IndexSettings.MAX_RESULT_WINDOW_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_WARMER_ENABLED_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_RESCORE_WINDOW_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING); - nonReplicatedSettings.add(IndexSettings.DEFAULT_FIELD_SETTING); - nonReplicatedSettings.add(IndexSettings.QUERY_STRING_LENIENT_SETTING); - nonReplicatedSettings.add(IndexSettings.QUERY_STRING_ANALYZE_WILDCARD); - nonReplicatedSettings.add(IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD); - nonReplicatedSettings.add(IndexSettings.ALLOW_UNMAPPED); - nonReplicatedSettings.add(IndexSettings.INDEX_SEARCH_IDLE_AFTER); - nonReplicatedSettings.add(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_SCRIPT_FIELDS_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_REGEX_LENGTH_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_TERMS_COUNT_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_ANALYZED_OFFSET_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_DOCVALUE_FIELDS_SEARCH_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_TOKEN_COUNT_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_SLICES_PER_SCROLL); - nonReplicatedSettings.add(IndexSettings.MAX_ADJACENCY_MATRIX_FILTERS_SETTING); - nonReplicatedSettings.add(IndexSettings.DEFAULT_PIPELINE); - nonReplicatedSettings.add(IndexSettings.INDEX_SEARCH_THROTTLED); - nonReplicatedSettings.add(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING); - nonReplicatedSettings.add(IndexSettings.INDEX_GC_DELETES_SETTING); - nonReplicatedSettings.add(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD); - - nonReplicatedSettings.add(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING); - nonReplicatedSettings.add(BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING); - - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE_SETTING); - nonReplicatedSettings.add(SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG_SETTING); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO_SETTING); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE_SETTING); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL_SETTING); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING); - nonReplicatedSettings.add(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG_SETTING); - - nonReplicatedSettings.add(MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_DELETES_PCT_ALLOWED_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING); - nonReplicatedSettings.add(MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING); - - nonReplicatedSettings.add(MergeSchedulerConfig.AUTO_THROTTLE_SETTING); - nonReplicatedSettings.add(MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING); - nonReplicatedSettings.add(MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING); - nonReplicatedSettings.add(EngineConfig.INDEX_CODEC_SETTING); - - NON_REPLICATED_SETTINGS = Collections.unmodifiableSet(nonReplicatedSettings); - } + static final Set> NON_REPLICATED_SETTINGS = Set.of( + IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING, + IndexMetaData.INDEX_AUTO_EXPAND_REPLICAS_SETTING, + IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING, + IndexMetaData.INDEX_ROUTING_INCLUDE_GROUP_SETTING, + IndexMetaData.INDEX_ROUTING_REQUIRE_GROUP_SETTING, + IndexMetaData.INDEX_READ_ONLY_SETTING, + IndexMetaData.INDEX_BLOCKS_READ_SETTING, + IndexMetaData.INDEX_BLOCKS_WRITE_SETTING, + IndexMetaData.INDEX_BLOCKS_METADATA_SETTING, + IndexMetaData.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING, + IndexMetaData.INDEX_PRIORITY_SETTING, + IndexMetaData.SETTING_WAIT_FOR_ACTIVE_SHARDS, + EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, + EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING, + ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, + MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY, + UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, + IndexSettings.MAX_RESULT_WINDOW_SETTING, + IndexSettings.INDEX_WARMER_ENABLED_SETTING, + IndexSettings.INDEX_REFRESH_INTERVAL_SETTING, + IndexSettings.MAX_RESCORE_WINDOW_SETTING, + IndexSettings.MAX_INNER_RESULT_WINDOW_SETTING, + IndexSettings.DEFAULT_FIELD_SETTING, + IndexSettings.QUERY_STRING_LENIENT_SETTING, + IndexSettings.QUERY_STRING_ANALYZE_WILDCARD, + IndexSettings.QUERY_STRING_ALLOW_LEADING_WILDCARD, + IndexSettings.ALLOW_UNMAPPED, + IndexSettings.INDEX_SEARCH_IDLE_AFTER, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING, + IndexSettings.MAX_SCRIPT_FIELDS_SETTING, + IndexSettings.MAX_REGEX_LENGTH_SETTING, + IndexSettings.MAX_TERMS_COUNT_SETTING, + IndexSettings.MAX_ANALYZED_OFFSET_SETTING, + IndexSettings.MAX_DOCVALUE_FIELDS_SEARCH_SETTING, + IndexSettings.MAX_TOKEN_COUNT_SETTING, + IndexSettings.MAX_SLICES_PER_SCROLL, + IndexSettings.MAX_ADJACENCY_MATRIX_FILTERS_SETTING, + IndexSettings.DEFAULT_PIPELINE, + IndexSettings.INDEX_SEARCH_THROTTLED, + IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING, + IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING, + IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING, + IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, + IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING, + IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING, + IndexSettings.INDEX_GC_DELETES_SETTING, + IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD, + IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, + BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE_SETTING, + SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING, + IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG_SETTING, + MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_DELETES_PCT_ALLOWED_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING, + MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING, + MergeSchedulerConfig.AUTO_THROTTLE_SETTING, + MergeSchedulerConfig.MAX_MERGE_COUNT_SETTING, + MergeSchedulerConfig.MAX_THREAD_COUNT_SETTING, + EngineConfig.INDEX_CODEC_SETTING); static Settings filter(Settings originalSettings) { Settings.Builder settings = Settings.builder().put(originalSettings); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportXPackUsageAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportXPackUsageAction.java index 337c2281d5579..dd1b73415b541 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportXPackUsageAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportXPackUsageAction.java @@ -38,7 +38,7 @@ public TransportXPackUsageAction(ThreadPool threadPool, TransportService transpo IndexNameExpressionResolver indexNameExpressionResolver, Set featureSets) { super(XPackUsageAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, XPackUsageRequest::new); - this.featureSets = Collections.unmodifiableList(new ArrayList<>(featureSets)); + this.featureSets = List.copyOf(featureSets); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java index e9c71d3718b22..78ce0ee58c961 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/GetDataFrameTransformsStatsAction.java @@ -25,7 +25,6 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -90,7 +89,7 @@ public List getExpandedIds() { } public void setExpandedIds(List expandedIds) { - this.expandedIds = Collections.unmodifiableList(new ArrayList<>(expandedIds)); + this.expandedIds = List.copyOf(expandedIds); } public final void setPageParams(PageParams pageParams) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java index acaceace0472b..31098223d4b7b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/filestructurefinder/FileStructure.java @@ -15,7 +15,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -179,22 +178,20 @@ public FileStructure(int numLinesAnalyzed, int numMessagesAnalyzed, String sampl this.format = Objects.requireNonNull(format); this.multilineStartPattern = multilineStartPattern; this.excludeLinesPattern = excludeLinesPattern; - this.columnNames = (columnNames == null) ? null : Collections.unmodifiableList(new ArrayList<>(columnNames)); + this.columnNames = (columnNames == null) ? null : List.copyOf(columnNames); this.hasHeaderRow = hasHeaderRow; this.delimiter = delimiter; this.quote = quote; this.shouldTrimFields = shouldTrimFields; this.grokPattern = grokPattern; this.timestampField = timestampField; - this.jodaTimestampFormats = - (jodaTimestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(jodaTimestampFormats)); - this.javaTimestampFormats = - (javaTimestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(javaTimestampFormats)); + this.jodaTimestampFormats = (jodaTimestampFormats == null) ? null : List.copyOf(jodaTimestampFormats); + this.javaTimestampFormats = (javaTimestampFormats == null) ? null : List.copyOf(javaTimestampFormats); this.needClientTimezone = needClientTimezone; this.mappings = Collections.unmodifiableSortedMap(new TreeMap<>(mappings)); this.ingestPipeline = (ingestPipeline == null) ? null : Collections.unmodifiableMap(new LinkedHashMap<>(ingestPipeline)); this.fieldStats = Collections.unmodifiableSortedMap(new TreeMap<>(fieldStats)); - this.explanation = Collections.unmodifiableList(new ArrayList<>(explanation)); + this.explanation = List.copyOf(explanation); } public FileStructure(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/DeletePrivilegesResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/DeletePrivilegesResponse.java index 18efb2ac5fac3..45ca0509f5f9b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/DeletePrivilegesResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/DeletePrivilegesResponse.java @@ -14,7 +14,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.Set; /** @@ -29,7 +28,7 @@ public DeletePrivilegesResponse() { } public DeletePrivilegesResponse(Collection found) { - this.found = Collections.unmodifiableSet(new HashSet<>(found)); + this.found = Set.copyOf(found); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesRequest.java index 651c695db6a41..62027045b116c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/privilege/PutPrivilegesRequest.java @@ -16,7 +16,6 @@ import org.elasticsearch.xpack.core.security.support.MetadataUtils; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -96,7 +95,7 @@ public List getPrivileges() { } public void setPrivileges(Collection privileges) { - this.privileges = Collections.unmodifiableList(new ArrayList<>(privileges)); + this.privileges = List.copyOf(privileges); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/oidc/OpenIdConnectRealmSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/oidc/OpenIdConnectRealmSettings.java index b88056a4f24e7..7c87198b96446 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/oidc/OpenIdConnectRealmSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/oidc/OpenIdConnectRealmSettings.java @@ -29,9 +29,9 @@ public class OpenIdConnectRealmSettings { private OpenIdConnectRealmSettings() { } - private static final List SUPPORTED_SIGNATURE_ALGORITHMS = Collections.unmodifiableList( - Arrays.asList("HS256", "HS384", "HS512", "RS256", "RS384", "RS512", "ES256", "ES384", "ES512", "PS256", "PS384", "PS512")); - private static final List RESPONSE_TYPES = Arrays.asList("code", "id_token", "id_token token"); + private static final List SUPPORTED_SIGNATURE_ALGORITHMS = + List.of("HS256", "HS384", "HS512", "RS256", "RS384", "RS512", "ES256", "ES384", "ES512", "PS256", "PS384", "PS512"); + private static final List RESPONSE_TYPES = List.of("code", "id_token", "id_token token"); public static final String TYPE = "oidc"; public static final Setting.AffixSetting RP_CLIENT_ID diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/ApplicationPermission.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/ApplicationPermission.java index da6af4ec7cbdc..79f751140beb8 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/ApplicationPermission.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/permission/ApplicationPermission.java @@ -15,7 +15,6 @@ import org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor; import org.elasticsearch.xpack.core.security.support.Automatons; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -54,7 +53,7 @@ public final class ApplicationPermission { Automatons.unionAndMinimize(Arrays.asList(existing.resourceAutomaton, patterns))); } })); - this.permissions = Collections.unmodifiableList(new ArrayList<>(permissionsByPrivilege.values())); + this.permissions = List.copyOf(permissionsByPrivilege.values()); } /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ApplicationPrivilegeDescriptor.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ApplicationPrivilegeDescriptor.java index 613f64f93b54a..d80e2a1904558 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ApplicationPrivilegeDescriptor.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ApplicationPrivilegeDescriptor.java @@ -18,7 +18,7 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -167,7 +167,7 @@ private Builder privilegeName(String privilegeName) { } private Builder actions(Collection actions) { - this.actions = new HashSet<>(actions); + this.actions = new LinkedHashSet<>(actions); return this; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java index 3611b6663a38f..fa5ec665043ae 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java @@ -37,8 +37,8 @@ import java.net.Socket; import java.security.GeneralSecurityException; import java.security.KeyManagementException; -import java.security.NoSuchAlgorithmException; import java.security.KeyStore; +import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -327,7 +327,7 @@ public Set getTransportProfileContextNames() { * Accessor to the loaded ssl configuration objects at the current point in time. This is useful for testing */ Collection getLoadedSSLConfigurations() { - return Collections.unmodifiableSet(new HashSet<>(sslContexts.keySet())); + return Set.copyOf(sslContexts.keySet()); } /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/xcontent/WatcherParams.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/xcontent/WatcherParams.java index a530108a6fae6..007fcc30c3d24 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/xcontent/WatcherParams.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/watcher/support/xcontent/WatcherParams.java @@ -12,7 +12,6 @@ import java.util.Map; import static java.util.Collections.emptyMap; -import static java.util.Collections.unmodifiableMap; public class WatcherParams extends ToXContent.DelegatingMapParams { @@ -99,7 +98,7 @@ public Builder put(String key, Object value) { } public WatcherParams build() { - return new WatcherParams(unmodifiableMap(new HashMap<>(params)), delegate); + return new WatcherParams(Map.copyOf(params), delegate); } } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java index 2eff0384b6f92..94661819e5008 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfigTests; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -86,18 +85,10 @@ public void testFrom() throws IOException { boolean indexIssueFound = randomBoolean(); boolean mlIssueFound = randomBoolean(); DeprecationIssue foundIssue = DeprecationIssueTests.createTestInstance(); - List> clusterSettingsChecks = - Collections.unmodifiableList(Arrays.asList( - (s) -> clusterIssueFound ? foundIssue : null - )); - List> indexSettingsChecks = - Collections.unmodifiableList(Arrays.asList( - (idx) -> indexIssueFound ? foundIssue : null - )); + List> clusterSettingsChecks = List.of((s) -> clusterIssueFound ? foundIssue : null); + List> indexSettingsChecks = List.of((idx) -> indexIssueFound ? foundIssue : null); List> mlSettingsChecks = - Collections.unmodifiableList(Arrays.asList( - (idx, unused) -> mlIssueFound ? foundIssue : null - )); + List.of((idx, unused) -> mlIssueFound ? foundIssue : null); NodesDeprecationCheckResponse nodeDeprecationIssues = new NodesDeprecationCheckResponse( new ClusterName(randomAlphaOfLength(5)), diff --git a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/Deprecation.java b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/Deprecation.java index 9bfbe352f839a..5b87f4153cbba 100644 --- a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/Deprecation.java +++ b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/Deprecation.java @@ -21,7 +21,6 @@ import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction; import org.elasticsearch.xpack.core.deprecation.NodesDeprecationCheckAction; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.function.Supplier; @@ -30,12 +29,12 @@ * The plugin class for the Deprecation API */ public class Deprecation extends Plugin implements ActionPlugin { + @Override public List> getActions() { - return Collections.unmodifiableList(Arrays.asList( - new ActionHandler<>(DeprecationInfoAction.INSTANCE, TransportDeprecationInfoAction.class), - new ActionHandler<>(NodesDeprecationCheckAction.INSTANCE, TransportNodeDeprecationCheckAction.class) - )); + return List.of( + new ActionHandler<>(DeprecationInfoAction.INSTANCE, TransportDeprecationInfoAction.class), + new ActionHandler<>(NodesDeprecationCheckAction.INSTANCE, TransportNodeDeprecationCheckAction.class)); } @Override diff --git a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java index 5567956c3e9a0..88ce82729df10 100644 --- a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java +++ b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java @@ -14,7 +14,6 @@ import org.elasticsearch.xpack.core.deprecation.DeprecationIssue; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -34,20 +33,13 @@ private DeprecationChecks() { static List> CLUSTER_SETTINGS_CHECKS = Collections.emptyList(); - static List> NODE_SETTINGS_CHECKS = - Collections.unmodifiableList(Arrays.asList( - // STUB - )); + static List> NODE_SETTINGS_CHECKS = Collections.emptyList(); static List> INDEX_SETTINGS_CHECKS = - Collections.unmodifiableList(Arrays.asList( - IndexDeprecationChecks::oldIndicesCheck)); + Collections.singletonList(IndexDeprecationChecks::oldIndicesCheck); static List> ML_SETTINGS_CHECKS = - Collections.unmodifiableList(Arrays.asList( - MlDeprecationChecks::checkDataFeedAggregations, - MlDeprecationChecks::checkDataFeedQuery - )); + List.of(MlDeprecationChecks::checkDataFeedAggregations, MlDeprecationChecks::checkDataFeedQuery); /** * helper utility function to reduce repeat of running a specific {@link List} of checks. diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 281a374b97b2f..1ff478782a8e8 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -323,18 +323,18 @@ public static boolean isMlNode(DiscoveryNode node) { } public List> getSettings() { - return Collections.unmodifiableList( - Arrays.asList(MachineLearningField.AUTODETECT_PROCESS, - ML_ENABLED, - CONCURRENT_JOB_ALLOCATIONS, - MachineLearningField.MAX_MODEL_MEMORY_LIMIT, - MAX_LAZY_ML_NODES, - MAX_MACHINE_MEMORY_PERCENT, - AutodetectBuilder.DONT_PERSIST_MODEL_STATE_SETTING, - AutodetectBuilder.MAX_ANOMALY_RECORDS_SETTING_DYNAMIC, - MAX_OPEN_JOBS_PER_NODE, - AutodetectProcessManager.MIN_DISK_SPACE_OFF_HEAP, - MlConfigMigrationEligibilityCheck.ENABLE_CONFIG_MIGRATION)); + return List.of( + MachineLearningField.AUTODETECT_PROCESS, + ML_ENABLED, + CONCURRENT_JOB_ALLOCATIONS, + MachineLearningField.MAX_MODEL_MEMORY_LIMIT, + MAX_LAZY_ML_NODES, + MAX_MACHINE_MEMORY_PERCENT, + AutodetectBuilder.DONT_PERSIST_MODEL_STATE_SETTING, + AutodetectBuilder.MAX_ANOMALY_RECORDS_SETTING_DYNAMIC, + MAX_OPEN_JOBS_PER_NODE, + AutodetectProcessManager.MIN_DISK_SPACE_OFF_HEAP, + MlConfigMigrationEligibilityCheck.ENABLE_CONFIG_MIGRATION); } public Settings additionalSettings() { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureFinderManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureFinderManager.java index 5332f18e9f05a..1a0e41c5e135e 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureFinderManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureFinderManager.java @@ -23,7 +23,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Objects; @@ -44,45 +43,244 @@ public final class FileStructureFinderManager { public static final int MIN_SAMPLE_LINE_COUNT = 2; public static final int DEFAULT_IDEAL_SAMPLE_LINE_COUNT = 1000; - static final Set FILEBEAT_SUPPORTED_ENCODINGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - "866", "ansi_x3.4-1968", "arabic", "ascii", "asmo-708", "big5", "big5-hkscs", "chinese", "cn-big5", "cp1250", "cp1251", "cp1252", - "cp1253", "cp1254", "cp1255", "cp1256", "cp1257", "cp1258", "cp819", "cp866", "csbig5", "cseuckr", "cseucpkdfmtjapanese", - "csgb2312", "csibm866", "csiso2022jp", "csiso2022kr", "csiso58gb231280", "csiso88596e", "csiso88596i", "csiso88598e", "csiso88598i", - "csisolatin1", "csisolatin2", "csisolatin3", "csisolatin4", "csisolatin5", "csisolatin6", "csisolatin9", "csisolatinarabic", - "csisolatincyrillic", "csisolatingreek", "csisolatinhebrew", "cskoi8r", "csksc56011987", "csmacintosh", "csshiftjis", "cyrillic", - "dos-874", "ecma-114", "ecma-118", "elot_928", "euc-jp", "euc-kr", "gb18030", "gb2312", "gb_2312", "gb_2312-80", "gbk", "greek", - "greek8", "hebrew", "hz-gb-2312", "ibm819", "ibm866", "iso-2022-cn", "iso-2022-cn-ext", "iso-2022-jp", "iso-2022-kr", "iso-8859-1", - "iso-8859-10", "iso-8859-11", "iso-8859-13", "iso-8859-14", "iso-8859-15", "iso-8859-16", "iso-8859-2", "iso-8859-3", "iso-8859-4", - "iso-8859-5", "iso-8859-6", "iso-8859-6-e", "iso-8859-6-i", "iso-8859-7", "iso-8859-8", "iso-8859-8-e", "iso-8859-8-i", - "iso-8859-9", "iso-ir-100", "iso-ir-101", "iso-ir-109", "iso-ir-110", "iso-ir-126", "iso-ir-127", "iso-ir-138", "iso-ir-144", - "iso-ir-148", "iso-ir-149", "iso-ir-157", "iso-ir-58", "iso8859-1", "iso8859-10", "iso8859-11", "iso8859-13", "iso8859-14", - "iso8859-15", "iso8859-2", "iso8859-3", "iso8859-4", "iso8859-5", "iso8859-6", "iso8859-6e", "iso8859-6i", "iso8859-7", "iso8859-8", - "iso8859-8e", "iso8859-8i", "iso8859-9", "iso88591", "iso885910", "iso885911", "iso885913", "iso885914", "iso885915", "iso88592", - "iso88593", "iso88594", "iso88595", "iso88596", "iso88597", "iso88598", "iso88599", "iso_8859-1", "iso_8859-15", "iso_8859-1:1987", - "iso_8859-2", "iso_8859-2:1987", "iso_8859-3", "iso_8859-3:1988", "iso_8859-4", "iso_8859-4:1988", "iso_8859-5", "iso_8859-5:1988", - "iso_8859-6", "iso_8859-6:1987", "iso_8859-7", "iso_8859-7:1987", "iso_8859-8", "iso_8859-8:1988", "iso_8859-9", "iso_8859-9:1989", - "koi", "koi8", "koi8-r", "koi8-ru", "koi8-u", "koi8_r", "korean", "ks_c_5601-1987", "ks_c_5601-1989", "ksc5601", "ksc_5601", "l1", - "l2", "l3", "l4", "l5", "l6", "l9", "latin1", "latin2", "latin3", "latin4", "latin5", "latin6", "logical", "mac", "macintosh", - "ms932", "ms_kanji", "shift-jis", "shift_jis", "sjis", "sun_eu_greek", "tis-620", "unicode-1-1-utf-8", "us-ascii", "utf-16", - "utf-16-bom", "utf-16be", "utf-16be-bom", "utf-16le", "utf-16le-bom", "utf-8", "utf8", "visual", "windows-1250", "windows-1251", - "windows-1252", "windows-1253", "windows-1254", "windows-1255", "windows-1256", "windows-1257", "windows-1258", "windows-31j", - "windows-874", "windows-949", "x-cp1250", "x-cp1251", "x-cp1252", "x-cp1253", "x-cp1254", "x-cp1255", "x-cp1256", "x-cp1257", - "x-cp1258", "x-euc-jp", "x-gbk", "x-mac-cyrillic", "x-mac-roman", "x-mac-ukrainian", "x-sjis", "x-x-big5" - ))); + static final Set FILEBEAT_SUPPORTED_ENCODINGS = Set.of( + "866", + "ansi_x3.4-1968", + "arabic", + "ascii", + "asmo-708", + "big5", + "big5-hkscs", + "chinese", + "cn-big5", + "cp1250", + "cp1251", + "cp1252", + "cp1253", + "cp1254", + "cp1255", + "cp1256", + "cp1257", + "cp1258", + "cp819", + "cp866", + "csbig5", + "cseuckr", + "cseucpkdfmtjapanese", + "csgb2312", + "csibm866", + "csiso2022jp", + "csiso2022kr", + "csiso58gb231280", + "csiso88596e", + "csiso88596i", + "csiso88598e", + "csiso88598i", + "csisolatin1", + "csisolatin2", + "csisolatin3", + "csisolatin4", + "csisolatin5", + "csisolatin6", + "csisolatin9", + "csisolatinarabic", + "csisolatincyrillic", + "csisolatingreek", + "csisolatinhebrew", + "cskoi8r", + "csksc56011987", + "csmacintosh", + "csshiftjis", + "cyrillic", + "dos-874", + "ecma-114", + "ecma-118", + "elot_928", + "euc-jp", + "euc-kr", + "gb18030", + "gb2312", + "gb_2312", + "gb_2312-80", + "gbk", + "greek", + "greek8", + "hebrew", + "hz-gb-2312", + "ibm819", + "ibm866", + "iso-2022-cn", + "iso-2022-cn-ext", + "iso-2022-jp", + "iso-2022-kr", + "iso-8859-1", + "iso-8859-10", + "iso-8859-11", + "iso-8859-13", + "iso-8859-14", + "iso-8859-15", + "iso-8859-16", + "iso-8859-2", + "iso-8859-3", + "iso-8859-4", + "iso-8859-5", + "iso-8859-6", + "iso-8859-6-e", + "iso-8859-6-i", + "iso-8859-7", + "iso-8859-8", + "iso-8859-8-e", + "iso-8859-8-i", + "iso-8859-9", + "iso-ir-100", + "iso-ir-101", + "iso-ir-109", + "iso-ir-110", + "iso-ir-126", + "iso-ir-127", + "iso-ir-138", + "iso-ir-144", + "iso-ir-148", + "iso-ir-149", + "iso-ir-157", + "iso-ir-58", + "iso8859-1", + "iso8859-10", + "iso8859-11", + "iso8859-13", + "iso8859-14", + "iso8859-15", + "iso8859-2", + "iso8859-3", + "iso8859-4", + "iso8859-5", + "iso8859-6", + "iso8859-6e", + "iso8859-6i", + "iso8859-7", + "iso8859-8", + "iso8859-8e", + "iso8859-8i", + "iso8859-9", + "iso88591", + "iso885910", + "iso885911", + "iso885913", + "iso885914", + "iso885915", + "iso88592", + "iso88593", + "iso88594", + "iso88595", + "iso88596", + "iso88597", + "iso88598", + "iso88599", + "iso_8859-1", + "iso_8859-15", + "iso_8859-1:1987", + "iso_8859-2", + "iso_8859-2:1987", + "iso_8859-3", + "iso_8859-3:1988", + "iso_8859-4", + "iso_8859-4:1988", + "iso_8859-5", + "iso_8859-5:1988", + "iso_8859-6", + "iso_8859-6:1987", + "iso_8859-7", + "iso_8859-7:1987", + "iso_8859-8", + "iso_8859-8:1988", + "iso_8859-9", + "iso_8859-9:1989", + "koi", + "koi8", + "koi8-r", + "koi8-ru", + "koi8-u", + "koi8_r", + "korean", + "ks_c_5601-1987", + "ks_c_5601-1989", + "ksc5601", + "ksc_5601", + "l1", + "l2", + "l3", + "l4", + "l5", + "l6", + "l9", + "latin1", + "latin2", + "latin3", + "latin4", + "latin5", + "latin6", + "logical", + "mac", + "macintosh", + "ms932", + "ms_kanji", + "shift-jis", + "shift_jis", + "sjis", + "sun_eu_greek", + "tis-620", + "unicode-1-1-utf-8", + "us-ascii", + "utf-16", + "utf-16-bom", + "utf-16be", + "utf-16be-bom", + "utf-16le", + "utf-16le-bom", + "utf-8", + "utf8", + "visual", + "windows-1250", + "windows-1251", + "windows-1252", + "windows-1253", + "windows-1254", + "windows-1255", + "windows-1256", + "windows-1257", + "windows-1258", + "windows-31j", + "windows-874", + "windows-949", + "x-cp1250", + "x-cp1251", + "x-cp1252", + "x-cp1253", + "x-cp1254", + "x-cp1255", + "x-cp1256", + "x-cp1257", + "x-cp1258", + "x-euc-jp", + "x-gbk", + "x-mac-cyrillic", + "x-mac-roman", + "x-mac-ukrainian", + "x-sjis", + "x-x-big5"); /** * These need to be ordered so that the more generic formats come after the more specific ones */ - private static final List ORDERED_STRUCTURE_FACTORIES = Collections.unmodifiableList(Arrays.asList( - new NdJsonFileStructureFinderFactory(), - new XmlFileStructureFinderFactory(), - // NDJSON will often also be valid (although utterly weird) CSV, so NDJSON must come before CSV - new DelimitedFileStructureFinderFactory(',', '"', 2, false), - new DelimitedFileStructureFinderFactory('\t', '"', 2, false), - new DelimitedFileStructureFinderFactory(';', '"', 4, false), - new DelimitedFileStructureFinderFactory('|', '"', 5, true), - new TextLogFileStructureFinderFactory() - )); + private static final List ORDERED_STRUCTURE_FACTORIES = List.of( + // NDJSON will often also be valid (although utterly weird) CSV, so NDJSON must come before CSV + new NdJsonFileStructureFinderFactory(), + new XmlFileStructureFinderFactory(), + new DelimitedFileStructureFinderFactory(',', '"', 2, false), + new DelimitedFileStructureFinderFactory('\t', '"', 2, false), + new DelimitedFileStructureFinderFactory(';', '"', 4, false), + new DelimitedFileStructureFinderFactory('|', '"', 5, true), + new TextLogFileStructureFinderFactory()); private static final int BUFFER_SIZE = 8192; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureOverrides.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureOverrides.java index e30699c69b7f8..bf80c1076896f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureOverrides.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/filestructurefinder/FileStructureOverrides.java @@ -8,8 +8,6 @@ import org.elasticsearch.xpack.core.ml.action.FindFileStructureAction; import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Objects; @@ -48,7 +46,7 @@ private FileStructureOverrides(String charset, FileStructure.Format format, List String timestampFormat, String timestampField) { this.charset = charset; this.format = format; - this.columnNames = (columnNames == null) ? null : Collections.unmodifiableList(new ArrayList<>(columnNames)); + this.columnNames = (columnNames == null) ? null : List.copyOf(columnNames); this.hasHeaderRow = hasHeaderRow; this.delimiter = delimiter; this.quote = quote; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java index 64c6583f7c2df..585b4d1f87dfa 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java @@ -70,7 +70,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -95,12 +94,7 @@ public class JobConfigProvider { private static final Logger logger = LogManager.getLogger(JobConfigProvider.class); - public static final Map TO_XCONTENT_PARAMS; - static { - Map modifiable = new HashMap<>(); - modifiable.put(ToXContentParams.FOR_INTERNAL_STORAGE, "true"); - TO_XCONTENT_PARAMS = Collections.unmodifiableMap(modifiable); - } + public static final Map TO_XCONTENT_PARAMS = Collections.singletonMap(ToXContentParams.FOR_INTERNAL_STORAGE, "true"); private final Client client; private final NamedXContentRegistry xContentRegistry; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java index 3ccf0f22da922..7a68514e25d23 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java @@ -63,7 +63,8 @@ private synchronized void execute(T previousValue, ActionListener> liste collectedResponses.add(previousValue); if (continuationPredicate.test(previousValue)) { if (tasks.isEmpty()) { - listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); + // noinspection Java9CollectionFactory (because the list can contain null entries) + listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); return; } ChainTask task = tasks.pop(); @@ -83,6 +84,7 @@ protected void doRun() { } }); } else { + // noinspection Java9CollectionFactory (because the list can contain null entries) listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); } } @@ -120,6 +122,6 @@ protected void doRun() { } public synchronized List getCollectedResponses() { - return Collections.unmodifiableList(new ArrayList<>(collectedResponses)); + return List.copyOf(collectedResponses); } } diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/Monitoring.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/Monitoring.java index b054eca1cc319..e2b137bd95d19 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/Monitoring.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/Monitoring.java @@ -191,6 +191,6 @@ public List> getSettings() { @Override public List getSettingsFilter() { final String exportersKey = "xpack.monitoring.exporters."; - return Collections.unmodifiableList(Arrays.asList(exportersKey + "*.auth.*", exportersKey + "*.ssl.*")); + return List.of(exportersKey + "*.auth.*", exportersKey + "*.ssl.*"); } } diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkAction.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkAction.java index 434b0bd9fa0ff..0a1576393ba26 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkAction.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkAction.java @@ -26,8 +26,6 @@ import java.io.IOException; import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -58,11 +56,10 @@ public RestMonitoringBulkAction(Settings settings, RestController controller) { MonitoringTemplateUtils.OLD_TEMPLATE_VERSION ); - final Map> versionsMap = new HashMap<>(); - versionsMap.put(MonitoredSystem.KIBANA, allVersions); - versionsMap.put(MonitoredSystem.LOGSTASH, allVersions); - versionsMap.put(MonitoredSystem.BEATS, allVersions); - supportedApiVersions = Collections.unmodifiableMap(versionsMap); + supportedApiVersions = Map.of( + MonitoredSystem.KIBANA, allVersions, + MonitoredSystem.LOGSTASH, allVersions, + MonitoredSystem.BEATS, allVersions); } @Override diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestRollupSearchAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestRollupSearchAction.java index 1d9960e711f61..5481ac3a43172 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestRollupSearchAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/rest/RestRollupSearchAction.java @@ -16,16 +16,11 @@ import org.elasticsearch.xpack.core.rollup.action.RollupSearchAction; import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; import java.util.Set; public class RestRollupSearchAction extends BaseRestHandler { - private static final Set RESPONSE_PARAMS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - RestSearchAction.TYPED_KEYS_PARAM, - RestSearchAction.TOTAL_HITS_AS_INT_PARAM))); + private static final Set RESPONSE_PARAMS = Set.of(RestSearchAction.TYPED_KEYS_PARAM, RestSearchAction.TOTAL_HITS_AS_INT_PARAM); public RestRollupSearchAction(Settings settings, RestController controller) { super(settings); 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 ff92122626630..51dafd433caf5 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 @@ -200,7 +200,9 @@ import org.elasticsearch.xpack.security.rest.action.RestInvalidateApiKeyAction; import org.elasticsearch.xpack.security.rest.action.oauth2.RestGetTokenAction; import org.elasticsearch.xpack.security.rest.action.oauth2.RestInvalidateTokenAction; +import org.elasticsearch.xpack.security.rest.action.oidc.RestOpenIdConnectAuthenticateAction; import org.elasticsearch.xpack.security.rest.action.oidc.RestOpenIdConnectLogoutAction; +import org.elasticsearch.xpack.security.rest.action.oidc.RestOpenIdConnectPrepareAuthenticationAction; import org.elasticsearch.xpack.security.rest.action.privilege.RestDeletePrivilegesAction; import org.elasticsearch.xpack.security.rest.action.privilege.RestGetPrivilegesAction; import org.elasticsearch.xpack.security.rest.action.privilege.RestPutPrivilegesAction; @@ -212,8 +214,6 @@ import org.elasticsearch.xpack.security.rest.action.rolemapping.RestDeleteRoleMappingAction; import org.elasticsearch.xpack.security.rest.action.rolemapping.RestGetRoleMappingsAction; import org.elasticsearch.xpack.security.rest.action.rolemapping.RestPutRoleMappingAction; -import org.elasticsearch.xpack.security.rest.action.oidc.RestOpenIdConnectAuthenticateAction; -import org.elasticsearch.xpack.security.rest.action.oidc.RestOpenIdConnectPrepareAuthenticationAction; import org.elasticsearch.xpack.security.rest.action.saml.RestSamlAuthenticateAction; import org.elasticsearch.xpack.security.rest.action.saml.RestSamlInvalidateSessionAction; import org.elasticsearch.xpack.security.rest.action.saml.RestSamlLogoutAction; @@ -894,13 +894,31 @@ public Map> getTransports(Settings settings, ThreadP } IPFilter ipFilter = this.ipFilter.get(); - Map> transports = new HashMap<>(); - transports.put(SecurityField.NAME4, () -> new SecurityNetty4ServerTransport(settings, Version.CURRENT, threadPool, - networkService, pageCacheRecycler, namedWriteableRegistry, circuitBreakerService, ipFilter, getSslService())); - transports.put(SecurityField.NIO, () -> new SecurityNioTransport(settings, Version.CURRENT, threadPool, networkService, - pageCacheRecycler, namedWriteableRegistry, circuitBreakerService, ipFilter, getSslService(), getNioGroupFactory(settings))); - - return Collections.unmodifiableMap(transports); + return Map.of( + // security based on Netty 4 + SecurityField.NAME4, + () -> new SecurityNetty4ServerTransport( + settings, + Version.CURRENT, + threadPool, + networkService, + pageCacheRecycler, + namedWriteableRegistry, + circuitBreakerService, + ipFilter, + getSslService()), + // security based on NIO + SecurityField.NIO, + () -> new SecurityNioTransport(settings, + Version.CURRENT, + threadPool, + networkService, + pageCacheRecycler, + namedWriteableRegistry, + circuitBreakerService, + ipFilter, + getSslService(), + getNioGroupFactory(settings))); } @Override diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java index 66206d5013713..3a27b6670aeb1 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java @@ -37,7 +37,6 @@ import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -102,23 +101,35 @@ public static Map getFactories(ThreadPool threadPool, Res NativeRoleMappingStore nativeRoleMappingStore, SecurityIndexManager securityIndex) { - Map map = new HashMap<>(); - map.put(FileRealmSettings.TYPE, config -> new FileRealm(config, resourceWatcherService, threadPool)); - map.put(NativeRealmSettings.TYPE, config -> { - final NativeRealm nativeRealm = new NativeRealm(config, nativeUsersStore, threadPool); - securityIndex.addIndexStateListener(nativeRealm::onSecurityIndexStateChange); - return nativeRealm; - }); - map.put(LdapRealmSettings.AD_TYPE, config -> new LdapRealm(config, sslService, - resourceWatcherService, nativeRoleMappingStore, threadPool)); - map.put(LdapRealmSettings.LDAP_TYPE, config -> new LdapRealm(config, - sslService, resourceWatcherService, nativeRoleMappingStore, threadPool)); - map.put(PkiRealmSettings.TYPE, config -> new PkiRealm(config, resourceWatcherService, nativeRoleMappingStore)); - map.put(SamlRealmSettings.TYPE, config -> SamlRealm.create(config, sslService, resourceWatcherService, nativeRoleMappingStore)); - map.put(KerberosRealmSettings.TYPE, config -> new KerberosRealm(config, nativeRoleMappingStore, threadPool)); - map.put(OpenIdConnectRealmSettings.TYPE, config -> new OpenIdConnectRealm(config, sslService, nativeRoleMappingStore, - resourceWatcherService)); - return Collections.unmodifiableMap(map); + return Map.of( + // file realm + FileRealmSettings.TYPE, + config -> new FileRealm(config, resourceWatcherService, threadPool), + // native realm + NativeRealmSettings.TYPE, + config -> { + final NativeRealm nativeRealm = new NativeRealm(config, nativeUsersStore, threadPool); + securityIndex.addIndexStateListener(nativeRealm::onSecurityIndexStateChange); + return nativeRealm; + }, + // active directory realm + LdapRealmSettings.AD_TYPE, + config -> new LdapRealm(config, sslService, resourceWatcherService, nativeRoleMappingStore, threadPool), + // LDAP realm + LdapRealmSettings.LDAP_TYPE, + config -> new LdapRealm(config, sslService, resourceWatcherService, nativeRoleMappingStore, threadPool), + // PKI realm + PkiRealmSettings.TYPE, + config -> new PkiRealm(config, resourceWatcherService, nativeRoleMappingStore), + // SAML realm + SamlRealmSettings.TYPE, + config -> SamlRealm.create(config, sslService, resourceWatcherService, nativeRoleMappingStore), + // Kerberos realm + KerberosRealmSettings.TYPE, + config -> new KerberosRealm(config, nativeRoleMappingStore, threadPool), + // OpenID Connect realm + OpenIdConnectRealmSettings.TYPE, + config -> new OpenIdConnectRealm(config, sslService, nativeRoleMappingStore, resourceWatcherService)); } private InternalRealms() { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTicketValidator.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTicketValidator.java index 9f2a182839857..f24aaedcbf71d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTicketValidator.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/kerberos/KerberosTicketValidator.java @@ -6,8 +6,8 @@ package org.elasticsearch.xpack.security.authc.kerberos; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.collect.Tuple; @@ -17,21 +17,20 @@ import org.ietf.jgss.GSSManager; import org.ietf.jgss.Oid; +import javax.security.auth.Subject; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; + import java.nio.file.Path; import java.security.AccessController; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; import java.util.Base64; import java.util.Collections; -import java.util.HashMap; import java.util.Map; -import javax.security.auth.Subject; -import javax.security.auth.login.AppConfigurationEntry; -import javax.security.auth.login.Configuration; -import javax.security.auth.login.LoginContext; -import javax.security.auth.login.LoginException; - /** * Utility class that validates kerberos ticket for peer authentication. *

    @@ -254,21 +253,18 @@ static class KeytabJaasConf extends Configuration { @Override public AppConfigurationEntry[] getAppConfigurationEntry(final String name) { - final Map options = new HashMap<>(); - options.put("keyTab", keytabFilePath); - /* - * As acceptor, we can have multiple SPNs, we do not want to use particular - * principal so it uses "*" - */ - options.put("principal", "*"); - options.put("useKeyTab", Boolean.TRUE.toString()); - options.put("storeKey", Boolean.TRUE.toString()); - options.put("doNotPrompt", Boolean.TRUE.toString()); - options.put("isInitiator", Boolean.FALSE.toString()); - options.put("debug", Boolean.toString(krbDebug)); - - return new AppConfigurationEntry[] { new AppConfigurationEntry(SUN_KRB5_LOGIN_MODULE, - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, Collections.unmodifiableMap(options)) }; + return new AppConfigurationEntry[]{new AppConfigurationEntry( + SUN_KRB5_LOGIN_MODULE, + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + Map.of( + "keyTab", keytabFilePath, + // as acceptor, we can have multiple SPNs, we do not want to use any particular principal so it uses "*" + "principal", "*", + "useKeyTab", Boolean.TRUE.toString(), + "storeKey", Boolean.TRUE.toString(), + "doNotPrompt", Boolean.TRUE.toString(), + "isInitiator", Boolean.FALSE.toString(), + "debug", Boolean.toString(krbDebug)))}; } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/UserAttributeGroupsResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/UserAttributeGroupsResolver.java index d2b2a346a9588..32e7cf890a322 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/UserAttributeGroupsResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/UserAttributeGroupsResolver.java @@ -56,7 +56,7 @@ public void resolve(LDAPInterface connection, String userDn, TimeValue timeout, if (entry == null || entry.hasAttribute(attribute) == false) { listener.onResponse(Collections.emptyList()); } else { - listener.onResponse(Collections.unmodifiableList(Arrays.asList(entry.getAttributeValues(attribute)))); + listener.onResponse(List.of(entry.getAttributeValues(attribute))); } }, listener::onFailure), attribute); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DnRoleMapper.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DnRoleMapper.java index 78094f518c53c..1018c591617a9 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DnRoleMapper.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/DnRoleMapper.java @@ -5,25 +5,10 @@ */ package org.elasticsearch.xpack.security.authc.support; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.stream.Collectors; - import com.unboundid.ldap.sdk.DN; import com.unboundid.ldap.sdk.LDAPException; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.ElasticsearchException; @@ -37,6 +22,20 @@ import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.support.DnRoleMapperSettings; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.stream.Collectors; + import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.xpack.security.authc.ldap.support.LdapUtils.dn; @@ -153,7 +152,7 @@ public static Map> parseFile(Path path, Logger logger, Stri realmName); Map> normalizedMap = dnToRoles.entrySet().stream().collect(Collectors.toMap( entry -> entry.getKey().toNormalizedString(), - entry -> Collections.unmodifiableList(new ArrayList<>(entry.getValue())))); + entry -> List.copyOf(entry.getValue()))); return unmodifiableMap(normalizedMap); } catch (IOException | SettingsException e) { throw new ElasticsearchException("could not read realm [" + realmType + "/" + realmName + "] role mappings file [" + diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java index dbc323810610f..c23cae1aa878d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/UserRoleMapper.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Locale; import java.util.Map; @@ -62,10 +63,12 @@ public UserData(String username, @Nullable String dn, Collection groups, Map metadata, RealmConfig realm) { this.username = username; this.dn = dn; + // noinspection Java9CollectionFactory (because null values happen in some tests, is this realistic?) this.groups = groups == null || groups.isEmpty() ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(groups)); + // noinspection Java9CollectionFactory (because null values happen in production code, can such keys be dropped?) this.metadata = metadata == null || metadata.isEmpty() - ? Collections.emptyMap() : Collections.unmodifiableMap(metadata); + ? Collections.emptyMap() : Collections.unmodifiableMap(new HashMap<>(metadata)); this.realm = realm; } 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 781072b95a254..7454ec59da55f 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 @@ -139,7 +139,7 @@ public CompositeRolesStore(Settings settings, FileRolesStore fileRolesStore, Nat nlcBuilder.setMaximumWeight(nlcCacheSize); } this.negativeLookupCache = nlcBuilder.build(); - this.builtInRoleProviders = Collections.unmodifiableList(Arrays.asList(reservedRolesStore, fileRolesStore, nativeRolesStore)); + this.builtInRoleProviders = List.of(reservedRolesStore, fileRolesStore, nativeRolesStore); if (rolesProviders.isEmpty()) { this.allRoleProviders = this.builtInRoleProviders; } else { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java index 55d5bd579c12d..0aaf9db7f90d3 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java @@ -36,11 +36,11 @@ import org.elasticsearch.xpack.core.security.authc.Authentication; import org.elasticsearch.xpack.core.security.authc.Authentication.RealmRef; import org.elasticsearch.xpack.core.security.authc.AuthenticationToken; +import org.elasticsearch.xpack.core.security.authz.AuthorizationEngine.AuthorizationInfo; import org.elasticsearch.xpack.core.security.user.SystemUser; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.security.audit.AuditTrail; import org.elasticsearch.xpack.security.audit.AuditUtil; -import org.elasticsearch.xpack.core.security.authz.AuthorizationEngine.AuthorizationInfo; import org.elasticsearch.xpack.security.rest.RemoteHostHeader; import org.elasticsearch.xpack.security.transport.filter.IPFilter; import org.elasticsearch.xpack.security.transport.filter.SecurityIpFilterRule; @@ -66,10 +66,10 @@ import java.util.regex.Pattern; import static org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail.PRINCIPAL_ROLES_FIELD_NAME; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.containsString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -178,6 +178,7 @@ public void init() throws Exception { .put("xpack.security.audit.logfile.events.emit_request_body", includeRequestBody) .build(); localNode = mock(DiscoveryNode.class); + when(localNode.getId()).thenReturn(randomAlphaOfLength(16)); when(localNode.getAddress()).thenReturn(buildNewFakeTransportAddress()); clusterService = mock(ClusterService.class); when(clusterService.localNode()).thenReturn(localNode); @@ -249,11 +250,11 @@ public void testAnonymousAccessDeniedRest() throws Exception { .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) - .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, - includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) - .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri") - .put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, null); + .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); + if (includeRequestBody && Strings.hasLength(expectedMessage)) { + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, expectedMessage); + } opaqueId(threadContext, checkedFields); forwardedFor(threadContext, checkedFields); assertMsg(logger, checkedFields.immutableMap()); @@ -346,16 +347,18 @@ public void testAuthenticationFailedRest() throws Exception { final MapBuilder checkedFields = new MapBuilder<>(commonFields); checkedFields.put(LoggingAuditTrail.EVENT_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.EVENT_ACTION_FIELD_NAME, "authentication_failed") - .put(LoggingAuditTrail.ACTION_FIELD_NAME, null) .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, mockToken.principal()) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) - .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, - includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) - .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri") - .put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, params.isEmpty() ? null : "foo=bar"); + .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); + if (includeRequestBody && Strings.hasLength(expectedMessage)) { + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, expectedMessage); + } + if (params.isEmpty() == false) { + checkedFields.put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, "foo=bar"); + } opaqueId(threadContext, checkedFields); forwardedFor(threadContext, checkedFields); assertMsg(logger, checkedFields.immutableMap()); @@ -387,16 +390,17 @@ public void testAuthenticationFailedRestNoToken() throws Exception { final MapBuilder checkedFields = new MapBuilder<>(commonFields); checkedFields.put(LoggingAuditTrail.EVENT_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.EVENT_ACTION_FIELD_NAME, "authentication_failed") - .put(LoggingAuditTrail.ACTION_FIELD_NAME, null) - .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, null) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) - .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, - includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) - .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri") - .put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, params.isEmpty() ? null : "bar=baz"); + .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); + if (includeRequestBody && Strings.hasLength(expectedMessage)) { + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, expectedMessage); + } + if (params.isEmpty() == false) { + checkedFields.put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, "bar=baz"); + } opaqueId(threadContext, checkedFields); forwardedFor(threadContext, checkedFields); assertMsg(logger, checkedFields.immutableMap()); @@ -473,13 +477,15 @@ public void testAuthenticationFailedRealmRest() throws Exception { .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, mockToken.principal()) - .put(LoggingAuditTrail.ACTION_FIELD_NAME, null) .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) - .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, - includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) - .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri") - .put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, params.isEmpty() ? null : "_param=baz"); + .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); + if (includeRequestBody && Strings.hasLength(expectedMessage)) { + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, expectedMessage); + } + if (params.isEmpty() == false) { + checkedFields.put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, "_param=baz"); + } opaqueId(threadContext, checkedFields); forwardedFor(threadContext, checkedFields); assertMsg(logger, checkedFields.immutableMap()); @@ -639,11 +645,14 @@ public void testTamperedRequestRest() throws Exception { .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) - .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, - includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) - .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri") - .put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, params.isEmpty() ? null : "_param=baz"); + .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); + if (includeRequestBody && Strings.hasLength(expectedMessage)) { + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, expectedMessage); + } + if (params.isEmpty() == false) { + checkedFields.put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, "_param=baz"); + } opaqueId(threadContext, checkedFields); forwardedFor(threadContext, checkedFields); assertMsg(logger, checkedFields.immutableMap()); @@ -906,11 +915,14 @@ public void testAuthenticationSuccessRest() throws Exception { .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) - .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, - includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) - .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri") - .put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, params.isEmpty() ? null : "foo=bar&evac=true"); + .put(LoggingAuditTrail.URL_PATH_FIELD_NAME, "_uri"); + if (includeRequestBody && Strings.hasLength(expectedMessage)) { + checkedFields.put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, expectedMessage); + } + if (params.isEmpty() == false) { + checkedFields.put(LoggingAuditTrail.URL_QUERY_FIELD_NAME, "foo=bar&evac=true"); + } if (user.isRunAs()) { checkedFields.put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, "running as"); checkedFields.put(LoggingAuditTrail.PRINCIPAL_RUN_BY_FIELD_NAME, "_username"); @@ -1216,8 +1228,6 @@ private static void opaqueId(ThreadContext threadContext, MapBuilder> BUILDER_APPLIERS; static { - HashMap> appliers = new HashMap<>(11); // TODO: it'd be great if these could be constants instead of Strings, needs a core change to make the fields public first // TODO: add zero terms query support, I'm not sure the best way to parse it yet... // appliers.put("zero_terms_query", (qb, s) -> qb.zeroTermsQuery(s)); - appliers.put("analyzer", (qb, s) -> qb.analyzer(s)); - appliers.put("auto_generate_synonyms_phrase_query", (qb, s) -> qb.autoGenerateSynonymsPhraseQuery(Booleans.parseBoolean(s))); - appliers.put("cutoff_frequency", (qb, s) -> qb.cutoffFrequency(Float.valueOf(s))); - appliers.put("fuzziness", (qb, s) -> qb.fuzziness(Fuzziness.fromString(s))); - appliers.put("fuzzy_transpositions", (qb, s) -> qb.fuzzyTranspositions(Booleans.parseBoolean(s))); - appliers.put("fuzzy_rewrite", (qb, s) -> qb.fuzzyRewrite(s)); - appliers.put("lenient", (qb, s) -> qb.lenient(Booleans.parseBoolean(s))); - appliers.put("max_expansions", (qb, s) -> qb.maxExpansions(Integer.valueOf(s))); - appliers.put("minimum_should_match", (qb, s) -> qb.minimumShouldMatch(s)); - appliers.put("operator", (qb, s) -> qb.operator(Operator.fromString(s))); - appliers.put("prefix_length", (qb, s) -> qb.prefixLength(Integer.valueOf(s))); - BUILDER_APPLIERS = Collections.unmodifiableMap(appliers); + BUILDER_APPLIERS = Map.ofEntries( + entry("analyzer", MatchQueryBuilder::analyzer), + entry("auto_generate_synonyms_phrase_query", (qb, s) -> qb.autoGenerateSynonymsPhraseQuery(Booleans.parseBoolean(s))), + entry("cutoff_frequency", (qb, s) -> qb.cutoffFrequency(Float.valueOf(s))), + entry("fuzziness", (qb, s) -> qb.fuzziness(Fuzziness.fromString(s))), + entry("fuzzy_transpositions", (qb, s) -> qb.fuzzyTranspositions(Booleans.parseBoolean(s))), + entry("fuzzy_rewrite", MatchQueryBuilder::fuzzyRewrite), + entry("lenient", (qb, s) -> qb.lenient(Booleans.parseBoolean(s))), + entry("max_expansions", (qb, s) -> qb.maxExpansions(Integer.valueOf(s))), + entry("minimum_should_match", MatchQueryBuilder::minimumShouldMatch), + entry("operator", (qb, s) -> qb.operator(Operator.fromString(s))), + entry("prefix_length", (qb, s) -> qb.prefixLength(Integer.valueOf(s)))); } private final String name; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/MultiMatchQuery.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/MultiMatchQuery.java index 141dc4776a5ff..f51f8275a898c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/MultiMatchQuery.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/MultiMatchQuery.java @@ -14,36 +14,35 @@ import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MultiMatchQueryPredicate; import org.elasticsearch.xpack.sql.tree.Source; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.function.BiConsumer; +import static java.util.Map.entry; + public class MultiMatchQuery extends LeafQuery { private static final Map> BUILDER_APPLIERS; static { - HashMap> appliers = new HashMap<>(14); // TODO: it'd be great if these could be constants instead of Strings, needs a core change to make the fields public first - appliers.put("slop", (qb, s) -> qb.slop(Integer.valueOf(s))); - // TODO: add zero terms query support, I'm not sure the best way to parse it yet... - // appliers.put("zero_terms_query", (qb, s) -> qb.zeroTermsQuery(s)); - appliers.put("analyzer", (qb, s) -> qb.analyzer(s)); - appliers.put("auto_generate_synonyms_phrase_query", (qb, s) -> qb.autoGenerateSynonymsPhraseQuery(Booleans.parseBoolean(s))); - appliers.put("cutoff_frequency", (qb, s) -> qb.cutoffFrequency(Float.valueOf(s))); - appliers.put("fuzziness", (qb, s) -> qb.fuzziness(Fuzziness.fromString(s))); - appliers.put("fuzzy_rewrite", (qb, s) -> qb.fuzzyRewrite(s)); - appliers.put("fuzzy_transpositions", (qb, s) -> qb.fuzzyTranspositions(Booleans.parseBoolean(s))); - appliers.put("lenient", (qb, s) -> qb.lenient(Booleans.parseBoolean(s))); - appliers.put("max_expansions", (qb, s) -> qb.maxExpansions(Integer.valueOf(s))); - appliers.put("minimum_should_match", (qb, s) -> qb.minimumShouldMatch(s)); - appliers.put("operator", (qb, s) -> qb.operator(Operator.fromString(s))); - appliers.put("prefix_length", (qb, s) -> qb.prefixLength(Integer.valueOf(s))); - appliers.put("tie_breaker", (qb, s) -> qb.tieBreaker(Float.valueOf(s))); - appliers.put("type", (qb, s) -> qb.type(s)); - BUILDER_APPLIERS = Collections.unmodifiableMap(appliers); + BUILDER_APPLIERS = Map.ofEntries( + entry("slop", (qb, s) -> qb.slop(Integer.valueOf(s))), + // TODO: add zero terms query support, I'm not sure the best way to parse it yet... + // appliers.put("zero_terms_query", (qb, s) -> qb.zeroTermsQuery(s)); + entry("analyzer", MultiMatchQueryBuilder::analyzer), + entry("auto_generate_synonyms_phrase_query", (qb, s) -> qb.autoGenerateSynonymsPhraseQuery(Booleans.parseBoolean(s))), + entry("cutoff_frequency", (qb, s) -> qb.cutoffFrequency(Float.valueOf(s))), + entry("fuzziness", (qb, s) -> qb.fuzziness(Fuzziness.fromString(s))), + entry("fuzzy_rewrite", MultiMatchQueryBuilder::fuzzyRewrite), + entry("fuzzy_transpositions", (qb, s) -> qb.fuzzyTranspositions(Booleans.parseBoolean(s))), + entry("lenient", (qb, s) -> qb.lenient(Booleans.parseBoolean(s))), + entry("max_expansions", (qb, s) -> qb.maxExpansions(Integer.valueOf(s))), + entry("minimum_should_match", MultiMatchQueryBuilder::minimumShouldMatch), + entry("operator", (qb, s) -> qb.operator(Operator.fromString(s))), + entry("prefix_length", (qb, s) -> qb.prefixLength(Integer.valueOf(s))), + entry("tie_breaker", (qb, s) -> qb.tieBreaker(Float.valueOf(s))), + entry("type", MultiMatchQueryBuilder::type)); } private final String query; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/QueryStringQuery.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/QueryStringQuery.java index 94a9716b7cc8a..a81d1d1c9dae7 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/QueryStringQuery.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/query/QueryStringQuery.java @@ -17,43 +17,39 @@ import org.elasticsearch.xpack.sql.tree.Source; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.function.BiConsumer; +import static java.util.Map.entry; + public class QueryStringQuery extends LeafQuery { - private static final Map> BUILDER_APPLIERS; - - static { - HashMap> appliers = new HashMap<>(28); - // TODO: it'd be great if these could be constants instead of Strings, needs a core change to make the fields public first - appliers.put("allow_leading_wildcard", (qb, s) -> qb.allowLeadingWildcard(Booleans.parseBoolean(s))); - appliers.put("analyze_wildcard", (qb, s) -> qb.analyzeWildcard(Booleans.parseBoolean(s))); - appliers.put("analyzer", (qb, s) -> qb.analyzer(s)); - appliers.put("auto_generate_synonyms_phrase_query", (qb, s) -> qb.autoGenerateSynonymsPhraseQuery(Booleans.parseBoolean(s))); - appliers.put("default_field", (qb, s) -> qb.defaultField(s)); - appliers.put("default_operator", (qb, s) -> qb.defaultOperator(Operator.fromString(s))); - appliers.put("enable_position_increments", (qb, s) -> qb.enablePositionIncrements(Booleans.parseBoolean(s))); - appliers.put("escape", (qb, s) -> qb.escape(Booleans.parseBoolean(s))); - appliers.put("fuzziness", (qb, s) -> qb.fuzziness(Fuzziness.fromString(s))); - appliers.put("fuzzy_max_expansions", (qb, s) -> qb.fuzzyMaxExpansions(Integer.valueOf(s))); - appliers.put("fuzzy_prefix_length", (qb, s) -> qb.fuzzyPrefixLength(Integer.valueOf(s))); - appliers.put("fuzzy_rewrite", (qb, s) -> qb.fuzzyRewrite(s)); - appliers.put("fuzzy_transpositions", (qb, s) -> qb.fuzzyTranspositions(Booleans.parseBoolean(s))); - appliers.put("lenient", (qb, s) -> qb.lenient(Booleans.parseBoolean(s))); - appliers.put("max_determinized_states", (qb, s) -> qb.maxDeterminizedStates(Integer.valueOf(s))); - appliers.put("minimum_should_match", (qb, s) -> qb.minimumShouldMatch(s)); - appliers.put("phrase_slop", (qb, s) -> qb.phraseSlop(Integer.valueOf(s))); - appliers.put("rewrite", (qb, s) -> qb.rewrite(s)); - appliers.put("quote_analyzer", (qb, s) -> qb.quoteAnalyzer(s)); - appliers.put("quote_field_suffix", (qb, s) -> qb.quoteFieldSuffix(s)); - appliers.put("tie_breaker", (qb, s) -> qb.tieBreaker(Float.valueOf(s))); - appliers.put("time_zone", (qb, s) -> qb.timeZone(s)); - appliers.put("type", (qb, s) -> qb.type(MultiMatchQueryBuilder.Type.parse(s, LoggingDeprecationHandler.INSTANCE))); - BUILDER_APPLIERS = Collections.unmodifiableMap(appliers); - } + // TODO: it'd be great if these could be constants instead of Strings, needs a core change to make the fields public first + private static final Map> BUILDER_APPLIERS = Map.ofEntries( + entry("allow_leading_wildcard", (qb, s) -> qb.allowLeadingWildcard(Booleans.parseBoolean(s))), + entry("analyze_wildcard", (qb, s) -> qb.analyzeWildcard(Booleans.parseBoolean(s))), + entry("analyzer", QueryStringQueryBuilder::analyzer), + entry("auto_generate_synonyms_phrase_query", (qb, s) -> qb.autoGenerateSynonymsPhraseQuery(Booleans.parseBoolean(s))), + entry("default_field", QueryStringQueryBuilder::defaultField), + entry("default_operator", (qb, s) -> qb.defaultOperator(Operator.fromString(s))), + entry("enable_position_increments", (qb, s) -> qb.enablePositionIncrements(Booleans.parseBoolean(s))), + entry("escape", (qb, s) -> qb.escape(Booleans.parseBoolean(s))), + entry("fuzziness", (qb, s) -> qb.fuzziness(Fuzziness.fromString(s))), + entry("fuzzy_max_expansions", (qb, s) -> qb.fuzzyMaxExpansions(Integer.valueOf(s))), + entry("fuzzy_prefix_length", (qb, s) -> qb.fuzzyPrefixLength(Integer.valueOf(s))), + entry("fuzzy_rewrite", QueryStringQueryBuilder::fuzzyRewrite), + entry("fuzzy_transpositions", (qb, s) -> qb.fuzzyTranspositions(Booleans.parseBoolean(s))), + entry("lenient", (qb, s) -> qb.lenient(Booleans.parseBoolean(s))), + entry("max_determinized_states", (qb, s) -> qb.maxDeterminizedStates(Integer.valueOf(s))), + entry("minimum_should_match", QueryStringQueryBuilder::minimumShouldMatch), + entry("phrase_slop", (qb, s) -> qb.phraseSlop(Integer.valueOf(s))), + entry("rewrite", QueryStringQueryBuilder::rewrite), + entry("quote_analyzer", QueryStringQueryBuilder::quoteAnalyzer), + entry("quote_field_suffix", QueryStringQueryBuilder::quoteFieldSuffix), + entry("tie_breaker", (qb, s) -> qb.tieBreaker(Float.valueOf(s))), + entry("time_zone", QueryStringQueryBuilder::timeZone), + entry("type", (qb, s) -> qb.type(MultiMatchQueryBuilder.Type.parse(s, LoggingDeprecationHandler.INSTANCE)))); private final String query; private final Map fields; diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index f5f12d4fd244a..dd7f268e1a6f5 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -62,13 +62,11 @@ import org.elasticsearch.xpack.core.watcher.WatcherField; import org.elasticsearch.xpack.core.watcher.actions.ActionFactory; import org.elasticsearch.xpack.core.watcher.actions.ActionRegistry; -import org.elasticsearch.xpack.core.watcher.condition.ConditionFactory; import org.elasticsearch.xpack.core.watcher.condition.ConditionRegistry; import org.elasticsearch.xpack.core.watcher.crypto.CryptoService; import org.elasticsearch.xpack.core.watcher.execution.TriggeredWatchStoreField; import org.elasticsearch.xpack.core.watcher.history.HistoryStoreField; import org.elasticsearch.xpack.core.watcher.input.none.NoneInput; -import org.elasticsearch.xpack.core.watcher.transform.TransformFactory; import org.elasticsearch.xpack.core.watcher.transform.TransformRegistry; import org.elasticsearch.xpack.core.watcher.transport.actions.ack.AckWatchAction; import org.elasticsearch.xpack.core.watcher.transport.actions.activate.ActivateWatchAction; @@ -291,18 +289,18 @@ public Collection createComponents(Client client, ClusterService cluster EmailAttachmentsParser emailAttachmentsParser = new EmailAttachmentsParser(emailAttachmentParsers); // conditions - final Map parsers = new HashMap<>(); - parsers.put(InternalAlwaysCondition.TYPE, (c, id, p) -> InternalAlwaysCondition.parse(id, p)); - parsers.put(NeverCondition.TYPE, (c, id, p) -> NeverCondition.parse(id, p)); - parsers.put(ArrayCompareCondition.TYPE, ArrayCompareCondition::parse); - parsers.put(CompareCondition.TYPE, CompareCondition::parse); - parsers.put(ScriptCondition.TYPE, (c, id, p) -> ScriptCondition.parse(scriptService, id, p)); - - final ConditionRegistry conditionRegistry = new ConditionRegistry(Collections.unmodifiableMap(parsers), getClock()); - final Map transformFactories = new HashMap<>(); - transformFactories.put(ScriptTransform.TYPE, new ScriptTransformFactory(scriptService)); - transformFactories.put(SearchTransform.TYPE, new SearchTransformFactory(settings, client, xContentRegistry, scriptService)); - final TransformRegistry transformRegistry = new TransformRegistry(Collections.unmodifiableMap(transformFactories)); + + final ConditionRegistry conditionRegistry = new ConditionRegistry( + Map.of( + InternalAlwaysCondition.TYPE, (c, id, p) -> InternalAlwaysCondition.parse(id, p), + NeverCondition.TYPE, (c, id, p) -> NeverCondition.parse(id, p), + ArrayCompareCondition.TYPE, ArrayCompareCondition::parse, + CompareCondition.TYPE, CompareCondition::parse, + ScriptCondition.TYPE, (c, id, p) -> ScriptCondition.parse(scriptService, id, p)), + getClock()); + final TransformRegistry transformRegistry = new TransformRegistry(Map.of( + ScriptTransform.TYPE, new ScriptTransformFactory(scriptService), + SearchTransform.TYPE, new SearchTransformFactory(settings, client, xContentRegistry, scriptService))); // actions final Map actionFactoryMap = new HashMap<>(); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTemplate.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTemplate.java index 8ca87395248b1..55e5d07a71c7b 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTemplate.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTemplate.java @@ -29,7 +29,6 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.singletonMap; -import static java.util.Collections.unmodifiableMap; public class HttpRequestTemplate implements ToXContentObject { @@ -469,8 +468,8 @@ public Builder proxy(HttpProxy proxy) { } public HttpRequestTemplate build() { - return new HttpRequestTemplate(host, port, scheme, method, path, unmodifiableMap(new HashMap<>(params)), - unmodifiableMap(new HashMap<>(headers)), auth, body, connectionTimeout, readTimeout, proxy); + return new HttpRequestTemplate(host, port, scheme, method, path, Map.copyOf(params), + Map.copyOf(headers), auth, body, connectionTimeout, readTimeout, proxy); } public Builder fromUrl(String supposedUrl) { diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/WatcherMockScriptPlugin.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/WatcherMockScriptPlugin.java index 5a03a3c0e6433..2515a827bcc60 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/WatcherMockScriptPlugin.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/WatcherMockScriptPlugin.java @@ -15,8 +15,6 @@ import org.elasticsearch.xpack.watcher.condition.WatcherConditionScript; import org.elasticsearch.xpack.watcher.transform.script.WatcherTransformScript; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -29,35 +27,31 @@ public abstract class WatcherMockScriptPlugin extends MockScriptPlugin { public static final Map, MockScriptEngine.ContextCompiler> CONTEXT_COMPILERS; static { - Map, MockScriptEngine.ContextCompiler> compilers = new HashMap<>(); - compilers.put(WatcherConditionScript.CONTEXT, (script, options) -> - (WatcherConditionScript.Factory) (params, watcherContext) -> - new WatcherConditionScript(params, watcherContext) { - @Override - public boolean execute() { - Map vars = new HashMap<>(); - vars.put("params", getParams()); - vars.put("ctx", getCtx()); - return (boolean) script.apply(vars); - } - }); - compilers.put(WatcherTransformScript.CONTEXT, (script, options) -> - (WatcherTransformScript.Factory) (params, watcherContext, payload) -> - new WatcherTransformScript(params, watcherContext, payload) { - @Override - public Object execute() { - Map vars = new HashMap<>(); - vars.put("params", getParams()); - vars.put("ctx", getCtx()); - return script.apply(vars); - } - }); - CONTEXT_COMPILERS = Collections.unmodifiableMap(compilers); + CONTEXT_COMPILERS = Map.of(WatcherConditionScript.CONTEXT, (script, options) -> + (WatcherConditionScript.Factory) (params, watcherContext) -> + new WatcherConditionScript(params, watcherContext) { + @Override + public boolean execute() { + Map vars = new HashMap<>(); + vars.put("params", getParams()); + vars.put("ctx", getCtx()); + return (boolean) script.apply(vars); + } + }, WatcherTransformScript.CONTEXT, (script, options) -> + (WatcherTransformScript.Factory) (params, watcherContext, payload) -> + new WatcherTransformScript(params, watcherContext, payload) { + @Override + public Object execute() { + Map vars = new HashMap<>(); + vars.put("params", getParams()); + vars.put("ctx", getCtx()); + return script.apply(vars); + } + }); } - public static final List> CONTEXTS = Collections.unmodifiableList(Arrays.asList( - WatcherConditionScript.CONTEXT, WatcherTransformScript.CONTEXT, Watcher.SCRIPT_TEMPLATE_CONTEXT - )); + public static final List> CONTEXTS = + List.of(WatcherConditionScript.CONTEXT, WatcherTransformScript.CONTEXT, Watcher.SCRIPT_TEMPLATE_CONTEXT); @Override protected Map, MockScriptEngine.ContextCompiler> pluginContextCompilers() { diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/watch/WatchTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/watch/WatchTests.java index ac944755b9c4d..84144b2866b32 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/watch/WatchTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/watch/WatchTests.java @@ -37,7 +37,6 @@ import org.elasticsearch.xpack.core.watcher.input.ExecutableInput; import org.elasticsearch.xpack.core.watcher.input.none.NoneInput; import org.elasticsearch.xpack.core.watcher.transform.ExecutableTransform; -import org.elasticsearch.xpack.core.watcher.transform.TransformFactory; import org.elasticsearch.xpack.core.watcher.transform.TransformRegistry; import org.elasticsearch.xpack.core.watcher.transform.chain.ChainTransform; import org.elasticsearch.xpack.core.watcher.transform.chain.ExecutableChainTransform; @@ -572,10 +571,11 @@ private ExecutableTransform randomTransform() { } private TransformRegistry transformRegistry() { - Map factories = new HashMap<>(); - factories.put(ScriptTransform.TYPE, new ScriptTransformFactory(scriptService)); - factories.put(SearchTransform.TYPE, new SearchTransformFactory(settings, client, xContentRegistry(), scriptService)); - return new TransformRegistry(unmodifiableMap(factories)); + return new TransformRegistry(Map.of( + ScriptTransform.TYPE, + new ScriptTransformFactory(scriptService), + SearchTransform.TYPE, + new SearchTransformFactory(settings, client, xContentRegistry(), scriptService))); } private List randomActions() { diff --git a/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/SpnegoClient.java b/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/SpnegoClient.java index 839b25c783690..42fde573c3d07 100644 --- a/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/SpnegoClient.java +++ b/x-pack/qa/evil-tests/src/test/java/org/elasticsearch/xpack/security/authc/kerberos/SpnegoClient.java @@ -18,17 +18,6 @@ import org.ietf.jgss.GSSName; import org.ietf.jgss.Oid; -import java.io.IOException; -import java.security.AccessController; -import java.security.Principal; -import java.security.PrivilegedActionException; -import java.security.PrivilegedExceptionAction; -import java.util.Base64; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - import javax.security.auth.Subject; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; @@ -40,6 +29,16 @@ import javax.security.auth.login.LoginContext; import javax.security.auth.login.LoginException; +import java.io.IOException; +import java.security.AccessController; +import java.security.Principal; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.Base64; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + /** * This class is used as a Spnego client during testing and handles SPNEGO * interactions using GSS context negotiation.
    @@ -196,16 +195,17 @@ static class PasswordJaasConf extends Configuration { @Override public AppConfigurationEntry[] getAppConfigurationEntry(final String name) { - final Map options = new HashMap<>(); - options.put("principal", principal); - options.put("storeKey", Boolean.TRUE.toString()); - options.put("isInitiator", Boolean.TRUE.toString()); - options.put("debug", Boolean.TRUE.toString()); - // Refresh Krb5 config during tests as the port keeps changing for kdc server - options.put("refreshKrb5Config", Boolean.TRUE.toString()); - return new AppConfigurationEntry[] { new AppConfigurationEntry(SUN_KRB5_LOGIN_MODULE, - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, Collections.unmodifiableMap(options)) }; + return new AppConfigurationEntry[]{new AppConfigurationEntry( + SUN_KRB5_LOGIN_MODULE, + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + Map.of( + "principal", principal, + "storeKey", Boolean.TRUE.toString(), + "isInitiator", Boolean.TRUE.toString(), + "debug", Boolean.TRUE.toString(), + // refresh Krb5 config during tests as the port keeps changing for kdc server + "refreshKrb5Config", Boolean.TRUE.toString()))}; } } From 06bc25e44734c8c30b5be02a29a0b3a2880a1a1d Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 26 Apr 2019 14:41:05 -0500 Subject: [PATCH 214/260] [ML] cleanup + adding description field to transforms (#41554) * [ML] cleanup + adding description field to transforms * making description length have a max of 1k --- .../transforms/DataFrameTransformConfig.java | 36 ++++- .../client/DataFrameTransformIT.java | 8 +- ...PreviewDataFrameTransformRequestTests.java | 5 +- .../PutDataFrameTransformRequestTests.java | 2 +- .../DataFrameTransformConfigTests.java | 2 +- .../DataFrameTransformDocumentationIT.java | 23 ++-- .../dataframe/preview_data_frame.asciidoc | 4 +- .../dataframe/put_data_frame.asciidoc | 1 + .../data-frames/apis/put-transform.asciidoc | 5 +- .../DeleteDataFrameTransformAction.java | 2 +- .../action/StartDataFrameTransformAction.java | 2 +- .../StartDataFrameTransformTaskAction.java | 2 +- .../action/StopDataFrameTransformAction.java | 2 +- .../notifications/DataFrameAuditMessage.java | 15 +- .../transforms/DataFrameTransformConfig.java | 33 ++++- .../core/dataframe/transforms/DestConfig.java | 2 +- .../dataframe/transforms/SourceConfig.java | 2 +- .../transforms/pivot/GroupConfig.java | 2 +- .../transforms/pivot/PivotConfig.java | 2 +- .../xpack/core/dataframe/utils/TimeUtils.java | 129 ++++++++++++++++++ ...wDataFrameTransformActionRequestTests.java | 2 +- .../DataFrameTransformConfigTests.java | 22 ++- .../integration/DataFrameIntegTestCase.java | 3 +- .../DataFrameGetAndGetStatsIT.java | 1 + .../DataFrameTransformProgressIT.java | 6 +- .../persistence/DataFrameInternalIndex.java | 4 + ...portStopDataFrameTransformActionTests.java | 31 ++++- .../test/data_frame/transforms_crud.yml | 5 +- 28 files changed, 290 insertions(+), 63 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/TimeUtils.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java index 88670a7b36d1f..4c7f29afc8678 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java @@ -20,6 +20,7 @@ package org.elasticsearch.client.dataframe.transforms; import org.elasticsearch.client.dataframe.transforms.pivot.PivotConfig; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.ConstructingObjectParser; @@ -38,6 +39,7 @@ public class DataFrameTransformConfig implements ToXContentObject { public static final ParseField ID = new ParseField("id"); public static final ParseField SOURCE = new ParseField("source"); public static final ParseField DEST = new ParseField("dest"); + public static final ParseField DESCRIPTION = new ParseField("description"); // types of transforms public static final ParseField PIVOT_TRANSFORM = new ParseField("pivot"); @@ -45,6 +47,7 @@ public class DataFrameTransformConfig implements ToXContentObject { private final SourceConfig source; private final DestConfig dest; private final PivotConfig pivotConfig; + private final String description; public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("data_frame_transform", true, @@ -53,7 +56,8 @@ public class DataFrameTransformConfig implements ToXContentObject { SourceConfig source = (SourceConfig) args[1]; DestConfig dest = (DestConfig) args[2]; PivotConfig pivotConfig = (PivotConfig) args[3]; - return new DataFrameTransformConfig(id, source, dest, pivotConfig); + String description = (String)args[4]; + return new DataFrameTransformConfig(id, source, dest, pivotConfig, description); }); static { @@ -61,21 +65,38 @@ public class DataFrameTransformConfig implements ToXContentObject { PARSER.declareObject(constructorArg(), (p, c) -> SourceConfig.PARSER.apply(p, null), SOURCE); PARSER.declareObject(constructorArg(), (p, c) -> DestConfig.PARSER.apply(p, null), DEST); PARSER.declareObject(optionalConstructorArg(), (p, c) -> PivotConfig.fromXContent(p), PIVOT_TRANSFORM); + PARSER.declareString(optionalConstructorArg(), DESCRIPTION); } public static DataFrameTransformConfig fromXContent(final XContentParser parser) { return PARSER.apply(parser, null); } + /** + * Helper method for previewing a data frame transform configuration + * + * The DataFrameTransformConfig returned from this method should only be used for previewing the resulting data. + * + * A new, valid, DataFrameTransformConfig with an appropriate destination and ID will have to be constructed to create + * the transform. + * @param source Source configuration for gathering the data + * @param pivotConfig Pivot config to preview + * @return A DataFrameTransformConfig to preview, NOTE it will have a {@code null} id, destination and index. + */ + public static DataFrameTransformConfig forPreview(final SourceConfig source, final PivotConfig pivotConfig) { + return new DataFrameTransformConfig(null, source, null, pivotConfig, null); + } public DataFrameTransformConfig(final String id, final SourceConfig source, final DestConfig dest, - final PivotConfig pivotConfig) { + final PivotConfig pivotConfig, + final String description) { this.id = id; this.source = source; this.dest = dest; this.pivotConfig = pivotConfig; + this.description = description; } public String getId() { @@ -94,6 +115,11 @@ public PivotConfig getPivotConfig() { return pivotConfig; } + @Nullable + public String getDescription() { + return description; + } + @Override public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException { builder.startObject(); @@ -109,6 +135,9 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (pivotConfig != null) { builder.field(PIVOT_TRANSFORM.getPreferredName(), pivotConfig); } + if (description != null) { + builder.field(DESCRIPTION.getPreferredName(), description); + } builder.endObject(); return builder; } @@ -128,12 +157,13 @@ public boolean equals(Object other) { return Objects.equals(this.id, that.id) && Objects.equals(this.source, that.source) && Objects.equals(this.dest, that.dest) + && Objects.equals(this.description, that.description) && Objects.equals(this.pivotConfig, that.pivotConfig); } @Override public int hashCode() { - return Objects.hash(id, source, dest, pivotConfig); + return Objects.hash(id, source, dest, pivotConfig, description); } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java index 3d7f5e3dbcbbd..9cfa42da1a800 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java @@ -312,7 +312,8 @@ private DataFrameTransformConfig validDataFrameTransformConfig(String id, String return new DataFrameTransformConfig(id, new SourceConfig(new String[]{source}, queryConfig), destConfig, - pivotConfig); + pivotConfig, + "this is a test transform"); } public void testGetStats() throws Exception { @@ -329,7 +330,10 @@ public void testGetStats() throws Exception { String id = "test-get-stats"; DataFrameTransformConfig transform = new DataFrameTransformConfig(id, - new SourceConfig(new String[]{sourceIndex}, queryConfig), new DestConfig("pivot-dest"), pivotConfig); + new SourceConfig(new String[]{sourceIndex}, queryConfig), + new DestConfig("pivot-dest"), + pivotConfig, + "transform for testing stats"); DataFrameClient client = highLevelClient().dataFrame(); AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java index 768014e4e210c..690e93697b54b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java @@ -65,13 +65,12 @@ public void testValidate() { containsString("preview requires a non-null data frame config")); // null id and destination is valid - DataFrameTransformConfig config = new DataFrameTransformConfig(null, randomSourceConfig(), null, - PivotConfigTests.randomPivotConfig()); + DataFrameTransformConfig config = DataFrameTransformConfig.forPreview(randomSourceConfig(), PivotConfigTests.randomPivotConfig()); assertFalse(new PreviewDataFrameTransformRequest(config).validate().isPresent()); // null source is not valid - config = new DataFrameTransformConfig(null, null, null, PivotConfigTests.randomPivotConfig()); + config = new DataFrameTransformConfig(null, null, null, PivotConfigTests.randomPivotConfig(), null); Optional error = new PreviewDataFrameTransformRequest(config).validate(); assertTrue(error.isPresent()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java index 9782418e322bf..245e8e94e5599 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java @@ -40,7 +40,7 @@ public class PutDataFrameTransformRequestTests extends AbstractXContentTestCase< public void testValidate() { assertFalse(createTestInstance().validate().isPresent()); - DataFrameTransformConfig config = new DataFrameTransformConfig(null, null, null, PivotConfigTests.randomPivotConfig()); + DataFrameTransformConfig config = new DataFrameTransformConfig(null, null, null, PivotConfigTests.randomPivotConfig(), null); Optional error = new PutDataFrameTransformRequest(config).validate(); assertTrue(error.isPresent()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java index fd744b2f9f8a6..1b5228d96229f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfigTests.java @@ -37,7 +37,7 @@ public class DataFrameTransformConfigTests extends AbstractXContentTestCase sourceConfig, // <2> new DestConfig("pivot-destination"), // <3> - pivotConfig); // <4> + pivotConfig, // <4> + "This is my test transform"); // <5> // end::put-data-frame-transform-config { @@ -161,7 +162,7 @@ public void testPutDataFrameTransform() throws IOException, InterruptedException { DataFrameTransformConfig configWithDifferentId = new DataFrameTransformConfig("reviewer-avg-rating2", transformConfig.getSource(), transformConfig.getDestination(), - transformConfig.getPivotConfig()); + transformConfig.getPivotConfig(), null); PutDataFrameTransformRequest request = new PutDataFrameTransformRequest(configWithDifferentId); // tag::put-data-frame-transform-execute-listener @@ -205,7 +206,7 @@ public void testStartStop() throws IOException, InterruptedException { PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); DataFrameTransformConfig transformConfig = new DataFrameTransformConfig("mega-transform", - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest"), pivotConfig); + new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest"), pivotConfig, null); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig), RequestOptions.DEFAULT); transformsToClean.add(transformConfig.getId()); @@ -320,9 +321,9 @@ public void testDeleteDataFrameTransform() throws IOException, InterruptedExcept PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); DataFrameTransformConfig transformConfig1 = new DataFrameTransformConfig("mega-transform", - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest"), pivotConfig); + new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest"), pivotConfig, null); DataFrameTransformConfig transformConfig2 = new DataFrameTransformConfig("mega-transform2", - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest2"), pivotConfig); + new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest2"), pivotConfig, null); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig1), RequestOptions.DEFAULT); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig2), RequestOptions.DEFAULT); @@ -386,11 +387,9 @@ public void testPreview() throws IOException, InterruptedException { // tag::preview-data-frame-transform-request DataFrameTransformConfig transformConfig = - new DataFrameTransformConfig(null, // <1> - new SourceConfig(new String[]{"source-data"}, queryConfig), - null, // <2> - pivotConfig); - + DataFrameTransformConfig.forPreview( + new SourceConfig(new String[]{"source-data"}, queryConfig), // <1> + pivotConfig); // <2> PreviewDataFrameTransformRequest request = new PreviewDataFrameTransformRequest(transformConfig); // <3> // end::preview-data-frame-transform-request @@ -447,7 +446,7 @@ public void testGetStats() throws IOException, InterruptedException { String id = "statisitcal-transform"; DataFrameTransformConfig transformConfig = new DataFrameTransformConfig(id, - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("dest"), pivotConfig); + new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("dest"), pivotConfig, null); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig), RequestOptions.DEFAULT); // tag::get-data-frame-transform-stats-request @@ -526,7 +525,7 @@ public void testGetDataFrameTransform() throws IOException, InterruptedException DataFrameTransformConfig putTransformConfig = new DataFrameTransformConfig("mega-transform", new SourceConfig(new String[]{"source-data"}, queryConfig), - new DestConfig("pivot-dest"), pivotConfig); + new DestConfig("pivot-dest"), pivotConfig, null); RestHighLevelClient client = highLevelClient(); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(putTransformConfig), RequestOptions.DEFAULT); diff --git a/docs/java-rest/high-level/dataframe/preview_data_frame.asciidoc b/docs/java-rest/high-level/dataframe/preview_data_frame.asciidoc index 2bfb672991def..df7e461fa0919 100644 --- a/docs/java-rest/high-level/dataframe/preview_data_frame.asciidoc +++ b/docs/java-rest/high-level/dataframe/preview_data_frame.asciidoc @@ -20,8 +20,8 @@ A +{request}+ takes a single argument: a valid data frame transform config. -------------------------------------------------- include-tagged::{doc-tests-file}[{api}-request] -------------------------------------------------- -<1> The transform Id may be null for the preview -<2> The destination may be null for the preview +<1> The source config from which the data should be gathered +<2> The pivot config used to transform the data <3> The configuration of the {dataframe-job} to preview include::../execution.asciidoc[] diff --git a/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc b/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc index 2db0bbea7ac81..bb1b20aaa1a52 100644 --- a/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc +++ b/docs/java-rest/high-level/dataframe/put_data_frame.asciidoc @@ -35,6 +35,7 @@ include-tagged::{doc-tests-file}[{api}-config] <2> The source indices and query from which to gather data <3> The destination index <4> The PivotConfig +<5> Optional free text description of the transform [id="{upid}-{api}-query-config"] diff --git a/docs/reference/data-frames/apis/put-transform.asciidoc b/docs/reference/data-frames/apis/put-transform.asciidoc index 1889121ae5d38..7158af959fc22 100644 --- a/docs/reference/data-frames/apis/put-transform.asciidoc +++ b/docs/reference/data-frames/apis/put-transform.asciidoc @@ -33,6 +33,8 @@ a `query`. `pivot`:: Defines the pivot function `group by` fields and the aggregation to reduce the data. +`description`:: Optional free text description of the data frame transform + //==== Authorization @@ -73,7 +75,8 @@ PUT _data_frame/transforms/ecommerce_transform } } } - } + }, + "description": "Maximum priced ecommerce data by customer_id in Asia" } -------------------------------------------------- // CONSOLE diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java index 8ac1e9c6c513f..6b7de0ab80f3a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/DeleteDataFrameTransformAction.java @@ -18,7 +18,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Collections; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java index 29529c1d67452..6216b4489db92 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformAction.java @@ -16,7 +16,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Collections; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java index 52f061e380e82..fe453200fb2bc 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StartDataFrameTransformTaskAction.java @@ -17,7 +17,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Collections; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java index ee7bb830660f3..e7a43f252d637 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/action/StopDataFrameTransformAction.java @@ -18,7 +18,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.tasks.Task; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Arrays; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/notifications/DataFrameAuditMessage.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/notifications/DataFrameAuditMessage.java index e6ac6cbc57b15..dd6aee25580df 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/notifications/DataFrameAuditMessage.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/notifications/DataFrameAuditMessage.java @@ -12,7 +12,7 @@ import org.elasticsearch.xpack.core.common.notifications.AbstractAuditMessage; import org.elasticsearch.xpack.core.common.notifications.Level; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.ml.utils.time.TimeUtils; +import org.elasticsearch.xpack.core.dataframe.utils.TimeUtils; import java.util.Date; @@ -36,15 +36,10 @@ public class DataFrameAuditMessage extends AbstractAuditMessage { } throw new IllegalArgumentException("Unsupported token [" + p.currentToken() + "]"); }, LEVEL, ObjectParser.ValueType.STRING); - PARSER.declareField(constructorArg(), parser -> { - if (parser.currentToken() == XContentParser.Token.VALUE_NUMBER) { - return new Date(parser.longValue()); - } else if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { - return new Date(TimeUtils.dateStringToEpoch(parser.text())); - } - throw new IllegalArgumentException( - "unexpected token [" + parser.currentToken() + "] for [" + TIMESTAMP.getPreferredName() + "]"); - }, TIMESTAMP, ObjectParser.ValueType.VALUE); + PARSER.declareField(constructorArg(), + p -> TimeUtils.parseTimeField(p, TIMESTAMP.getPreferredName()), + TIMESTAMP, + ObjectParser.ValueType.VALUE); PARSER.declareString(optionalConstructorArg(), NODE_NAME); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java index 89eab5605ca90..ee35fe3d21ec7 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfig.java @@ -20,7 +20,7 @@ import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.transforms.pivot.PivotConfig; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Collections; @@ -41,13 +41,15 @@ public class DataFrameTransformConfig extends AbstractDiffable STRICT_PARSER = createParser(false); private static final ConstructingObjectParser LENIENT_PARSER = createParser(true); + private static final int MAX_DESCRIPTION_LENGTH = 1_000; private final String id; private final SourceConfig source; private final DestConfig dest; - + private final String description; // headers store the user context from the creating user, which allows us to run the transform as this user // the header only contains name, groups and other context but no authorization keys private Map headers; @@ -81,7 +83,8 @@ private static ConstructingObjectParser create Map headers = (Map) args[4]; PivotConfig pivotConfig = (PivotConfig) args[5]; - return new DataFrameTransformConfig(id, source, dest, headers, pivotConfig); + String description = (String)args[6]; + return new DataFrameTransformConfig(id, source, dest, headers, pivotConfig, description); }); parser.declareString(optionalConstructorArg(), DataFrameField.ID); @@ -91,6 +94,7 @@ private static ConstructingObjectParser create parser.declareString(optionalConstructorArg(), DataFrameField.INDEX_DOC_TYPE); parser.declareObject(optionalConstructorArg(), (p, c) -> p.mapStrings(), HEADERS); parser.declareObject(optionalConstructorArg(), (p, c) -> PivotConfig.fromXContent(p, lenient), PIVOT_TRANSFORM); + parser.declareString(optionalConstructorArg(), DESCRIPTION); return parser; } @@ -103,17 +107,22 @@ public DataFrameTransformConfig(final String id, final SourceConfig source, final DestConfig dest, final Map headers, - final PivotConfig pivotConfig) { + final PivotConfig pivotConfig, + final String description) { this.id = ExceptionsHelper.requireNonNull(id, DataFrameField.ID.getPreferredName()); this.source = ExceptionsHelper.requireNonNull(source, DataFrameField.SOURCE.getPreferredName()); this.dest = ExceptionsHelper.requireNonNull(dest, DataFrameField.DESTINATION.getPreferredName()); this.setHeaders(headers == null ? Collections.emptyMap() : headers); this.pivotConfig = pivotConfig; + this.description = description; // at least one function must be defined if (this.pivotConfig == null) { throw new IllegalArgumentException(DataFrameMessages.DATA_FRAME_TRANSFORM_CONFIGURATION_NO_TRANSFORM); } + if (this.description != null && this.description.length() > MAX_DESCRIPTION_LENGTH) { + throw new IllegalArgumentException("[description] must be less than 1000 characters in length."); + } } public DataFrameTransformConfig(final StreamInput in) throws IOException { @@ -122,6 +131,7 @@ public DataFrameTransformConfig(final StreamInput in) throws IOException { dest = new DestConfig(in); setHeaders(in.readMap(StreamInput::readString, StreamInput::readString)); pivotConfig = in.readOptionalWriteable(PivotConfig::new); + description = in.readOptionalString(); } public String getId() { @@ -148,6 +158,11 @@ public PivotConfig getPivotConfig() { return pivotConfig; } + @Nullable + public String getDescription() { + return description; + } + public boolean isValid() { if (pivotConfig != null && pivotConfig.isValid() == false) { return false; @@ -163,6 +178,7 @@ public void writeTo(final StreamOutput out) throws IOException { dest.writeTo(out); out.writeMap(headers, StreamOutput::writeString, StreamOutput::writeString); out.writeOptionalWriteable(pivotConfig); + out.writeOptionalString(description); } @Override @@ -180,7 +196,9 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (headers.isEmpty() == false && params.paramAsBoolean(DataFrameField.FOR_INTERNAL_STORAGE, false) == true) { builder.field(HEADERS.getPreferredName(), headers); } - + if (description != null) { + builder.field(DESCRIPTION.getPreferredName(), description); + } builder.endObject(); return builder; } @@ -201,12 +219,13 @@ public boolean equals(Object other) { && Objects.equals(this.source, that.source) && Objects.equals(this.dest, that.dest) && Objects.equals(this.headers, that.headers) - && Objects.equals(this.pivotConfig, that.pivotConfig); + && Objects.equals(this.pivotConfig, that.pivotConfig) + && Objects.equals(this.description, that.description); } @Override public int hashCode(){ - return Objects.hash(id, source, dest, headers, pivotConfig); + return Objects.hash(id, source, dest, headers, pivotConfig, description); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DestConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DestConfig.java index 285f970a4a4e9..29b2b8c5dc0e1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DestConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/DestConfig.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Objects; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/SourceConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/SourceConfig.java index a11950701d156..19671fd552c80 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/SourceConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/SourceConfig.java @@ -15,7 +15,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Arrays; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java index e5ba14c381a4a..807c2e8d339dd 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/GroupConfig.java @@ -22,7 +22,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.LinkedHashMap; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java index 993ba78482aed..c1c894e2971ae 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/PivotConfig.java @@ -15,7 +15,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder; import org.elasticsearch.xpack.core.dataframe.DataFrameField; -import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; +import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper; import java.io.IOException; import java.util.Map.Entry; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/TimeUtils.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/TimeUtils.java new file mode 100644 index 0000000000000..4cf3859380cd8 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/utils/TimeUtils.java @@ -0,0 +1,129 @@ +/* + * 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.core.dataframe.utils; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.DateFieldMapper; + +import java.io.IOException; +import java.util.Date; +import java.util.concurrent.TimeUnit; + +public final class TimeUtils { + + private TimeUtils() { + // Do nothing + } + + public static Date parseTimeField(XContentParser parser, String fieldName) throws IOException { + if (parser.currentToken() == XContentParser.Token.VALUE_NUMBER) { + return new Date(parser.longValue()); + } else if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { + return new Date(TimeUtils.dateStringToEpoch(parser.text())); + } + throw new IllegalArgumentException( + "unexpected token [" + parser.currentToken() + "] for [" + fieldName + "]"); + } + + /** + * First tries to parse the date first as a Long and convert that to an + * epoch time. If the long number has more than 10 digits it is considered a + * time in milliseconds else if 10 or less digits it is in seconds. If that + * fails it tries to parse the string using + * {@link DateFieldMapper#DEFAULT_DATE_TIME_FORMATTER} + * + * If the date string cannot be parsed -1 is returned. + * + * @return The epoch time in milliseconds or -1 if the date cannot be + * parsed. + */ + public static long dateStringToEpoch(String date) { + try { + long epoch = Long.parseLong(date); + if (date.trim().length() <= 10) { // seconds + return epoch * 1000; + } else { + return epoch; + } + } catch (NumberFormatException nfe) { + // not a number + } + + try { + return DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parseMillis(date); + } catch (ElasticsearchParseException | IllegalArgumentException e) { + } + // Could not do the conversion + return -1; + } + + /** + * Checks that the given {@code timeValue} is a non-negative multiple value of the {@code baseUnit}. + * + *
      + *
    • 400ms is valid for base unit of seconds
    • + *
    • 450ms is invalid for base unit of seconds but valid for base unit of milliseconds
    • + *
    + */ + public static void checkNonNegativeMultiple(TimeValue timeValue, TimeUnit baseUnit, ParseField field) { + checkNonNegative(timeValue, field); + checkMultiple(timeValue, baseUnit, field); + } + + /** + * Checks that the given {@code timeValue} is a positive multiple value of the {@code baseUnit}. + * + *
      + *
    • 400ms is valid for base unit of seconds
    • + *
    • 450ms is invalid for base unit of seconds but valid for base unit of milliseconds
    • + *
    + */ + public static void checkPositiveMultiple(TimeValue timeValue, TimeUnit baseUnit, ParseField field) { + checkPositive(timeValue, field); + checkMultiple(timeValue, baseUnit, field); + } + + /** + * Checks that the given {@code timeValue} is positive. + * + *
      + *
    • 1s is valid
    • + *
    • -1s is invalid
    • + *
    + */ + public static void checkPositive(TimeValue timeValue, ParseField field) { + long nanos = timeValue.getNanos(); + if (nanos <= 0) { + throw new IllegalArgumentException(field.getPreferredName() + " cannot be less or equal than 0. Value = " + + timeValue.toString()); + } + } + + private static void checkNonNegative(TimeValue timeValue, ParseField field) { + long nanos = timeValue.getNanos(); + if (nanos < 0) { + throw new IllegalArgumentException(field.getPreferredName() + " cannot be less than 0. Value = " + timeValue.toString()); + } + } + + + + /** + * Check the given {@code timeValue} is a multiple of the {@code baseUnit} + */ + public static void checkMultiple(TimeValue timeValue, TimeUnit baseUnit, ParseField field) { + long nanos = timeValue.getNanos(); + TimeValue base = new TimeValue(1, baseUnit); + long baseNanos = base.getNanos(); + if (nanos % baseNanos != 0) { + throw new IllegalArgumentException(field.getPreferredName() + " has to be a multiple of " + base.toString() + "; actual was '" + + timeValue.toString() + "'"); + } + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java index 4cab28105a899..267725d9a69d1 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/action/PreviewDataFrameTransformActionRequestTests.java @@ -68,7 +68,7 @@ protected boolean supportsUnknownFields() { @Override protected Request createTestInstance() { DataFrameTransformConfig config = new DataFrameTransformConfig("transform-preview", randomSourceConfig(), - new DestConfig("unused-transform-preview-index"), null, PivotConfigTests.randomPivotConfig()); + new DestConfig("unused-transform-preview-index"), null, PivotConfigTests.randomPivotConfig(), null); return new Request(config); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfigTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfigTests.java index cb0f25ea89c01..a735b5a02acb8 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfigTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/dataframe/transforms/DataFrameTransformConfigTests.java @@ -25,6 +25,7 @@ import static org.elasticsearch.xpack.core.dataframe.transforms.DestConfigTests.randomDestConfig; import static org.elasticsearch.xpack.core.dataframe.transforms.SourceConfigTests.randomInvalidSourceConfig; import static org.elasticsearch.xpack.core.dataframe.transforms.SourceConfigTests.randomSourceConfig; +import static org.hamcrest.Matchers.equalTo; public class DataFrameTransformConfigTests extends AbstractSerializingDataFrameTestCase { @@ -41,21 +42,23 @@ public static DataFrameTransformConfig randomDataFrameTransformConfig() { public static DataFrameTransformConfig randomDataFrameTransformConfigWithoutHeaders(String id) { return new DataFrameTransformConfig(id, randomSourceConfig(), randomDestConfig(), null, - PivotConfigTests.randomPivotConfig()); + PivotConfigTests.randomPivotConfig(), randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000)); } public static DataFrameTransformConfig randomDataFrameTransformConfig(String id) { return new DataFrameTransformConfig(id, randomSourceConfig(), randomDestConfig(), randomHeaders(), - PivotConfigTests.randomPivotConfig()); + PivotConfigTests.randomPivotConfig(), randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000)); } public static DataFrameTransformConfig randomInvalidDataFrameTransformConfig() { if (randomBoolean()) { return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), randomInvalidSourceConfig(), - randomDestConfig(), randomHeaders(), PivotConfigTests.randomPivotConfig()); + randomDestConfig(), randomHeaders(), PivotConfigTests.randomPivotConfig(), + randomBoolean() ? null : randomAlphaOfLengthBetween(1, 100)); } // else return new DataFrameTransformConfig(randomAlphaOfLengthBetween(1, 10), randomSourceConfig(), - randomDestConfig(), randomHeaders(), PivotConfigTests.randomInvalidPivotConfig()); + randomDestConfig(), randomHeaders(), PivotConfigTests.randomInvalidPivotConfig(), + randomBoolean() ? null : randomAlphaOfLengthBetween(1, 100)); } @Before @@ -162,6 +165,16 @@ public void testXContentForInternalStorage() throws IOException { } } + public void testMaxLengthDescription() { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> new DataFrameTransformConfig("id", + randomSourceConfig(), randomDestConfig(), null, PivotConfigTests.randomPivotConfig(), randomAlphaOfLength(1001))); + assertThat(exception.getMessage(), equalTo("[description] must be less than 1000 characters in length.")); + String description = randomAlphaOfLength(1000); + DataFrameTransformConfig config = new DataFrameTransformConfig("id", + randomSourceConfig(), randomDestConfig(), null, PivotConfigTests.randomPivotConfig(), description); + assertThat(description, equalTo(config.getDescription())); + } + public void testSetIdInBody() throws IOException { String pivotTransform = "{" + " \"id\" : \"body_id\"," @@ -189,6 +202,7 @@ public void testSetIdInBody() throws IOException { ex.getCause().getMessage()); } + private DataFrameTransformConfig createDataFrameTransformConfigFromString(String json, String id) throws IOException { final XContentParser parser = XContentType.JSON.xContent().createParser(xContentRegistry(), DeprecationHandler.THROW_UNSUPPORTED_OPERATION, json); diff --git a/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java b/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java index b45325fffaf08..84f3e05de5cd1 100644 --- a/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java +++ b/x-pack/plugin/data-frame/qa/multi-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameIntegTestCase.java @@ -193,7 +193,8 @@ protected DataFrameTransformConfig createTransformConfig(String id, new SourceConfig(sourceIndices, createQueryConfig(queryBuilder)), new DestConfig(destinationIndex), Collections.emptyMap(), - createPivotConfig(groups, aggregations)); + createPivotConfig(groups, aggregations), + "Test data frame transform config id: " + id); } protected void createReviewsIndex() throws Exception { 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 8de2fc5961a6c..62101e4e12064 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 @@ -57,6 +57,7 @@ public void clearOutTransforms() throws Exception { wipeDataFrameTransforms(); } + @SuppressWarnings("unchecked") public void testGetAndGetStats() throws Exception { createPivotReviewsTransform("pivot_1", "pivot_reviews_1", null); createPivotReviewsTransform("pivot_2", "pivot_reviews_2", null); diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java index e32842af0cc9f..d338d6949f07b 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameTransformProgressIT.java @@ -135,7 +135,8 @@ public void testGetProgress() throws Exception { sourceConfig, destConfig, null, - pivotConfig); + pivotConfig, + null); PlainActionFuture progressFuture = new PlainActionFuture<>(); TransformProgressGatherer.getInitialProgress(client(), config, progressFuture); @@ -154,7 +155,8 @@ public void testGetProgress() throws Exception { sourceConfig, destConfig, null, - pivotConfig); + pivotConfig, + null); progressFuture = new PlainActionFuture<>(); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameInternalIndex.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameInternalIndex.java index 26847c4881c3f..17a49d8b7e834 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameInternalIndex.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataFrameInternalIndex.java @@ -16,6 +16,7 @@ import org.elasticsearch.xpack.core.common.notifications.AbstractAuditMessage; import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats; +import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; import org.elasticsearch.xpack.core.dataframe.transforms.DestConfig; import org.elasticsearch.xpack.core.dataframe.transforms.SourceConfig; @@ -194,6 +195,9 @@ private static XContentBuilder addDataFrameTransformsConfigMappings(XContentBuil .field(TYPE, KEYWORD) .endObject() .endObject() + .endObject() + .startObject(DataFrameTransformConfig.DESCRIPTION.getPreferredName()) + .field(TYPE, TEXT) .endObject(); } diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java index 4be532de2a2be..ddc7ddd4f1b9c 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/action/TransportStopDataFrameTransformActionTests.java @@ -6,16 +6,19 @@ package org.elasticsearch.xpack.dataframe.action; +import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.persistent.PersistentTaskParams; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform; -import org.elasticsearch.xpack.core.ml.MlTasks; -import org.elasticsearch.xpack.core.ml.action.OpenJobAction; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; @@ -34,8 +37,28 @@ DataFrameField.TASK_NAME, new DataFrameTransform(dataFrameIdFoo), tasksBuilder.addTask(dataFrameIdBar, DataFrameField.TASK_NAME, new DataFrameTransform(dataFrameIdBar), new PersistentTasksCustomMetaData.Assignment("node-2", "test assignment")); - tasksBuilder.addTask(MlTasks.jobTaskId("foo-1"), MlTasks.JOB_TASK_NAME, new OpenJobAction.JobParams("foo-1"), - new PersistentTasksCustomMetaData.Assignment("node-3", "test assignment")); + tasksBuilder.addTask("test-task1", "testTasks", new PersistentTaskParams() { + @Override + public String getWriteableName() { + return "testTasks"; + } + + @Override + public Version getMinimalSupportedVersion() { + return null; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + }, + new PersistentTasksCustomMetaData.Assignment("node-3", "test assignment")); ClusterState cs = ClusterState.builder(new ClusterName("_name")) .metaData(MetaData.builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml index 353fc0aa9326a..b7f951679d1c5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/data_frame/transforms_crud.yml @@ -63,7 +63,8 @@ setup: "pivot": { "group_by": { "airline": {"terms": {"field": "airline"}}}, "aggs": {"avg_response": {"avg": {"field": "responsetime"}}} - } + }, + "description": "yaml test transform on airline-data" } - match: { acknowledged: true } @@ -91,6 +92,7 @@ setup: - is_true: transforms.0.source.query.match_all - match: { transforms.0.pivot.group_by.airline.terms.field: "airline" } - match: { transforms.0.pivot.aggregations.avg_response.avg.field: "responsetime" } + - match: { transforms.0.description: "yaml test transform on airline-data" } - do: data_frame.get_data_frame_transform: @@ -98,6 +100,7 @@ setup: - match: { count: 2 } - match: { transforms.0.id: "airline-transform" } - match: { transforms.1.id: "airline-transform-dos" } + - is_false: transforms.1.description - do: data_frame.get_data_frame_transform: From 39284f58c16f534c48408784c6bba2c925dd74b1 Mon Sep 17 00:00:00 2001 From: Tomas Della Vedova Date: Sat, 27 Apr 2019 07:09:24 +0200 Subject: [PATCH 215/260] Add missing skip: arbitrary_key (#41492) In `indices.split/30_copy_settings.yml` we are using the `arbitrary_key` feature, but it is not declared inside the skip list. --- .../rest-api-spec/test/indices.split/30_copy_settings.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml index ba876b0ec4e11..1f6a71c6b085c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/30_copy_settings.yml @@ -1,5 +1,7 @@ --- "Copy settings during split index": + - skip: + features: [arbitrary_key] - do: nodes.info: From 64e1f07a43842cf58d8f4aa0e07fa77dea1b416a Mon Sep 17 00:00:00 2001 From: Michael Morello Date: Sat, 27 Apr 2019 12:18:04 +0200 Subject: [PATCH 216/260] Fix multi-node parsing in voting config exclusions REST API (#41588) Fixes an issue where multiple nodes where not properly parsed in the voting config exclusions REST API. Closes #41587 --- .../rest/discovery/Zen2RestApiIT.java | 14 +++++ .../RestAddVotingConfigExclusionAction.java | 15 ++++-- ...stAddVotingConfigExclusionActionTests.java | 54 +++++++++++++++++++ 3 files changed, 78 insertions(+), 5 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java index 7944735dd4dfe..83d4c3419ef64 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java @@ -160,4 +160,18 @@ public void testFailsOnUnknownNode() throws Exception { ); } } + + public void testRemoveTwoNodesAtOnce() throws Exception { + internalCluster().setBootstrapMasterNodeIndex(2); + List nodes = internalCluster().startNodes(3); + ensureStableCluster(3); + RestClient restClient = getRestClient(); + Response response = restClient.performRequest(new Request("POST", "/_cluster/voting_config_exclusions/" + + nodes.get(2) + "," + nodes.get(0))); + assertThat(response.getStatusLine().getStatusCode(), is(200)); + assertThat(response.getEntity().getContentLength(), is(0L)); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodes.get(2))); + ensureStableCluster(1); + } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java index 2a58ebf52d4b0..c0d8c1cf698d6 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.rest.BaseRestHandler; @@ -47,15 +48,19 @@ public String getName() { @Override protected RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - String nodeName = request.param("node_name"); - AddVotingConfigExclusionsRequest votingConfigExclusionsRequest = new AddVotingConfigExclusionsRequest( - new String[]{nodeName}, - TimeValue.parseTimeValue(request.param("timeout"), DEFAULT_TIMEOUT, getClass().getSimpleName() + ".timeout") - ); + AddVotingConfigExclusionsRequest votingConfigExclusionsRequest = resolveVotingConfigExclusionsRequest(request); return channel -> client.execute( AddVotingConfigExclusionsAction.INSTANCE, votingConfigExclusionsRequest, new RestToXContentListener<>(channel) ); } + + AddVotingConfigExclusionsRequest resolveVotingConfigExclusionsRequest(final RestRequest request) { + String nodeName = request.param("node_name"); + return new AddVotingConfigExclusionsRequest( + Strings.splitStringByCommaToArray(nodeName), + TimeValue.parseTimeValue(request.param("timeout"), DEFAULT_TIMEOUT, getClass().getSimpleName() + ".timeout") + ); + } } diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java new file mode 100644 index 0000000000000..2dc6a5b7136fc --- /dev/null +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java @@ -0,0 +1,54 @@ +/* + * 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.rest.action.admin.cluster; + +import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.test.rest.RestActionTestCase; +import org.junit.Before; +import java.util.HashMap; +import java.util.Map; + + +public class RestAddVotingConfigExclusionActionTests extends RestActionTestCase { + + private RestAddVotingConfigExclusionAction action; + + @Before + public void setupAction() { + action = new RestAddVotingConfigExclusionAction(Settings.EMPTY, controller()); + } + + public void testResolveVotingConfigExclusionsRequest() { + Map params = new HashMap<>(); + params.put("node_name", "node-1,node-2,node-3"); + RestRequest deprecatedRequest = new FakeRestRequest.Builder(xContentRegistry()) + .withMethod(RestRequest.Method.PUT) + .withPath("/_cluster/voting_config_exclusions") + .withParams(params) + .build(); + + AddVotingConfigExclusionsRequest addVotingConfigExclusionsRequest = action.resolveVotingConfigExclusionsRequest(deprecatedRequest); + String[] expected = {"node-1","node-2", "node-3"}; + assertArrayEquals(expected, addVotingConfigExclusionsRequest.getNodeDescriptions()); + } +} From b4c6643e71988ebcb8042e34d84f7894245015d9 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 27 Apr 2019 10:54:00 -0400 Subject: [PATCH 217/260] Adjust bwc version (#41099) Relates #41099 --- .../indices/recovery/RecoveryTranslogOperationsRequest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java index a9484af5cc75a..7f0fa23f2431c 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java @@ -106,7 +106,7 @@ long mappingVersionOnPrimary() { maxSeenAutoIdTimestampOnPrimary = in.readZLong(); maxSeqNoOfUpdatesOrDeletesOnPrimary = in.readZLong(); retentionLeases = new RetentionLeases(in); - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_7_1_0)) { mappingVersionOnPrimary = in.readVLong(); } else { mappingVersionOnPrimary = Long.MAX_VALUE; @@ -123,7 +123,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeZLong(maxSeenAutoIdTimestampOnPrimary); out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary); retentionLeases.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_7_1_0)) { out.writeVLong(mappingVersionOnPrimary); } } From 545e56d1d1187669c7da810a356cba9dcb04822c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 28 Apr 2019 19:55:17 -0400 Subject: [PATCH 218/260] Remove some usages of MapBuilder#immutableMap (#41593) This method can be replaced by some convenience collection factory methods available in the JDK now. This commit does not attempt to replace all usages, only enough that the size of the commit is kept to a reasonable size. --- .../elasticsearch/cloud/gce/GCEFixture.java | 31 +-- .../azure/AzureStorageService.java | 9 +- .../azure/AzureStorageSettings.java | 8 +- .../azure/AzureStorageServiceMock.java | 6 +- .../org/elasticsearch/index/IndexService.java | 7 +- .../index/codec/CodecService.java | 6 +- .../index/engine/CommitStats.java | 16 +- .../elasticsearch/rest/RestRequestTests.java | 48 ++-- .../analysis/AnalysisFactoryTestCase.java | 258 +++++++++--------- .../authz/privilege/ClusterPrivilege.java | 55 ++-- .../authz/privilege/IndexPrivilege.java | 37 ++- .../security/transport/filter/IPFilter.java | 10 +- .../authc/esnative/NativeRealmIntegTests.java | 12 +- .../xpack/watcher/actions/ActionBuilders.java | 7 +- .../watcher/execution/ExecutionService.java | 9 +- 15 files changed, 253 insertions(+), 266 deletions(-) diff --git a/plugins/discovery-gce/qa/gce/src/test/java/org/elasticsearch/cloud/gce/GCEFixture.java b/plugins/discovery-gce/qa/gce/src/test/java/org/elasticsearch/cloud/gce/GCEFixture.java index f52e613ee29c0..93fff86a05bec 100644 --- a/plugins/discovery-gce/qa/gce/src/test/java/org/elasticsearch/cloud/gce/GCEFixture.java +++ b/plugins/discovery-gce/qa/gce/src/test/java/org/elasticsearch/cloud/gce/GCEFixture.java @@ -129,26 +129,23 @@ private PathTrie defaultHandlers() { // https://cloud.google.com/compute/docs/reference/rest/v1/instances handlers.insert(authPath(HttpGet.METHOD_NAME, "/compute/v1/projects/{project}/zones/{zone}/instances"), request -> { - final List items = new ArrayList(); + final var items = new ArrayList>(); int count = 0; for (String address : Files.readAllLines(nodes)) { count++; - items.add(MapBuilder.newMapBuilder() - .put("id", Long.toString(9309873766405L + count)) - .put("description", "ES node" + count) - .put("name", "test" + count) - .put("kind", "compute#instance") - .put("machineType", "n1-standard-1") - .put("networkInterfaces", - Collections.singletonList(MapBuilder.newMapBuilder() - .put("accessConfigs", Collections.emptyList()) - .put("name", "nic0") - .put("network", "default") - .put("networkIP", address) - .immutableMap())) - .put("status", "RUNNING") - .put("zone", ZONE) - .immutableMap()); + items.add(Map.of( + "id", Long.toString(9309873766405L + count), + "description", "ES node" + count, + "name", "test" + count, + "kind", "compute#instance", + "machineType", "n1-standard-1", + "networkInterfaces", List.of(Map.of( + "accessConfigs", Collections.emptyList(), + "name", "nic0", + "network", "default", + "networkIP", address)), + "status", "RUNNING", + "zone", ZONE)); } final String json = Strings.toString(jsonBuilder() diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java index 79e8d4c6235ae..0d18592b8a7bb 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureStorageService.java @@ -34,13 +34,11 @@ import com.microsoft.azure.storage.blob.CloudBlockBlob; import com.microsoft.azure.storage.blob.DeleteSnapshotsOption; import com.microsoft.azure.storage.blob.ListBlobItem; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.support.PlainBlobMetaData; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; @@ -55,6 +53,7 @@ import java.nio.file.FileAlreadyExistsException; import java.security.InvalidKeyException; import java.util.EnumSet; +import java.util.HashMap; import java.util.Map; import java.util.function.Supplier; @@ -136,7 +135,7 @@ protected OperationContext buildOperationContext(AzureStorageSettings azureStora */ public Map refreshAndClearCache(Map clientsSettings) { final Map prevSettings = this.storageSettings; - this.storageSettings = MapBuilder.newMapBuilder(clientsSettings).immutableMap(); + this.storageSettings = Map.copyOf(clientsSettings); // clients are built lazily by {@link client(String)} return prevSettings; } @@ -220,7 +219,7 @@ public Map listBlobsByPrefix(String account, String contai // NOTE: this should be here: if (prefix == null) prefix = ""; // however, this is really inefficient since deleteBlobsByPrefix enumerates everything and // then does a prefix match on the result; it should just call listBlobsByPrefix with the prefix! - final MapBuilder blobsBuilder = MapBuilder.newMapBuilder(); + final var blobsBuilder = new HashMap(); final EnumSet enumBlobListingDetails = EnumSet.of(BlobListingDetails.METADATA); final Tuple> client = client(account); final CloudBlobContainer blobContainer = client.v1().getContainerReference(container); @@ -239,7 +238,7 @@ public Map listBlobsByPrefix(String account, String contai blobsBuilder.put(name, new PlainBlobMetaData(name, properties.getLength())); } }); - return blobsBuilder.immutableMap(); + return Map.copyOf(blobsBuilder); } public void writeBlob(String account, String container, String blobName, InputStream inputStream, long blobSize, 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 1c90f97a43728..3f7a5df8f14b2 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 @@ -22,7 +22,6 @@ import com.microsoft.azure.storage.LocationMode; import com.microsoft.azure.storage.RetryPolicy; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.settings.SecureSetting; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Setting; @@ -31,6 +30,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.unit.TimeValue; + import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.Proxy; @@ -234,13 +234,13 @@ public static T getValue(Settings settings, String groupName, Setting set static Map overrideLocationMode(Map clientsSettings, LocationMode locationMode) { - final MapBuilder mapBuilder = new MapBuilder<>(); + final var map = new HashMap(); for (final Map.Entry entry : clientsSettings.entrySet()) { final AzureStorageSettings azureSettings = new AzureStorageSettings(entry.getValue().account, entry.getValue().key, entry.getValue().endpointSuffix, entry.getValue().timeout, entry.getValue().maxRetries, entry.getValue().proxy, locationMode); - mapBuilder.put(entry.getKey(), azureSettings); + map.put(entry.getKey(), azureSettings); } - return mapBuilder.immutableMap(); + return Map.copyOf(map); } } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java index 18eb529c0eebe..5f3072e1ad9ed 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureStorageServiceMock.java @@ -24,7 +24,6 @@ import com.microsoft.azure.storage.blob.CloudBlobClient; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.support.PlainBlobMetaData; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.Streams; @@ -38,6 +37,7 @@ import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; import java.security.AccessController; +import java.util.HashMap; import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -91,7 +91,7 @@ public InputStream getInputStream(String account, String container, String blob) @Override public Map listBlobsByPrefix(String account, String container, String keyPath, String prefix) { - MapBuilder blobsBuilder = MapBuilder.newMapBuilder(); + final var blobsBuilder = new HashMap(); blobs.forEach((String blobName, ByteArrayOutputStream bos) -> { final String checkBlob; if (keyPath != null && !keyPath.isEmpty()) { @@ -104,7 +104,7 @@ public Map listBlobsByPrefix(String account, String contai blobsBuilder.put(blobName, new PlainBlobMetaData(checkBlob, bos.size())); } }); - return blobsBuilder.immutableMap(); + return Map.copyOf(blobsBuilder); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 1a49fd418735d..94c45927486e6 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -91,10 +91,12 @@ import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; -import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; +import static java.util.Map.entry; public class IndexService extends AbstractIndexComponent implements IndicesClusterStateService.AllocatedIndex { @@ -428,7 +430,8 @@ public synchronized IndexShard createShard( circuitBreakerService); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); - shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); + shards = Stream.concat(shards.entrySet().stream(), Stream.of(entry(shardId.id(), indexShard))) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); success = true; return indexShard; } catch (ShardLockObtainFailedException e) { diff --git a/server/src/main/java/org/elasticsearch/index/codec/CodecService.java b/server/src/main/java/org/elasticsearch/index/codec/CodecService.java index 6c3a2138e89bf..c43f733f916cb 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/CodecService.java +++ b/server/src/main/java/org/elasticsearch/index/codec/CodecService.java @@ -25,9 +25,9 @@ import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode; import org.apache.lucene.codecs.lucene80.Lucene80Codec; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.index.mapper.MapperService; +import java.util.HashMap; import java.util.Map; /** @@ -46,7 +46,7 @@ public class CodecService { public static final String LUCENE_DEFAULT_CODEC = "lucene_default"; public CodecService(@Nullable MapperService mapperService, Logger logger) { - final MapBuilder codecs = MapBuilder.newMapBuilder(); + final var codecs = new HashMap(); if (mapperService == null) { codecs.put(DEFAULT_CODEC, new Lucene80Codec()); codecs.put(BEST_COMPRESSION_CODEC, new Lucene80Codec(Mode.BEST_COMPRESSION, FSTLoadMode.AUTO)); @@ -60,7 +60,7 @@ public CodecService(@Nullable MapperService mapperService, Logger logger) { for (String codec : Codec.availableCodecs()) { codecs.put(codec, Codec.forName(codec)); } - this.codecs = codecs.immutableMap(); + this.codecs = Map.copyOf(codecs); } public Codec codec(String name) { diff --git a/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java b/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java index 21025046b8c57..79b468f28a98b 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CommitStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.engine; import org.apache.lucene.index.SegmentInfos; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -28,9 +27,12 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; +import java.util.ArrayList; import java.util.Base64; import java.util.Map; +import static java.util.Map.entry; + /** a class the returns dynamic information with respect to the last commit point of this shard */ public final class CommitStats implements Streamable, ToXContentFragment { @@ -41,7 +43,7 @@ public final class CommitStats implements Streamable, ToXContentFragment { public CommitStats(SegmentInfos segmentInfos) { // clone the map to protect against concurrent changes - userData = MapBuilder.newMapBuilder().putAll(segmentInfos.getUserData()).immutableMap(); + userData = Map.copyOf(segmentInfos.getUserData()); // lucene calls the current generation, last generation. generation = segmentInfos.getLastGeneration(); id = Base64.getEncoder().encodeToString(segmentInfos.getId()); @@ -92,11 +94,13 @@ public int getNumDocs() { @Override public void readFrom(StreamInput in) throws IOException { - MapBuilder builder = MapBuilder.newMapBuilder(); - for (int i = in.readVInt(); i > 0; i--) { - builder.put(in.readString(), in.readString()); + final int length = in.readVInt(); + final var entries = new ArrayList>(length); + for (int i = length; i > 0; i--) { + entries.add(entry(in.readString(), in.readString())); } - userData = builder.immutableMap(); + // noinspection unchecked + userData = Map.ofEntries(entries.toArray((Map.Entry[])new Map.Entry[0])); generation = in.readLong(); id = in.readOptionalString(); numDocs = in.readInt(); diff --git a/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java b/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java index 8a2994a69816b..4dbe3a1f6f17b 100644 --- a/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java +++ b/server/src/test/java/org/elasticsearch/rest/RestRequestTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; @@ -126,16 +125,17 @@ public void testContentOrSourceParam() throws IOException { contentRestRequest("", emptyMap()).contentOrSourceParam()); assertEquals("request body or source parameter is required", e.getMessage()); assertEquals(new BytesArray("stuff"), contentRestRequest("stuff", emptyMap()).contentOrSourceParam().v2()); - assertEquals(new BytesArray("stuff"), - contentRestRequest("stuff", MapBuilder.newMapBuilder() - .put("source", "stuff2").put("source_content_type", "application/json").immutableMap()).contentOrSourceParam().v2()); + assertEquals( + new BytesArray("stuff"), + contentRestRequest( + "stuff", + Map.of("source", "stuff2", "source_content_type", "application/json")) + .contentOrSourceParam().v2()); assertEquals(new BytesArray("{\"foo\": \"stuff\"}"), - contentRestRequest("", MapBuilder.newMapBuilder() - .put("source", "{\"foo\": \"stuff\"}").put("source_content_type", "application/json").immutableMap()) + contentRestRequest("", Map.of("source", "{\"foo\": \"stuff\"}", "source_content_type", "application/json")) .contentOrSourceParam().v2()); e = expectThrows(IllegalStateException.class, () -> - contentRestRequest("", MapBuilder.newMapBuilder() - .put("source", "stuff2").immutableMap()).contentOrSourceParam()); + contentRestRequest("", Map.of("source", "stuff2")).contentOrSourceParam()); assertEquals("source and source_content_type parameters are required", e.getMessage()); } @@ -152,8 +152,12 @@ public void testContentOrSourceParamParser() throws IOException { assertEquals("request body or source parameter is required", e.getMessage()); assertEquals(emptyMap(), contentRestRequest("{}", emptyMap()).contentOrSourceParamParser().map()); assertEquals(emptyMap(), contentRestRequest("{}", singletonMap("source", "stuff2")).contentOrSourceParamParser().map()); - assertEquals(emptyMap(), contentRestRequest("", MapBuilder.newMapBuilder() - .put("source", "{}").put("source_content_type", "application/json").immutableMap()).contentOrSourceParamParser().map()); + assertEquals( + emptyMap(), + contentRestRequest( + "", + Map.of("source", "{}", "source_content_type", "application/json")) + .contentOrSourceParamParser().map()); } public void testWithContentOrSourceParamParserOrNull() throws IOException { @@ -161,10 +165,10 @@ public void testWithContentOrSourceParamParserOrNull() throws IOException { contentRestRequest("{}", emptyMap()).withContentOrSourceParamParserOrNull(parser -> assertEquals(emptyMap(), parser.map())); contentRestRequest("{}", singletonMap("source", "stuff2")).withContentOrSourceParamParserOrNull(parser -> assertEquals(emptyMap(), parser.map())); - contentRestRequest("", MapBuilder.newMapBuilder().put("source_content_type", "application/json") - .put("source", "{}").immutableMap()) - .withContentOrSourceParamParserOrNull(parser -> - assertEquals(emptyMap(), parser.map())); + contentRestRequest( + "", + Map.of("source_content_type", "application/json", "source", "{}")) + .withContentOrSourceParamParserOrNull(parser -> assertEquals(emptyMap(), parser.map())); } public void testContentTypeParsing() { @@ -221,13 +225,15 @@ public void testRequiredContent() { contentRestRequest("", emptyMap()).requiredContent()); assertEquals("request body is required", e.getMessage()); assertEquals(new BytesArray("stuff"), contentRestRequest("stuff", emptyMap()).requiredContent()); - assertEquals(new BytesArray("stuff"), - contentRestRequest("stuff", MapBuilder.newMapBuilder() - .put("source", "stuff2").put("source_content_type", "application/json").immutableMap()).requiredContent()); - e = expectThrows(ElasticsearchParseException.class, () -> - contentRestRequest("", MapBuilder.newMapBuilder() - .put("source", "{\"foo\": \"stuff\"}").put("source_content_type", "application/json").immutableMap()) - .requiredContent()); + assertEquals( + new BytesArray("stuff"), + contentRestRequest("stuff", Map.of("source", "stuff2", "source_content_type", "application/json")).requiredContent()); + e = expectThrows( + ElasticsearchParseException.class, + () -> contentRestRequest( + "", + Map.of("source", "{\"foo\": \"stuff\"}", "source_content_type", "application/json")) + .requiredContent()); assertEquals("request body is required", e.getMessage()); e = expectThrows(IllegalStateException.class, () -> contentRestRequest("test", null, Collections.emptyMap()).requiredContent()); diff --git a/test/framework/src/main/java/org/elasticsearch/indices/analysis/AnalysisFactoryTestCase.java b/test/framework/src/main/java/org/elasticsearch/indices/analysis/AnalysisFactoryTestCase.java index 0238526cab4dd..86b6014b87f87 100644 --- a/test/framework/src/main/java/org/elasticsearch/indices/analysis/AnalysisFactoryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/analysis/AnalysisFactoryTestCase.java @@ -21,7 +21,6 @@ import org.apache.lucene.analysis.util.TokenFilterFactory; import org.apache.lucene.analysis.util.TokenizerFactory; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.index.analysis.HunspellTokenFilterFactory; import org.elasticsearch.index.analysis.ShingleTokenFilterFactory; import org.elasticsearch.index.analysis.StandardTokenizerFactory; @@ -39,6 +38,7 @@ import java.util.regex.Pattern; import static java.util.Collections.emptyMap; +import static java.util.Map.entry; /** * Alerts us if new analysis components are added to Lucene, so we don't miss them. @@ -61,164 +61,156 @@ private static String toCamelCase(String s) { return sb.toString(); } - static final Map> KNOWN_TOKENIZERS = new MapBuilder>() + static final Map> KNOWN_TOKENIZERS = Map.ofEntries( // exposed in ES - .put("classic", MovedToAnalysisCommon.class) - .put("edgengram", MovedToAnalysisCommon.class) - .put("keyword", MovedToAnalysisCommon.class) - .put("letter", MovedToAnalysisCommon.class) - .put("ngram", MovedToAnalysisCommon.class) - .put("pathhierarchy", MovedToAnalysisCommon.class) - .put("pattern", MovedToAnalysisCommon.class) - .put("simplepattern", MovedToAnalysisCommon.class) - .put("simplepatternsplit", MovedToAnalysisCommon.class) - .put("standard", StandardTokenizerFactory.class) - .put("thai", MovedToAnalysisCommon.class) - .put("uax29urlemail", MovedToAnalysisCommon.class) - .put("whitespace", MovedToAnalysisCommon.class) - + entry("classic", MovedToAnalysisCommon.class), + entry("edgengram", MovedToAnalysisCommon.class), + entry("keyword", MovedToAnalysisCommon.class), + entry("letter", MovedToAnalysisCommon.class), + entry("ngram", MovedToAnalysisCommon.class), + entry("pathhierarchy", MovedToAnalysisCommon.class), + entry("pattern", MovedToAnalysisCommon.class), + entry("simplepattern", MovedToAnalysisCommon.class), + entry("simplepatternsplit", MovedToAnalysisCommon.class), + entry("standard", StandardTokenizerFactory.class), + entry("thai", MovedToAnalysisCommon.class), + entry("uax29urlemail", MovedToAnalysisCommon.class), + entry("whitespace", MovedToAnalysisCommon.class), // this one "seems to mess up offsets". probably shouldn't be a tokenizer... - .put("wikipedia", Void.class) - .immutableMap(); + entry("wikipedia", Void.class)); - static final Map> KNOWN_TOKENFILTERS = new MapBuilder>() + static final Map> KNOWN_TOKENFILTERS = Map.ofEntries( // exposed in ES - .put("apostrophe", MovedToAnalysisCommon.class) - .put("arabicnormalization", MovedToAnalysisCommon.class) - .put("arabicstem", MovedToAnalysisCommon.class) - .put("asciifolding", MovedToAnalysisCommon.class) - .put("bengalinormalization", MovedToAnalysisCommon.class) - .put("bengalistem", MovedToAnalysisCommon.class) - .put("brazilianstem", MovedToAnalysisCommon.class) - .put("bulgarianstem", MovedToAnalysisCommon.class) - .put("cjkbigram", MovedToAnalysisCommon.class) - .put("cjkwidth", MovedToAnalysisCommon.class) - .put("classic", MovedToAnalysisCommon.class) - .put("commongrams", MovedToAnalysisCommon.class) - .put("commongramsquery", MovedToAnalysisCommon.class) - .put("czechstem", MovedToAnalysisCommon.class) - .put("decimaldigit", MovedToAnalysisCommon.class) - .put("delimitedpayload", MovedToAnalysisCommon.class) - .put("dictionarycompoundword", MovedToAnalysisCommon.class) - .put("edgengram", MovedToAnalysisCommon.class) - .put("elision", MovedToAnalysisCommon.class) - .put("englishminimalstem", MovedToAnalysisCommon.class) - .put("englishpossessive", MovedToAnalysisCommon.class) - .put("finnishlightstem", MovedToAnalysisCommon.class) - .put("fixedshingle", MovedToAnalysisCommon.class) - .put("frenchlightstem", MovedToAnalysisCommon.class) - .put("frenchminimalstem", MovedToAnalysisCommon.class) - .put("galicianminimalstem", MovedToAnalysisCommon.class) - .put("galicianstem", MovedToAnalysisCommon.class) - .put("germanstem", MovedToAnalysisCommon.class) - .put("germanlightstem", MovedToAnalysisCommon.class) - .put("germanminimalstem", MovedToAnalysisCommon.class) - .put("germannormalization", MovedToAnalysisCommon.class) - .put("greeklowercase", MovedToAnalysisCommon.class) - .put("greekstem", MovedToAnalysisCommon.class) - .put("hindinormalization", MovedToAnalysisCommon.class) - .put("hindistem", MovedToAnalysisCommon.class) - .put("hungarianlightstem", MovedToAnalysisCommon.class) - .put("hunspellstem", HunspellTokenFilterFactory.class) - .put("hyphenationcompoundword", MovedToAnalysisCommon.class) - .put("indicnormalization", MovedToAnalysisCommon.class) - .put("irishlowercase", MovedToAnalysisCommon.class) - .put("indonesianstem", MovedToAnalysisCommon.class) - .put("italianlightstem", MovedToAnalysisCommon.class) - .put("keepword", MovedToAnalysisCommon.class) - .put("keywordmarker", MovedToAnalysisCommon.class) - .put("kstem", MovedToAnalysisCommon.class) - .put("latvianstem", MovedToAnalysisCommon.class) - .put("length", MovedToAnalysisCommon.class) - .put("limittokencount", MovedToAnalysisCommon.class) - .put("lowercase", MovedToAnalysisCommon.class) - .put("ngram", MovedToAnalysisCommon.class) - .put("norwegianlightstem", MovedToAnalysisCommon.class) - .put("norwegianminimalstem", MovedToAnalysisCommon.class) - .put("patterncapturegroup", MovedToAnalysisCommon.class) - .put("patternreplace", MovedToAnalysisCommon.class) - .put("persiannormalization", MovedToAnalysisCommon.class) - .put("porterstem", MovedToAnalysisCommon.class) - .put("portuguesestem", MovedToAnalysisCommon.class) - .put("portugueselightstem", MovedToAnalysisCommon.class) - .put("portugueseminimalstem", MovedToAnalysisCommon.class) - .put("reversestring", MovedToAnalysisCommon.class) - .put("russianlightstem", MovedToAnalysisCommon.class) - .put("scandinavianfolding", MovedToAnalysisCommon.class) - .put("scandinaviannormalization", MovedToAnalysisCommon.class) - .put("serbiannormalization", MovedToAnalysisCommon.class) - .put("shingle", ShingleTokenFilterFactory.class) - .put("minhash", MovedToAnalysisCommon.class) - .put("snowballporter", MovedToAnalysisCommon.class) - .put("soraninormalization", MovedToAnalysisCommon.class) - .put("soranistem", MovedToAnalysisCommon.class) - .put("spanishlightstem", MovedToAnalysisCommon.class) - .put("stemmeroverride", MovedToAnalysisCommon.class) - .put("stop", StopTokenFilterFactory.class) - .put("swedishlightstem", MovedToAnalysisCommon.class) - .put("synonym", MovedToAnalysisCommon.class) - .put("synonymgraph", MovedToAnalysisCommon.class) - .put("trim", MovedToAnalysisCommon.class) - .put("truncate", MovedToAnalysisCommon.class) - .put("turkishlowercase", MovedToAnalysisCommon.class) - .put("type", MovedToAnalysisCommon.class) - .put("uppercase", MovedToAnalysisCommon.class) - .put("worddelimiter", MovedToAnalysisCommon.class) - .put("worddelimitergraph", MovedToAnalysisCommon.class) - .put("flattengraph", MovedToAnalysisCommon.class) - + entry("apostrophe", MovedToAnalysisCommon.class), + entry("arabicnormalization", MovedToAnalysisCommon.class), + entry("arabicstem", MovedToAnalysisCommon.class), + entry("asciifolding", MovedToAnalysisCommon.class), + entry("bengalinormalization", MovedToAnalysisCommon.class), + entry("bengalistem", MovedToAnalysisCommon.class), + entry("brazilianstem", MovedToAnalysisCommon.class), + entry("bulgarianstem", MovedToAnalysisCommon.class), + entry("cjkbigram", MovedToAnalysisCommon.class), + entry("cjkwidth", MovedToAnalysisCommon.class), + entry("classic", MovedToAnalysisCommon.class), + entry("commongrams", MovedToAnalysisCommon.class), + entry("commongramsquery", MovedToAnalysisCommon.class), + entry("czechstem", MovedToAnalysisCommon.class), + entry("decimaldigit", MovedToAnalysisCommon.class), + entry("delimitedpayload", MovedToAnalysisCommon.class), + entry("dictionarycompoundword", MovedToAnalysisCommon.class), + entry("edgengram", MovedToAnalysisCommon.class), + entry("elision", MovedToAnalysisCommon.class), + entry("englishminimalstem", MovedToAnalysisCommon.class), + entry("englishpossessive", MovedToAnalysisCommon.class), + entry("finnishlightstem", MovedToAnalysisCommon.class), + entry("fixedshingle", MovedToAnalysisCommon.class), + entry("frenchlightstem", MovedToAnalysisCommon.class), + entry("frenchminimalstem", MovedToAnalysisCommon.class), + entry("galicianminimalstem", MovedToAnalysisCommon.class), + entry("galicianstem", MovedToAnalysisCommon.class), + entry("germanstem", MovedToAnalysisCommon.class), + entry("germanlightstem", MovedToAnalysisCommon.class), + entry("germanminimalstem", MovedToAnalysisCommon.class), + entry("germannormalization", MovedToAnalysisCommon.class), + entry("greeklowercase", MovedToAnalysisCommon.class), + entry("greekstem", MovedToAnalysisCommon.class), + entry("hindinormalization", MovedToAnalysisCommon.class), + entry("hindistem", MovedToAnalysisCommon.class), + entry("hungarianlightstem", MovedToAnalysisCommon.class), + entry("hunspellstem", HunspellTokenFilterFactory.class), + entry("hyphenationcompoundword", MovedToAnalysisCommon.class), + entry("indicnormalization", MovedToAnalysisCommon.class), + entry("irishlowercase", MovedToAnalysisCommon.class), + entry("indonesianstem", MovedToAnalysisCommon.class), + entry("italianlightstem", MovedToAnalysisCommon.class), + entry("keepword", MovedToAnalysisCommon.class), + entry("keywordmarker", MovedToAnalysisCommon.class), + entry("kstem", MovedToAnalysisCommon.class), + entry("latvianstem", MovedToAnalysisCommon.class), + entry("length", MovedToAnalysisCommon.class), + entry("limittokencount", MovedToAnalysisCommon.class), + entry("lowercase", MovedToAnalysisCommon.class), + entry("ngram", MovedToAnalysisCommon.class), + entry("norwegianlightstem", MovedToAnalysisCommon.class), + entry("norwegianminimalstem", MovedToAnalysisCommon.class), + entry("patterncapturegroup", MovedToAnalysisCommon.class), + entry("patternreplace", MovedToAnalysisCommon.class), + entry("persiannormalization", MovedToAnalysisCommon.class), + entry("porterstem", MovedToAnalysisCommon.class), + entry("portuguesestem", MovedToAnalysisCommon.class), + entry("portugueselightstem", MovedToAnalysisCommon.class), + entry("portugueseminimalstem", MovedToAnalysisCommon.class), + entry("reversestring", MovedToAnalysisCommon.class), + entry("russianlightstem", MovedToAnalysisCommon.class), + entry("scandinavianfolding", MovedToAnalysisCommon.class), + entry("scandinaviannormalization", MovedToAnalysisCommon.class), + entry("serbiannormalization", MovedToAnalysisCommon.class), + entry("shingle", ShingleTokenFilterFactory.class), + entry("minhash", MovedToAnalysisCommon.class), + entry("snowballporter", MovedToAnalysisCommon.class), + entry("soraninormalization", MovedToAnalysisCommon.class), + entry("soranistem", MovedToAnalysisCommon.class), + entry("spanishlightstem", MovedToAnalysisCommon.class), + entry("stemmeroverride", MovedToAnalysisCommon.class), + entry("stop", StopTokenFilterFactory.class), + entry("swedishlightstem", MovedToAnalysisCommon.class), + entry("synonym", MovedToAnalysisCommon.class), + entry("synonymgraph", MovedToAnalysisCommon.class), + entry("trim", MovedToAnalysisCommon.class), + entry("truncate", MovedToAnalysisCommon.class), + entry("turkishlowercase", MovedToAnalysisCommon.class), + entry("type", MovedToAnalysisCommon.class), + entry("uppercase", MovedToAnalysisCommon.class), + entry("worddelimiter", MovedToAnalysisCommon.class), + entry("worddelimitergraph", MovedToAnalysisCommon.class), + entry("flattengraph", MovedToAnalysisCommon.class), // TODO: these tokenfilters are not yet exposed: useful? // suggest stop - .put("suggeststop", Void.class) + entry("suggeststop", Void.class), // capitalizes tokens - .put("capitalization", Void.class) + entry("capitalization", Void.class), // like length filter (but codepoints) - .put("codepointcount", Void.class) + entry("codepointcount", Void.class), // puts hyphenated words back together - .put("hyphenatedwords", Void.class) + entry("hyphenatedwords", Void.class), // repeats anything marked as keyword - .put("keywordrepeat", Void.class) + entry("keywordrepeat", Void.class), // like limittokencount, but by offset - .put("limittokenoffset", Void.class) + entry("limittokenoffset", Void.class), // like limittokencount, but by position - .put("limittokenposition", Void.class) + entry("limittokenposition", Void.class), // ??? - .put("numericpayload", Void.class) + entry("numericpayload", Void.class), // removes duplicates at the same position (this should be used by the existing factory) - .put("removeduplicates", Void.class) + entry("removeduplicates", Void.class), // ??? - .put("tokenoffsetpayload", Void.class) + entry("tokenoffsetpayload", Void.class), // puts the type into the payload - .put("typeaspayload", Void.class) + entry("typeaspayload", Void.class), // puts the type as a synonym - .put("typeassynonym", Void.class) + entry("typeassynonym", Void.class), // fingerprint - .put("fingerprint", Void.class) + entry("fingerprint", Void.class), // for tee-sinks - .put("daterecognizer", Void.class) + entry("daterecognizer", Void.class), // for token filters that generate bad offsets, which are now rejected since Lucene 7 - .put("fixbrokenoffsets", Void.class) + entry("fixbrokenoffsets", Void.class), // should we expose it, or maybe think about higher level integration of the // fake term frequency feature (LUCENE-7854) - .put("delimitedtermfrequency", Void.class) + entry("delimitedtermfrequency", Void.class), // LUCENE-8273: ProtectedTermFilterFactory allows analysis chains to skip // particular token filters based on the attributes of the current token. - .put("protectedterm", Void.class) + entry("protectedterm", Void.class), // LUCENE-8332 - .put("concatenategraph", Void.class) - - .immutableMap(); - - static final Map> KNOWN_CHARFILTERS = new MapBuilder>() - // exposed in ES - .put("htmlstrip", MovedToAnalysisCommon.class) - .put("mapping", MovedToAnalysisCommon.class) - .put("patternreplace", MovedToAnalysisCommon.class) + entry("concatenategraph", Void.class)); - // TODO: these charfilters are not yet exposed: useful? - // handling of zwnj for persian - .put("persian", Void.class) - .immutableMap(); + static final Map> KNOWN_CHARFILTERS = Map.of( + "htmlstrip", MovedToAnalysisCommon.class, + "mapping", MovedToAnalysisCommon.class, + "patternreplace", MovedToAnalysisCommon.class, + // TODO: these charfilters are not yet exposed: useful? + // handling of zwnj for persian + "persian", Void.class); /** * The plugin being tested. Core uses an "empty" plugin so we don't have to throw null checks all over the place. diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ClusterPrivilege.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ClusterPrivilege.java index c929fb3bfd348..f7d03c2356e5b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ClusterPrivilege.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/ClusterPrivilege.java @@ -12,7 +12,6 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.xpack.core.indexlifecycle.action.GetLifecycleAction; import org.elasticsearch.xpack.core.indexlifecycle.action.GetStatusAction; import org.elasticsearch.xpack.core.security.action.token.InvalidateTokenAction; @@ -28,6 +27,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.Predicate; +import static java.util.Map.entry; import static org.elasticsearch.xpack.core.security.support.Automatons.minusAndMinimize; import static org.elasticsearch.xpack.core.security.support.Automatons.patterns; @@ -93,33 +93,32 @@ public final class ClusterPrivilege extends Privilege { public static final Predicate ACTION_MATCHER = ClusterPrivilege.ALL.predicate(); - private static final Map VALUES = MapBuilder.newMapBuilder() - .put("none", NONE) - .put("all", ALL) - .put("monitor", MONITOR) - .put("monitor_ml", MONITOR_ML) - .put("monitor_data_frame_transforms", MONITOR_DATA_FRAME) - .put("monitor_watcher", MONITOR_WATCHER) - .put("monitor_rollup", MONITOR_ROLLUP) - .put("manage", MANAGE) - .put("manage_ml", MANAGE_ML) - .put("manage_data_frame_transforms", MANAGE_DATA_FRAME) - .put("manage_token", MANAGE_TOKEN) - .put("manage_watcher", MANAGE_WATCHER) - .put("manage_index_templates", MANAGE_IDX_TEMPLATES) - .put("manage_ingest_pipelines", MANAGE_INGEST_PIPELINES) - .put("transport_client", TRANSPORT_CLIENT) - .put("manage_security", MANAGE_SECURITY) - .put("manage_saml", MANAGE_SAML) - .put("manage_oidc", MANAGE_OIDC) - .put("manage_pipeline", MANAGE_PIPELINE) - .put("manage_rollup", MANAGE_ROLLUP) - .put("manage_ccr", MANAGE_CCR) - .put("read_ccr", READ_CCR) - .put("create_snapshot", CREATE_SNAPSHOT) - .put("manage_ilm", MANAGE_ILM) - .put("read_ilm", READ_ILM) - .immutableMap(); + private static final Map VALUES = Map.ofEntries( + entry("none", NONE), + entry("all", ALL), + entry("monitor", MONITOR), + entry("monitor_ml", MONITOR_ML), + entry("monitor_data_frame_transforms", MONITOR_DATA_FRAME), + entry("monitor_watcher", MONITOR_WATCHER), + entry("monitor_rollup", MONITOR_ROLLUP), + entry("manage", MANAGE), + entry("manage_ml", MANAGE_ML), + entry("manage_data_frame_transforms", MANAGE_DATA_FRAME), + entry("manage_token", MANAGE_TOKEN), + entry("manage_watcher", MANAGE_WATCHER), + entry("manage_index_templates", MANAGE_IDX_TEMPLATES), + entry("manage_ingest_pipelines", MANAGE_INGEST_PIPELINES), + entry("transport_client", TRANSPORT_CLIENT), + entry("manage_security", MANAGE_SECURITY), + entry("manage_saml", MANAGE_SAML), + entry("manage_oidc", MANAGE_OIDC), + entry("manage_pipeline", MANAGE_PIPELINE), + entry("manage_rollup", MANAGE_ROLLUP), + entry("manage_ccr", MANAGE_CCR), + entry("read_ccr", READ_CCR), + entry("create_snapshot", CREATE_SNAPSHOT), + entry("manage_ilm", MANAGE_ILM), + entry("read_ilm", READ_ILM)); private static final ConcurrentHashMap, ClusterPrivilege> CACHE = new ConcurrentHashMap<>(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java index e20e76ee47b37..c4092d8947f78 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.admin.indices.settings.get.GetSettingsAction; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.xpack.core.ccr.action.ForgetFollowerAction; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; import org.elasticsearch.xpack.core.ccr.action.UnfollowAction; @@ -37,6 +36,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.Predicate; +import static java.util.Map.entry; import static org.elasticsearch.xpack.core.security.support.Automatons.patterns; import static org.elasticsearch.xpack.core.security.support.Automatons.unionAndMinimize; @@ -83,24 +83,23 @@ public final class IndexPrivilege extends Privilege { public static final IndexPrivilege MANAGE_LEADER_INDEX = new IndexPrivilege("manage_leader_index", MANAGE_LEADER_INDEX_AUTOMATON); public static final IndexPrivilege MANAGE_ILM = new IndexPrivilege("manage_ilm", MANAGE_ILM_AUTOMATON); - private static final Map VALUES = MapBuilder.newMapBuilder() - .put("none", NONE) - .put("all", ALL) - .put("manage", MANAGE) - .put("create_index", CREATE_INDEX) - .put("monitor", MONITOR) - .put("read", READ) - .put("index", INDEX) - .put("delete", DELETE) - .put("write", WRITE) - .put("create", CREATE) - .put("delete_index", DELETE_INDEX) - .put("view_index_metadata", VIEW_METADATA) - .put("read_cross_cluster", READ_CROSS_CLUSTER) - .put("manage_follow_index", MANAGE_FOLLOW_INDEX) - .put("manage_leader_index", MANAGE_LEADER_INDEX) - .put("manage_ilm", MANAGE_ILM) - .immutableMap(); + private static final Map VALUES = Map.ofEntries( + entry("none", NONE), + entry("all", ALL), + entry("manage", MANAGE), + entry("create_index", CREATE_INDEX), + entry("monitor", MONITOR), + entry("read", READ), + entry("index", INDEX), + entry("delete", DELETE), + entry("write", WRITE), + entry("create", CREATE), + entry("delete_index", DELETE_INDEX), + entry("view_index_metadata", VIEW_METADATA), + entry("read_cross_cluster", READ_CROSS_CLUSTER), + entry("manage_follow_index", MANAGE_FOLLOW_INDEX), + entry("manage_leader_index", MANAGE_LEADER_INDEX), + entry("manage_ilm", MANAGE_ILM)); public static final Predicate ACTION_MATCHER = ALL.predicate(); public static final Predicate CREATE_INDEX_MATCHER = CREATE_INDEX.predicate(); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/filter/IPFilter.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/filter/IPFilter.java index c9f60dce95cef..1dfedd3f87d42 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/filter/IPFilter.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/filter/IPFilter.java @@ -7,10 +7,9 @@ import io.netty.handler.ipfilter.IpFilterRuleType; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -78,10 +77,9 @@ public class IPFilter { public static final Setting> HTTP_FILTER_DENY_SETTING = Setting.listSetting(setting("http.filter.deny"), HTTP_FILTER_DENY_FALLBACK, Function.identity(), Property.Dynamic, Property.NodeScope); - public static final Map DISABLED_USAGE_STATS = new MapBuilder() - .put("http", false) - .put("transport", false) - .immutableMap(); + public static final Map DISABLED_USAGE_STATS = Map.of( + "http", false, + "transport", false); public static final SecurityIpFilterRule DEFAULT_PROFILE_ACCEPT_ALL = new SecurityIpFilterRule(true, "default:accept_all") { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java index d547fe5a83960..2d5aaa73f2c58 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java @@ -16,7 +16,6 @@ import org.elasticsearch.client.Client; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -67,11 +66,11 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_SECURITY_INDEX; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; import static org.hamcrest.Matchers.arrayContaining; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -846,10 +845,9 @@ public void testConcurrentRunAs() throws Exception { .TEST_ROLE).get(); securityClient().preparePutUser("executor", "s3krit".toCharArray(), hasher, "superuser").get(); final String token = basicAuthHeaderValue("executor", new SecureString("s3krit".toCharArray())); - final Client client = client().filterWithHeader(MapBuilder.newMapBuilder() - .put("Authorization", token) - .put("es-security-runas-user", "joe") - .immutableMap()); + final Client client = client().filterWithHeader(Map.of( + "Authorization", token, + "es-security-runas-user", "joe")); final CountDownLatch latch = new CountDownLatch(1); final int numberOfProcessors = Runtime.getRuntime().availableProcessors(); final int numberOfThreads = scaledRandomIntBetween(numberOfProcessors, numberOfProcessors * 3); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/ActionBuilders.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/ActionBuilders.java index 4d54b32e53d45..ae2795bf2122c 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/ActionBuilders.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/actions/ActionBuilders.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.watcher.actions; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.xpack.watcher.actions.email.EmailAction; import org.elasticsearch.xpack.watcher.actions.index.IndexAction; import org.elasticsearch.xpack.watcher.actions.jira.JiraAction; @@ -46,12 +45,8 @@ public static IndexAction.Builder indexAction(String index) { return IndexAction.builder(index); } - public static JiraAction.Builder jiraAction(String account, MapBuilder fields) { - return jiraAction(account, fields.immutableMap()); - } - public static JiraAction.Builder jiraAction(String account, Map fields) { - return JiraAction.builder(account, fields); + return JiraAction.builder(account, Map.copyOf(fields)); } public static WebhookAction.Builder webhookAction(HttpRequestTemplate.Builder httpRequest) { diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java index 4e33e15044c3e..1fca64c776773 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.watcher.execution; import com.google.common.collect.Iterables; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; @@ -23,7 +22,6 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.routing.Preference; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.metrics.MeanMetric; import org.elasticsearch.common.settings.Setting; @@ -342,10 +340,9 @@ public void updateWatchStatus(Watch watch) throws IOException { // at the moment we store the status together with the watch, // so we just need to update the watch itself // we do not want to update the status.state field, as it might have been deactivated in-between - Map parameters = MapBuilder.newMapBuilder() - .put(Watch.INCLUDE_STATUS_KEY, "true") - .put(WatchStatus.INCLUDE_STATE, "false") - .immutableMap(); + Map parameters = Map.of( + Watch.INCLUDE_STATUS_KEY, "true", + WatchStatus.INCLUDE_STATE, "false"); ToXContent.MapParams params = new ToXContent.MapParams(parameters); XContentBuilder source = JsonXContent.contentBuilder(). startObject() From 5fa17aaf8ad1b303bf81611c44e5b51c97dab18e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 28 Apr 2019 20:14:17 -0400 Subject: [PATCH 219/260] Change JDK distribution source (#41626) We had been obtaining JDK distributions from download.java.net. This site is now presenting a certificate that does not list download.java.net as a SAN. Therefore with host verification, the build can not use this site. This commit switches to using download.oracle.com which appears to be an alternative name for the same CNAME download.oracle.com.edgekey.net. This allows our builds to resume. --- Vagrantfile | 4 ++-- distribution/build.gradle | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Vagrantfile b/Vagrantfile index 96745ddfc5f05..9f4571d8d6db2 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -244,7 +244,7 @@ def linux_common(config, SHELL config.vm.provision 'jdk-11', type: 'shell', inline: <<-SHELL - curl -sSL https://download.java.net/java/GA/jdk11/9/GPL/openjdk-11.0.2_linux-x64_bin.tar.gz | tar xz -C /opt/ + curl -sSL https://download.oracle.com/java/GA/jdk11/9/GPL/openjdk-11.0.2_linux-x64_bin.tar.gz | tar xz -C /opt/ SHELL # This prevents leftovers from previous tests using the @@ -405,7 +405,7 @@ def windows_common(config, name) config.vm.provision 'windows-jdk-11', type: 'shell', inline: <<-SHELL New-Item -ItemType Directory -Force -Path "C:/java" - Invoke-WebRequest "https://download.java.net/java/GA/jdk11/9/GPL/openjdk-11.0.2_windows-x64_bin.zip" -OutFile "C:/java/jdk-11.zip" + Invoke-WebRequest "https://download.oracle.com/java/GA/jdk11/9/GPL/openjdk-11.0.2_windows-x64_bin.zip" -OutFile "C:/java/jdk-11.zip" Expand-Archive -Path "C:/java/jdk-11.zip" -DestinationPath "C:/java/" SHELL diff --git a/distribution/build.gradle b/distribution/build.gradle index bbf11923011c9..f5b87e0beb03b 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -235,7 +235,7 @@ String jdkBuild = jdkVersionMatcher.group(3) repositories { ivy { - url "https://download.java.net" + url "https://download.oracle.com" patternLayout { artifact "java/GA/jdk${jdkMajor}/${jdkBuild}/GPL/openjdk-[revision]_[module]-x64_bin.[ext]" } From e6f4611e62d0cb48aef6f3543d836f999327d5ed Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 28 Apr 2019 21:04:08 -0400 Subject: [PATCH 220/260] Bump the bundled JDK to 12.0.1 (#41627) This commit bumps the bundled JDK to version 12.0.1. Note that we had to add a new pattern here as Oracle has changed the source of the builds. This commit will be backported to 6.7 in a different form to bump the bundled JDK in the Docker images too. --- buildSrc/version.properties | 2 +- distribution/build.gradle | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index d08086603bb59..8b5e54b581c79 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,7 +1,7 @@ elasticsearch = 8.0.0 lucene = 8.1.0-snapshot-e460356abe -bundled_jdk = 12+33 +bundled_jdk = 12.0.1+12@69cfe15208a647278a19ef0990eea691 # optional dependencies spatial4j = 0.7 diff --git a/distribution/build.gradle b/distribution/build.gradle index f5b87e0beb03b..d277badf4516e 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -224,7 +224,7 @@ xpack.subprojects.findAll { it.parent == xpack }.each { Project xpackModule -> *****************************************************************************/ // extract the bundled jdk version, broken into elements as: [feature, interim, update, build] // Note the "patch" version is not yet handled here, as it has not yet been used by java. -Pattern JDK_VERSION = Pattern.compile("(\\d+)(\\.\\d+\\.\\d+)?\\+(\\d+)") +Pattern JDK_VERSION = Pattern.compile("(\\d+)(\\.\\d+\\.\\d+)?\\+(\\d+)@([a-f0-9]{32})?") Matcher jdkVersionMatcher = JDK_VERSION.matcher(VersionProperties.bundledJdk) if (jdkVersionMatcher.matches() == false) { throw new IllegalArgumentException("Malformed jdk version [" + VersionProperties.bundledJdk + "]") @@ -232,14 +232,23 @@ if (jdkVersionMatcher.matches() == false) { String jdkVersion = jdkVersionMatcher.group(1) + (jdkVersionMatcher.group(2) != null ? (jdkVersionMatcher.group(2)) : "") String jdkMajor = jdkVersionMatcher.group(1) String jdkBuild = jdkVersionMatcher.group(3) +String hash = jdkVersionMatcher.group(4) repositories { + // simpler legacy pattern from JDK 9 to JDK 12 that we are advocating to Oracle to bring back ivy { url "https://download.oracle.com" patternLayout { artifact "java/GA/jdk${jdkMajor}/${jdkBuild}/GPL/openjdk-[revision]_[module]-x64_bin.[ext]" } } + // current pattern since 12.0.1 + ivy { + url "https://download.oracle.com" + patternLayout { + artifact "java/GA/jdk${jdkVersion}/${hash}/${jdkBuild}/GPL/openjdk-[revision]_[module]-x64_bin.[ext]" + } + } } for (String platform : ['linux', 'darwin', 'windows']) { String jdkConfigName = "jdk_${platform}" From 47d5c50f8318777b5f84fdb21773231cd679d7cc Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 28 Apr 2019 23:03:34 -0400 Subject: [PATCH 221/260] Suppress illegal access in plugin install (#41620) We use Bouncy Castle to verify signatures when installing official plugins. This leads to illegal access warnings because Bouncy Castle accesses the Sun security provider constructor. This commit adds an add-opens flag to suppress this illegal access. --- distribution/src/bin/elasticsearch-plugin | 3 ++- distribution/src/bin/elasticsearch-plugin.bat | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/distribution/src/bin/elasticsearch-plugin b/distribution/src/bin/elasticsearch-plugin index 1c11cfb35f235..df8626e7c5314 100755 --- a/distribution/src/bin/elasticsearch-plugin +++ b/distribution/src/bin/elasticsearch-plugin @@ -1,6 +1,7 @@ #!/bin/bash -ES_MAIN_CLASS=org.elasticsearch.plugins.PluginCli \ +ES_JAVA_OPTS="--add-opens java.base/sun.security.provider=ALL-UNNAMED $ES_JAVA_OPTS" \ + ES_MAIN_CLASS=org.elasticsearch.plugins.PluginCli \ ES_ADDITIONAL_CLASSPATH_DIRECTORIES=lib/tools/plugin-cli \ "`dirname "$0"`"/elasticsearch-cli \ "$@" diff --git a/distribution/src/bin/elasticsearch-plugin.bat b/distribution/src/bin/elasticsearch-plugin.bat index e447c7e847cf2..964fef0e29bfc 100644 --- a/distribution/src/bin/elasticsearch-plugin.bat +++ b/distribution/src/bin/elasticsearch-plugin.bat @@ -3,6 +3,7 @@ setlocal enabledelayedexpansion setlocal enableextensions +set ES_JAVA_OPTS="--add-opens java.base/sun.security.provider=ALL-UNNAMED %ES_JAVA_OPTS%" set ES_MAIN_CLASS=org.elasticsearch.plugins.PluginCli set ES_ADDITIONAL_CLASSPATH_DIRECTORIES=lib/tools/plugin-cli call "%~dp0elasticsearch-cli.bat" ^ From b3493631a8b24680d722be3ac81330111a62572f Mon Sep 17 00:00:00 2001 From: Russ Cam Date: Mon, 29 Apr 2019 19:57:13 +1000 Subject: [PATCH 222/260] [ML] Specify from and size as querystring parameters (#40575) This commit updates the header for from and size to indicate that they can be specified as querystring parameters. --- docs/reference/ml/apis/get-filter.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/ml/apis/get-filter.asciidoc b/docs/reference/ml/apis/get-filter.asciidoc index 07fc25d756a6a..53dfab2530a10 100644 --- a/docs/reference/ml/apis/get-filter.asciidoc +++ b/docs/reference/ml/apis/get-filter.asciidoc @@ -28,7 +28,7 @@ You can get a single filter or all filters. For more information, see (string) Identifier for the filter. -==== Request Body +==== Querystring Parameters `from`::: (integer) Skips the specified number of filters. From eda31b0ac00c952a52885902be59ac429b0ca81a Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Mon, 29 Apr 2019 13:40:46 +0200 Subject: [PATCH 223/260] Close and acquire commit during reset engine fix (#41584) * Close shard during reset engine fix If closing a shard while resetting engine, IndexEventListener.afterIndexShardClosed would be called while there is still an active IndexWriter on the shard. For integration tests, this leads to an exception during check index called from MockFSIndexStore .Listener. Fixed. Relates to #38561 --- .../elasticsearch/index/shard/IndexShard.java | 84 ++++++++----- .../index/shard/IndexShardTests.java | 117 ++++++++++++++++++ 2 files changed, 169 insertions(+), 32 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index c0b482b176853..2476eabf980c6 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -32,6 +32,7 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.UsageTrackingQueryCachingPolicy; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.ThreadInterruptedException; import org.elasticsearch.Assertions; import org.elasticsearch.ElasticsearchException; @@ -3083,43 +3084,62 @@ assert getActiveOperationsCount() == OPERATIONS_BLOCKED final TranslogStats translogStats = translogStats(); // flush to make sure the latest commit, which will be opened by the read-only engine, includes all operations. flush(new FlushRequest().waitIfOngoing(true)); + + SetOnce newEngineReference = new SetOnce<>(); + final long globalCheckpoint = getGlobalCheckpoint(); synchronized (mutex) { verifyNotClosed(); - // we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata). - final Engine readOnlyEngine = new ReadOnlyEngine(newEngineConfig(), seqNoStats, translogStats, false, Function.identity()); + // we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata, + // acquireXXXCommit and close works. + final Engine readOnlyEngine = + new ReadOnlyEngine(newEngineConfig(), seqNoStats, translogStats, false, Function.identity()) { + @Override + public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) { + synchronized (mutex) { + // ignore flushFirst since we flushed above and we do not want to interfere with ongoing translog replay + return newEngineReference.get().acquireLastIndexCommit(false); + } + } + + @Override + public IndexCommitRef acquireSafeIndexCommit() { + synchronized (mutex) { + return newEngineReference.get().acquireSafeIndexCommit(); + } + } + + @Override + public void close() throws IOException { + assert Thread.holdsLock(mutex); + + Engine newEngine = newEngineReference.get(); + if (newEngine == currentEngineReference.get()) { + // we successfully installed the new engine so do not close it. + newEngine = null; + } + IOUtils.close(super::close, newEngine); + } + }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); + newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig())); + onNewEngine(newEngineReference.get()); } - - Engine newEngine = null; - try { - final long globalCheckpoint = getGlobalCheckpoint(); - synchronized (mutex) { - assert currentEngineReference.get() instanceof ReadOnlyEngine : "another write engine is running"; - verifyNotClosed(); - // we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata). - newEngine = engineFactory.newReadWriteEngine(newEngineConfig()); - onNewEngine(newEngine); - } - newEngine.advanceMaxSeqNoOfUpdatesOrDeletes(globalCheckpoint); - final Engine.TranslogRecoveryRunner translogRunner = (engine, snapshot) -> runTranslogRecovery( - engine, snapshot, Engine.Operation.Origin.LOCAL_RESET, () -> { - // TODO: add a dedicate recovery stats for the reset translog - }); - newEngine.recoverFromTranslog(translogRunner, globalCheckpoint); - synchronized (mutex) { - verifyNotClosed(); - IOUtils.close(currentEngineReference.getAndSet(newEngine)); - // We set active because we are now writing operations to the engine; this way, - // if we go idle after some time and become inactive, we still give sync'd flush a chance to run. - active.set(true); - newEngine = null; - } - // time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during - // which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine. - onSettingsChanged(); - } finally { - IOUtils.close(newEngine); + newEngineReference.get().advanceMaxSeqNoOfUpdatesOrDeletes(globalCheckpoint); + final Engine.TranslogRecoveryRunner translogRunner = (engine, snapshot) -> runTranslogRecovery( + engine, snapshot, Engine.Operation.Origin.LOCAL_RESET, () -> { + // TODO: add a dedicate recovery stats for the reset translog + }); + newEngineReference.get().recoverFromTranslog(translogRunner, globalCheckpoint); + synchronized (mutex) { + verifyNotClosed(); + IOUtils.close(currentEngineReference.getAndSet(newEngineReference.get())); + // We set active because we are now writing operations to the engine; this way, + // if we go idle after some time and become inactive, we still give sync'd flush a chance to run. + active.set(true); } + // time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during + // which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine. + onSettingsChanged(); } /** diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index e7567ce495147..1d3cfeabd306c 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -125,6 +125,7 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.FieldMaskingReader; import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.threadpool.ThreadPool; import org.junit.Assert; @@ -3699,6 +3700,122 @@ public void testResetEngine() throws Exception { closeShard(shard, false); } + /** + * This test simulates a scenario seen rarely in ConcurrentSeqNoVersioningIT. Closing a shard while engine is inside + * resetEngineToGlobalCheckpoint can lead to check index failure in integration tests. + */ + public void testCloseShardWhileResettingEngine() throws Exception { + CountDownLatch readyToCloseLatch = new CountDownLatch(1); + CountDownLatch closeDoneLatch = new CountDownLatch(1); + IndexShard shard = newStartedShard(false, Settings.EMPTY, config -> new InternalEngine(config) { + @Override + public InternalEngine recoverFromTranslog(TranslogRecoveryRunner translogRecoveryRunner, + long recoverUpToSeqNo) throws IOException { + readyToCloseLatch.countDown(); + try { + closeDoneLatch.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + return super.recoverFromTranslog(translogRecoveryRunner, recoverUpToSeqNo); + } + }); + + Thread closeShardThread = new Thread(() -> { + try { + readyToCloseLatch.await(); + shard.close("testing", false); + // in integration tests, this is done as a listener on IndexService. + MockFSDirectoryService.checkIndex(logger, shard.store(), shard.shardId); + } catch (InterruptedException | IOException e) { + throw new AssertionError(e); + } finally { + closeDoneLatch.countDown(); + } + }); + + closeShardThread.start(); + + final CountDownLatch engineResetLatch = new CountDownLatch(1); + shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), shard.getGlobalCheckpoint(), 0L, + ActionListener.wrap(r -> { + try (r) { + shard.resetEngineToGlobalCheckpoint(); + } finally { + engineResetLatch.countDown(); + } + }, Assert::assertNotNull), TimeValue.timeValueMinutes(1L)); + + engineResetLatch.await(); + + closeShardThread.join(); + + // close store. + closeShard(shard, false); + } + + /** + * This test simulates a scenario seen rarely in ConcurrentSeqNoVersioningIT. While engine is inside + * resetEngineToGlobalCheckpoint snapshot metadata could fail + */ + public void testSnapshotWhileResettingEngine() throws Exception { + CountDownLatch readyToSnapshotLatch = new CountDownLatch(1); + CountDownLatch snapshotDoneLatch = new CountDownLatch(1); + IndexShard shard = newStartedShard(false, Settings.EMPTY, config -> new InternalEngine(config) { + @Override + public InternalEngine recoverFromTranslog(TranslogRecoveryRunner translogRecoveryRunner, + long recoverUpToSeqNo) throws IOException { + InternalEngine internalEngine = super.recoverFromTranslog(translogRecoveryRunner, recoverUpToSeqNo); + readyToSnapshotLatch.countDown(); + try { + snapshotDoneLatch.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + return internalEngine; + } + }); + + indexOnReplicaWithGaps(shard, between(0, 1000), Math.toIntExact(shard.getLocalCheckpoint())); + final long globalCheckpoint = randomLongBetween(shard.getGlobalCheckpoint(), shard.getLocalCheckpoint()); + shard.updateGlobalCheckpointOnReplica(globalCheckpoint, "test"); + + Thread snapshotThread = new Thread(() -> { + try { + readyToSnapshotLatch.await(); + shard.snapshotStoreMetadata(); + try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(randomBoolean())) { + shard.store().getMetadata(indexCommitRef.getIndexCommit()); + } + try (Engine.IndexCommitRef indexCommitRef = shard.acquireSafeIndexCommit()) { + shard.store().getMetadata(indexCommitRef.getIndexCommit()); + } + } catch (InterruptedException | IOException e) { + throw new AssertionError(e); + } finally { + snapshotDoneLatch.countDown(); + } + }); + + snapshotThread.start(); + + final CountDownLatch engineResetLatch = new CountDownLatch(1); + shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), shard.getGlobalCheckpoint(), 0L, + ActionListener.wrap(r -> { + try (r) { + shard.resetEngineToGlobalCheckpoint(); + } finally { + engineResetLatch.countDown(); + } + }, Assert::assertNotNull), TimeValue.timeValueMinutes(1L)); + + engineResetLatch.await(); + + snapshotThread.join(); + + closeShard(shard, false); + } + public void testConcurrentAcquireAllReplicaOperationsPermitsWithPrimaryTermUpdate() throws Exception { final IndexShard replica = newStartedShard(false); indexOnReplicaWithGaps(replica, between(0, 1000), Math.toIntExact(replica.getLocalCheckpoint())); From 60b6ebacc8845aef72668864754b9526b0afa8aa Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 29 Apr 2019 08:09:14 -0400 Subject: [PATCH 224/260] Fix quoting of ES_JAVA_OPTS in plugin script This commit fixes the quoting of ES_JAVA_OPTS in the plugin script on Windows. --- distribution/src/bin/elasticsearch-plugin.bat | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/src/bin/elasticsearch-plugin.bat b/distribution/src/bin/elasticsearch-plugin.bat index 964fef0e29bfc..5803d3bdd9248 100644 --- a/distribution/src/bin/elasticsearch-plugin.bat +++ b/distribution/src/bin/elasticsearch-plugin.bat @@ -3,7 +3,7 @@ setlocal enabledelayedexpansion setlocal enableextensions -set ES_JAVA_OPTS="--add-opens java.base/sun.security.provider=ALL-UNNAMED %ES_JAVA_OPTS%" +set ES_JAVA_OPTS=--add-opens java.base/sun.security.provider=ALL-UNNAMED %ES_JAVA_OPTS% set ES_MAIN_CLASS=org.elasticsearch.plugins.PluginCli set ES_ADDITIONAL_CLASSPATH_DIRECTORIES=lib/tools/plugin-cli call "%~dp0elasticsearch-cli.bat" ^ From 20cc0e00a202c0d0dd5a622e546bcb2fea503227 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Mon, 29 Apr 2019 13:19:56 +0100 Subject: [PATCH 225/260] Limit complexity of IntervalQueryBuilderTests#testRandomSource() (#41538) IntervalsSources can throw IllegalArgumentExceptions if they would produce too many disjunctions. To mitigate against this when building random sources, we limit the depth of the randomly generated source to four nested sources Fixes #41402 --- .../query/IntervalQueryBuilderTests.java | 67 +++++++++---------- 1 file changed, 31 insertions(+), 36 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java index 9f9a8f1d20716..03e64eb7ac415 100644 --- a/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java @@ -49,12 +49,7 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase 18) { - return new IntervalsSourceProvider.IntervalFilter(createRandomSource(), randomFrom(filters)); - } - return null; - } - private static final String MASKED_FIELD = "masked_field"; private static final String NO_POSITIONS_FIELD = "no_positions_field"; @@ -88,40 +76,54 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws new CompressedXContent(Strings.toString(mapping)), MapperService.MergeReason.MAPPING_UPDATE); } - private IntervalsSourceProvider createRandomSource() { - String useField = rarely() ? MASKED_FIELD : null; + private IntervalsSourceProvider createRandomSource(int depth) { + if (depth > 3) { + return createRandomMatch(depth + 1); + } switch (randomInt(20)) { case 0: case 1: int orCount = randomInt(4) + 1; List orSources = new ArrayList<>(); for (int i = 0; i < orCount; i++) { - orSources.add(createRandomSource()); + orSources.add(createRandomSource(depth + 1)); } - return new IntervalsSourceProvider.Disjunction(orSources, createRandomFilter()); + return new IntervalsSourceProvider.Disjunction(orSources, createRandomFilter(depth + 1)); case 2: case 3: int count = randomInt(5) + 1; List subSources = new ArrayList<>(); for (int i = 0; i < count; i++) { - subSources.add(createRandomSource()); + subSources.add(createRandomSource(depth + 1)); } boolean ordered = randomBoolean(); int maxGaps = randomInt(5) - 1; - IntervalsSourceProvider.IntervalFilter filter = createRandomFilter(); + IntervalsSourceProvider.IntervalFilter filter = createRandomFilter(depth + 1); return new IntervalsSourceProvider.Combine(subSources, ordered, maxGaps, filter); default: - int wordCount = randomInt(4) + 1; - List words = new ArrayList<>(); - for (int i = 0; i < wordCount; i++) { - words.add(randomRealisticUnicodeOfLengthBetween(4, 20)); - } - String text = String.join(" ", words); - boolean mOrdered = randomBoolean(); - int maxMGaps = randomInt(5) - 1; - String analyzer = randomFrom("simple", "keyword", "whitespace"); - return new IntervalsSourceProvider.Match(text, maxMGaps, mOrdered, analyzer, createRandomFilter(), useField); + return createRandomMatch(depth + 1); + } + } + + private IntervalsSourceProvider.IntervalFilter createRandomFilter(int depth) { + if (depth < 3 && randomInt(20) > 18) { + return new IntervalsSourceProvider.IntervalFilter(createRandomSource(depth + 1), randomFrom(filters)); } + return null; + } + + private IntervalsSourceProvider createRandomMatch(int depth) { + String useField = rarely() ? MASKED_FIELD : null; + int wordCount = randomInt(4) + 1; + List words = new ArrayList<>(); + for (int i = 0; i < wordCount; i++) { + words.add(randomRealisticUnicodeOfLengthBetween(4, 20)); + } + String text = String.join(" ", words); + boolean mOrdered = randomBoolean(); + int maxMGaps = randomInt(5) - 1; + String analyzer = randomFrom("simple", "keyword", "whitespace"); + return new IntervalsSourceProvider.Match(text, maxMGaps, mOrdered, analyzer, createRandomFilter(depth + 1), useField); } @Override @@ -383,11 +385,4 @@ public FactoryType compile(Script script, ScriptContext Date: Mon, 29 Apr 2019 08:28:03 -0400 Subject: [PATCH 226/260] [DOCS] Document 200 http code response for partial results (#40367) --- docs/reference/docs/bulk.asciidoc | 5 +++++ docs/reference/docs/data-replication.asciidoc | 22 ++++++++++++++----- docs/reference/docs/multi-get.asciidoc | 7 +++++- docs/reference/search/multi-search.asciidoc | 5 +++++ docs/reference/search/search.asciidoc | 7 +++++- 5 files changed, 39 insertions(+), 7 deletions(-) diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index 382d644023bb7..6087a1ba3b3b3 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -283,3 +283,8 @@ POST _bulk === Security See <>. + +[float] +[[bulk-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 diff --git a/docs/reference/docs/data-replication.asciidoc b/docs/reference/docs/data-replication.asciidoc index 84375467726fa..b83858cecfd8d 100644 --- a/docs/reference/docs/data-replication.asciidoc +++ b/docs/reference/docs/data-replication.asciidoc @@ -102,12 +102,24 @@ is as follows: . Combine the results and respond. Note that in the case of get by ID look up, only one shard is relevant and this step can be skipped. [float] -==== Failure handling +[[shard-failures]] +==== Shard failures + +When a shard fails to respond to a read request, the coordinating node sends the +request to another shard copy in the same replication group. Repeated failures +can result in no available shard copies. + +To ensure fast responses, the following APIs will +respond with partial results if one or more shards fail: + +* <> +* <> +* <> +* <> -When a shard fails to respond to a read request, the coordinating node will select another copy from the same replication group -and send the shard level search request to that copy instead. Repetitive failures can result in no shard copies being available. -In some cases, such as `_search`, Elasticsearch will prefer to respond fast, albeit with partial results, instead of waiting -for the issue to be resolved (partial results are indicated in the `_shards` header of the response). +Responses containing partial results still provide a `200 OK` HTTP status code. +Shard failures are indicated by the `timed_out` and `_shards` fields of +the response header. [float] === A few simple implications diff --git a/docs/reference/docs/multi-get.asciidoc b/docs/reference/docs/multi-get.asciidoc index fc8cc667999c1..8d5dd2ad74a3d 100644 --- a/docs/reference/docs/multi-get.asciidoc +++ b/docs/reference/docs/multi-get.asciidoc @@ -1,7 +1,7 @@ [[docs-multi-get]] == Multi Get API -Multi Get API allows to get multiple documents based on an index, type, +The Multi get API returns multiple documents based on an index, type, (optional) and id (and possibly routing). The response includes a `docs` array with all the fetched documents in order corresponding to the original multi-get request (if there was a failure for a specific get, an object containing this @@ -212,3 +212,8 @@ document `test/_doc/1` will be fetched from the shard corresponding to routing k === Security See <>. + +[float] +[[multi-get-partial-responses]] +=== Partial responses +To ensure fast responses, the multi get API will respond with partial results if one or more shards fail. See <> for more information. \ No newline at end of file diff --git a/docs/reference/search/multi-search.asciidoc b/docs/reference/search/multi-search.asciidoc index 9e3bff3c0c063..34dc37d794cad 100644 --- a/docs/reference/search/multi-search.asciidoc +++ b/docs/reference/search/multi-search.asciidoc @@ -173,3 +173,8 @@ 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 diff --git a/docs/reference/search/search.asciidoc b/docs/reference/search/search.asciidoc index 8154f2e701be2..f72eb7d68227e 100644 --- a/docs/reference/search/search.asciidoc +++ b/docs/reference/search/search.asciidoc @@ -37,4 +37,9 @@ Or we can search across all available indices using `_all`: GET /_all/_search?q=tag:wow --------------------------------------------------- // CONSOLE -// TEST[setup:twitter] \ No newline at end of file +// TEST[setup:twitter] + +[float] +[[search-partial-responses]] +=== Partial responses +To ensure fast responses, the search API will respond with partial results if one or more shards fail. See <> for more information. \ No newline at end of file From e49dd06a2b96527c80e73e5909800f17d85f65fe Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 29 Apr 2019 08:18:16 -0500 Subject: [PATCH 227/260] [ML] Add bucket_script agg support to data frames (#41594) --- .../transforms/pivot/AggregationConfig.java | 5 + .../integration/DataFramePivotRestIT.java | 51 +++++ .../pivot/AggregationResultUtils.java | 17 +- .../transforms/pivot/Aggregations.java | 3 +- .../dataframe/transforms/pivot/Pivot.java | 4 + .../transforms/pivot/SchemaUtil.java | 7 + .../pivot/AggregationResultUtilsTests.java | 181 +++++++++++++++--- .../transforms/pivot/AggregationsTests.java | 4 + .../transforms/pivot/PivotTests.java | 6 + 9 files changed, 248 insertions(+), 30 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/AggregationConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/AggregationConfig.java index a6d618e55321f..4942f335eeeef 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/AggregationConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/dataframe/transforms/pivot/AggregationConfig.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import java.io.IOException; @@ -66,6 +67,10 @@ public Collection getAggregatorFactories() { return aggregations.getAggregatorFactories(); } + public Collection getPipelineAggregatorFactories() { + return aggregations.getPipelineAggregatorFactories(); + } + public static AggregationConfig fromXContent(final XContentParser parser, boolean lenient) throws IOException { NamedXContentRegistry registry = parser.getXContentRegistry(); Map source = parser.mapOrdered(); diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java index fd89a32a81713..3a273cde27e19 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java @@ -368,6 +368,57 @@ public void testPivotWithScriptedMetricAgg() throws Exception { assertEquals(711.0, actual.doubleValue(), 0.000001); } + public void testPivotWithBucketScriptAgg() throws Exception { + String transformId = "bucketScriptPivot"; + String dataFrameIndex = "bucket_script_pivot_reviews"; + setupDataAccessRole(DATA_ACCESS_ROLE, REVIEWS_INDEX_NAME, dataFrameIndex); + + final Request createDataframeTransformRequest = createRequestWithAuth("PUT", DATAFRAME_ENDPOINT + transformId, + BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); + + String config = "{" + + " \"source\": {\"index\":\"" + REVIEWS_INDEX_NAME + "\"}," + + " \"dest\": {\"index\":\"" + dataFrameIndex + "\"},"; + + config += " \"pivot\": {" + + " \"group_by\": {" + + " \"reviewer\": {" + + " \"terms\": {" + + " \"field\": \"user_id\"" + + " } } }," + + " \"aggregations\": {" + + " \"avg_rating\": {" + + " \"avg\": {" + + " \"field\": \"stars\"" + + " } }," + + " \"avg_rating_again\": {" + + " \"bucket_script\": {" + + " \"buckets_path\": {\"param_1\": \"avg_rating\"}," + + " \"script\": \"return params.param_1\"" + + " } }" + + " } }" + + "}"; + + createDataframeTransformRequest.setJsonEntity(config); + Map createDataframeTransformResponse = entityAsMap(client().performRequest(createDataframeTransformRequest)); + assertThat(createDataframeTransformResponse.get("acknowledged"), equalTo(Boolean.TRUE)); + + startAndWaitForTransform(transformId, dataFrameIndex, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); + assertTrue(indexExists(dataFrameIndex)); + + // we expect 27 documents as there shall be 27 user_id's + Map indexStats = getAsMap(dataFrameIndex + "/_stats"); + assertEquals(27, XContentMapValues.extractValue("_all.total.docs.count", indexStats)); + + // get and check some users + Map searchResult = getAsMap(dataFrameIndex + "/_search?q=reviewer:user_4"); + assertEquals(1, XContentMapValues.extractValue("hits.total.value", searchResult)); + Number actual = (Number) ((List) XContentMapValues.extractValue("hits.hits._source.avg_rating", searchResult)).get(0); + assertEquals(3.878048780, actual.doubleValue(), 0.000001); + actual = (Number) ((List) XContentMapValues.extractValue("hits.hits._source.avg_rating_again", searchResult)).get(0); + assertEquals(3.878048780, actual.doubleValue(), 0.000001); + } + private void assertOnePivotValue(String query, double expected) throws IOException { Map searchResult = getAsMap(query); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtils.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtils.java index 574afd4f2fdf5..b17a65fc4daf1 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtils.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtils.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.search.aggregations.Aggregation; import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation.SingleValue; @@ -21,7 +22,9 @@ import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.elasticsearch.xpack.dataframe.transforms.pivot.SchemaUtil.isNumericType; @@ -42,6 +45,7 @@ final class AggregationResultUtils { public static Stream> extractCompositeAggregationResults(CompositeAggregation agg, GroupConfig groups, Collection aggregationBuilders, + Collection pipelineAggs, Map fieldTypeMap, DataFrameIndexerTransformStats stats) { return agg.getBuckets().stream().map(bucket -> { @@ -58,8 +62,10 @@ public static Stream> extractCompositeAggregationResults(Com document.put(destinationFieldName, value); }); - for (AggregationBuilder aggregationBuilder : aggregationBuilders) { - String aggName = aggregationBuilder.getName(); + List aggNames = aggregationBuilders.stream().map(AggregationBuilder::getName).collect(Collectors.toList()); + aggNames.addAll(pipelineAggs.stream().map(PipelineAggregationBuilder::getName).collect(Collectors.toList())); + + for (String aggName: aggNames) { final String fieldType = fieldTypeMap.get(aggName); // TODO: support other aggregation types @@ -67,9 +73,10 @@ public static Stream> extractCompositeAggregationResults(Com if (aggResult instanceof NumericMetricsAggregation.SingleValue) { NumericMetricsAggregation.SingleValue aggResultSingleValue = (SingleValue) aggResult; - // If the type is numeric, simply gather the `value` type, otherwise utilize `getValueAsString` so we don't lose - // formatted outputs. - if (isNumericType(fieldType)) { + // If the type is numeric or if the formatted string is the same as simply making the value a string, + // gather the `value` type, otherwise utilize `getValueAsString` so we don't lose formatted outputs. + if (isNumericType(fieldType) || + (aggResultSingleValue.getValueAsString().equals(String.valueOf(aggResultSingleValue.value())))) { document.put(aggName, aggResultSingleValue.value()); } else { document.put(aggName, aggResultSingleValue.getValueAsString()); diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Aggregations.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Aggregations.java index 39b139314d433..e7257c463ce7d 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Aggregations.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Aggregations.java @@ -35,7 +35,8 @@ enum AggregationType { MAX("max", SOURCE), MIN("min", SOURCE), SUM("sum", SOURCE), - SCRIPTED_METRIC("scripted_metric", DYNAMIC); + SCRIPTED_METRIC("scripted_metric", DYNAMIC), + BUCKET_SCRIPT("bucket_script", DYNAMIC); private final String aggregationType; private final String targetMapping; diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java index aa63ea92e7a53..9a8bf3f234dcd 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/Pivot.java @@ -19,6 +19,7 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -102,10 +103,12 @@ public Stream> extractResults(CompositeAggregation agg, GroupConfig groups = config.getGroupConfig(); Collection aggregationBuilders = config.getAggregationConfig().getAggregatorFactories(); + Collection pipelineAggregationBuilders = config.getAggregationConfig().getPipelineAggregatorFactories(); return AggregationResultUtils.extractCompositeAggregationResults(agg, groups, aggregationBuilders, + pipelineAggregationBuilders, fieldTypeMap, dataFrameIndexerTransformStats); } @@ -148,6 +151,7 @@ private static CompositeAggregationBuilder createCompositeAggregation(PivotConfi LoggingDeprecationHandler.INSTANCE, BytesReference.bytes(builder).streamInput()); compositeAggregation = CompositeAggregationBuilder.parse(COMPOSITE_AGGREGATION_NAME, parser); config.getAggregationConfig().getAggregatorFactories().forEach(agg -> compositeAggregation.subAggregation(agg)); + config.getAggregationConfig().getPipelineAggregatorFactories().forEach(agg -> compositeAggregation.subAggregation(agg)); } catch (IOException e) { throw new RuntimeException(DataFrameMessages.DATA_FRAME_TRANSFORM_PIVOT_FAILED_TO_CREATE_COMPOSITE_AGGREGATION, e); } diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/SchemaUtil.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/SchemaUtil.java index 95798e4c5640a..304f35b8c4c8a 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/SchemaUtil.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/pivot/SchemaUtil.java @@ -15,6 +15,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.ScriptedMetricAggregationBuilder; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; import org.elasticsearch.xpack.core.ClientHelper; @@ -85,6 +86,12 @@ public static void deduceMappings(final Client client, } } + // For pipeline aggs, since they are referencing other aggregations in the payload, they do not have any + // sourcefieldnames to put into the payload. Though, certain ones, i.e. avg_bucket, do have determinant value types + for (PipelineAggregationBuilder agg : config.getAggregationConfig().getPipelineAggregatorFactories()) { + aggregationTypes.put(agg.getName(), agg.getType()); + } + Map allFieldNames = new HashMap<>(); allFieldNames.putAll(aggregationSourceFieldNames); allFieldNames.putAll(fieldNamesForGrouping); diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtilsTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtilsTests.java index 62a4de353bc62..7eb4295111324 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtilsTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationResultUtilsTests.java @@ -14,9 +14,12 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.script.Script; import org.elasticsearch.search.aggregations.Aggregation; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; +import org.elasticsearch.search.aggregations.PipelineAggregatorBuilders; import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation; import org.elasticsearch.search.aggregations.bucket.composite.ParsedComposite; import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms; @@ -43,6 +46,8 @@ import org.elasticsearch.search.aggregations.metrics.StatsAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.pipeline.BucketScriptPipelineAggregationBuilder; +import org.elasticsearch.search.aggregations.pipeline.ParsedSimpleValue; import org.elasticsearch.search.aggregations.pipeline.ParsedStatsBucket; import org.elasticsearch.search.aggregations.pipeline.StatsBucketPipelineAggregationBuilder; import org.elasticsearch.test.ESTestCase; @@ -78,6 +83,7 @@ public class AggregationResultUtilsTests extends ESTestCase { map.put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c)); map.put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c)); map.put(AvgAggregationBuilder.NAME, (p, c) -> ParsedAvg.fromXContent(p, (String) c)); + map.put(BucketScriptPipelineAggregationBuilder.NAME, (p, c) -> ParsedSimpleValue.fromXContent(p, (String) c)); map.put(ScriptedMetricAggregationBuilder.NAME, (p, c) -> ParsedScriptedMetric.fromXContent(p, (String) c)); map.put(ValueCountAggregationBuilder.NAME, (p, c) -> ParsedValueCount.fromXContent(p, (String) c)); map.put(StatsAggregationBuilder.NAME, (p, c) -> ParsedStats.fromXContent(p, (String) c)); @@ -150,7 +156,7 @@ aggTypedName, asMap( targetField, "keyword", aggName, "double" ); - executeTest(groupBy, aggregationBuilders, input, fieldTypeMap, expected, 20); + executeTest(groupBy, aggregationBuilders, Collections.emptyList(), input, fieldTypeMap, expected, 20); } public void testExtractCompositeAggregationResultsMultipleGroups() throws IOException { @@ -236,7 +242,7 @@ aggTypedName, asMap( targetField, "keyword", targetField2, "keyword" ); - executeTest(groupBy, aggregationBuilders, input, fieldTypeMap, expected, 10); + executeTest(groupBy, aggregationBuilders, Collections.emptyList(), input, fieldTypeMap, expected, 10); } public void testExtractCompositeAggregationResultsMultiAggregations() throws IOException { @@ -306,7 +312,7 @@ aggTypedName2, asMap( aggName, "double", aggName2, "double" ); - executeTest(groupBy, aggregationBuilders, input, fieldTypeMap, expected, 200); + executeTest(groupBy, aggregationBuilders, Collections.emptyList(), input, fieldTypeMap, expected, 200); } public void testExtractCompositeAggregationResultsMultiAggregationsAndTypes() throws IOException { @@ -343,7 +349,8 @@ KEY, asMap( aggTypedName, asMap( "value", 42.33), aggTypedName2, asMap( - "value", 9.9), + "value", 9.9, + "value_as_string", "9.9F"), DOC_COUNT, 1), asMap( KEY, asMap( @@ -353,7 +360,8 @@ KEY, asMap( aggTypedName, asMap( "value", 8.4), aggTypedName2, asMap( - "value", 222.33), + "value", 222.33, + "value_as_string", "222.33F"), DOC_COUNT, 2), asMap( KEY, asMap( @@ -363,7 +371,8 @@ KEY, asMap( aggTypedName, asMap( "value", 28.99), aggTypedName2, asMap( - "value", -2.44), + "value", -2.44, + "value_as_string", "-2.44F"), DOC_COUNT, 3), asMap( KEY, asMap( @@ -373,7 +382,8 @@ KEY, asMap( aggTypedName, asMap( "value", 12.55), aggTypedName2, asMap( - "value", -100.44), + "value", -100.44, + "value_as_string", "-100.44F"), DOC_COUNT, 4) )); @@ -382,25 +392,25 @@ aggTypedName2, asMap( targetField, "ID1", targetField2, "ID1_2", aggName, 42.33, - aggName2, "9.9" + aggName2, "9.9F" ), asMap( targetField, "ID1", targetField2, "ID2_2", aggName, 8.4, - aggName2, "222.33" + aggName2, "222.33F" ), asMap( targetField, "ID2", targetField2, "ID1_2", aggName, 28.99, - aggName2, "-2.44" + aggName2, "-2.44F" ), asMap( targetField, "ID3", targetField2, "ID2_2", aggName, 12.55, - aggName2, "-100.44" + aggName2, "-100.44F" ) ); Map fieldTypeMap = asStringMap( @@ -409,7 +419,7 @@ aggTypedName2, asMap( targetField, "keyword", targetField2, "keyword" ); - executeTest(groupBy, aggregationBuilders, input, fieldTypeMap, expected, 10); + executeTest(groupBy, aggregationBuilders, Collections.emptyList(), input, fieldTypeMap, expected, 10); } public void testExtractCompositeAggregationResultsWithDynamicType() throws IOException { @@ -495,7 +505,112 @@ aggName, asMap("field", 12.0) targetField, "keyword", targetField2, "keyword" ); - executeTest(groupBy, aggregationBuilders, input, fieldTypeMap, expected, 10); + executeTest(groupBy, aggregationBuilders, Collections.emptyList(), input, fieldTypeMap, expected, 10); + } + + public void testExtractCompositeAggregationResultsWithPipelineAggregation() throws IOException { + String targetField = randomAlphaOfLengthBetween(5, 10); + String targetField2 = randomAlphaOfLengthBetween(5, 10) + "_2"; + + GroupConfig groupBy = parseGroupConfig("{" + + "\"" + targetField + "\" : {" + + " \"terms\" : {" + + " \"field\" : \"doesn't_matter_for_this_test\"" + + " } }," + + "\"" + targetField2 + "\" : {" + + " \"terms\" : {" + + " \"field\" : \"doesn't_matter_for_this_test\"" + + " } }" + + "}"); + + String aggName = randomAlphaOfLengthBetween(5, 10); + String aggTypedName = "avg#" + aggName; + String pipelineAggName = randomAlphaOfLengthBetween(5, 10) + "_2"; + String pipelineAggTypedName = "bucket_script#" + pipelineAggName; + + Collection aggregationBuilders = asList(AggregationBuilders.scriptedMetric(aggName)); + Collection pipelineAggregationBuilders = + asList(PipelineAggregatorBuilders.bucketScript(pipelineAggName, + Collections.singletonMap("param_1", aggName), + new Script("return params.param_1"))); + + Map input = asMap( + "buckets", + asList( + asMap( + KEY, asMap( + targetField, "ID1", + targetField2, "ID1_2" + ), + aggTypedName, asMap( + "value", 123.0), + pipelineAggTypedName, asMap( + "value", 123.0), + DOC_COUNT, 1), + asMap( + KEY, asMap( + targetField, "ID1", + targetField2, "ID2_2" + ), + aggTypedName, asMap( + "value", 1.0), + pipelineAggTypedName, asMap( + "value", 1.0), + DOC_COUNT, 2), + asMap( + KEY, asMap( + targetField, "ID2", + targetField2, "ID1_2" + ), + aggTypedName, asMap( + "value", 2.13), + pipelineAggTypedName, asMap( + "value", 2.13), + DOC_COUNT, 3), + asMap( + KEY, asMap( + targetField, "ID3", + targetField2, "ID2_2" + ), + aggTypedName, asMap( + "value", 12.0), + pipelineAggTypedName, asMap( + "value", 12.0), + DOC_COUNT, 4) + )); + + List> expected = asList( + asMap( + targetField, "ID1", + targetField2, "ID1_2", + aggName, 123.0, + pipelineAggName, 123.0 + ), + asMap( + targetField, "ID1", + targetField2, "ID2_2", + aggName, 1.0, + pipelineAggName, 1.0 + ), + asMap( + targetField, "ID2", + targetField2, "ID1_2", + aggName, 2.13, + pipelineAggName, 2.13 + ), + asMap( + targetField, "ID3", + targetField2, "ID2_2", + aggName, 12.0, + pipelineAggName, 12.0 + ) + ); + Map fieldTypeMap = asStringMap( + targetField, "keyword", + targetField2, "keyword", + aggName, "double" + ); + executeTest(groupBy, aggregationBuilders, pipelineAggregationBuilders, input, fieldTypeMap, expected, 10); } public void testExtractCompositeAggregationResultsDocIDs() throws IOException { @@ -598,8 +713,10 @@ aggTypedName, asMap( targetField2, "keyword" ); - List> resultFirstRun = runExtraction(groupBy, aggregationBuilders, inputFirstRun, fieldTypeMap, stats); - List> resultSecondRun = runExtraction(groupBy, aggregationBuilders, inputSecondRun, fieldTypeMap, stats); + List> resultFirstRun = + runExtraction(groupBy, aggregationBuilders, Collections.emptyList(), inputFirstRun, fieldTypeMap, stats); + List> resultSecondRun = + runExtraction(groupBy, aggregationBuilders, Collections.emptyList(), inputSecondRun, fieldTypeMap, stats); assertNotEquals(resultFirstRun, resultSecondRun); @@ -619,15 +736,23 @@ aggTypedName, asMap( assertEquals(documentIdsFirstRun, documentIdsSecondRun); } - - - private void executeTest(GroupConfig groups, Collection aggregationBuilders, Map input, - Map fieldTypeMap, List> expected, long expectedDocCounts) throws IOException { + private void executeTest(GroupConfig groups, + Collection aggregationBuilders, + Collection pipelineAggregationBuilders, + Map input, + Map fieldTypeMap, + List> expected, + long expectedDocCounts) throws IOException { DataFrameIndexerTransformStats stats = DataFrameIndexerTransformStats.withDefaultTransformId(); XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values())); builder.map(input); - List> result = runExtraction(groups, aggregationBuilders, input, fieldTypeMap, stats); + List> result = runExtraction(groups, + aggregationBuilders, + pipelineAggregationBuilders, + input, + fieldTypeMap, + stats); // remove the document ids and test uniqueness Set documentIds = new HashSet<>(); @@ -641,16 +766,24 @@ private void executeTest(GroupConfig groups, Collection aggr } - private List> runExtraction(GroupConfig groups, Collection aggregationBuilders, - Map input, Map fieldTypeMap, DataFrameIndexerTransformStats stats) throws IOException { + private List> runExtraction(GroupConfig groups, + Collection aggregationBuilders, + Collection pipelineAggregationBuilders, + Map input, + Map fieldTypeMap, + DataFrameIndexerTransformStats stats) throws IOException { XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values())); builder.map(input); try (XContentParser parser = createParser(builder)) { CompositeAggregation agg = ParsedComposite.fromXContent(parser, "my_feature"); - return AggregationResultUtils.extractCompositeAggregationResults(agg, groups, aggregationBuilders, fieldTypeMap, stats) - .collect(Collectors.toList()); + return AggregationResultUtils.extractCompositeAggregationResults(agg, + groups, + aggregationBuilders, + pipelineAggregationBuilders, + fieldTypeMap, + stats).collect(Collectors.toList()); } } diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationsTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationsTests.java index 47476baebdd61..5fb8463ae5412 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationsTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/AggregationsTests.java @@ -41,5 +41,9 @@ public void testResolveTargetMapping() { // scripted_metric assertEquals("_dynamic", Aggregations.resolveTargetMapping("scripted_metric", null)); assertEquals("_dynamic", Aggregations.resolveTargetMapping("scripted_metric", "int")); + + // scripted_metric + assertEquals("_dynamic", Aggregations.resolveTargetMapping("bucket_script", null)); + assertEquals("_dynamic", Aggregations.resolveTargetMapping("bucket_script", "int")); } } diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/PivotTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/PivotTests.java index be23f515ac8d6..d4607d7adc3e3 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/PivotTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/pivot/PivotTests.java @@ -184,6 +184,12 @@ private AggregationConfig getAggregationConfig(String agg) throws IOException { " }\n" + "}}"); } + if (agg.equals(AggregationType.BUCKET_SCRIPT.getName())) { + return parseAggregations("{\"pivot_bucket_script\":{" + + "\"bucket_script\":{" + + "\"buckets_path\":{\"param_1\":\"other_bucket\"}," + + "\"script\":\"return params.param_1\"}}}"); + } return parseAggregations("{\n" + " \"pivot_" + agg + "\": {\n" + " \"" + agg + "\": {\n" + " \"field\": \"values\"\n" + " }\n" + " }" + "}"); } From be399151ba3a8bea963cf3854b248e5951c786cf Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 29 Apr 2019 11:30:18 -0400 Subject: [PATCH 228/260] Verify consistency of version and source in disruption tests (#41614) With this change, we will verify the consistency of version and source (besides id, seq_no, and term) of live documents between shard copies at the end of disruption tests. --- .../discovery/ClusterDisruptionIT.java | 8 +++-- .../index/engine/InternalEngineTests.java | 6 ++-- .../engine/LuceneChangesSnapshotTests.java | 10 ++++++- .../index/engine/ReadOnlyEngineTests.java | 2 +- .../RecoveryDuringReplicationTests.java | 6 ++-- .../index/shard/IndexShardTests.java | 4 +-- ...oAndTerm.java => DocIdSeqNoAndSource.java} | 30 ++++++++++++++----- .../index/engine/EngineTestCase.java | 23 +++++++++----- .../ESIndexLevelReplicationTestCase.java | 4 +-- .../index/shard/IndexShardTestCase.java | 6 ++-- .../test/InternalTestCluster.java | 6 ++-- .../elasticsearch/xpack/CcrIntegTestCase.java | 24 +++++++-------- .../index/engine/FollowingEngineTests.java | 6 ++-- 13 files changed, 85 insertions(+), 50 deletions(-) rename test/framework/src/main/java/org/elasticsearch/index/engine/{DocIdSeqNoAndTerm.java => DocIdSeqNoAndSource.java} (58%) diff --git a/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java index f60e108c34ccd..3a257ec5973f8 100644 --- a/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java @@ -56,6 +56,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; @@ -66,6 +67,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.elasticsearch.action.DocWriteResponse.Result.CREATED; import static org.elasticsearch.action.DocWriteResponse.Result.UPDATED; @@ -134,6 +136,7 @@ public void testAckedIndexing() throws Exception { final List exceptedExceptions = new CopyOnWriteArrayList<>(); final ConflictMode conflictMode = ConflictMode.randomMode(); + final List fieldNames = IntStream.rangeClosed(0, randomInt(10)).mapToObj(n -> "f" + n).collect(Collectors.toList()); logger.info("starting indexers using conflict mode " + conflictMode); try { @@ -156,7 +159,7 @@ public void testAckedIndexing() throws Exception { int shard = Math.floorMod(Murmur3HashFunction.hash(id), numPrimaries); logger.trace("[{}] indexing id [{}] through node [{}] targeting shard [{}]", name, id, node, shard); IndexRequestBuilder indexRequestBuilder = client.prepareIndex("test", "type", id) - .setSource("{}", XContentType.JSON) + .setSource(Map.of(randomFrom(fieldNames), randomNonNegativeLong()), XContentType.JSON) .setTimeout(timeout); if (conflictMode == ConflictMode.external) { @@ -459,7 +462,8 @@ public void testRestartNodeWhileIndexing() throws Exception { while (stopped.get() == false && docID.get() < 5000) { String id = Integer.toString(docID.incrementAndGet()); try { - IndexResponse response = client().prepareIndex(index, "_doc", id).setSource("{}", XContentType.JSON).get(); + IndexResponse response = client().prepareIndex(index, "_doc", id) + .setSource(Map.of("f" + randomIntBetween(1, 10), randomNonNegativeLong()), XContentType.JSON).get(); assertThat(response.getResult(), isOneOf(CREATED, UPDATED)); logger.info("--> index id={} seq_no={}", response.getId(), response.getSeqNo()); ackedDocs.add(response.getId()); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 11b9ef7baa518..23763426b3745 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -4410,7 +4410,7 @@ public void testRestoreLocalHistoryFromTranslog() throws IOException { Randomness.shuffle(seqNos); final EngineConfig engineConfig; final SeqNoStats prevSeqNoStats; - final List prevDocs; + final List prevDocs; final int totalTranslogOps; try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) { engineConfig = engine.config(); @@ -5522,7 +5522,7 @@ public void testRebuildLocalCheckpointTracker() throws Exception { commits.add(new ArrayList<>()); try (Store store = createStore()) { EngineConfig config = config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get); - final List docs; + final List docs; try (InternalEngine engine = createEngine(config)) { List flushedOperations = new ArrayList<>(); for (Engine.Operation op : operations) { @@ -5570,7 +5570,7 @@ public void testOpenSoftDeletesIndexWithSoftDeletesDisabled() throws Exception { final IndexSettings softDeletesEnabled = IndexSettingsModule.newIndexSettings( IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(Settings.builder(). put(defaultSettings.getSettings()).put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true)).build()); - final List docs; + final List docs; try (InternalEngine engine = createEngine( config(softDeletesEnabled, store, translogPath, newMergePolicy(), null, null, globalCheckpoint::get))) { List ops = generateHistoryOnReplica(between(1, 100), randomBoolean(), randomBoolean(), randomBoolean()); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index f179cd840c60e..d1840c4d97cff 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -36,6 +36,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -274,7 +275,14 @@ public void run() { pullOperations(engine); } assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - assertThat(getDocIds(engine, true), equalTo(getDocIds(leader, true))); + // have to verify without source since we are randomly testing without _source + List docsWithoutSourceOnFollower = getDocIds(engine, true).stream() + .map(d -> new DocIdSeqNoAndSource(d.getId(), null, d.getSeqNo(), d.getPrimaryTerm(), d.getVersion())) + .collect(Collectors.toList()); + List docsWithoutSourceOnLeader = getDocIds(leader, true).stream() + .map(d -> new DocIdSeqNoAndSource(d.getId(), null, d.getSeqNo(), d.getPrimaryTerm(), d.getVersion())) + .collect(Collectors.toList()); + assertThat(docsWithoutSourceOnFollower, equalTo(docsWithoutSourceOnLeader)); } catch (Exception ex) { throw new AssertionError(ex); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java index e66094d7321a7..b689400601dc6 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -44,7 +44,7 @@ public void testReadOnlyEngine() throws Exception { EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); int numDocs = scaledRandomIntBetween(10, 1000); final SeqNoStats lastSeqNoStats; - final List lastDocIds; + final List lastDocIds; try (InternalEngine engine = createEngine(config)) { Engine.Get get = null; for (int i = 0; i < numDocs; i++) { diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index de6ab82892f51..d499cf6e83f90 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -40,7 +40,7 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineFactory; @@ -770,7 +770,7 @@ public void testRollbackOnPromotion() throws Exception { } } shards.refresh("test"); - List docsBelowGlobalCheckpoint = EngineTestCase.getDocIds(getEngine(newPrimary), randomBoolean()) + List docsBelowGlobalCheckpoint = EngineTestCase.getDocIds(getEngine(newPrimary), randomBoolean()) .stream().filter(doc -> doc.getSeqNo() <= newPrimary.getGlobalCheckpoint()).collect(Collectors.toList()); CountDownLatch latch = new CountDownLatch(1); final AtomicBoolean done = new AtomicBoolean(); @@ -780,7 +780,7 @@ public void testRollbackOnPromotion() throws Exception { latch.countDown(); while (done.get() == false) { try { - List exposedDocs = EngineTestCase.getDocIds(getEngine(randomFrom(replicas)), randomBoolean()); + List exposedDocs = EngineTestCase.getDocIds(getEngine(randomFrom(replicas)), randomBoolean()); assertThat(docsBelowGlobalCheckpoint, everyItem(isIn(exposedDocs))); assertThat(randomFrom(replicas).getLocalCheckpoint(), greaterThanOrEqualTo(initDocs - 1L)); } catch (AlreadyClosedException ignored) { diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 1d3cfeabd306c..bf1bfa668829e 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -79,7 +79,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.CommitStats; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine.DeleteResult; import org.elasticsearch.index.engine.EngineException; @@ -3666,7 +3666,7 @@ public void testResetEngine() throws Exception { while (done.get() == false) { try { List exposedDocIds = EngineTestCase.getDocIds(getEngine(shard), rarely()) - .stream().map(DocIdSeqNoAndTerm::getId).collect(Collectors.toList()); + .stream().map(DocIdSeqNoAndSource::getId).collect(Collectors.toList()); assertThat("every operations before the global checkpoint must be reserved", docBelowGlobalCheckpoint, everyItem(isIn(exposedDocIds))); } catch (AlreadyClosedException ignored) { diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/DocIdSeqNoAndTerm.java b/test/framework/src/main/java/org/elasticsearch/index/engine/DocIdSeqNoAndSource.java similarity index 58% rename from test/framework/src/main/java/org/elasticsearch/index/engine/DocIdSeqNoAndTerm.java rename to test/framework/src/main/java/org/elasticsearch/index/engine/DocIdSeqNoAndSource.java index b24a010c1a0d6..a48e813c1a68e 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/DocIdSeqNoAndTerm.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/DocIdSeqNoAndSource.java @@ -20,24 +20,34 @@ package org.elasticsearch.index.engine; +import org.apache.lucene.util.BytesRef; + import java.util.Objects; -/** A tuple of document id, sequence number and primary term of a document */ -public final class DocIdSeqNoAndTerm { +/** A tuple of document id, sequence number, primary term, source and version of a document */ +public final class DocIdSeqNoAndSource { private final String id; + private final BytesRef source; private final long seqNo; private final long primaryTerm; + private final long version; - public DocIdSeqNoAndTerm(String id, long seqNo, long primaryTerm) { + public DocIdSeqNoAndSource(String id, BytesRef source, long seqNo, long primaryTerm, long version) { this.id = id; + this.source = source; this.seqNo = seqNo; this.primaryTerm = primaryTerm; + this.version = version; } public String getId() { return id; } + public BytesRef getSource() { + return source; + } + public long getSeqNo() { return seqNo; } @@ -46,21 +56,27 @@ public long getPrimaryTerm() { return primaryTerm; } + public long getVersion() { + return version; + } + @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - DocIdSeqNoAndTerm that = (DocIdSeqNoAndTerm) o; - return Objects.equals(id, that.id) && seqNo == that.seqNo && primaryTerm == that.primaryTerm; + DocIdSeqNoAndSource that = (DocIdSeqNoAndSource) o; + return Objects.equals(id, that.id) && Objects.equals(source, that.source) + && seqNo == that.seqNo && primaryTerm == that.primaryTerm && version == that.version; } @Override public int hashCode() { - return Objects.hash(id, seqNo, primaryTerm); + return Objects.hash(id, source, seqNo, primaryTerm, version); } @Override public String toString() { - return "DocIdSeqNoAndTerm{" + "id='" + id + " seqNo=" + seqNo + " primaryTerm=" + primaryTerm + "}"; + return "doc{" + "id='" + id + " seqNo=" + seqNo + " primaryTerm=" + primaryTerm + + " version=" + version + " source= " + (source != null ? source.utf8ToString() : null) + "}"; } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 7fb2d50302c11..2a5b110795628 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -112,6 +112,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -996,16 +997,17 @@ public static Engine.Result applyOperation(Engine engine, Engine.Operation opera /** * Gets a collection of tuples of docId, sequence number, and primary term of all live documents in the provided engine. */ - public static List getDocIds(Engine engine, boolean refresh) throws IOException { + public static List getDocIds(Engine engine, boolean refresh) throws IOException { if (refresh) { engine.refresh("test_get_doc_ids"); } try (Engine.Searcher searcher = engine.acquireSearcher("test_get_doc_ids")) { - List docs = new ArrayList<>(); + List docs = new ArrayList<>(); for (LeafReaderContext leafContext : searcher.reader().leaves()) { LeafReader reader = leafContext.reader(); NumericDocValues seqNoDocValues = reader.getNumericDocValues(SeqNoFieldMapper.NAME); NumericDocValues primaryTermDocValues = reader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); + NumericDocValues versionDocValues = reader.getNumericDocValues(VersionFieldMapper.NAME); Bits liveDocs = reader.getLiveDocs(); for (int i = 0; i < reader.maxDoc(); i++) { if (liveDocs == null || liveDocs.get(i)) { @@ -1014,20 +1016,25 @@ public static List getDocIds(Engine engine, boolean refresh) continue; } final long primaryTerm = primaryTermDocValues.longValue(); - Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); - BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); + Document doc = reader.document(i, Set.of(IdFieldMapper.NAME, SourceFieldMapper.NAME)); + BytesRef binaryID = doc.getBinaryValue(IdFieldMapper.NAME); String id = Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length)); + final BytesRef source = doc.getBinaryValue(SourceFieldMapper.NAME); if (seqNoDocValues.advanceExact(i) == false) { throw new AssertionError("seqNoDocValues not found for doc[" + i + "] id[" + id + "]"); } final long seqNo = seqNoDocValues.longValue(); - docs.add(new DocIdSeqNoAndTerm(id, seqNo, primaryTerm)); + if (versionDocValues.advanceExact(i) == false) { + throw new AssertionError("versionDocValues not found for doc[" + i + "] id[" + id + "]"); + } + final long version = versionDocValues.longValue(); + docs.add(new DocIdSeqNoAndSource(id, source, seqNo, primaryTerm, version)); } } } - docs.sort(Comparator.comparingLong(DocIdSeqNoAndTerm::getSeqNo) - .thenComparingLong(DocIdSeqNoAndTerm::getPrimaryTerm) - .thenComparing((DocIdSeqNoAndTerm::getId))); + docs.sort(Comparator.comparingLong(DocIdSeqNoAndSource::getSeqNo) + .thenComparingLong(DocIdSeqNoAndSource::getPrimaryTerm) + .thenComparing((DocIdSeqNoAndSource::getId))); return docs; } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 9691677f643b9..0063f2a6d9b0c 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -65,7 +65,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; @@ -479,7 +479,7 @@ public synchronized void close() throws Exception { if (closed == false) { closed = true; try { - final List docsOnPrimary = getDocIdAndSeqNos(primary); + final List docsOnPrimary = getDocIdAndSeqNos(primary); for (IndexShard replica : replicas) { assertThat(replica.getMaxSeenAutoIdTimestamp(), equalTo(primary.getMaxSeenAutoIdTimestamp())); assertThat(replica.getMaxSeqNoOfUpdatesOrDeletes(), greaterThanOrEqualTo(primary.getMaxSeqNoOfUpdatesOrDeletes())); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 5b5ff8de01d03..6a39896199cf2 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -51,7 +51,7 @@ import org.elasticsearch.index.VersionType; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.query.DisabledQueryCache; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineTestCase; @@ -703,10 +703,10 @@ private Store.MetadataSnapshot getMetadataSnapshotOrEmpty(IndexShard replica) th } public static Set getShardDocUIDs(final IndexShard shard) throws IOException { - return getDocIdAndSeqNos(shard).stream().map(DocIdSeqNoAndTerm::getId).collect(Collectors.toSet()); + return getDocIdAndSeqNos(shard).stream().map(DocIdSeqNoAndSource::getId).collect(Collectors.toSet()); } - public static List getDocIdAndSeqNos(final IndexShard shard) throws IOException { + public static List getDocIdAndSeqNos(final IndexShard shard) throws IOException { return EngineTestCase.getDocIds(shard.getEngine(), true); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 8de9989fd6652..7ff928c4413d2 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -82,7 +82,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.CommitStats; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; @@ -1396,7 +1396,7 @@ public void assertSameDocIdsOnShards() throws Exception { if (primaryShard == null) { continue; } - final List docsOnPrimary; + final List docsOnPrimary; try { docsOnPrimary = IndexShardTestCase.getDocIdAndSeqNos(primaryShard); } catch (AlreadyClosedException ex) { @@ -1407,7 +1407,7 @@ public void assertSameDocIdsOnShards() throws Exception { if (replicaShard == null) { continue; } - final List docsOnReplica; + final List docsOnReplica; try { docsOnReplica = IndexShardTestCase.getDocIdAndSeqNos(replicaShard); } catch (AlreadyClosedException ex) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index fd84725e4bd6e..dea3da2a3ba1b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -49,7 +49,7 @@ import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -490,13 +490,13 @@ public static ResumeFollowAction.Request resumeFollow(String followerIndex) { protected void assertIndexFullyReplicatedToFollower(String leaderIndex, String followerIndex) throws Exception { logger.info("--> asserting <> between {} and {}", leaderIndex, followerIndex); assertBusy(() -> { - Map> docsOnFollower = getDocIdAndSeqNos(clusterGroup.followerCluster, followerIndex); - Map> docsOnLeader = getDocIdAndSeqNos(clusterGroup.leaderCluster, leaderIndex); - Map> mismatchedDocs = new HashMap<>(); - for (Map.Entry> fe : docsOnFollower.entrySet()) { - Set d1 = Sets.difference( + Map> docsOnFollower = getDocIdAndSeqNos(clusterGroup.followerCluster, followerIndex); + Map> docsOnLeader = getDocIdAndSeqNos(clusterGroup.leaderCluster, leaderIndex); + Map> mismatchedDocs = new HashMap<>(); + for (Map.Entry> fe : docsOnFollower.entrySet()) { + Set d1 = Sets.difference( Sets.newHashSet(fe.getValue()), Sets.newHashSet(docsOnLeader.getOrDefault(fe.getKey(), Collections.emptyList()))); - Set d2 = Sets.difference( + Set d2 = Sets.difference( Sets.newHashSet(docsOnLeader.getOrDefault(fe.getKey(), Collections.emptyList())), Sets.newHashSet(fe.getValue())); if (d1.isEmpty() == false || d2.isEmpty() == false) { mismatchedDocs.put(fe.getKey(), Sets.union(d1, d2)); @@ -525,11 +525,11 @@ protected void assertIndexFullyReplicatedToFollower(String leaderIndex, String f }, 120, TimeUnit.SECONDS); } - private Map> getDocIdAndSeqNos(InternalTestCluster cluster, String index) throws IOException { + private Map> getDocIdAndSeqNos(InternalTestCluster cluster, String index) throws IOException { final ClusterState state = cluster.client().admin().cluster().prepareState().get().getState(); List shardRoutings = state.routingTable().allShards(index); Randomness.shuffle(shardRoutings); - final Map> docs = new HashMap<>(); + final Map> docs = new HashMap<>(); for (ShardRouting shardRouting : shardRoutings) { if (shardRouting == null || shardRouting.assignedToNode() == false) { continue; @@ -537,14 +537,14 @@ private Map> getDocIdAndSeqNos(InternalTestClus IndexShard indexShard = cluster.getInstance(IndicesService.class, state.nodes().get(shardRouting.currentNodeId()).getName()) .indexServiceSafe(shardRouting.index()).getShard(shardRouting.id()); try { - final List docsOnShard = IndexShardTestCase.getDocIdAndSeqNos(indexShard); + final List docsOnShard = IndexShardTestCase.getDocIdAndSeqNos(indexShard); logger.info("--> shard {} docs {} seq_no_stats {}", shardRouting, docsOnShard, indexShard.seqNoStats()); docs.put(shardRouting.shardId().id(), docsOnShard.stream() // normalize primary term as the follower use its own term - .map(d -> new DocIdSeqNoAndTerm(d.getId(), d.getSeqNo(), 1L)) + .map(d -> new DocIdSeqNoAndSource(d.getId(), d.getSource(), d.getSeqNo(), 1L, d.getVersion())) .collect(Collectors.toList())); } catch (AlreadyClosedException e) { - // Ignore this exception and try getting List from other IndexShard instance. + // Ignore this exception and try getting List from other IndexShard instance. } } return docs; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index dfac5ef2654b8..e3d997886334b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -24,7 +24,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; -import org.elasticsearch.index.engine.DocIdSeqNoAndTerm; +import org.elasticsearch.index.engine.DocIdSeqNoAndSource; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineTestCase; @@ -623,7 +623,7 @@ public void testProcessOnceOnPrimary() throws Exception { assertThat(failure.getExistingPrimaryTerm().getAsLong(), equalTo(operationWithTerms.get(op.seqNo()))); } } - for (DocIdSeqNoAndTerm docId : getDocIds(followingEngine, true)) { + for (DocIdSeqNoAndSource docId : getDocIds(followingEngine, true)) { assertThat(docId.getPrimaryTerm(), equalTo(operationWithTerms.get(docId.getSeqNo()))); } // Replica should accept duplicates @@ -635,7 +635,7 @@ public void testProcessOnceOnPrimary() throws Exception { Engine.Result result = applyOperation(followingEngine, op, newTerm, nonPrimary); assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); } - for (DocIdSeqNoAndTerm docId : getDocIds(followingEngine, true)) { + for (DocIdSeqNoAndSource docId : getDocIds(followingEngine, true)) { assertThat(docId.getPrimaryTerm(), equalTo(operationWithTerms.get(docId.getSeqNo()))); } } From b8b58117508cfb6b894b1841f67c6faf1566bff2 Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Mon, 29 Apr 2019 11:11:57 -0500 Subject: [PATCH 229/260] Fix Watcher deadlock that can cause in-abilty to index documents. (#41418) * Fix Watcher deadlock that can cause in-abilty to index documents. This commit removes the usage of the `BulkProcessor` to write history documents and delete triggered watches on a `EsRejectedExecutionException`. Since the exception could be handled on the write thread, the write thread can be blocked waiting on watcher threads (due to a synchronous method). This is problematic since those watcher threads can be blocked waiting on write threads. This commit also moves the handling of the exception to the generic threadpool to avoid submitting write requests from the write thread pool. fixes #41390 --- .../watcher/execution/ExecutionService.java | 80 +++++++++++++++--- .../execution/ExecutionServiceTests.java | 75 +++++++++++++++-- .../integration/RejectedExecutionTests.java | 81 +++++++++++++++++++ 3 files changed, 216 insertions(+), 20 deletions(-) create mode 100644 x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/RejectedExecutionTests.java diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java index 1fca64c776773..b1ba8c1522acf 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/execution/ExecutionService.java @@ -15,8 +15,10 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.Client; @@ -31,19 +33,25 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.engine.DocumentMissingException; +import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.xpack.core.watcher.actions.ActionWrapper; import org.elasticsearch.xpack.core.watcher.actions.ActionWrapperResult; import org.elasticsearch.xpack.core.watcher.common.stats.Counters; import org.elasticsearch.xpack.core.watcher.condition.Condition; import org.elasticsearch.xpack.core.watcher.execution.ExecutionState; import org.elasticsearch.xpack.core.watcher.execution.QueuedWatch; +import org.elasticsearch.xpack.core.watcher.execution.TriggeredWatchStoreField; import org.elasticsearch.xpack.core.watcher.execution.WatchExecutionContext; import org.elasticsearch.xpack.core.watcher.execution.WatchExecutionSnapshot; +import org.elasticsearch.xpack.core.watcher.execution.Wid; +import org.elasticsearch.xpack.core.watcher.history.HistoryStoreField; import org.elasticsearch.xpack.core.watcher.history.WatchRecord; import org.elasticsearch.xpack.core.watcher.input.Input; +import org.elasticsearch.xpack.core.watcher.support.xcontent.WatcherParams; import org.elasticsearch.xpack.core.watcher.transform.Transform; import org.elasticsearch.xpack.core.watcher.trigger.TriggerEvent; import org.elasticsearch.xpack.core.watcher.watch.Watch; @@ -65,8 +73,10 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -396,22 +406,68 @@ private void executeAsync(WatchExecutionContext ctx, final TriggeredWatch trigge try { executor.execute(new WatchExecutionTask(ctx, () -> execute(ctx))); } catch (EsRejectedExecutionException e) { - String message = "failed to run triggered watch [" + triggeredWatch.id() + "] due to thread pool capacity"; - WatchRecord record = ctx.abortBeforeExecution(ExecutionState.THREADPOOL_REJECTION, message); - try { - if (ctx.overrideRecordOnConflict()) { - historyStore.forcePut(record); - } else { - historyStore.put(record); + //Using the generic pool here since this can happen from a write thread and we don't want to block a write + //thread to kick off these additional write/delete requests. + //Intentionally not using the HistoryStore or TriggerWatchStore to avoid re-using the same synchronous + //BulkProcessor which can cause a deadlock see #41390 + genericExecutor.execute(new WatchExecutionTask(ctx, () -> { + String message = "failed to run triggered watch [" + triggeredWatch.id() + "] due to thread pool capacity"; + logger.warn(message); + WatchRecord record = ctx.abortBeforeExecution(ExecutionState.THREADPOOL_REJECTION, message); + try { + forcePutHistory(record); + } catch (Exception exc) { + logger.error((Supplier) () -> + new ParameterizedMessage( + "Error storing watch history record for watch [{}] after thread pool rejection", + triggeredWatch.id()), exc); } - } catch (Exception exc) { - logger.error((Supplier) () -> - new ParameterizedMessage("Error storing watch history record for watch [{}] after thread pool rejection", - triggeredWatch.id()), exc); + deleteTrigger(triggeredWatch.id()); + })); + } + } + + /** + * Stores the specified watchRecord. + * Any existing watchRecord will be overwritten. + */ + private void forcePutHistory(WatchRecord watchRecord) { + String index = HistoryStoreField.getHistoryIndexNameForTime(watchRecord.triggerEvent().triggeredTime()); + try { + try (XContentBuilder builder = XContentFactory.jsonBuilder(); + ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(WATCHER_ORIGIN)) { + watchRecord.toXContent(builder, WatcherParams.HIDE_SECRETS); + IndexRequest request = new IndexRequest(index) + .id(watchRecord.id().value()) + .source(builder) + .opType(IndexRequest.OpType.CREATE); + client.index(request).get(30, TimeUnit.SECONDS); + logger.debug("indexed watch history record [{}]", watchRecord.id().value()); + } catch (VersionConflictEngineException vcee) { + watchRecord = new WatchRecord.MessageWatchRecord(watchRecord, ExecutionState.EXECUTED_MULTIPLE_TIMES, + "watch record [{ " + watchRecord.id() + " }] has been stored before, previous state [" + watchRecord.state() + "]"); + try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder(); + ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(WATCHER_ORIGIN)) { + IndexRequest request = new IndexRequest(index) + .id(watchRecord.id().value()) + .source(xContentBuilder.value(watchRecord)); + client.index(request).get(30, TimeUnit.SECONDS); + } + logger.debug("overwrote watch history record [{}]", watchRecord.id().value()); } + } catch (InterruptedException | ExecutionException | TimeoutException | IOException ioe) { + final WatchRecord wr = watchRecord; + logger.error((Supplier) () -> new ParameterizedMessage("failed to persist watch record [{}]", wr), ioe); + } + } - triggeredWatchStore.delete(triggeredWatch.id()); + private void deleteTrigger(Wid watcherId) { + DeleteRequest request = new DeleteRequest(TriggeredWatchStoreField.INDEX_NAME); + request.id(watcherId.value()); + try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(WATCHER_ORIGIN)) { + client.delete(request).actionGet(30, TimeUnit.SECONDS); } + logger.trace("successfully deleted triggered watch with id [{}]", watcherId); } WatchRecord executeInner(WatchExecutionContext ctx) { diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java index 321cf979bca73..80cb657a5762e 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java @@ -5,11 +5,13 @@ */ package org.elasticsearch.xpack.watcher.execution; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateResponse; @@ -28,8 +30,11 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.get.GetResult; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.Authentication; @@ -48,6 +53,7 @@ import org.elasticsearch.xpack.core.watcher.execution.ExecutionPhase; import org.elasticsearch.xpack.core.watcher.execution.ExecutionState; import org.elasticsearch.xpack.core.watcher.execution.QueuedWatch; +import org.elasticsearch.xpack.core.watcher.execution.TriggeredWatchStoreField; import org.elasticsearch.xpack.core.watcher.execution.WatchExecutionContext; import org.elasticsearch.xpack.core.watcher.execution.WatchExecutionSnapshot; import org.elasticsearch.xpack.core.watcher.execution.Wid; @@ -91,6 +97,7 @@ import static java.util.Collections.singletonMap; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasSize; @@ -844,11 +851,15 @@ public void testThatTriggeredWatchDeletionWorksOnExecutionRejection() throws Exc when(getResponse.isExists()).thenReturn(true); when(getResponse.getId()).thenReturn("foo"); mockGetWatchResponse(client, "foo", getResponse); + ActionFuture actionFuture = mock(ActionFuture.class); + when(actionFuture.get()).thenReturn(""); + when(client.index(any())).thenReturn(actionFuture); + when(client.delete(any())).thenReturn(actionFuture); + when(parser.parseWithSecrets(eq("foo"), eq(true), any(), any(), any(), anyLong(), anyLong())).thenReturn(watch); - // execute needs to fail as well as storing the history + // execute needs to fail doThrow(new EsRejectedExecutionException()).when(executor).execute(any()); - doThrow(new ElasticsearchException("whatever")).when(historyStore).forcePut(any()); Wid wid = new Wid(watch.id(), ZonedDateTime.now(ZoneOffset.UTC)); @@ -856,10 +867,58 @@ public void testThatTriggeredWatchDeletionWorksOnExecutionRejection() throws Exc new ScheduleTriggerEvent(ZonedDateTime.now(ZoneOffset.UTC) ,ZonedDateTime.now(ZoneOffset.UTC))); executionService.executeTriggeredWatches(Collections.singleton(triggeredWatch)); - verify(triggeredWatchStore, times(1)).delete(wid); - ArgumentCaptor captor = ArgumentCaptor.forClass(WatchRecord.class); - verify(historyStore, times(1)).forcePut(captor.capture()); - assertThat(captor.getValue().state(), is(ExecutionState.THREADPOOL_REJECTION)); + ArgumentCaptor deleteCaptor = ArgumentCaptor.forClass(DeleteRequest.class); + verify(client).delete(deleteCaptor.capture()); + assertThat(deleteCaptor.getValue().index(), equalTo(TriggeredWatchStoreField.INDEX_NAME)); + assertThat(deleteCaptor.getValue().id(), equalTo(wid.value())); + + ArgumentCaptor watchHistoryCaptor = ArgumentCaptor.forClass(IndexRequest.class); + verify(client).index(watchHistoryCaptor.capture()); + + assertThat(watchHistoryCaptor.getValue().source().utf8ToString(), containsString(ExecutionState.THREADPOOL_REJECTION.toString())); + assertThat(watchHistoryCaptor.getValue().index(), containsString(".watcher-history")); + } + + public void testForcePutHistoryOnExecutionRejection() throws Exception { + Watch watch = mock(Watch.class); + when(watch.id()).thenReturn("foo"); + WatchStatus status = new WatchStatus(ZonedDateTime.now(ZoneOffset.UTC), Collections.emptyMap()); + when(watch.status()).thenReturn(status); + GetResponse getResponse = mock(GetResponse.class); + when(getResponse.isExists()).thenReturn(true); + when(getResponse.getId()).thenReturn("foo"); + mockGetWatchResponse(client, "foo", getResponse); + ActionFuture actionFuture = mock(ActionFuture.class); + when(actionFuture.get()).thenReturn(""); + when(client.index(any())) + .thenThrow(new VersionConflictEngineException( + new ShardId(new Index("mockindex", "mockuuid"), 0), "id", "explaination")) + .thenReturn(actionFuture); + when(client.delete(any())).thenReturn(actionFuture); + + when(parser.parseWithSecrets(eq("foo"), eq(true), any(), any(), any(), anyLong(), anyLong())).thenReturn(watch); + + // execute needs to fail + doThrow(new EsRejectedExecutionException()).when(executor).execute(any()); + + Wid wid = new Wid(watch.id(), ZonedDateTime.now(ZoneOffset.UTC)); + + TriggeredWatch triggeredWatch = new TriggeredWatch(wid, + new ScheduleTriggerEvent(ZonedDateTime.now(ZoneOffset.UTC), ZonedDateTime.now(ZoneOffset.UTC))); + executionService.executeTriggeredWatches(Collections.singleton(triggeredWatch)); + + ArgumentCaptor deleteCaptor = ArgumentCaptor.forClass(DeleteRequest.class); + verify(client).delete(deleteCaptor.capture()); + assertThat(deleteCaptor.getValue().index(), equalTo(TriggeredWatchStoreField.INDEX_NAME)); + assertThat(deleteCaptor.getValue().id(), equalTo(wid.value())); + + ArgumentCaptor watchHistoryCaptor = ArgumentCaptor.forClass(IndexRequest.class); + verify(client, times(2)).index(watchHistoryCaptor.capture()); + List indexRequests = watchHistoryCaptor.getAllValues(); + + assertThat(indexRequests.get(0).id(), equalTo(indexRequests.get(1).id())); + assertThat(indexRequests.get(0).source().utf8ToString(), containsString(ExecutionState.THREADPOOL_REJECTION.toString())); + assertThat(indexRequests.get(1).source().utf8ToString(), containsString(ExecutionState.EXECUTED_MULTIPLE_TIMES.toString())); } public void testThatTriggeredWatchDeletionHappensOnlyIfWatchExists() throws Exception { @@ -898,7 +957,7 @@ public void testThatTriggeredWatchDeletionHappensOnlyIfWatchExists() throws Exce when(watch.status()).thenReturn(watchStatus); executionService.execute(context); - verify(triggeredWatchStore, never()).delete(any()); + verify(client, never()).delete(any()); } public void testThatSingleWatchCannotBeExecutedConcurrently() throws Exception { diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/RejectedExecutionTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/RejectedExecutionTests.java new file mode 100644 index 0000000000000..a457c1052cadb --- /dev/null +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/RejectedExecutionTests.java @@ -0,0 +1,81 @@ +/* + * 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.watcher.test.integration; + +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.license.LicenseService; +import org.elasticsearch.xpack.core.XPackSettings; +import org.elasticsearch.xpack.core.watcher.client.WatcherClient; +import org.elasticsearch.xpack.watcher.condition.CompareCondition; +import org.elasticsearch.xpack.watcher.support.search.WatcherSearchTemplateRequest; +import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase; +import org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule; + +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.index.query.QueryBuilders.termQuery; +import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource; +import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.loggingAction; +import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder; +import static org.elasticsearch.xpack.watcher.input.InputBuilders.searchInput; +import static org.elasticsearch.xpack.watcher.test.WatcherTestUtils.templateRequest; +import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule; +import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interval; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; + +public class RejectedExecutionTests extends AbstractWatcherIntegrationTestCase { + + @Override + protected boolean timeWarped() { + //need to use the real scheduler + return false; + } + + public void testHistoryAndTriggeredOnRejection() throws Exception { + WatcherClient watcherClient = watcherClient(); + createIndex("idx"); + client().prepareIndex("idx", "_doc").setSource("field", "a").get(); + refresh(); + WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "a")), "idx"); + watcherClient.preparePutWatch(randomAlphaOfLength(5)) + .setSource(watchBuilder() + .trigger(schedule(interval(1, IntervalSchedule.Interval.Unit.SECONDS))) + .input(searchInput(request)) + .condition(new CompareCondition("ctx.payload.hits.total", CompareCondition.Op.EQ, 1L)) + .addAction("_logger", loggingAction("_logging") + .setCategory("_category"))) + .get(); + + assertBusy(() -> { + flushAndRefresh(".watcher-history-*"); + SearchResponse searchResponse = client().prepareSearch(".watcher-history-*").get(); + assertThat(searchResponse.getHits().getTotalHits().value, greaterThanOrEqualTo(2L)); + }, 10, TimeUnit.SECONDS); + + flushAndRefresh(".triggered_watches"); + SearchResponse searchResponse = client().prepareSearch(".triggered_watches").get(); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(0L)); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(XPackSettings.MONITORING_ENABLED.getKey(), false) + .put(XPackSettings.SECURITY_ENABLED.getKey(), false) + .put(LicenseService.SELF_GENERATED_LICENSE_TYPE.getKey(), "trial") + .put("thread_pool.write.size", 1) + .put("thread_pool.write.queue_size", 1) + .put("xpack.watcher.thread_pool.size", 1) + .put("xpack.watcher.thread_pool.queue_size", 0) + .build(); + } + + +} From 6d3fd8401d5b2429878e81a9a0428a7792910407 Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Mon, 29 Apr 2019 13:40:30 -0400 Subject: [PATCH 230/260] Geo: Add GeoJson parser to libs/geo classes (#41575) Adds GeoJson parser for Geometry classes defined in libs/geo. Relates #40908 and #29872 --- .../elasticsearch/geo/geometry/Circle.java | 2 +- .../elasticsearch/geo/geometry/Geometry.java | 2 +- .../geo/geometry/GeometryCollection.java | 2 +- .../geo/geometry/GeometryVisitor.java | 22 +- .../org/elasticsearch/geo/geometry/Line.java | 2 +- .../geo/geometry/LinearRing.java | 2 +- .../elasticsearch/geo/geometry/MultiLine.java | 2 +- .../geo/geometry/MultiPoint.java | 2 +- .../geo/geometry/MultiPolygon.java | 2 +- .../org/elasticsearch/geo/geometry/Point.java | 2 +- .../elasticsearch/geo/geometry/Polygon.java | 2 +- .../elasticsearch/geo/geometry/Rectangle.java | 2 +- .../elasticsearch/geo/geometry/ShapeType.java | 6 + .../geo/utils/WellKnownText.java | 4 +- .../geo/geometry/BaseGeometryTestCase.java | 2 +- .../org/elasticsearch/common/geo/GeoJson.java | 612 ++++++++++++++ .../common/geo/GeometryParser.java | 54 ++ .../index/mapper/GeoShapeFieldMapper.java | 2 +- .../index/query/GeoShapeQueryBuilder.java | 2 +- .../common/geo/BaseGeoParsingTestCase.java | 7 + .../common/geo/GeoJsonParserTests.java | 757 ++++++++++++++++++ .../common/geo/GeoJsonSerializationTests.java | 269 +++++++ .../common/geo/GeometryParserTests.java | 137 ++++ 23 files changed, 1869 insertions(+), 27 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/geo/GeoJson.java create mode 100644 server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java create mode 100644 server/src/test/java/org/elasticsearch/common/geo/GeoJsonParserTests.java create mode 100644 server/src/test/java/org/elasticsearch/common/geo/GeoJsonSerializationTests.java create mode 100644 server/src/test/java/org/elasticsearch/common/geo/GeometryParserTests.java diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java index 7140540f5c140..cb8e2c4cb33e1 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java @@ -102,7 +102,7 @@ public int hashCode() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java index 9322193326fc5..140dd13427294 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java @@ -26,7 +26,7 @@ public interface Geometry { ShapeType type(); - T visit(GeometryVisitor visitor); + T visit(GeometryVisitor visitor) throws E; boolean isEmpty(); diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java index 56e59f94983ed..cc27d7c0972dd 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java @@ -57,7 +57,7 @@ public ShapeType type() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java index 8317b23d1feca..f4c189fbe288f 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java @@ -44,26 +44,26 @@ * * @see Visitor Pattern */ -public interface GeometryVisitor { +public interface GeometryVisitor { - T visit(Circle circle); + T visit(Circle circle) throws E; - T visit(GeometryCollection collection); + T visit(GeometryCollection collection) throws E; - T visit(Line line); + T visit(Line line) throws E; - T visit(LinearRing ring); + T visit(LinearRing ring) throws E; - T visit(MultiLine multiLine); + T visit(MultiLine multiLine) throws E; - T visit(MultiPoint multiPoint); + T visit(MultiPoint multiPoint) throws E; - T visit(MultiPolygon multiPolygon); + T visit(MultiPolygon multiPolygon) throws E; - T visit(Point point); + T visit(Point point) throws E; - T visit(Polygon polygon); + T visit(Polygon polygon) throws E; - T visit(Rectangle rectangle); + T visit(Rectangle rectangle) throws E; } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java index e06ccc555aa2c..c2c9cb4b83a18 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java @@ -103,7 +103,7 @@ public ShapeType type() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java index 7d66a93ea6d57..d27e512ef34cc 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java @@ -54,7 +54,7 @@ public ShapeType type() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java index 995c43d0c1c80..ac1f956397bb0 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java @@ -40,7 +40,7 @@ public ShapeType type() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java index 7d57b66ca564f..748902bd9eb72 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java @@ -40,7 +40,7 @@ public ShapeType type() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java index 01c68d6dd0b32..a843d90165b4b 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java @@ -40,7 +40,7 @@ public ShapeType type() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java index 189968fdd40b3..248f433b96a13 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java @@ -93,7 +93,7 @@ public int hashCode() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java index 1dee1c69fc840..ec6f564774ca9 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java @@ -92,7 +92,7 @@ public LinearRing getHole(int i) { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java index 120bf9e2eb862..ca7ec2e57c98d 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java @@ -207,7 +207,7 @@ public int hashCode() { } @Override - public T visit(GeometryVisitor visitor) { + public T visit(GeometryVisitor visitor) throws E { return visitor.visit(this); } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java index 2272f1ad89410..48a262a8316e3 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java @@ -19,6 +19,8 @@ package org.elasticsearch.geo.geometry; +import java.util.Locale; + /** * Shape types supported by elasticsearch */ @@ -33,4 +35,8 @@ public enum ShapeType { LINEARRING, // not serialized by itself in WKT or WKB ENVELOPE, // not part of the actual WKB spec CIRCLE; // not part of the actual WKB spec + + public static ShapeType forName(String shapeName) { + return ShapeType.valueOf(shapeName.toUpperCase(Locale.ROOT)); + } } diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java index ac0b8d9abfea4..e1af54e3383e0 100644 --- a/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java @@ -68,7 +68,7 @@ public static void toWKT(Geometry geometry, StringBuilder sb) { if (geometry.isEmpty()) { sb.append(EMPTY); } else { - geometry.visit(new GeometryVisitor() { + geometry.visit(new GeometryVisitor() { @Override public Void visit(Circle circle) { sb.append(LPAREN); @@ -543,7 +543,7 @@ private static String nextCloserOrComma(StreamTokenizer stream) throws IOExcepti } public static String getWKTName(Geometry geometry) { - return geometry.visit(new GeometryVisitor() { + return geometry.visit(new GeometryVisitor() { @Override public String visit(Circle circle) { return "circle"; diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java index b3cc834faea24..cc7dcd340c734 100644 --- a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java @@ -67,7 +67,7 @@ public void testVisitor() { public static void testVisitor(Geometry geom) { AtomicBoolean called = new AtomicBoolean(false); - Object result = geom.visit(new GeometryVisitor() { + Object result = geom.visit(new GeometryVisitor() { private Object verify(Geometry geometry, String expectedClass) { assertFalse("Visitor should be called only once", called.getAndSet(true)); assertSame(geom, geometry); diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java b/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java new file mode 100644 index 0000000000000..3489eca8b58e4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoJson.java @@ -0,0 +1,612 @@ +/* + * 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.geo; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.geo.parsers.ShapeParser; +import org.elasticsearch.common.unit.DistanceUnit; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentSubParser; +import org.elasticsearch.geo.geometry.Circle; +import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.geometry.GeometryCollection; +import org.elasticsearch.geo.geometry.GeometryVisitor; +import org.elasticsearch.geo.geometry.Line; +import org.elasticsearch.geo.geometry.LinearRing; +import org.elasticsearch.geo.geometry.MultiLine; +import org.elasticsearch.geo.geometry.MultiPoint; +import org.elasticsearch.geo.geometry.MultiPolygon; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Polygon; +import org.elasticsearch.geo.geometry.Rectangle; +import org.elasticsearch.geo.geometry.ShapeType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +/** + * Utility class for converting libs/geo shapes to and from GeoJson + */ +public final class GeoJson { + + private static final ParseField FIELD_TYPE = new ParseField("type"); + private static final ParseField FIELD_COORDINATES = new ParseField("coordinates"); + private static final ParseField FIELD_GEOMETRIES = new ParseField("geometries"); + private static final ParseField FIELD_ORIENTATION = new ParseField("orientation"); + private static final ParseField FIELD_RADIUS = new ParseField("radius"); + + private GeoJson() { + + } + + public static Geometry fromXContent(XContentParser parser, boolean rightOrientation, boolean coerce, boolean ignoreZValue) + throws IOException { + try (XContentSubParser subParser = new XContentSubParser(parser)) { + return PARSER.apply(subParser, new ParserContext(rightOrientation, coerce, ignoreZValue)); + } + } + + public static XContentBuilder toXContent(Geometry geometry, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(FIELD_TYPE.getPreferredName(), getGeoJsonName(geometry)); + geometry.visit(new GeometryVisitor() { + @Override + public XContentBuilder visit(Circle circle) throws IOException { + builder.field(FIELD_RADIUS.getPreferredName(), DistanceUnit.METERS.toString(circle.getRadiusMeters())); + builder.field(ShapeParser.FIELD_COORDINATES.getPreferredName()); + return coordinatesToXContent(circle.getLat(), circle.getLon(), circle.getAlt()); + } + + @Override + public XContentBuilder visit(GeometryCollection collection) throws IOException { + builder.startArray(FIELD_GEOMETRIES.getPreferredName()); + for (Geometry g : collection) { + toXContent(g, builder, params); + } + return builder.endArray(); + } + + @Override + public XContentBuilder visit(Line line) throws IOException { + builder.field(ShapeParser.FIELD_COORDINATES.getPreferredName()); + return coordinatesToXContent(line); + } + + @Override + public XContentBuilder visit(LinearRing ring) { + throw new UnsupportedOperationException("linearRing cannot be serialized using GeoJson"); + } + + @Override + public XContentBuilder visit(MultiLine multiLine) throws IOException { + builder.field(ShapeParser.FIELD_COORDINATES.getPreferredName()); + builder.startArray(); + for (int i = 0; i < multiLine.size(); i++) { + coordinatesToXContent(multiLine.get(i)); + } + return builder.endArray(); + } + + @Override + public XContentBuilder visit(MultiPoint multiPoint) throws IOException { + builder.startArray(ShapeParser.FIELD_COORDINATES.getPreferredName()); + for (int i = 0; i < multiPoint.size(); i++) { + Point p = multiPoint.get(i); + builder.startArray().value(p.getLon()).value(p.getLat()); + if (p.hasAlt()) { + builder.value(p.getAlt()); + } + builder.endArray(); + } + return builder.endArray(); + } + + @Override + public XContentBuilder visit(MultiPolygon multiPolygon) throws IOException { + builder.startArray(ShapeParser.FIELD_COORDINATES.getPreferredName()); + for (int i = 0; i < multiPolygon.size(); i++) { + builder.startArray(); + coordinatesToXContent(multiPolygon.get(i)); + builder.endArray(); + } + return builder.endArray(); + } + + @Override + public XContentBuilder visit(Point point) throws IOException { + builder.field(ShapeParser.FIELD_COORDINATES.getPreferredName()); + return coordinatesToXContent(point.getLat(), point.getLon(), point.getAlt()); + } + + @Override + public XContentBuilder visit(Polygon polygon) throws IOException { + builder.startArray(ShapeParser.FIELD_COORDINATES.getPreferredName()); + coordinatesToXContent(polygon.getPolygon()); + for (int i = 0; i < polygon.getNumberOfHoles(); i++) { + coordinatesToXContent(polygon.getHole(i)); + } + return builder.endArray(); + } + + @Override + public XContentBuilder visit(Rectangle rectangle) throws IOException { + builder.startArray(ShapeParser.FIELD_COORDINATES.getPreferredName()); + coordinatesToXContent(rectangle.getMaxLat(), rectangle.getMinLon(), rectangle.getMinAlt()); // top left + coordinatesToXContent(rectangle.getMinLat(), rectangle.getMaxLon(), rectangle.getMaxAlt()); // bottom right + return builder.endArray(); + } + + private XContentBuilder coordinatesToXContent(double lat, double lon, double alt) throws IOException { + builder.startArray().value(lon).value(lat); + if (Double.isNaN(alt) == false) { + builder.value(alt); + } + return builder.endArray(); + } + + private XContentBuilder coordinatesToXContent(Line line) throws IOException { + builder.startArray(); + for (int i = 0; i < line.length(); i++) { + builder.startArray().value(line.getLon(i)).value(line.getLat(i)); + if (line.hasAlt()) { + builder.value(line.getAlt(i)); + } + builder.endArray(); + } + return builder.endArray(); + } + + private XContentBuilder coordinatesToXContent(Polygon polygon) throws IOException { + coordinatesToXContent(polygon.getPolygon()); + for (int i = 0; i < polygon.getNumberOfHoles(); i++) { + coordinatesToXContent(polygon.getHole(i)); + } + return builder; + } + + }); + return builder.endObject(); + } + + private static class ParserContext { + public final boolean defaultOrientation; + public final boolean coerce; + public final boolean ignoreZValue; + + ParserContext(boolean defaultOrientation, boolean coerce, boolean ignoreZValue) { + this.defaultOrientation = defaultOrientation; + this.coerce = coerce; + this.ignoreZValue = ignoreZValue; + } + } + + private static ConstructingObjectParser PARSER = + new ConstructingObjectParser<>("geojson", true, (a, c) -> { + String type = (String) a[0]; + CoordinateNode coordinates = (CoordinateNode) a[1]; + @SuppressWarnings("unchecked") List geometries = (List) a[2]; + Boolean orientation = orientationFromString((String) a[3]); + DistanceUnit.Distance radius = (DistanceUnit.Distance) a[4]; + return createGeometry(type, geometries, coordinates, orientation, c.defaultOrientation, c.coerce, radius); + }); + + static { + PARSER.declareString(constructorArg(), FIELD_TYPE); + PARSER.declareField(optionalConstructorArg(), (p, c) -> parseCoordinates(p, c.ignoreZValue), FIELD_COORDINATES, + ObjectParser.ValueType.VALUE_ARRAY); + PARSER.declareObjectArray(optionalConstructorArg(), PARSER, FIELD_GEOMETRIES); + PARSER.declareString(optionalConstructorArg(), FIELD_ORIENTATION); + PARSER.declareField(optionalConstructorArg(), p -> DistanceUnit.Distance.parseDistance(p.text()), FIELD_RADIUS, + ObjectParser.ValueType.STRING); + } + + private static Geometry createGeometry(String type, List geometries, CoordinateNode coordinates, Boolean orientation, + boolean defaultOrientation, boolean coerce, DistanceUnit.Distance radius) { + + ShapeType shapeType = ShapeType.forName(type); + if (shapeType == ShapeType.GEOMETRYCOLLECTION) { + if (geometries == null) { + throw new ElasticsearchParseException("geometries not included"); + } + if (coordinates != null) { + throw new ElasticsearchParseException("parameter coordinates is not supported for type " + type); + } + verifyNulls(type, null, orientation, radius); + return new GeometryCollection<>(geometries); + } + + // We expect to have coordinates for all the rest + if (coordinates == null) { + throw new ElasticsearchParseException("coordinates not included"); + } + + switch (shapeType) { + case CIRCLE: + if (radius == null) { + throw new ElasticsearchParseException("radius is not specified"); + } + verifyNulls(type, geometries, orientation, null); + Point point = coordinates.asPoint(); + return new Circle(point.getLat(), point.getLon(), point.getAlt(), radius.convert(DistanceUnit.METERS).value); + case POINT: + verifyNulls(type, geometries, orientation, radius); + return coordinates.asPoint(); + case MULTIPOINT: + verifyNulls(type, geometries, orientation, radius); + return coordinates.asMultiPoint(); + case LINESTRING: + verifyNulls(type, geometries, orientation, radius); + return coordinates.asLineString(coerce); + case MULTILINESTRING: + verifyNulls(type, geometries, orientation, radius); + return coordinates.asMultiLineString(coerce); + case POLYGON: + verifyNulls(type, geometries, null, radius); + // handle possible null in orientation + return coordinates.asPolygon(orientation != null ? orientation : defaultOrientation, coerce); + case MULTIPOLYGON: + verifyNulls(type, geometries, null, radius); + // handle possible null in orientation + return coordinates.asMultiPolygon(orientation != null ? orientation : defaultOrientation, coerce); + case ENVELOPE: + verifyNulls(type, geometries, orientation, radius); + return coordinates.asRectangle(); + default: + throw new ElasticsearchParseException("unsuppoted shape type " + type); + } + } + + /** + * Checks that all passed parameters except type are null, generates corresponding error messages if they are not + */ + private static void verifyNulls(String type, List geometries, Boolean orientation, DistanceUnit.Distance radius) { + if (geometries != null) { + throw new ElasticsearchParseException("parameter geometries is not supported for type " + type); + } + if (orientation != null) { + throw new ElasticsearchParseException("parameter orientation is not supported for type " + type); + } + if (radius != null) { + throw new ElasticsearchParseException("parameter radius is not supported for type " + type); + } + } + + /** + * Recursive method which parses the arrays of coordinates used to define + * Shapes + */ + private static CoordinateNode parseCoordinates(XContentParser parser, boolean ignoreZValue) throws IOException { + XContentParser.Token token = parser.nextToken(); + // Base cases + if (token != XContentParser.Token.START_ARRAY && + token != XContentParser.Token.END_ARRAY && + token != XContentParser.Token.VALUE_NULL) { + return new CoordinateNode(parseCoordinate(parser, ignoreZValue)); + } else if (token == XContentParser.Token.VALUE_NULL) { + throw new IllegalArgumentException("coordinates cannot contain NULL values)"); + } + + List nodes = new ArrayList<>(); + while (token != XContentParser.Token.END_ARRAY) { + CoordinateNode node = parseCoordinates(parser, ignoreZValue); + if (nodes.isEmpty() == false && nodes.get(0).numDimensions() != node.numDimensions()) { + throw new ElasticsearchParseException("Exception parsing coordinates: number of dimensions do not match"); + } + nodes.add(node); + token = parser.nextToken(); + } + + return new CoordinateNode(nodes); + } + + /** + * Parser a singe set of 2 or 3 coordinates + */ + private static Point parseCoordinate(XContentParser parser, boolean ignoreZValue) throws IOException { + // Add support for coerce here + if (parser.currentToken() != XContentParser.Token.VALUE_NUMBER) { + throw new ElasticsearchParseException("geo coordinates must be numbers"); + } + double lon = parser.doubleValue(); + if (parser.nextToken() != XContentParser.Token.VALUE_NUMBER) { + throw new ElasticsearchParseException("geo coordinates must be numbers"); + } + double lat = parser.doubleValue(); + XContentParser.Token token = parser.nextToken(); + // alt (for storing purposes only - future use includes 3d shapes) + double alt = Double.NaN; + if (token == XContentParser.Token.VALUE_NUMBER) { + alt = GeoPoint.assertZValue(ignoreZValue, parser.doubleValue()); + parser.nextToken(); + } + // do not support > 3 dimensions + if (parser.currentToken() == XContentParser.Token.VALUE_NUMBER) { + throw new ElasticsearchParseException("geo coordinates greater than 3 dimensions are not supported"); + } + return new Point(lat, lon, alt); + } + + /** + * Returns true for right orientation and false for left + */ + private static Boolean orientationFromString(String orientation) { + if (orientation == null) { + return null; + } + orientation = orientation.toLowerCase(Locale.ROOT); + switch (orientation) { + case "right": + case "counterclockwise": + case "ccw": + return true; + case "left": + case "clockwise": + case "cw": + return false; + default: + throw new IllegalArgumentException("Unknown orientation [" + orientation + "]"); + } + } + + public static String getGeoJsonName(Geometry geometry) { + return geometry.visit(new GeometryVisitor<>() { + @Override + public String visit(Circle circle) { + return "Circle"; + } + + @Override + public String visit(GeometryCollection collection) { + return "GeometryCollection"; + } + + @Override + public String visit(Line line) { + return "LineString"; + } + + @Override + public String visit(LinearRing ring) { + throw new UnsupportedOperationException("line ring cannot be serialized using GeoJson"); + } + + @Override + public String visit(MultiLine multiLine) { + return "MultiLineString"; + } + + @Override + public String visit(MultiPoint multiPoint) { + return "MultiPoint"; + } + + @Override + public String visit(MultiPolygon multiPolygon) { + return "MultiPolygon"; + } + + @Override + public String visit(Point point) { + return "Point"; + } + + @Override + public String visit(Polygon polygon) { + return "Polygon"; + } + + @Override + public String visit(Rectangle rectangle) { + return "Envelope"; + } + }); + } + + private static class CoordinateNode implements ToXContentObject { + public final Point coordinate; + public final List children; + + /** + * Creates a new leaf CoordinateNode + * + * @param coordinate Coordinate for the Node + */ + CoordinateNode(Point coordinate) { + this.coordinate = coordinate; + this.children = null; + } + + /** + * Creates a new parent CoordinateNode + * + * @param children Children of the Node + */ + CoordinateNode(List children) { + this.children = children; + this.coordinate = null; + } + + public boolean isEmpty() { + return (coordinate == null && (children == null || children.isEmpty())); + } + + protected int numDimensions() { + if (isEmpty()) { + throw new ElasticsearchException("attempting to get number of dimensions on an empty coordinate node"); + } + if (coordinate != null) { + return coordinate.hasAlt() ? 3 : 2; + } + return children.get(0).numDimensions(); + } + + public Point asPoint() { + if (children != null) { + throw new ElasticsearchException("expected a single points but got a list"); + } + return coordinate; + } + + public MultiPoint asMultiPoint() { + if (coordinate != null) { + throw new ElasticsearchException("expected a list of points but got a point"); + } + List points = new ArrayList<>(); + for (CoordinateNode node : children) { + points.add(node.asPoint()); + } + return new MultiPoint(points); + } + + private double[][] asLineComponents(boolean orientation, boolean coerce) { + if (coordinate != null) { + throw new ElasticsearchException("expected a list of points but got a point"); + } + + if (children.size() < 2) { + throw new ElasticsearchException("not enough points to build a line"); + } + + boolean needsClosing; + int resultSize; + if (coerce && children.get(0).asPoint().equals(children.get(children.size() - 1).asPoint()) == false) { + needsClosing = true; + resultSize = children.size() + 1; + } else { + needsClosing = false; + resultSize = children.size(); + } + + double[] lats = new double[resultSize]; + double[] lons = new double[resultSize]; + double[] alts = numDimensions() == 3 ? new double[resultSize] : null; + int i = orientation ? 0 : lats.length - 1; + for (CoordinateNode node : children) { + Point point = node.asPoint(); + lats[i] = point.getLat(); + lons[i] = point.getLon(); + if (alts != null) { + alts[i] = point.getAlt(); + } + i = orientation ? i + 1 : i - 1; + } + if (needsClosing) { + lats[resultSize - 1] = lats[0]; + lons[resultSize - 1] = lons[0]; + if (alts != null) { + alts[resultSize - 1] = alts[0]; + } + } + double[][] components = new double[3][]; + components[0] = lats; + components[1] = lons; + components[2] = alts; + return components; + } + + public Line asLineString(boolean coerce) { + double[][] components = asLineComponents(true, coerce); + return new Line(components[0], components[1], components[2]); + } + + public LinearRing asLinearRing(boolean orientation, boolean coerce) { + double[][] components = asLineComponents(orientation, coerce); + return new LinearRing(components[0], components[1], components[2]); + } + + public MultiLine asMultiLineString(boolean coerce) { + if (coordinate != null) { + throw new ElasticsearchException("expected a list of points but got a point"); + } + List lines = new ArrayList<>(); + for (CoordinateNode node : children) { + lines.add(node.asLineString(coerce)); + } + return new MultiLine(lines); + } + + + public Polygon asPolygon(boolean orientation, boolean coerce) { + if (coordinate != null) { + throw new ElasticsearchException("expected a list of points but got a point"); + } + List lines = new ArrayList<>(); + for (CoordinateNode node : children) { + lines.add(node.asLinearRing(orientation, coerce)); + } + if (lines.size() == 1) { + return new Polygon(lines.get(0)); + } else { + LinearRing shell = lines.remove(0); + return new Polygon(shell, lines); + } + } + + public MultiPolygon asMultiPolygon(boolean orientation, boolean coerce) { + if (coordinate != null) { + throw new ElasticsearchException("expected a list of points but got a point"); + } + List polygons = new ArrayList<>(); + for (CoordinateNode node : children) { + polygons.add(node.asPolygon(orientation, coerce)); + } + return new MultiPolygon(polygons); + } + + public Rectangle asRectangle() { + if (children.size() != 2) { + throw new ElasticsearchParseException( + "invalid number of points [{}] provided for geo_shape [{}] when expecting an array of 2 coordinates", + children.size(), ShapeType.ENVELOPE); + } + // verify coordinate bounds, correct if necessary + Point uL = children.get(0).coordinate; + Point lR = children.get(1).coordinate; + return new Rectangle(lR.getLat(), uL.getLat(), uL.getLon(), lR.getLon()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (children == null) { + builder.startArray().value(coordinate.getLon()).value(coordinate.getLat()).endArray(); + } else { + builder.startArray(); + for (CoordinateNode child : children) { + child.toXContent(builder, params); + } + builder.endArray(); + } + return builder; + } + } + +} diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java b/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java new file mode 100644 index 0000000000000..8e1db18ccdd97 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/geo/GeometryParser.java @@ -0,0 +1,54 @@ +/* + * 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.geo; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; + +/** + * An utility class with a geometry parser methods supporting different shape representation formats + */ +public final class GeometryParser { + + private GeometryParser() { + + } + + /** + * Parses supplied XContent into Geometry + */ + public static Geometry parse(XContentParser parser, boolean orientation, boolean coerce, boolean ignoreZValue) throws IOException, + ParseException { + if (parser.currentToken() == XContentParser.Token.VALUE_NULL) { + return null; + } else if (parser.currentToken() == XContentParser.Token.START_OBJECT) { + return GeoJson.fromXContent(parser, orientation, coerce, ignoreZValue); + } else if (parser.currentToken() == XContentParser.Token.VALUE_STRING) { + // TODO: Add support for ignoreZValue and coerce to WKT + return WellKnownText.fromWKT(parser.text()); + } + throw new ElasticsearchParseException("shape must be an object consisting of type and coordinates"); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java index 72b3e68fa025e..6449c06fbe1ad 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/GeoShapeFieldMapper.java @@ -134,7 +134,7 @@ private void indexShape(ParseContext context, Object luceneShape) { } } - private class LuceneGeometryIndexer implements GeometryVisitor { + private class LuceneGeometryIndexer implements GeometryVisitor { private ParseContext context; private LuceneGeometryIndexer(ParseContext context) { diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java index b13666296b55d..b651a26d7e280 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java @@ -450,7 +450,7 @@ private Query getVectorQuery(QueryShardContext context, ShapeBuilder queryShapeB } private Query getVectorQueryFromShape(QueryShardContext context, Geometry queryShape) { - return queryShape.visit(new GeometryVisitor() { + return queryShape.visit(new GeometryVisitor() { @Override public Query visit(Circle circle) { throw new QueryShardException(context, "Field [" + fieldName + "] found and unknown shape Circle"); diff --git a/server/src/test/java/org/elasticsearch/common/geo/BaseGeoParsingTestCase.java b/server/src/test/java/org/elasticsearch/common/geo/BaseGeoParsingTestCase.java index 9359101128883..7f6c56855ec70 100644 --- a/server/src/test/java/org/elasticsearch/common/geo/BaseGeoParsingTestCase.java +++ b/server/src/test/java/org/elasticsearch/common/geo/BaseGeoParsingTestCase.java @@ -67,6 +67,13 @@ protected void assertGeometryEquals(Object expected, XContentBuilder geoJson, bo } } + protected void assertGeometryEquals(org.elasticsearch.geo.geometry.Geometry expected, XContentBuilder geoJson) throws IOException { + try (XContentParser parser = createParser(geoJson)) { + parser.nextToken(); + assertEquals(expected, GeoJson.fromXContent(parser, true, false, false)); + } + } + protected ShapeCollection shapeCollection(Shape... shapes) { return new ShapeCollection<>(Arrays.asList(shapes), SPATIAL_CONTEXT); } diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoJsonParserTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonParserTests.java new file mode 100644 index 0000000000000..4e2c2e50e2cf1 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonParserTests.java @@ -0,0 +1,757 @@ +/* + * 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.geo; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParseException; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.geo.geometry.Circle; +import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.geometry.GeometryCollection; +import org.elasticsearch.geo.geometry.Line; +import org.elasticsearch.geo.geometry.LinearRing; +import org.elasticsearch.geo.geometry.MultiLine; +import org.elasticsearch.geo.geometry.MultiPoint; +import org.elasticsearch.geo.geometry.MultiPolygon; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Polygon; +import org.elasticsearch.geo.geometry.Rectangle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; + + +/** + * Tests for {@code GeoJSONShapeParser} + */ +public class GeoJsonParserTests extends BaseGeoParsingTestCase { + + @Override + public void testParsePoint() throws IOException { + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Point") + .startArray("coordinates").value(100.0).value(0.0).endArray() + .endObject(); + Point expected = new Point(0.0, 100.0); + assertGeometryEquals(expected, pointGeoJson); + } + + @Override + public void testParseLineString() throws IOException { + XContentBuilder lineGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "LineString") + .startArray("coordinates") + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .endArray() + .endObject(); + + Line expected = new Line(new double[] {0.0, 1.0}, new double[] { 100.0, 101.0}); + try (XContentParser parser = createParser(lineGeoJson)) { + parser.nextToken(); + assertEquals(expected, GeoJson.fromXContent(parser, false, false, true)); + } + } + + @Override + public void testParseMultiLineString() throws IOException { + XContentBuilder multilinesGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "MultiLineString") + .startArray("coordinates") + .startArray() + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .endArray() + .startArray() + .startArray().value(102.0).value(2.0).endArray() + .startArray().value(103.0).value(3.0).endArray() + .endArray() + .endArray() + .endObject(); + + MultiLine expected = new MultiLine(Arrays.asList( + new Line(new double[] {0.0, 1.0}, new double[] { 100.0, 101.0}), + new Line(new double[] {2.0, 3.0}, new double[] { 102.0, 103.0}) + + )); + + assertGeometryEquals(expected, multilinesGeoJson); + } + + public void testParseCircle() throws IOException { + XContentBuilder multilinesGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "circle") + .startArray("coordinates").value(100.0).value(0.0).endArray() + .field("radius", "200m") + .endObject(); + + Circle expected = new Circle(0.0, 100.0, 200); + assertGeometryEquals(expected, multilinesGeoJson); + } + + public void testParseMultiDimensionShapes() throws IOException { + // multi dimension point + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Point") + .startArray("coordinates").value(100.0).value(0.0).value(15.0).value(18.0).endArray() + .endObject(); + + try (XContentParser parser = createParser(pointGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, false, false, false)); + assertNull(parser.nextToken()); + } + + // multi dimension linestring + XContentBuilder lineGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "LineString") + .startArray("coordinates") + .startArray().value(100.0).value(0.0).value(15.0).endArray() + .startArray().value(101.0).value(1.0).value(18.0).value(19.0).endArray() + .endArray() + .endObject(); + + try (XContentParser parser = createParser(lineGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, false, false, false)); + assertNull(parser.nextToken()); + } + } + + @Override + public void testParseEnvelope() throws IOException { + // test #1: envelope with expected coordinate order (TopLeft, BottomRight) + XContentBuilder multilinesGeoJson = XContentFactory.jsonBuilder().startObject().field("type", "envelope") + .startArray("coordinates") + .startArray().value(-50).value(30).endArray() + .startArray().value(50).value(-30).endArray() + .endArray() + .endObject(); + Rectangle expected = new Rectangle(-30, 30, -50, 50); + assertGeometryEquals(expected, multilinesGeoJson); + + // test #2: envelope that spans dateline + multilinesGeoJson = XContentFactory.jsonBuilder().startObject().field("type", "envelope") + .startArray("coordinates") + .startArray().value(50).value(30).endArray() + .startArray().value(-50).value(-30).endArray() + .endArray() + .endObject(); + + expected = new Rectangle(-30, 30, 50, -50); + assertGeometryEquals(expected, multilinesGeoJson); + + // test #3: "envelope" (actually a triangle) with invalid number of coordinates (TopRight, BottomLeft, BottomRight) + multilinesGeoJson = XContentFactory.jsonBuilder().startObject().field("type", "envelope") + .startArray("coordinates") + .startArray().value(50).value(30).endArray() + .startArray().value(-50).value(-30).endArray() + .startArray().value(50).value(-39).endArray() + .endArray() + .endObject(); + try (XContentParser parser = createParser(multilinesGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, false, false, false)); + assertNull(parser.nextToken()); + } + + // test #4: "envelope" with empty coordinates + multilinesGeoJson = XContentFactory.jsonBuilder().startObject().field("type", "envelope") + .startArray("coordinates") + .endArray() + .endObject(); + try (XContentParser parser = createParser(multilinesGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, false, false, false)); + assertNull(parser.nextToken()); + } + } + + @Override + public void testParsePolygon() throws IOException { + XContentBuilder polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray().value(100.0).value(1.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .startArray().value(101.0).value(0.0).endArray() + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(100.0).value(1.0).endArray() + .endArray() + .endArray() + .endObject(); + + Polygon p = new Polygon( + new LinearRing( + new double[] {1d, 1d, 0d, 0d, 1d}, + new double[] {100d, 101d, 101d, 100d, 100d})); + assertGeometryEquals(p, polygonGeoJson); + } + + public void testParse3DPolygon() throws IOException { + XContentBuilder polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray().value(100.0).value(1.0).value(10.0).endArray() + .startArray().value(101.0).value(1.0).value(10.0).endArray() + .startArray().value(101.0).value(0.0).value(10.0).endArray() + .startArray().value(100.0).value(0.0).value(10.0).endArray() + .startArray().value(100.0).value(1.0).value(10.0).endArray() + .endArray() + .endArray() + .endObject(); + + Polygon expected = new Polygon(new LinearRing( + new double[]{1.0, 1.0, 0.0, 0.0, 1.0}, + new double[]{100.0, 101.0, 101.0, 100.0, 100.0}, + new double[]{10.0, 10.0, 10.0, 10.0, 10.0} + )); + try (XContentParser parser = createParser(polygonGeoJson)) { + parser.nextToken(); + assertEquals(expected, GeoJson.fromXContent(parser, true, false, true)); + } + } + + public void testInvalidDimensionalPolygon() throws IOException { + XContentBuilder polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray().value(100.0).value(1.0).value(10.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .startArray().value(101.0).value(0.0).value(10.0).endArray() + .startArray().value(100.0).value(0.0).value(10.0).endArray() + .startArray().value(100.0).value(1.0).value(10.0).endArray() + .endArray() + .endArray() + .endObject(); + try (XContentParser parser = createParser(polygonGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, true)); + assertNull(parser.nextToken()); + } + } + + public void testParseInvalidPoint() throws IOException { + // test case 1: create an invalid point object with multipoint data format + XContentBuilder invalidPoint1 = XContentFactory.jsonBuilder() + .startObject() + .field("type", "point") + .startArray("coordinates") + .startArray().value(-74.011).value(40.753).endArray() + .endArray() + .endObject(); + try (XContentParser parser = createParser(invalidPoint1)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 2: create an invalid point object with an empty number of coordinates + XContentBuilder invalidPoint2 = XContentFactory.jsonBuilder() + .startObject() + .field("type", "point") + .startArray("coordinates") + .endArray() + .endObject(); + try (XContentParser parser = createParser(invalidPoint2)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + } + + public void testParseInvalidMultipoint() throws IOException { + // test case 1: create an invalid multipoint object with single coordinate + XContentBuilder invalidMultipoint1 = XContentFactory.jsonBuilder() + .startObject() + .field("type", "multipoint") + .startArray("coordinates").value(-74.011).value(40.753).endArray() + .endObject(); + try (XContentParser parser = createParser(invalidMultipoint1)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 2: create an invalid multipoint object with null coordinate + XContentBuilder invalidMultipoint2 = XContentFactory.jsonBuilder() + .startObject() + .field("type", "multipoint") + .startArray("coordinates") + .endArray() + .endObject(); + try (XContentParser parser = createParser(invalidMultipoint2)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 3: create a valid formatted multipoint object with invalid number (0) of coordinates + XContentBuilder invalidMultipoint3 = XContentFactory.jsonBuilder() + .startObject() + .field("type", "multipoint") + .startArray("coordinates") + .startArray().endArray() + .endArray() + .endObject(); + try (XContentParser parser = createParser(invalidMultipoint3)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + } + + public void testParseInvalidDimensionalMultiPolygon() throws IOException { + // test invalid multipolygon (an "accidental" polygon with inner rings outside outer ring) + String multiPolygonGeoJson = Strings.toString(XContentFactory.jsonBuilder() + .startObject() + .field("type", "MultiPolygon") + .startArray("coordinates") + .startArray()//first poly (without holes) + .startArray() + .startArray().value(102.0).value(2.0).endArray() + .startArray().value(103.0).value(2.0).endArray() + .startArray().value(103.0).value(3.0).endArray() + .startArray().value(102.0).value(3.0).endArray() + .startArray().value(102.0).value(2.0).endArray() + .endArray() + .endArray() + .startArray()//second poly (with hole) + .startArray() + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(101.0).value(0.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .startArray().value(100.0).value(1.0).endArray() + .startArray().value(100.0).value(0.0).endArray() + .endArray() + .startArray()//hole + .startArray().value(100.2).value(0.8).endArray() + .startArray().value(100.2).value(0.2).value(10.0).endArray() + .startArray().value(100.8).value(0.2).endArray() + .startArray().value(100.8).value(0.8).endArray() + .startArray().value(100.2).value(0.8).endArray() + .endArray() + .endArray() + .endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, multiPolygonGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + } + + public void testParseInvalidPolygon() throws IOException { + /* + * The following 3 test cases ensure proper error handling of invalid polygons + * per the GeoJSON specification + */ + // test case 1: create an invalid polygon with only 2 points + String invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates") + .startArray() + .startArray().value(-74.011).value(40.753).endArray() + .startArray().value(-75.022).value(41.783).endArray() + .endArray() + .endArray() + .endObject()); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 2: create an invalid polygon with only 1 point + invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates") + .startArray() + .startArray().value(-74.011).value(40.753).endArray() + .endArray() + .endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 3: create an invalid polygon with 0 points + invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates") + .startArray() + .startArray().endArray() + .endArray() + .endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 4: create an invalid polygon with null value points + invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates") + .startArray() + .startArray().nullValue().nullValue().endArray() + .endArray() + .endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 5: create an invalid polygon with 1 invalid LinearRing + invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates") + .nullValue().nullValue() + .endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 6: create an invalid polygon with 0 LinearRings + invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates").endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // test case 7: create an invalid polygon with 0 LinearRings + invalidPoly = Strings.toString(XContentFactory.jsonBuilder().startObject().field("type", "polygon") + .startArray("coordinates") + .startArray().value(-74.011).value(40.753).endArray() + .endArray() + .endObject()); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, invalidPoly)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + } + + public void testParsePolygonWithHole() throws IOException { + XContentBuilder polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray().value(100.0).value(1.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .startArray().value(101.0).value(0.0).endArray() + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(100.0).value(1.0).endArray() + .endArray() + .startArray() + .startArray().value(100.2).value(0.8).endArray() + .startArray().value(100.2).value(0.2).endArray() + .startArray().value(100.8).value(0.2).endArray() + .startArray().value(100.8).value(0.8).endArray() + .startArray().value(100.2).value(0.8).endArray() + .endArray() + .endArray() + .endObject(); + + LinearRing hole = + new LinearRing( + new double[] {0.8d, 0.2d, 0.2d, 0.8d, 0.8d}, new double[] {100.2d, 100.2d, 100.8d, 100.8d, 100.2d}); + Polygon p = + new Polygon(new LinearRing( + new double[] {1d, 1d, 0d, 0d, 1d}, new double[] {100d, 101d, 101d, 100d, 100d}), Collections.singletonList(hole)); + assertGeometryEquals(p, polygonGeoJson); + } + + @Override + public void testParseMultiPoint() throws IOException { + XContentBuilder multiPointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "MultiPoint") + .startArray("coordinates") + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .endArray() + .endObject(); + + assertGeometryEquals(new MultiPoint(Arrays.asList( + new Point(0, 100), + new Point(1, 101))), multiPointGeoJson); + } + + @Override + public void testParseMultiPolygon() throws IOException { + // two polygons; one without hole, one with hole + XContentBuilder multiPolygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "MultiPolygon") + .startArray("coordinates") + .startArray()//first poly (without holes) + .startArray() + .startArray().value(102.0).value(2.0).endArray() + .startArray().value(103.0).value(2.0).endArray() + .startArray().value(103.0).value(3.0).endArray() + .startArray().value(102.0).value(3.0).endArray() + .startArray().value(102.0).value(2.0).endArray() + .endArray() + .endArray() + .startArray()//second poly (with hole) + .startArray() + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(101.0).value(0.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .startArray().value(100.0).value(1.0).endArray() + .startArray().value(100.0).value(0.0).endArray() + .endArray() + .startArray()//hole + .startArray().value(100.2).value(0.8).endArray() + .startArray().value(100.2).value(0.2).endArray() + .startArray().value(100.8).value(0.2).endArray() + .startArray().value(100.8).value(0.8).endArray() + .startArray().value(100.2).value(0.8).endArray() + .endArray() + .endArray() + .endArray() + .endObject(); + + LinearRing hole = new LinearRing( + new double[] {0.8d, 0.2d, 0.2d, 0.8d, 0.8d}, new double[] {100.2d, 100.2d, 100.8d, 100.8d, 100.2d}); + + MultiPolygon polygons = new MultiPolygon(Arrays.asList( + new Polygon(new LinearRing( + new double[] {2d, 2d, 3d, 3d, 2d}, new double[] {102d, 103d, 103d, 102d, 102d})), + new Polygon(new LinearRing( + new double[] {0d, 0d, 1d, 1d, 0d}, new double[] {100d, 101d, 101d, 100d, 100d}), + Collections.singletonList(hole)))); + + assertGeometryEquals(polygons, multiPolygonGeoJson); + } + + @Override + public void testParseGeometryCollection() throws IOException { + XContentBuilder geometryCollectionGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "GeometryCollection") + .startArray("geometries") + .startObject() + .field("type", "LineString") + .startArray("coordinates") + .startArray().value(100.0).value(0.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .endArray() + .endObject() + .startObject() + .field("type", "Point") + .startArray("coordinates").value(102.0).value(2.0).endArray() + .endObject() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray().value(-177.0).value(10.0).endArray() + .startArray().value(176.0).value(15.0).endArray() + .startArray().value(172.0).value(0.0).endArray() + .startArray().value(176.0).value(-15.0).endArray() + .startArray().value(-177.0).value(-10.0).endArray() + .startArray().value(-177.0).value(10.0).endArray() + .endArray() + .endArray() + .endObject() + .endArray() + .endObject(); + + GeometryCollection geometryExpected = new GeometryCollection<> (Arrays.asList( + new Line(new double[] {0d, 1d}, new double[] {100d, 101d}), + new Point(2d, 102d), + new Polygon(new LinearRing( + new double[] {10, 15, 0, -15, -10, 10}, + new double[] {-177, 176, 172, 176, -177, -177} + )) + )); + assertGeometryEquals(geometryExpected, geometryCollectionGeoJson); + } + + public void testThatParserExtractsCorrectTypeAndCoordinatesFromArbitraryJson() throws IOException { + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .startObject("crs") + .field("type", "name") + .startObject("properties") + .field("name", "urn:ogc:def:crs:OGC:1.3:CRS84") + .endObject() + .endObject() + .field("bbox", "foobar") + .field("type", "point") + .field("bubu", "foobar") + .startArray("coordinates").value(100.0).value(0.0).endArray() + .startObject("nested").startArray("coordinates").value(200.0).value(0.0).endArray().endObject() + .startObject("lala").field("type", "NotAPoint").endObject() + .endObject(); + + Point expectedPt = new Point(0, 100); + assertGeometryEquals(expectedPt, pointGeoJson, false); + } + + public void testParseOrientationOption() throws IOException { + // test 1: valid ccw (right handed system) poly not crossing dateline (with 'right' field) + XContentBuilder polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .field("orientation", randomFrom("ccw", "right")) + .startArray("coordinates") + .startArray() + .startArray().value(176.0).value(15.0).endArray() + .startArray().value(-177.0).value(10.0).endArray() + .startArray().value(-177.0).value(-10.0).endArray() + .startArray().value(176.0).value(-15.0).endArray() + .startArray().value(172.0).value(0.0).endArray() + .startArray().value(176.0).value(15.0).endArray() + .endArray() + .startArray() + .startArray().value(-172.0).value(8.0).endArray() + .startArray().value(174.0).value(10.0).endArray() + .startArray().value(-172.0).value(-8.0).endArray() + .startArray().value(-172.0).value(8.0).endArray() + .endArray() + .endArray() + .endObject(); + + Polygon expected = new Polygon( + new LinearRing(new double[]{15.0, 10.0, -10.0, -15.0, 0.0, 15.0}, new double[]{176.0, -177.0, -177.0, 176.0, 172.0, 176.0}), + Collections.singletonList( + new LinearRing(new double[]{8.0, 10.0, -8.0, 8.0}, new double[]{-172.0, 174.0, -172.0, -172.0}) + )); + assertGeometryEquals(expected, polygonGeoJson); + + // test 2: valid cw poly + polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .field("orientation", randomFrom("cw", "left")) + .startArray("coordinates") + .startArray() + .startArray().value(176.0).value(15.0).endArray() + .startArray().value(-177.0).value(10.0).endArray() + .startArray().value(-177.0).value(-10.0).endArray() + .startArray().value(176.0).value(-15.0).endArray() + .startArray().value(172.0).value(0.0).endArray() + .startArray().value(176.0).value(15.0).endArray() + .endArray() + .startArray() + .startArray().value(-172.0).value(8.0).endArray() + .startArray().value(174.0).value(10.0).endArray() + .startArray().value(-172.0).value(-8.0).endArray() + .startArray().value(-172.0).value(8.0).endArray() + .endArray() + .endArray() + .endObject(); + + expected = new Polygon( + new LinearRing(new double[]{15.0, 0.0, -15.0, -10.0, 10.0, 15.0}, new double[]{176.0, 172.0, 176.0, -177.0, -177.0, 176.0}), + Collections.singletonList( + new LinearRing(new double[]{8.0, -8.0, 10.0, 8.0}, new double[]{-172.0, -172.0, 174.0, -172.0}) + )); + assertGeometryEquals(expected, polygonGeoJson); + } + + public void testParseInvalidShapes() throws IOException { + // single dimensions point + XContentBuilder tooLittlePointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Point") + .startArray("coordinates").value(10.0).endArray() + .endObject(); + + try (XContentParser parser = createParser(tooLittlePointGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + + // zero dimensions point + XContentBuilder emptyPointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Point") + .startObject("coordinates").field("foo", "bar").endObject() + .endObject(); + + try (XContentParser parser = createParser(emptyPointGeoJson)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertNull(parser.nextToken()); + } + } + + public void testParseInvalidGeometryCollectionShapes() throws IOException { + // single dimensions point + XContentBuilder invalidPoints = XContentFactory.jsonBuilder() + .startObject() + .startObject("foo") + .field("type", "geometrycollection") + .startArray("geometries") + .startObject() + .field("type", "polygon") + .startArray("coordinates") + .startArray().value("46.6022226498514").value("24.7237442867977").endArray() + .startArray().value("46.6031857243798").value("24.722968774929").endArray() + .endArray() // coordinates + .endObject() + .endArray() // geometries + .endObject() + .endObject(); + try (XContentParser parser = createParser(invalidPoints)) { + parser.nextToken(); // foo + parser.nextToken(); // start object + parser.nextToken(); // start object + expectThrows(XContentParseException.class, () -> GeoJson.fromXContent(parser, true, false, false)); + assertEquals(XContentParser.Token.END_OBJECT, parser.nextToken()); // end of the document + assertNull(parser.nextToken()); // no more elements afterwards + } + } +} diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoJsonSerializationTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonSerializationTests.java new file mode 100644 index 0000000000000..ab6e3242654f5 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonSerializationTests.java @@ -0,0 +1,269 @@ +/* + * 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.geo; + +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.geo.geometry.Circle; +import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.geometry.GeometryCollection; +import org.elasticsearch.geo.geometry.Line; +import org.elasticsearch.geo.geometry.LinearRing; +import org.elasticsearch.geo.geometry.MultiLine; +import org.elasticsearch.geo.geometry.MultiPoint; +import org.elasticsearch.geo.geometry.MultiPolygon; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Polygon; +import org.elasticsearch.geo.geometry.Rectangle; +import org.elasticsearch.test.AbstractXContentTestCase; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.function.Supplier; + +public class GeoJsonSerializationTests extends ESTestCase { + + private static class GeometryWrapper implements ToXContentObject { + + private Geometry geometry; + + GeometryWrapper(Geometry geometry) { + this.geometry = geometry; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return GeoJson.toXContent(geometry, builder, params); + } + + public static GeometryWrapper fromXContent(XContentParser parser) throws IOException { + parser.nextToken(); + return new GeometryWrapper(GeoJson.fromXContent(parser, true, false, true)); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + GeometryWrapper that = (GeometryWrapper) o; + return Objects.equals(geometry, that.geometry); + } + + @Override + public int hashCode() { + return Objects.hash(geometry); + } + } + + + private void xContentTest(Supplier instanceSupplier) throws IOException { + AbstractXContentTestCase.xContentTester( + this::createParser, + () -> new GeometryWrapper(instanceSupplier.get()), + (geometryWrapper, xContentBuilder) -> { + geometryWrapper.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + }, + GeometryWrapper::fromXContent) + .supportsUnknownFields(true) + .test(); + } + + + public void testPoint() throws IOException { + xContentTest(() -> randomPoint(randomBoolean())); + } + + public void testMultiPoint() throws IOException { + xContentTest(() -> randomMultiPoint(randomBoolean())); + } + + public void testLineString() throws IOException { + xContentTest(() -> randomLine(randomBoolean())); + } + + public void testMultiLineString() throws IOException { + xContentTest(() -> randomMultiLine(randomBoolean())); + } + + public void testPolygon() throws IOException { + xContentTest(() -> randomPolygon(randomBoolean())); + } + + public void testMultiPolygon() throws IOException { + xContentTest(() -> randomMultiPolygon(randomBoolean())); + } + + public void testEnvelope() throws IOException { + xContentTest(GeoJsonSerializationTests::randomRectangle); + } + + public void testGeometryCollection() throws IOException { + xContentTest(() -> randomGeometryCollection(randomBoolean())); + } + + public void testCircle() throws IOException { + xContentTest(() -> randomCircle(randomBoolean())); + } + + public static double randomLat() { + return randomDoubleBetween(-90, 90, true); + } + + public static double randomLon() { + return randomDoubleBetween(-180, 180, true); + } + + public static Circle randomCircle(boolean hasAlt) { + if (hasAlt) { + return new Circle(randomDoubleBetween(-90, 90, true), randomDoubleBetween(-180, 180, true), randomDouble(), + randomDoubleBetween(0, 100, false)); + } else { + return new Circle(randomDoubleBetween(-90, 90, true), randomDoubleBetween(-180, 180, true), randomDoubleBetween(0, 100, false)); + } + } + + public static Line randomLine(boolean hasAlts) { + int size = randomIntBetween(2, 10); + double[] lats = new double[size]; + double[] lons = new double[size]; + double[] alts = hasAlts ? new double[size] : null; + for (int i = 0; i < size; i++) { + lats[i] = randomLat(); + lons[i] = randomLon(); + if (hasAlts) { + alts[i] = randomDouble(); + } + } + if (hasAlts) { + return new Line(lats, lons, alts); + } + return new Line(lats, lons); + } + + public static Point randomPoint(boolean hasAlt) { + if (hasAlt) { + return new Point(randomLat(), randomLon(), randomDouble()); + } else { + return new Point(randomLat(), randomLon()); + } + } + + public static MultiPoint randomMultiPoint(boolean hasAlt) { + int size = randomIntBetween(3, 10); + List points = new ArrayList<>(); + for (int i = 0; i < size; i++) { + points.add(randomPoint(hasAlt)); + } + return new MultiPoint(points); + } + + public static MultiLine randomMultiLine(boolean hasAlt) { + int size = randomIntBetween(3, 10); + List lines = new ArrayList<>(); + for (int i = 0; i < size; i++) { + lines.add(randomLine(hasAlt)); + } + return new MultiLine(lines); + } + + public static MultiPolygon randomMultiPolygon(boolean hasAlt) { + int size = randomIntBetween(3, 10); + List polygons = new ArrayList<>(); + for (int i = 0; i < size; i++) { + polygons.add(randomPolygon(hasAlt)); + } + return new MultiPolygon(polygons); + } + + public static LinearRing randomLinearRing(boolean hasAlt) { + int size = randomIntBetween(3, 10); + double[] lats = new double[size + 1]; + double[] lons = new double[size + 1]; + double[] alts; + if (hasAlt) { + alts = new double[size + 1]; + } else { + alts = null; + } + for (int i = 0; i < size; i++) { + lats[i] = randomLat(); + lons[i] = randomLon(); + if (hasAlt) { + alts[i] = randomDouble(); + } + } + lats[size] = lats[0]; + lons[size] = lons[0]; + if (hasAlt) { + alts[size] = alts[0]; + return new LinearRing(lats, lons, alts); + } else { + return new LinearRing(lats, lons); + } + } + + public static Polygon randomPolygon(boolean hasAlt) { + int size = randomIntBetween(0, 10); + List holes = new ArrayList<>(); + for (int i = 0; i < size; i++) { + holes.add(randomLinearRing(hasAlt)); + } + if (holes.size() > 0) { + return new Polygon(randomLinearRing(hasAlt), holes); + } else { + return new Polygon(randomLinearRing(hasAlt)); + } + } + + public static Rectangle randomRectangle() { + double lat1 = randomLat(); + double lat2 = randomLat(); + double minLon = randomLon(); + double maxLon = randomLon(); + return new Rectangle(Math.min(lat1, lat2), Math.max(lat1, lat2), minLon, maxLon); + } + + public static GeometryCollection randomGeometryCollection(boolean hasAlt) { + return randomGeometryCollection(0, hasAlt); + } + + private static GeometryCollection randomGeometryCollection(int level, boolean hasAlt) { + int size = randomIntBetween(1, 10); + List shapes = new ArrayList<>(); + for (int i = 0; i < size; i++) { + @SuppressWarnings("unchecked") Function geometry = randomFrom( + GeoJsonSerializationTests::randomCircle, + GeoJsonSerializationTests::randomLine, + GeoJsonSerializationTests::randomPoint, + GeoJsonSerializationTests::randomPolygon, + hasAlt ? GeoJsonSerializationTests::randomPoint : (b) -> randomRectangle(), + level < 3 ? (b) -> randomGeometryCollection(level + 1, b) : GeoJsonSerializationTests::randomPoint // don't build too deep + ); + shapes.add(geometry.apply(hasAlt)); + } + return new GeometryCollection<>(shapes); + } +} diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeometryParserTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeometryParserTests.java new file mode 100644 index 0000000000000..24ba7780cefd6 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/geo/GeometryParserTests.java @@ -0,0 +1,137 @@ +/* + * 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.geo; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParseException; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.geo.geometry.LinearRing; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Polygon; +import org.elasticsearch.test.ESTestCase; + +/** + * Tests for {@link GeometryParser} + */ +public class GeometryParserTests extends ESTestCase { + + public void testGeoJsonParsing() throws Exception { + + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Point") + .startArray("coordinates").value(100.0).value(0.0).endArray() + .endObject(); + + try (XContentParser parser = createParser(pointGeoJson)) { + parser.nextToken(); + assertEquals(new Point(0, 100), GeometryParser.parse(parser, true, randomBoolean(), randomBoolean())); + } + + XContentBuilder pointGeoJsonWithZ = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Point") + .startArray("coordinates").value(100.0).value(0.0).value(10.0).endArray() + .endObject(); + + try (XContentParser parser = createParser(pointGeoJsonWithZ)) { + parser.nextToken(); + assertEquals(new Point(0, 100, 10.0), GeometryParser.parse(parser, true, randomBoolean(), true)); + } + + + try (XContentParser parser = createParser(pointGeoJsonWithZ)) { + parser.nextToken(); + expectThrows(XContentParseException.class, () -> GeometryParser.parse(parser, true, randomBoolean(), false)); + } + + XContentBuilder polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray().value(100.0).value(1.0).endArray() + .startArray().value(101.0).value(1.0).endArray() + .startArray().value(101.0).value(0.0).endArray() + .startArray().value(100.0).value(0.0).endArray() + .endArray() + .endArray() + .endObject(); + + Polygon p = new Polygon(new LinearRing(new double[] {1d, 1d, 0d, 0d, 1d}, new double[] {100d, 101d, 101d, 100d, 100d})); + try (XContentParser parser = createParser(polygonGeoJson)) { + parser.nextToken(); + // Coerce should automatically close the polygon + assertEquals(p, GeometryParser.parse(parser, true, true, randomBoolean())); + } + + try (XContentParser parser = createParser(polygonGeoJson)) { + parser.nextToken(); + // No coerce - the polygon parsing should fail + expectThrows(XContentParseException.class, () -> GeometryParser.parse(parser, true, false, randomBoolean())); + } + } + + public void testWKTParsing() throws Exception { + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("foo", "Point (100 0)") + .endObject(); + + try (XContentParser parser = createParser(pointGeoJson)) { + parser.nextToken(); // Start object + parser.nextToken(); // Field Name + parser.nextToken(); // Field Value + assertEquals(new Point(0, 100), GeometryParser.parse(parser, true, randomBoolean(), randomBoolean())); + } + } + + public void testNullParsing() throws Exception { + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .nullField("foo") + .endObject(); + + try (XContentParser parser = createParser(pointGeoJson)) { + parser.nextToken(); // Start object + parser.nextToken(); // Field Name + parser.nextToken(); // Field Value + assertNull(GeometryParser.parse(parser, true, randomBoolean(), randomBoolean())); + } + } + + public void testUnsupportedValueParsing() throws Exception { + XContentBuilder pointGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("foo", 42) + .endObject(); + + try (XContentParser parser = createParser(pointGeoJson)) { + parser.nextToken(); // Start object + parser.nextToken(); // Field Name + parser.nextToken(); // Field Value + ElasticsearchParseException ex = expectThrows(ElasticsearchParseException.class, + () -> GeometryParser.parse(parser, true, randomBoolean(), randomBoolean())); + assertEquals("shape must be an object consisting of type and coordinates", ex.getMessage()); + } + } +} From 8a16b9195ede8b14ee48bc6aa27afcd331bb7471 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 29 Apr 2019 19:42:56 +0200 Subject: [PATCH 231/260] Upgrade to Netty 4.1.35 (#41499) * Some fixes and possible performance fixes in the last 3 versions -> upgrading --- buildSrc/version.properties | 2 +- modules/transport-netty4/build.gradle | 1 + .../licenses/netty-buffer-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-buffer-4.1.35.Final.jar.sha1 | 1 + .../transport-netty4/licenses/netty-codec-4.1.32.Final.jar.sha1 | 1 - .../transport-netty4/licenses/netty-codec-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-common-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-common-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.35.Final.jar.sha1 | 1 + plugins/transport-nio/build.gradle | 1 + .../transport-nio/licenses/netty-buffer-4.1.32.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-buffer-4.1.35.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-codec-4.1.32.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-codec-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.35.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-common-4.1.32.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-common-4.1.35.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-handler-4.1.32.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-handler-4.1.35.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-resolver-4.1.32.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-resolver-4.1.35.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.32.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.35.Final.jar.sha1 | 1 + 31 files changed, 17 insertions(+), 15 deletions(-) delete mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.35.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.35.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.35.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-common-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-common-4.1.35.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.35.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.35.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.32.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-common-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-common-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.35.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.32.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.35.Final.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 8b5e54b581c79..8f7911574979d 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -20,7 +20,7 @@ slf4j = 1.6.2 # when updating the JNA version, also update the version in buildSrc/build.gradle jna = 4.5.1 -netty = 4.1.32.Final +netty = 4.1.35.Final joda = 2.10.1 # when updating this version, you need to ensure compatibility with: diff --git a/modules/transport-netty4/build.gradle b/modules/transport-netty4/build.gradle index e8050c637f160..23de6a7f93b4f 100644 --- a/modules/transport-netty4/build.gradle +++ b/modules/transport-netty4/build.gradle @@ -81,6 +81,7 @@ thirdPartyAudit { 'io.netty.internal.tcnative.Library', 'io.netty.internal.tcnative.SSL', 'io.netty.internal.tcnative.SSLContext', + 'io.netty.internal.tcnative.SSLPrivateKeyMethod', // from io.netty.handler.ssl.util.BouncyCastleSelfSignedCertGenerator (netty) 'org.bouncycastle.cert.X509v3CertificateBuilder', diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.32.Final.jar.sha1 deleted file mode 100644 index 111093792d347..0000000000000 --- a/modules/transport-netty4/licenses/netty-buffer-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -046ede57693788181b2cafddc3a5967ed2f621c8 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..6112faf2d0103 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-buffer-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +a244722975cddaef5f9bbd45e7a44d0db5f058d8 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.32.Final.jar.sha1 deleted file mode 100644 index 5830dd05a5027..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8f32bd79c5a16f014a4372ed979dc62b39ede33a \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..811797decc1e8 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +b86f6b9eedbe38d6fa0bbbefa961d566e293e13e \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.32.Final.jar.sha1 deleted file mode 100644 index 6ff945b6c2de4..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0b9218adba7353ad5a75fcb639e4755d64bd6ddf \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..3b0f1f7daa3c5 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +f7a38b0a3ee2fff3d9dd2bb44f5e16140b70b354 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.32.Final.jar.sha1 deleted file mode 100644 index 02dd7ce15b843..0000000000000 --- a/modules/transport-netty4/licenses/netty-common-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e95de4f762606f492328e180c8ad5438565a5e3b \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..26576f8e9ccdd --- /dev/null +++ b/modules/transport-netty4/licenses/netty-common-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +c776487b782046e1399b00cd40c63ef51d26e953 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.32.Final.jar.sha1 deleted file mode 100644 index 06af1850f8cca..0000000000000 --- a/modules/transport-netty4/licenses/netty-handler-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b4e3fa13f219df14a9455cc2111f133374428be0 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..0956313b2aa40 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-handler-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +b23efe31416942718ac46ad00bb3e91e4b3f6ab7 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.32.Final.jar.sha1 deleted file mode 100644 index 58d0dfb949c4c..0000000000000 --- a/modules/transport-netty4/licenses/netty-resolver-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3e0114715cb125a12db8d982b2208e552a91256d \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..cdd335d059196 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-resolver-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +d60c4f4e12f0703dff477c9bf595f37a41ecacbc \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.32.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.32.Final.jar.sha1 deleted file mode 100644 index b248610a88623..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d5e5a8ff9c2bc7d91ddccc536a5aca1a4355bd8b \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.35.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..8f52a39c4f7a0 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +526b2646885c57adb54e2f89b2e2b80bebce3962 \ No newline at end of file diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle index 9e855995c3bfa..9f93d18a0e15e 100644 --- a/plugins/transport-nio/build.gradle +++ b/plugins/transport-nio/build.gradle @@ -120,6 +120,7 @@ thirdPartyAudit { 'io.netty.internal.tcnative.Library', 'io.netty.internal.tcnative.SSL', 'io.netty.internal.tcnative.SSLContext', + 'io.netty.internal.tcnative.SSLPrivateKeyMethod', 'io.netty.internal.tcnative.CertificateCallback', 'io.netty.internal.tcnative.CertificateVerifier', 'io.netty.internal.tcnative.SessionTicketKey', diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.32.Final.jar.sha1 deleted file mode 100644 index 111093792d347..0000000000000 --- a/plugins/transport-nio/licenses/netty-buffer-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -046ede57693788181b2cafddc3a5967ed2f621c8 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..6112faf2d0103 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-buffer-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +a244722975cddaef5f9bbd45e7a44d0db5f058d8 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.32.Final.jar.sha1 deleted file mode 100644 index 5830dd05a5027..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8f32bd79c5a16f014a4372ed979dc62b39ede33a \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..811797decc1e8 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +b86f6b9eedbe38d6fa0bbbefa961d566e293e13e \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.32.Final.jar.sha1 deleted file mode 100644 index 6ff945b6c2de4..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-http-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0b9218adba7353ad5a75fcb639e4755d64bd6ddf \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..3b0f1f7daa3c5 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-http-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +f7a38b0a3ee2fff3d9dd2bb44f5e16140b70b354 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.32.Final.jar.sha1 deleted file mode 100644 index 02dd7ce15b843..0000000000000 --- a/plugins/transport-nio/licenses/netty-common-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e95de4f762606f492328e180c8ad5438565a5e3b \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..26576f8e9ccdd --- /dev/null +++ b/plugins/transport-nio/licenses/netty-common-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +c776487b782046e1399b00cd40c63ef51d26e953 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.32.Final.jar.sha1 deleted file mode 100644 index 06af1850f8cca..0000000000000 --- a/plugins/transport-nio/licenses/netty-handler-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b4e3fa13f219df14a9455cc2111f133374428be0 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..0956313b2aa40 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-handler-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +b23efe31416942718ac46ad00bb3e91e4b3f6ab7 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.32.Final.jar.sha1 deleted file mode 100644 index 58d0dfb949c4c..0000000000000 --- a/plugins/transport-nio/licenses/netty-resolver-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3e0114715cb125a12db8d982b2208e552a91256d \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..cdd335d059196 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-resolver-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +d60c4f4e12f0703dff477c9bf595f37a41ecacbc \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.32.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.32.Final.jar.sha1 deleted file mode 100644 index b248610a88623..0000000000000 --- a/plugins/transport-nio/licenses/netty-transport-4.1.32.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d5e5a8ff9c2bc7d91ddccc536a5aca1a4355bd8b \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.35.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.35.Final.jar.sha1 new file mode 100644 index 0000000000000..8f52a39c4f7a0 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-transport-4.1.35.Final.jar.sha1 @@ -0,0 +1 @@ +526b2646885c57adb54e2f89b2e2b80bebce3962 \ No newline at end of file From 55c36540efd01d7af2d60131f90db3bceb353006 Mon Sep 17 00:00:00 2001 From: Henning Andersen <33268011+henningandersen@users.noreply.github.com> Date: Mon, 29 Apr 2019 19:56:16 +0200 Subject: [PATCH 232/260] SeqNo CAS linearizability (#38561) Add a test that stresses concurrent writes using ifSeqno/ifPrimaryTerm to do CAS style updates. Use linearizability checker to verify linearizability. Linearizability of successful CAS'es is guaranteed. Changed linearizability checker to allow collecting history concurrently. Changed unresponsive network simulation to wake up immediately when network disruption is cleared to ensure tests proceed in a timely manner (and this also seems more likely to provoke issues). --- .../elasticsearch/index/shard/IndexShard.java | 18 +- .../coordination/LinearizabilityChecker.java | 92 +- .../ConcurrentSeqNoVersioningIT.java | 791 ++++++++++++++++++ .../index/shard/IndexShardTestCase.java | 3 + .../test/transport/MockTransportService.java | 48 +- .../test/transport/StubbableTransport.java | 7 +- 6 files changed, 927 insertions(+), 32 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 2476eabf980c6..7d6faa73a9413 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -785,11 +785,25 @@ private Engine.IndexResult index(Engine engine, Engine.Index index) throws IOExc try { if (logger.isTraceEnabled()) { // don't use index.source().utf8ToString() here source might not be valid UTF-8 - logger.trace("index [{}][{}] seq# [{}] allocation-id {}", - index.type(), index.id(), index.seqNo(), routingEntry().allocationId()); + logger.trace("index [{}][{}] seq# [{}] allocation-id [{}] primaryTerm [{}] operationPrimaryTerm [{}] origin [{}]", + index.type(), index.id(), index.seqNo(), routingEntry().allocationId(), index.primaryTerm(), getOperationPrimaryTerm(), + index.origin()); } result = engine.index(index); + if (logger.isTraceEnabled()) { + logger.trace("index-done [{}][{}] seq# [{}] allocation-id [{}] primaryTerm [{}] operationPrimaryTerm [{}] origin [{}] " + + "result-seq# [{}] result-term [{}] failure [{}]", + index.type(), index.id(), index.seqNo(), routingEntry().allocationId(), index.primaryTerm(), getOperationPrimaryTerm(), + index.origin(), result.getSeqNo(), result.getTerm(), result.getFailure()); + } } catch (Exception e) { + if (logger.isTraceEnabled()) { + logger.trace(new ParameterizedMessage( + "index-fail [{}][{}] seq# [{}] allocation-id [{}] primaryTerm [{}] operationPrimaryTerm [{}] origin [{}]", + index.type(), index.id(), index.seqNo(), routingEntry().allocationId(), index.primaryTerm(), getOperationPrimaryTerm(), + index.origin() + ), e); + } indexingOperationListeners.postIndex(shardId, index, e); throw e; } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java b/server/src/test/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java index 38e930828923e..a854d40355739 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/LinearizabilityChecker.java @@ -19,7 +19,10 @@ package org.elasticsearch.cluster.coordination; import com.carrotsearch.hppc.LongObjectHashMap; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import java.util.ArrayList; @@ -32,7 +35,11 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Queue; import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.Function; /** @@ -44,6 +51,8 @@ */ public class LinearizabilityChecker { + private static final Logger logger = LogManager.getLogger(LinearizabilityChecker.class); + /** * Sequential specification of a datatype. Used as input for the linearizability checker. * All parameter and return values should be immutable and have proper equals / hashCode implementations @@ -114,12 +123,17 @@ default Collection> partition(List events) { * Sequence of invocations and responses, recording the run of a concurrent system. */ public static class History { - private final List events; - private int nextId; + private final Queue events; + private AtomicInteger nextId = new AtomicInteger(); public History() { - events = new ArrayList<>(); - nextId = 0; + events = new ConcurrentLinkedQueue<>(); + } + + public History(Collection events) { + this(); + this.events.addAll(events); + this.nextId.set(events.stream().mapToInt(e -> e.id).max().orElse(-1) + 1); } /** @@ -129,7 +143,7 @@ public History() { * @return an id that can be used to record the corresponding response event */ public int invoke(Object input) { - final int id = nextId++; + final int id = nextId.getAndIncrement(); events.add(new Event(EventType.INVOCATION, input, id)); return id; } @@ -153,6 +167,13 @@ public void remove(int id) { events.removeIf(e -> e.id == id); } + /** + * Copy the list of events for external use. + * @return list of events in the order recorded. + */ + public List copyEvents() { + return new ArrayList<>(events); + } /** * Completes the history with response events for invocations that are missing corresponding responses * @@ -177,10 +198,7 @@ public void complete(Function missingResponseGenerator) { @Override public History clone() { - final History history = new History(); - history.events.addAll(events); - history.nextId = nextId; - return history; + return new History(events); } /** @@ -197,6 +215,7 @@ public String toString() { ", nextId=" + nextId + '}'; } + } /** @@ -210,15 +229,16 @@ public String toString() { public boolean isLinearizable(SequentialSpec spec, History history, Function missingResponseGenerator) { history = history.clone(); // clone history before completing it history.complete(missingResponseGenerator); // complete history - final Collection> partitions = spec.partition(history.events); + final Collection> partitions = spec.partition(history.copyEvents()); return partitions.stream().allMatch(h -> isLinearizable(spec, h)); } private boolean isLinearizable(SequentialSpec spec, List history) { + logger.debug("Checking history of size: {}: {}", history.size(), history); Object state = spec.initialState(); // the current state of the datatype final FixedBitSet linearized = new FixedBitSet(history.size() / 2); // the linearized prefix of the history - final Cache cache = new Cache(); + final Cache cache = new Cache(); // cache of explored pairs final Deque> calls = new LinkedList<>(); // path we're currently exploring final Entry headEntry = createLinkedEntries(history); @@ -267,6 +287,54 @@ public boolean isLinearizable(SequentialSpec spec, History history) { }); } + /** + * Return a visual representation of the history + */ + public static String visualize(SequentialSpec spec, History history, Function missingResponseGenerator) { + history = history.clone(); + history.complete(missingResponseGenerator); + final Collection> partitions = spec.partition(history.copyEvents()); + StringBuilder builder = new StringBuilder(); + partitions.forEach(new Consumer>() { + int index = 0; + @Override + public void accept(List events) { + builder.append("Partition " ).append(index++).append("\n"); + builder.append(visualizePartition(events)); + } + }); + + return builder.toString(); + } + + private static String visualizePartition(List events) { + StringBuilder builder = new StringBuilder(); + Entry entry = createLinkedEntries(events).next; + Map, Integer> eventToPosition = new HashMap<>(); + for (Event event : events) { + eventToPosition.put(Tuple.tuple(event.type, event.id), eventToPosition.size()); + } + while (entry != null) { + if (entry.match != null) { + builder.append(visualizeEntry(entry, eventToPosition)).append("\n"); + } + entry = entry.next; + } + return builder.toString(); + } + + private static String visualizeEntry(Entry entry, Map, Integer> eventToPosition) { + String input = String.valueOf(entry.event.value); + String output = String.valueOf(entry.match.event.value); + int id = entry.event.id; + int beginIndex = eventToPosition.get(Tuple.tuple(EventType.INVOCATION, id)); + int endIndex = eventToPosition.get(Tuple.tuple(EventType.RESPONSE, id)); + input = input.substring(0, Math.min(beginIndex + 25, input.length())); + return Strings.padStart(input, beginIndex + 25, ' ') + + " " + Strings.padStart("", endIndex-beginIndex, 'X') + " " + + output + " (" + entry.event.id + ")"; + } + /** * Creates the internal linked data structure used by the linearizability checker. * Generates contiguous internal ids for the events so that they can be efficiently recorded in bit sets. @@ -314,7 +382,7 @@ private static Entry createLinkedEntries(List history) { return first; } - enum EventType { + public enum EventType { INVOCATION, RESPONSE } diff --git a/server/src/test/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java new file mode 100644 index 0000000000000..416f26e81972a --- /dev/null +++ b/server/src/test/java/org/elasticsearch/versioning/ConcurrentSeqNoVersioningIT.java @@ -0,0 +1,791 @@ +/* + * 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.versioning; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.cluster.coordination.LinearizabilityChecker; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.InputStreamStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteable; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.AbstractDisruptionTestCase; +import org.elasticsearch.index.engine.VersionConflictEngineException; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.disruption.ServiceDisruptionScheme; +import org.elasticsearch.test.junit.annotations.TestLogging; + +import java.io.FileInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Base64; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + + +/** + * This test stress tests CAS updates using sequence number based versioning (ifPrimaryTerm/ifSeqNo). + * + *

    The following is a summary of the expected CAS write behaviour of the system:

    + * + *
      + *
    • acknowledged CAS writes are guaranteed to have taken place between invocation and response and cannot be lost. It is + * guaranteed that the previous value had the specified primaryTerm and seqNo
    • + *
    • CAS writes resulting in a VersionConflictEngineException might or might not have taken place or may take place in the future + * provided the primaryTerm and seqNo still matches. The reason we cannot assume it will not take place after receiving the failure + * is that a request can fork into two because of retries on disconnect, and now race against itself. The retry might complete (and do a + * dirty or stale read) before the forked off request gets to execute, and that one might still subsequently succeed. + * + * Such writes are not necessarily fully replicated and can be lost. There is no + * guarantee that the previous value did not have the specified primaryTerm and seqNo
    • + *
    • CAS writes with other exceptions might or might not have taken place. If they have taken place, then after invocation but not + * necessarily before response. Such writes are not necessarily fully replicated and can be lost. + *
    • + *
    + * + * A deeper technical explanation of the behaviour is given here: + * + *
      + *
    • A CAS can fail on its own write in at least two ways. In both cases, the write might have taken place even though we get a + * version conflict response. Even though we might observe the write (by reading (not done in this test) or another CAS write), the + * write could be lost since it is not fully replicated. Details: + *
        + *
      • A write is successfully stored on primary and one replica (r1). Replication to second replica fails, primary is demoted + * and r1 is promoted to primary. The request is repeated on r1, but this time the request fails due to its own write.
      • + *
      • A coordinator sends write to primary, which stores write successfully (and replicates it). Connection is lost before + * response is sent back. Once connection is back, coordinator will retry against either same or new primary, but this time the + * request will fail due to its own write. + *
      • + *
      + *
    • + *
    • A CAS can fail on stale reads. A CAS failure is only checked on the supposedly primary node. However, the primary might not be + * the newest primary (could be isolated or just not have been told yet). So a CAS check is suspect to stale reads (like any + * read) and can thus fail due to reading stale data. Notice that a CAS success is fully replicated and thus guaranteed to not + * suffer from stale (or dirty) reads. + *
    • + *
    • A CAS can fail on a dirty read, i.e., a non-replicated write that ends up being discarded.
    • + *
    • For any other failure, we do not know if the write will succeed after the failure. However, we do know that if we + * subsequently get back a CAS success with seqNo s, any previous failures with ifSeqNo < s will not be able to succeed (but could + * produce dirty writes on a stale primary). + *
    • + *
    • A CAS failure or any other failure can eventually succeed after receiving the failure response due to reroute and retries, + * see above.
    • + *
    • A CAS failure throws a VersionConflictEngineException which does not directly contain the current seqno/primary-term to use for + * the next request. It is contained in the message (and we parse it out in the test), but notice that the numbers given here could be + * stale or dirty, i.e., come from a stale primary or belong to a write that ends up being discarded.
    • + *
    + */ +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, minNumDataNodes = 4, maxNumDataNodes = 6, + transportClientRatio = 0) +@TestLogging("_root:DEBUG,org.elasticsearch.action.bulk:TRACE,org.elasticsearch.action.get:TRACE," + + "org.elasticsearch.discovery:TRACE,org.elasticsearch.action.support.replication:TRACE," + + "org.elasticsearch.cluster.service:TRACE,org.elasticsearch.indices.recovery:TRACE," + + "org.elasticsearch.indices.cluster:TRACE,org.elasticsearch.index.shard:TRACE") +public class ConcurrentSeqNoVersioningIT extends AbstractDisruptionTestCase { + + private static final Pattern EXTRACT_VERSION = Pattern.compile("current document has seqNo \\[(\\d+)\\] and primary term \\[(\\d+)\\]"); + + // Test info: disrupt network for up to 8s in a number of rounds and check that we only get true positive CAS results when running + // multiple threads doing CAS updates. + // Wait up to 1 minute (+10s in thread to ensure it does not time out) for threads to complete previous round before initiating next + // round. + public void testSeqNoCASLinearizability() { + final int disruptTimeSeconds = scaledRandomIntBetween(1, 8); + + assertAcked(prepareCreate("test") + .setSettings(Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1 + randomInt(2)) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(3)) + )); + + ensureGreen(); + + int numberOfKeys = randomIntBetween(1, 10); + + logger.info("--> Indexing initial doc for {} keys", numberOfKeys); + List partitions = + IntStream.range(0, numberOfKeys) + .mapToObj(i -> client().prepareIndex("test", "type", "ID:" + i).setSource("value", -1).get()) + .map(response -> + new Partition(response.getId(), new Version(response.getPrimaryTerm(), response.getSeqNo()))) + .collect(Collectors.toList()); + + int threadCount = randomIntBetween(3, 20); + CyclicBarrier roundBarrier = new CyclicBarrier(threadCount + 1); // +1 for main thread. + + List threads = + IntStream.range(0, threadCount) + .mapToObj(i -> new CASUpdateThread(i, roundBarrier, partitions, disruptTimeSeconds + 1)) + .collect(Collectors.toList()); + + logger.info("--> Starting {} threads", threadCount); + threads.forEach(Thread::start); + + try { + int rounds = randomIntBetween(2, 5); + + logger.info("--> Running {} rounds", rounds); + + for (int i = 0; i < rounds; ++i) { + ServiceDisruptionScheme disruptionScheme = addRandomDisruptionScheme(); + roundBarrier.await(1, TimeUnit.MINUTES); + disruptionScheme.startDisrupting(); + logger.info("--> round {}", i); + try { + roundBarrier.await(disruptTimeSeconds, TimeUnit.SECONDS); + } catch (TimeoutException e) { + roundBarrier.reset(); + } + internalCluster().clearDisruptionScheme(false); + // heal cluster faster to reduce test time. + ensureFullyConnectedCluster(); + } + } catch (InterruptedException | BrokenBarrierException | TimeoutException e) { + logger.error("Timed out, dumping stack traces of all threads:"); + threads.forEach( + thread -> logger.info(thread.toString() + ":\n" + ExceptionsHelper.formatStackTrace(thread.getStackTrace()))); + throw new RuntimeException(e); + } finally { + logger.info("--> terminating test"); + threads.forEach(CASUpdateThread::terminate); + threads.forEach(CASUpdateThread::await); + threads.stream().filter(Thread::isAlive).forEach(t -> fail("Thread still alive: " + t)); + } + + partitions.forEach(Partition::assertLinearizable); + } + + + private class CASUpdateThread extends Thread { + private final CyclicBarrier roundBarrier; + private final List partitions; + private final int timeout; + + private volatile boolean stop; + private final Random random = new Random(randomLong()); + + private CASUpdateThread(int threadNum, CyclicBarrier roundBarrier, List partitions, int timeout) { + super("CAS-Update-" + threadNum); + this.roundBarrier = roundBarrier; + this.partitions = partitions; + this.timeout = timeout; + setDaemon(true); + } + + public void run() { + while (stop == false) { + try { + roundBarrier.await(70, TimeUnit.SECONDS); + + int numberOfUpdates = randomIntBetween(3, 13) * partitions.size(); + for (int i = 0; i < numberOfUpdates; ++i) { + final int keyIndex = random.nextInt(partitions.size()); + final Partition partition = partitions.get(keyIndex); + + final int seqNoChangePct = random.nextInt(100); + + // we use either the latest observed or the latest successful version, to increase chance of getting successful + // CAS'es and races. If we were to use only the latest successful version, any CAS fail on own write would mean that + // all future CAS'es would fail unless we guess the seqno/term below. On the other hand, using latest observed + // version exclusively we risk a single CAS fail on a dirty read to cause the same. Doing both randomly and adding + // variance to seqno/term should ensure we progress fine in most runs. + Version version = random.nextBoolean() ? partition.latestObservedVersion() : partition.latestSuccessfulVersion(); + + if (seqNoChangePct < 10) { + version = version.nextSeqNo(random.nextInt(4) + 1); + } else if (seqNoChangePct < 15) { + version = version.previousSeqNo(random.nextInt(4) + 1); + } + + final int termChangePct = random.nextInt(100); + if (termChangePct < 5) { + version = version.nextTerm(); + } else if (termChangePct < 10) { + version = version.previousTerm(); + } + + IndexRequest indexRequest = new IndexRequest("test", "type", partition.id) + .source("value", random.nextInt()) + .setIfPrimaryTerm(version.primaryTerm) + .setIfSeqNo(version.seqNo); + Consumer historyResponse = partition.invoke(version); + try { + // we should be able to remove timeout or fail hard on timeouts + IndexResponse indexResponse = client().index(indexRequest).actionGet(timeout, TimeUnit.SECONDS); + IndexResponseHistoryOutput historyOutput = new IndexResponseHistoryOutput(indexResponse); + historyResponse.accept(historyOutput); + // validate version and seqNo strictly increasing for successful CAS to avoid that overhead during + // linearizability checking. + assertThat(historyOutput.outputVersion, greaterThan(version)); + assertThat(historyOutput.outputVersion.seqNo, greaterThan(version.seqNo)); + } catch (VersionConflictEngineException e) { + // if we supplied an input version <= latest successful version, we can safely assume that any failed + // operation will no longer be able to complete after the next successful write and we can therefore terminate + // the operation wrt. linearizability. + // todo: collect the failed responses and terminate when CAS with higher output version is successful, since + // this is the guarantee we offer. + if (version.compareTo(partition.latestSuccessfulVersion()) <= 0) { + historyResponse.accept(new CASFailureHistoryOutput(e)); + } + } catch (RuntimeException e) { + // if we supplied an input version <= to latest successful version, we can safely assume that any failed + // operation will no longer be able to complete after the next successful write and we can therefore terminate + // the operation wrt. linearizability. + // todo: collect the failed responses and terminate when CAS with higher output version is successful, since + // this is the guarantee we offer. + if (version.compareTo(partition.latestSuccessfulVersion()) <= 0) { + historyResponse.accept(new FailureHistoryOutput()); + } + logger.info( + new ParameterizedMessage("Received failure for request [{}], version [{}]", indexRequest, version), + e); + if (stop) { + // interrupt often comes as a RuntimeException so check to stop here too. + return; + } + } + } + } catch (InterruptedException e) { + assert stop : "should only be interrupted when stopped"; + } catch (BrokenBarrierException e) { + // a thread can go here either because it completed before disruption ended, timeout on main thread causes broken + // barrier + } catch (TimeoutException e) { + // this is timeout on the barrier, unexpected. + throw new AssertionError("Unexpected timeout in thread: " + Thread.currentThread(), e); + } + } + } + + public void terminate() { + stop = true; + this.interrupt(); + } + + public void await() { + try { + join(60000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + /** + * Our version, which is primaryTerm,seqNo. + */ + private static final class Version implements NamedWriteable, Comparable { + public final long primaryTerm; + public final long seqNo; + + Version(long primaryTerm, long seqNo) { + this.primaryTerm = primaryTerm; + this.seqNo = seqNo; + } + + Version(StreamInput input) throws IOException { + this.primaryTerm = input.readLong(); + this.seqNo = input.readLong(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Version version = (Version) o; + return primaryTerm == version.primaryTerm && + seqNo == version.seqNo; + } + + @Override + public int hashCode() { + return Objects.hash(primaryTerm, seqNo); + } + + @Override + public int compareTo(Version other) { + int termCompare = Long.compare(primaryTerm, other.primaryTerm); + if (termCompare != 0) + return termCompare; + return Long.compare(seqNo, other.seqNo); + } + + @Override + public String toString() { + return "{" + "primaryTerm=" + primaryTerm + ", seqNo=" + seqNo + '}'; + } + + public Version nextSeqNo(int increment) { + return new Version(primaryTerm, seqNo + increment); + } + + public Version previousSeqNo(int decrement) { + return new Version(primaryTerm, Math.max(seqNo - decrement, 0)); + } + + @Override + public String getWriteableName() { + return "version"; + } + + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(primaryTerm); + out.writeLong(seqNo); + } + + public Version previousTerm() { + return new Version(primaryTerm - 1, seqNo); + } + + public Version nextTerm() { + return new Version(primaryTerm + 1, seqNo); + } + } + + private static class AtomicVersion { + private final AtomicReference current; + + private AtomicVersion(Version initialVersion) { + this.current = new AtomicReference<>(initialVersion); + } + + public Version get() { + return current.get(); + } + + public void consume(Version version) { + if (version == null) + return; + this.current.updateAndGet(current -> version.compareTo(current) <= 0 ? current : version); + } + } + + private class Partition { + private final String id; + private final AtomicVersion latestSuccessfulVersion; + private final AtomicVersion latestObservedVersion; + private final Version initialVersion; + private final LinearizabilityChecker.History history = new LinearizabilityChecker.History(); + + private Partition(String id, Version initialVersion) { + this.id = id; + this.latestSuccessfulVersion = new AtomicVersion(initialVersion); + this.latestObservedVersion = new AtomicVersion(initialVersion); + this.initialVersion = initialVersion; + } + + // latest version that was observed, possibly dirty read of a write that does not survive + public Version latestObservedVersion() { + return latestObservedVersion.get(); + } + + // latest version for which we got a successful response on a write. + public Version latestSuccessfulVersion() { + return latestSuccessfulVersion.get(); + } + + public Consumer invoke(Version version) { + int eventId = history.invoke(version); + logger.debug("invocation partition ({}) event ({}) version ({})", id, eventId, version); + return output -> consumeOutput(output, eventId); + } + + private void consumeOutput(HistoryOutput output, int eventId) { + history.respond(eventId, output); + logger.debug("response partition ({}) event ({}) output ({})", id, eventId, output); + latestObservedVersion.consume(output.getVersion()); + if (output instanceof IndexResponseHistoryOutput) { + latestSuccessfulVersion.consume(output.getVersion()); + } + } + + public boolean isLinearizable() { + logger.info("--> Linearizability checking history of size: {} for key: {} and initialVersion: {}: {}", history.size(), + id, initialVersion, history); + LinearizabilityChecker.SequentialSpec spec = new CASSequentialSpec(initialVersion); + boolean linearizable = new LinearizabilityChecker().isLinearizable(spec, history, missingResponseGenerator()); + // implicitly test that we can serialize all histories. + String serializedHistory = base64Serialize(history); + if (linearizable == false) { + // we dump base64 encoded data, since the nature of this test is that it does not reproduce even with same seed. + logger.error("Linearizability check failed. Spec: {}, initial version: {}, serialized history: {}", spec, initialVersion, + serializedHistory); + } + return linearizable; + } + + public void assertLinearizable() { + assertTrue("Must be linearizable", isLinearizable()); + } + } + + private static class CASSequentialSpec implements LinearizabilityChecker.SequentialSpec { + private final Version initialVersion; + + private CASSequentialSpec(Version initialVersion) { + this.initialVersion = initialVersion; + } + + @Override + public Object initialState() { + return casSuccess(initialVersion); + } + + @Override + public Optional nextState(Object currentState, Object input, Object output) { + State state = (State) currentState; + if (output instanceof IndexResponseHistoryOutput) { + if (input.equals(state.safeVersion) || + (state.lastFailed && ((Version) input).compareTo(state.safeVersion) > 0)) { + return Optional.of(casSuccess(((IndexResponseHistoryOutput) output).getVersion())); + } else { + return Optional.empty(); + } + } else { + return Optional.of(state.failed()); + } + } + } + + private static final class State { + private final Version safeVersion; + private final boolean lastFailed; + + private State(Version safeVersion, boolean lastFailed) { + this.safeVersion = safeVersion; + this.lastFailed = lastFailed; + } + + public State failed() { + return lastFailed ? this : casFail(safeVersion); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + State that = (State) o; + return lastFailed == that.lastFailed && + safeVersion.equals(that.safeVersion); + } + + @Override + public int hashCode() { + return Objects.hash(safeVersion, lastFailed); + } + + @Override + public String toString() { + return "State{" + + "safeVersion=" + safeVersion + + ", lastFailed=" + lastFailed + + '}'; + } + } + + private static State casFail(Version stateVersion) { + return new State(stateVersion, true); + } + + private static State casSuccess(Version version1) { + return new State(version1, false); + } + + /** + * HistoryOutput contains the information from the output of calls. + */ + private interface HistoryOutput extends NamedWriteable { + Version getVersion(); + } + + private static class IndexResponseHistoryOutput implements HistoryOutput { + private final Version outputVersion; + + private IndexResponseHistoryOutput(IndexResponse response) { + this(new Version(response.getPrimaryTerm(), response.getSeqNo())); + } + + private IndexResponseHistoryOutput(StreamInput input) throws IOException { + this(new Version(input)); + } + + private IndexResponseHistoryOutput(Version outputVersion) { + this.outputVersion = outputVersion; + } + + @Override + public Version getVersion() { + return outputVersion; + } + + @Override + public String toString() { + return "Index{" + outputVersion + "}"; + } + + @Override + public String getWriteableName() { + return "index"; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + outputVersion.writeTo(out); + } + } + + /** + * We treat CAS failures (version conflicts) identically to failures in linearizability checker, but keep this separate + * to parse out the latest observed version and to ease debugging. + */ + private static class CASFailureHistoryOutput implements HistoryOutput { + private Version outputVersion; + private CASFailureHistoryOutput(VersionConflictEngineException exception) { + this(parseException(exception.getMessage())); + } + + private CASFailureHistoryOutput(StreamInput input) throws IOException { + this(new Version(input)); + } + + private CASFailureHistoryOutput(Version outputVersion) { + this.outputVersion = outputVersion; + } + + private static Version parseException(String message) { + // parsing out the version increases chance of hitting races against CAS successes, since if we did not parse this out, no + // writes would succeed after a fail on own write failure (unless we were lucky enough to guess the seqNo/primaryTerm using the + // random futureTerm/futureSeqNo handling in CASUpdateThread). + try { + Matcher matcher = EXTRACT_VERSION.matcher(message); + matcher.find(); + return new Version(Long.parseLong(matcher.group(2)), Long.parseLong(matcher.group(1))); + } catch (RuntimeException e) { + throw new RuntimeException("Unable to parse message: " + message, e); + } + } + + @Override + public Version getVersion() { + return outputVersion; + } + + @Override + public String toString() { + return "CASFail{" + outputVersion + "}"; + } + + @Override + public String getWriteableName() { + return "casfail"; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + outputVersion.writeTo(out); + } + } + + /** + * A non version conflict failure. + */ + private static class FailureHistoryOutput implements HistoryOutput { + + private FailureHistoryOutput() { + } + + private FailureHistoryOutput(@SuppressWarnings("unused") StreamInput streamInput) { + } + + @Override + public Version getVersion() { + return null; + } + + @Override + public String toString() { + return "Fail"; + } + + @Override + public String getWriteableName() { + return "fail"; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + // nothing to write. + } + } + + private static Function missingResponseGenerator() { + return input -> new FailureHistoryOutput(); + } + + private String base64Serialize(LinearizabilityChecker.History history) { + BytesStreamOutput output = new BytesStreamOutput(); + try { + List events = history.copyEvents(); + output.writeInt(events.size()); + for (LinearizabilityChecker.Event event : events) { + writeEvent(event, output); + } + output.close(); + return Base64.getEncoder().encodeToString(BytesReference.toBytes(output.bytes())); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static LinearizabilityChecker.History readHistory(StreamInput input) throws IOException { + int size = input.readInt(); + List events = new ArrayList<>(size); + for (int i = 0; i < size; ++i) { + events.add(readEvent(input)); + } + return new LinearizabilityChecker.History(events); + } + + private static void writeEvent(LinearizabilityChecker.Event event, BytesStreamOutput output) throws IOException { + output.writeEnum(event.type); + output.writeNamedWriteable((NamedWriteable) event.value); + output.writeInt(event.id); + } + + private static LinearizabilityChecker.Event readEvent(StreamInput input) throws IOException { + return new LinearizabilityChecker.Event(input.readEnum(LinearizabilityChecker.EventType.class), + input.readNamedWriteable(NamedWriteable.class), input.readInt()); + } + + @SuppressForbidden(reason = "system err is ok for a command line tool") + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("usage: "); + } else { + runLinearizabilityChecker(new FileInputStream(args[0]), Long.parseLong(args[1]), Long.parseLong(args[2])); + } + } + + @SuppressForbidden(reason = "system out is ok for a command line tool") + private static void runLinearizabilityChecker(FileInputStream fileInputStream, long primaryTerm, long seqNo) throws IOException { + StreamInput is = new InputStreamStreamInput(Base64.getDecoder().wrap(fileInputStream)); + is = new NamedWriteableAwareStreamInput(is, createNamedWriteableRegistry()); + + LinearizabilityChecker.History history = readHistory(is); + + Version initialVersion = new Version(primaryTerm, seqNo); + boolean result = + new LinearizabilityChecker().isLinearizable(new CASSequentialSpec(initialVersion), history, + missingResponseGenerator()); + + System.out.println(LinearizabilityChecker.visualize(new CASSequentialSpec(initialVersion), history, + missingResponseGenerator())); + + System.out.println("Linearizable?: " + result); + } + + private static NamedWriteableRegistry createNamedWriteableRegistry() { + return new NamedWriteableRegistry(Arrays.asList( + new NamedWriteableRegistry.Entry(NamedWriteable.class, "version", Version::new), + new NamedWriteableRegistry.Entry(NamedWriteable.class, "index", IndexResponseHistoryOutput::new), + new NamedWriteableRegistry.Entry(NamedWriteable.class, "casfail", CASFailureHistoryOutput::new), + new NamedWriteableRegistry.Entry(NamedWriteable.class, "fail", FailureHistoryOutput::new) + )); + } + + public void testSequentialSpec() { + // Generate 3 increasing versions + Version version1 = new Version(randomIntBetween(1, 5), randomIntBetween(0, 100)); + Version version2 = futureVersion(version1); + Version version3 = futureVersion(version2); + + List versions = List.of(version1, version2, version3); + + LinearizabilityChecker.SequentialSpec spec = new CASSequentialSpec(version1); + + assertThat(spec.initialState(), equalTo(casSuccess(version1))); + + assertThat(spec.nextState(casSuccess(version1), version1, new IndexResponseHistoryOutput(version2)), + equalTo(Optional.of(casSuccess(version2)))); + assertThat(spec.nextState(casFail(version1), version2, new IndexResponseHistoryOutput(version3)), + equalTo(Optional.of(casSuccess(version3)))); + assertThat(spec.nextState(casSuccess(version1), version2, new IndexResponseHistoryOutput(version3)), + equalTo(Optional.empty())); + assertThat(spec.nextState(casSuccess(version2), version1, new IndexResponseHistoryOutput(version3)), + equalTo(Optional.empty())); + assertThat(spec.nextState(casFail(version2), version1, new IndexResponseHistoryOutput(version3)), + equalTo(Optional.empty())); + + // for version conflicts, we keep state version with lastFailed set, regardless of input/output version. + versions.forEach(stateVersion -> + versions.forEach(inputVersion -> + versions.forEach(outputVersion -> { + assertThat(spec.nextState(casSuccess(stateVersion), inputVersion, new CASFailureHistoryOutput(outputVersion)), + equalTo(Optional.of(casFail(stateVersion)))); + assertThat(spec.nextState(casFail(stateVersion), inputVersion, new CASFailureHistoryOutput(outputVersion)), + equalTo(Optional.of(casFail(stateVersion)))); + }) + ) + ); + + // for non version conflict failures, we keep state version with lastFailed set, regardless of input version. + versions.forEach(stateVersion -> + versions.forEach(inputVersion -> { + assertThat(spec.nextState(casSuccess(stateVersion), inputVersion, new FailureHistoryOutput()), + equalTo(Optional.of(casFail(stateVersion)))); + assertThat(spec.nextState(casFail(stateVersion), inputVersion, new FailureHistoryOutput()), + equalTo(Optional.of(casFail(stateVersion)))); + }) + ); + } + + private Version futureVersion(Version version) { + Version futureVersion = version.nextSeqNo(randomIntBetween(1,10)); + if (randomBoolean()) + futureVersion = futureVersion.nextTerm(); + return futureVersion; + } + } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 6a39896199cf2..105ec5415d686 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -721,6 +721,9 @@ protected void assertDocs(IndexShard shard, String... ids) throws IOException { } public static void assertConsistentHistoryBetweenTranslogAndLucene(IndexShard shard) throws IOException { + if (shard.state() != IndexShardState.POST_RECOVERY && shard.state() != IndexShardState.STARTED) { + return; + } final Engine engine = shard.getEngineOrNull(); if (engine != null) { EngineTestCase.assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, shard.mapperService()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index 408748d419354..d7aa6e5903e33 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -68,6 +69,7 @@ import java.util.Queue; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -307,26 +309,38 @@ public void addUnresponsiveRule(TransportAddress transportAddress, final TimeVal Supplier delaySupplier = () -> new TimeValue(duration.millis() - (System.currentTimeMillis() - startTime)); - transport().addConnectBehavior(transportAddress, (transport, discoveryNode, profile, listener) -> { - TimeValue delay = delaySupplier.get(); - if (delay.millis() <= 0) { - return original.openConnection(discoveryNode, profile, listener); - } - - // TODO: Replace with proper setting - TimeValue connectingTimeout = TransportSettings.CONNECT_TIMEOUT.getDefault(Settings.EMPTY); - try { - if (delay.millis() < connectingTimeout.millis()) { - Thread.sleep(delay.millis()); + transport().addConnectBehavior(transportAddress, new StubbableTransport.OpenConnectionBehavior() { + private CountDownLatch stopLatch = new CountDownLatch(1); + @Override + public Releasable openConnection(Transport transport, DiscoveryNode discoveryNode, + ConnectionProfile profile, ActionListener listener) { + TimeValue delay = delaySupplier.get(); + if (delay.millis() <= 0) { return original.openConnection(discoveryNode, profile, listener); - } else { - Thread.sleep(connectingTimeout.millis()); + } + + // TODO: Replace with proper setting + TimeValue connectingTimeout = TransportSettings.CONNECT_TIMEOUT.getDefault(Settings.EMPTY); + try { + if (delay.millis() < connectingTimeout.millis()) { + stopLatch.await(delay.millis(), TimeUnit.MILLISECONDS); + return original.openConnection(discoveryNode, profile, listener); + } else { + stopLatch.await(connectingTimeout.millis(), TimeUnit.MILLISECONDS); + listener.onFailure(new ConnectTransportException(discoveryNode, "UNRESPONSIVE: simulated")); + return () -> { + }; + } + } catch (InterruptedException e) { listener.onFailure(new ConnectTransportException(discoveryNode, "UNRESPONSIVE: simulated")); - return () -> {}; + return () -> { + }; } - } catch (InterruptedException e) { - listener.onFailure(new ConnectTransportException(discoveryNode, "UNRESPONSIVE: simulated")); - return () -> {}; + } + + @Override + public void clearCallback() { + stopLatch.countDown(); } }); diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java b/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java index 4ccc352158a73..1f29739d6284d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/StubbableTransport.java @@ -87,7 +87,10 @@ void clearBehavior(TransportAddress transportAddress) { if (behavior != null) { behavior.clearCallback(); } - connectBehaviors.remove(transportAddress); + OpenConnectionBehavior openConnectionBehavior = connectBehaviors.remove(transportAddress); + if (openConnectionBehavior != null) { + openConnectionBehavior.clearCallback(); + } } Transport getDelegate() { @@ -246,6 +249,8 @@ public interface OpenConnectionBehavior { Releasable openConnection(Transport transport, DiscoveryNode discoveryNode, ConnectionProfile profile, ActionListener listener); + + default void clearCallback() {} } @FunctionalInterface From 40216b4eca96121d85e76c7a0a263bd5e747a170 Mon Sep 17 00:00:00 2001 From: Henning Andersen Date: Mon, 29 Apr 2019 19:25:27 +0200 Subject: [PATCH 233/260] Mute TestClusterPluginIT.testMultiProject. Relates #41256 --- .../elasticsearch/gradle/testclusters/TestClustersPluginIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java b/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java index b06bd067edde8..91a240cab325b 100644 --- a/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java @@ -82,6 +82,7 @@ public void testUseClusterBySkippedAndWorkingTask() { ); } + @Ignore // https://github.com/elastic/elasticsearch/issues/41256 public void testMultiProject() { BuildResult result = getTestClustersRunner( "user1", "user2", "-s", "-i", "--parallel", "-Dlocal.repo.path=" + getLocalTestRepoPath() From 7a407f5609ed996f56cad4b4886cfc587b6d6ee1 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 29 Apr 2019 13:08:35 -0600 Subject: [PATCH 234/260] Handle WRAP ops during SSL read (#41611) It is possible that a WRAP operation can occur while decrypting handshake data in TLS 1.3. The SSLDriver does not currently handle this well as it does not have access to the outbound buffer during read call. This commit moves the buffer into the Driver to fix this issue. Data wrapped during a read call will be queued for writing after the read call is complete. --- .../transport/nio/SSLChannelContext.java | 42 ++++++++------- .../security/transport/nio/SSLDriver.java | 51 ++++++++++--------- .../transport/nio/SSLChannelContextTests.java | 34 ++++++------- .../transport/nio/SSLDriverTests.java | 21 ++++---- 4 files changed, 77 insertions(+), 71 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java index 2c00dd7092950..9372cb1ec54fc 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java @@ -9,17 +9,16 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; +import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; -import org.elasticsearch.nio.Page; import org.elasticsearch.nio.ReadWriteHandler; import org.elasticsearch.nio.SocketChannelContext; -import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.WriteOperation; import javax.net.ssl.SSLEngine; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; +import java.util.LinkedList; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -37,8 +36,7 @@ public final class SSLChannelContext extends SocketChannelContext { private static final Runnable DEFAULT_TIMEOUT_CANCELLER = () -> {}; private final SSLDriver sslDriver; - private final SSLOutboundBuffer outboundBuffer; - private FlushOperation encryptedFlush; + private final LinkedList encryptedFlushes = new LinkedList<>(); private Runnable closeTimeoutCanceller = DEFAULT_TIMEOUT_CANCELLER; SSLChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, SSLDriver sslDriver, @@ -51,14 +49,16 @@ public final class SSLChannelContext extends SocketChannelContext { Predicate allowChannelPredicate) { super(channel, selector, exceptionHandler, readWriteHandler, channelBuffer, allowChannelPredicate); this.sslDriver = sslDriver; - // TODO: When the bytes are actually recycled, we need to test that they are released on context close - this.outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); } @Override public void register() throws IOException { super.register(); sslDriver.init(); + SSLOutboundBuffer outboundBuffer = sslDriver.getOutboundBuffer(); + if (outboundBuffer.hasEncryptedBytesToFlush()) { + encryptedFlushes.addLast(outboundBuffer.buildNetworkFlushOperation()); + } } @Override @@ -98,11 +98,12 @@ public void flushChannel() throws IOException { try { // Attempt to encrypt application write data. The encrypted data ends up in the // outbound write buffer. - sslDriver.write(unencryptedFlush, outboundBuffer); + sslDriver.write(unencryptedFlush); + SSLOutboundBuffer outboundBuffer = sslDriver.getOutboundBuffer(); if (outboundBuffer.hasEncryptedBytesToFlush() == false) { break; } - encryptedFlush = outboundBuffer.buildNetworkFlushOperation(); + encryptedFlushes.addLast(outboundBuffer.buildNetworkFlushOperation()); // Flush the write buffer to the channel flushEncryptedOperation(); } catch (IOException e) { @@ -115,10 +116,11 @@ public void flushChannel() throws IOException { // We are not ready for application writes, check if the driver has non-application writes. We // only want to continue producing new writes if the outbound write buffer is fully flushed. while (pendingChannelFlush() == false && sslDriver.needsNonApplicationWrite()) { - sslDriver.nonApplicationWrite(outboundBuffer); + sslDriver.nonApplicationWrite(); // If non-application writes were produced, flush the outbound write buffer. + SSLOutboundBuffer outboundBuffer = sslDriver.getOutboundBuffer(); if (outboundBuffer.hasEncryptedBytesToFlush()) { - encryptedFlush = outboundBuffer.buildNetworkFlushOperation(); + encryptedFlushes.addFirst(outboundBuffer.buildNetworkFlushOperation()); flushEncryptedOperation(); } } @@ -127,14 +129,14 @@ public void flushChannel() throws IOException { private void flushEncryptedOperation() throws IOException { try { + FlushOperation encryptedFlush = encryptedFlushes.getFirst(); flushToChannel(encryptedFlush); if (encryptedFlush.isFullyFlushed()) { getSelector().executeListener(encryptedFlush.getListener(), null); - encryptedFlush = null; + encryptedFlushes.removeFirst(); } } catch (IOException e) { - getSelector().executeFailedListener(encryptedFlush.getListener(), e); - encryptedFlush = null; + getSelector().executeFailedListener(encryptedFlushes.removeFirst().getListener(), e); throw e; } } @@ -163,6 +165,11 @@ public int read() throws IOException { sslDriver.read(channelBuffer); handleReadBytes(); + // It is possible that a read call produced non-application bytes to flush + SSLOutboundBuffer outboundBuffer = sslDriver.getOutboundBuffer(); + if (outboundBuffer.hasEncryptedBytesToFlush()) { + encryptedFlushes.addLast(outboundBuffer.buildNetworkFlushOperation()); + } return bytesRead; } @@ -190,10 +197,11 @@ public void closeFromSelector() throws IOException { getSelector().assertOnSelectorThread(); if (channel.isOpen()) { closeTimeoutCanceller.run(); - if (encryptedFlush != null) { + for (FlushOperation encryptedFlush : encryptedFlushes) { getSelector().executeFailedListener(encryptedFlush.getListener(), new ClosedChannelException()); } - IOUtils.close(super::closeFromSelector, outboundBuffer::close, sslDriver::close); + encryptedFlushes.clear(); + IOUtils.close(super::closeFromSelector, sslDriver::close); } } @@ -208,7 +216,7 @@ private void channelCloseTimeout() { } private boolean pendingChannelFlush() { - return encryptedFlush != null; + return encryptedFlushes.isEmpty() == false; } private static class CloseNotifyOperation implements WriteOperation { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java index 4dbf1d1f03fdf..bc112dd3a60ad 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLDriver.java @@ -7,6 +7,7 @@ import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.utils.ExceptionsHelper; import javax.net.ssl.SSLEngine; @@ -32,14 +33,14 @@ * * Producing writes for a channel is more complicated. The method {@link #needsNonApplicationWrite()} can be * called to determine if this driver needs to produce more data to advance the handshake or close process. - * If that method returns true, {@link #nonApplicationWrite(SSLOutboundBuffer)} should be called (and the + * If that method returns true, {@link #nonApplicationWrite()} should be called (and the * data produced then flushed to the channel) until no further non-application writes are needed. * * If no non-application writes are needed, {@link #readyForApplicationWrites()} can be called to determine * if the driver is ready to consume application data. (Note: It is possible that * {@link #readyForApplicationWrites()} and {@link #needsNonApplicationWrite()} can both return false if the * driver is waiting on non-application data from the peer.) If the driver indicates it is ready for - * application writes, {@link #write(FlushOperation, SSLOutboundBuffer)} can be called. This method will + * application writes, {@link #write(FlushOperation)} can be called. This method will * encrypt flush operation application data and place it in the outbound buffer for flushing to a channel. * * If you are ready to close the channel {@link #initiateClose()} should be called. After that is called, the @@ -53,6 +54,8 @@ public class SSLDriver implements AutoCloseable { private static final FlushOperation EMPTY_FLUSH_OPERATION = new FlushOperation(EMPTY_BUFFERS, (r, t) -> {}); private final SSLEngine engine; + // TODO: When the bytes are actually recycled, we need to test that they are released on driver close + private final SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); private final boolean isClientMode; // This should only be accessed by the network thread associated with this channel, so nothing needs to // be volatile. @@ -107,6 +110,10 @@ public ByteBuffer getNetworkReadBuffer() { return networkReadBuffer; } + public SSLOutboundBuffer getOutboundBuffer() { + return outboundBuffer; + } + public void read(InboundChannelBuffer buffer) throws SSLException { Mode modePriorToRead; do { @@ -125,14 +132,14 @@ public boolean needsNonApplicationWrite() { return currentMode.needsNonApplicationWrite(); } - public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { - return currentMode.write(applicationBytes, outboundBuffer); + public int write(FlushOperation applicationBytes) throws SSLException { + return currentMode.write(applicationBytes); } - public void nonApplicationWrite(SSLOutboundBuffer outboundBuffer) throws SSLException { + public void nonApplicationWrite() throws SSLException { assert currentMode.isApplication() == false : "Should not be called if driver is in application mode"; if (currentMode.isApplication() == false) { - currentMode.write(EMPTY_FLUSH_OPERATION, outboundBuffer); + currentMode.write(EMPTY_FLUSH_OPERATION); } else { throw new AssertionError("Attempted to non-application write from invalid mode: " + currentMode.modeName()); } @@ -148,6 +155,7 @@ public boolean isClosed() { @Override public void close() throws SSLException { + outboundBuffer.close(); ArrayList closingExceptions = new ArrayList<>(2); closingInternal(); CloseMode closeMode = (CloseMode) this.currentMode; @@ -276,7 +284,7 @@ private interface Mode { void read(InboundChannelBuffer buffer) throws SSLException; - int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException; + int write(FlushOperation applicationBytes) throws SSLException; boolean needsNonApplicationWrite(); @@ -296,10 +304,9 @@ private class HandshakeMode implements Mode { private void startHandshake() throws SSLException { handshakeStatus = engine.getHandshakeStatus(); - if (handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_UNWRAP && - handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_WRAP) { + if (handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_UNWRAP) { try { - handshake(null); + handshake(); } catch (SSLException e) { closingInternal(); throw e; @@ -307,7 +314,7 @@ private void startHandshake() throws SSLException { } } - private void handshake(SSLOutboundBuffer outboundBuffer) throws SSLException { + private void handshake() throws SSLException { boolean continueHandshaking = true; while (continueHandshaking) { switch (handshakeStatus) { @@ -316,15 +323,7 @@ private void handshake(SSLOutboundBuffer outboundBuffer) throws SSLException { continueHandshaking = false; break; case NEED_WRAP: - if (outboundBuffer != null) { - handshakeStatus = wrap(outboundBuffer).getHandshakeStatus(); - // If we need NEED_TASK we should run the tasks immediately - if (handshakeStatus != SSLEngineResult.HandshakeStatus.NEED_TASK) { - continueHandshaking = false; - } - } else { - continueHandshaking = false; - } + handshakeStatus = wrap(outboundBuffer).getHandshakeStatus(); break; case NEED_TASK: runTasks(); @@ -351,7 +350,7 @@ public void read(InboundChannelBuffer buffer) throws SSLException { try { SSLEngineResult result = unwrap(buffer); handshakeStatus = result.getHandshakeStatus(); - handshake(null); + handshake(); // If we are done handshaking we should exit the handshake read continueUnwrap = result.bytesConsumed() > 0 && currentMode.isHandshake(); } catch (SSLException e) { @@ -362,9 +361,9 @@ public void read(InboundChannelBuffer buffer) throws SSLException { } @Override - public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { + public int write(FlushOperation applicationBytes) throws SSLException { try { - handshake(outboundBuffer); + handshake(); } catch (SSLException e) { closingInternal(); throw e; @@ -444,7 +443,7 @@ public void read(InboundChannelBuffer buffer) throws SSLException { } @Override - public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { + public int write(FlushOperation applicationBytes) throws SSLException { boolean continueWrap = true; int totalBytesProduced = 0; while (continueWrap && applicationBytes.isFullyFlushed() == false) { @@ -538,7 +537,7 @@ public void read(InboundChannelBuffer buffer) throws SSLException { } @Override - public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuffer) throws SSLException { + public int write(FlushOperation applicationBytes) throws SSLException { int bytesProduced = 0; if (engine.isOutboundDone() == false) { bytesProduced += wrap(outboundBuffer).bytesProduced(); @@ -549,6 +548,8 @@ public int write(FlushOperation applicationBytes, SSLOutboundBuffer outboundBuff closeInboundAndSwallowPeerDidNotCloseException(); } } + } else { + needToSendClose = false; } return bytesProduced; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java index 893af2140b9b0..dcccb23f1f665 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.Page; import org.elasticsearch.nio.TaskScheduler; import org.elasticsearch.nio.WriteOperation; import org.elasticsearch.test.ESTestCase; @@ -45,6 +46,7 @@ public class SSLChannelContextTests extends ESTestCase { private SocketChannel rawChannel; private SSLChannelContext context; private InboundChannelBuffer channelBuffer; + private SSLOutboundBuffer outboundBuffer; private NioSelector selector; private TaskScheduler nioTimer; private BiConsumer listener; @@ -67,6 +69,7 @@ public void init() { rawChannel = mock(SocketChannel.class); sslDriver = mock(SSLDriver.class); channelBuffer = InboundChannelBuffer.allocatingInstance(); + outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n), () -> {})); when(channel.getRawChannel()).thenReturn(rawChannel); exceptionHandler = mock(Consumer.class); context = new SSLChannelContext(channel, selector, exceptionHandler, sslDriver, readWriteHandler, channelBuffer); @@ -74,6 +77,7 @@ public void init() { when(selector.isOnCurrentThread()).thenReturn(true); when(selector.getTaskScheduler()).thenReturn(nioTimer); when(sslDriver.getNetworkReadBuffer()).thenReturn(readBuffer); + when(sslDriver.getOutboundBuffer()).thenReturn(outboundBuffer); ByteBuffer buffer = ByteBuffer.allocate(1 << 14); when(selector.getIoBuffer()).thenAnswer(invocationOnMock -> { buffer.clear(); @@ -183,7 +187,7 @@ public void testQueuedWritesAreIgnoredWhenNotReadyForAppWrites() { public void testPendingEncryptedFlushMeansWriteInterested() throws Exception { when(sslDriver.readyForApplicationWrites()).thenReturn(false); when(sslDriver.needsNonApplicationWrite()).thenReturn(true, false); - doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(); // Call will put bytes in buffer to flush context.flushChannel(); @@ -208,7 +212,7 @@ public void testNoNonAppWriteInterestInAppMode() { public void testFirstFlushMustFinishForWriteToContinue() throws Exception { when(sslDriver.readyForApplicationWrites()).thenReturn(false); when(sslDriver.needsNonApplicationWrite()).thenReturn(true); - doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(); // First call will put bytes in buffer to flush context.flushChannel(); @@ -217,30 +221,30 @@ public void testFirstFlushMustFinishForWriteToContinue() throws Exception { context.flushChannel(); assertTrue(context.readyForFlush()); - verify(sslDriver, times(1)).nonApplicationWrite(any(SSLOutboundBuffer.class)); + verify(sslDriver, times(1)).nonApplicationWrite(); } public void testNonAppWrites() throws Exception { when(sslDriver.needsNonApplicationWrite()).thenReturn(true, true, false); when(sslDriver.readyForApplicationWrites()).thenReturn(false); - doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(); when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(1); context.flushChannel(); - verify(sslDriver, times(2)).nonApplicationWrite(any(SSLOutboundBuffer.class)); + verify(sslDriver, times(2)).nonApplicationWrite(); verify(rawChannel, times(2)).write(same(selector.getIoBuffer())); } public void testNonAppWritesStopIfBufferNotFullyFlushed() throws Exception { when(sslDriver.needsNonApplicationWrite()).thenReturn(true); when(sslDriver.readyForApplicationWrites()).thenReturn(false); - doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(); when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(0); context.flushChannel(); - verify(sslDriver, times(1)).nonApplicationWrite(any(SSLOutboundBuffer.class)); + verify(sslDriver, times(1)).nonApplicationWrite(); verify(rawChannel, times(1)).write(same(selector.getIoBuffer())); } @@ -250,7 +254,7 @@ public void testQueuedWriteIsFlushedInFlushCall() throws Exception { context.queueWriteOperation(flushOperation); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - doAnswer(getWriteAnswer(10, true)).when(sslDriver).write(eq(flushOperation), any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(10, true)).when(sslDriver).write(eq(flushOperation)); when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(10); context.flushChannel(); @@ -266,7 +270,7 @@ public void testPartialFlush() throws IOException { context.queueWriteOperation(flushOperation); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(eq(flushOperation), any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(eq(flushOperation)); when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(4); context.flushChannel(); @@ -286,7 +290,7 @@ public void testMultipleWritesPartialFlushes() throws IOException { context.queueWriteOperation(flushOperation2); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(any(FlushOperation.class), any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(any(FlushOperation.class)); when(rawChannel.write(same(selector.getIoBuffer()))).thenReturn(5, 5, 2); context.flushChannel(); @@ -303,7 +307,7 @@ public void testWhenIOExceptionThrownListenerIsCalled() throws IOException { IOException exception = new IOException(); when(sslDriver.readyForApplicationWrites()).thenReturn(true); - doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(eq(flushOperation), any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(5, true)).when(sslDriver).write(eq(flushOperation)); when(rawChannel.write(any(ByteBuffer.class))).thenThrow(exception); expectThrows(IOException.class, () -> context.flushChannel()); @@ -314,7 +318,7 @@ public void testWhenIOExceptionThrownListenerIsCalled() throws IOException { public void testWriteIOExceptionMeansChannelReadyToClose() throws Exception { when(sslDriver.readyForApplicationWrites()).thenReturn(false); when(sslDriver.needsNonApplicationWrite()).thenReturn(true); - doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(any(SSLOutboundBuffer.class)); + doAnswer(getWriteAnswer(1, false)).when(sslDriver).nonApplicationWrite(); context.flushChannel(); @@ -406,12 +410,6 @@ public void testRegisterInitiatesDriver() throws IOException { private Answer getWriteAnswer(int bytesToEncrypt, boolean isApp) { return invocationOnMock -> { - SSLOutboundBuffer outboundBuffer; - if (isApp) { - outboundBuffer = (SSLOutboundBuffer) invocationOnMock.getArguments()[1]; - } else { - outboundBuffer = (SSLOutboundBuffer) invocationOnMock.getArguments()[0]; - } ByteBuffer byteBuffer = outboundBuffer.nextWriteBuffer(bytesToEncrypt + 1); for (int i = 0; i < bytesToEncrypt; ++i) { byteBuffer.put((byte) i); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java index 4b86d3223b061..5003d029043e9 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLDriverTests.java @@ -181,7 +181,7 @@ public void testCloseDuringHandshakeJDK11() throws Exception { clientDriver.init(); serverDriver.init(); - assertTrue(clientDriver.needsNonApplicationWrite()); + assertTrue(clientDriver.getOutboundBuffer().hasEncryptedBytesToFlush()); assertFalse(serverDriver.needsNonApplicationWrite()); sendHandshakeMessages(clientDriver, serverDriver); sendHandshakeMessages(serverDriver, clientDriver); @@ -296,12 +296,12 @@ private void normalClose(SSLDriver sendDriver, SSLDriver receiveDriver) throws I } private void sendNonApplicationWrites(SSLDriver sendDriver, SSLDriver receiveDriver) throws SSLException { - SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); + SSLOutboundBuffer outboundBuffer = sendDriver.getOutboundBuffer(); while (sendDriver.needsNonApplicationWrite() || outboundBuffer.hasEncryptedBytesToFlush()) { if (outboundBuffer.hasEncryptedBytesToFlush()) { sendData(outboundBuffer.buildNetworkFlushOperation(), receiveDriver); } else { - sendDriver.nonApplicationWrite(outboundBuffer); + sendDriver.nonApplicationWrite(); } } } @@ -316,7 +316,7 @@ private void handshake(SSLDriver clientDriver, SSLDriver serverDriver, boolean i serverDriver.init(); } - assertTrue(clientDriver.needsNonApplicationWrite()); + assertTrue(clientDriver.getOutboundBuffer().hasEncryptedBytesToFlush()); assertFalse(serverDriver.needsNonApplicationWrite()); sendHandshakeMessages(clientDriver, serverDriver); @@ -331,7 +331,6 @@ private void handshake(SSLDriver clientDriver, SSLDriver serverDriver, boolean i sendHandshakeMessages(clientDriver, serverDriver); assertTrue(clientDriver.isHandshaking()); - assertTrue(serverDriver.isHandshaking()); sendHandshakeMessages(serverDriver, clientDriver); @@ -340,20 +339,20 @@ private void handshake(SSLDriver clientDriver, SSLDriver serverDriver, boolean i } private void sendHandshakeMessages(SSLDriver sendDriver, SSLDriver receiveDriver) throws IOException { - assertTrue(sendDriver.needsNonApplicationWrite()); + assertTrue(sendDriver.needsNonApplicationWrite() || sendDriver.getOutboundBuffer().hasEncryptedBytesToFlush()); - SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); + SSLOutboundBuffer outboundBuffer = sendDriver.getOutboundBuffer(); while (sendDriver.needsNonApplicationWrite() || outboundBuffer.hasEncryptedBytesToFlush()) { if (outboundBuffer.hasEncryptedBytesToFlush()) { sendData(outboundBuffer.buildNetworkFlushOperation(), receiveDriver); receiveDriver.read(genericBuffer); } else { - sendDriver.nonApplicationWrite(outboundBuffer); + sendDriver.nonApplicationWrite(); } } if (receiveDriver.isHandshaking()) { - assertTrue(receiveDriver.needsNonApplicationWrite()); + assertTrue(receiveDriver.needsNonApplicationWrite() || receiveDriver.getOutboundBuffer().hasEncryptedBytesToFlush()); } } @@ -361,12 +360,12 @@ private void sendAppData(SSLDriver sendDriver, SSLDriver receiveDriver, ByteBuff assertFalse(sendDriver.needsNonApplicationWrite()); int bytesToEncrypt = Arrays.stream(message).mapToInt(Buffer::remaining).sum(); - SSLOutboundBuffer outboundBuffer = new SSLOutboundBuffer((n) -> new Page(ByteBuffer.allocate(n))); + SSLOutboundBuffer outboundBuffer = sendDriver.getOutboundBuffer(); FlushOperation flushOperation = new FlushOperation(message, (r, l) -> {}); int bytesEncrypted = 0; while (bytesToEncrypt > bytesEncrypted) { - bytesEncrypted += sendDriver.write(flushOperation, outboundBuffer); + bytesEncrypted += sendDriver.write(flushOperation); sendData(outboundBuffer.buildNetworkFlushOperation(), receiveDriver); } } From 73bfdc4066be080dc4cad1f0521bf6ea14cded93 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 29 Apr 2019 20:57:31 -0400 Subject: [PATCH 235/260] Simplify initialization of max_seq_no of updates (#41161) Today we choose to initialize max_seq_no_of_updates on primaries only so we can deal with a situation where a primary is on an old node (before 6.5) which does not have MUS while replicas on new nodes (6.5+). However, this strategy is quite complex and can lead to bugs (for example #40249) since we have to assign a correct value (not too low) to MSU in all possible situations (before recovering from translog, restoring history on promotion, and handing off relocation). Fortunately, we don't have to deal with this BWC in 7.0+ since all nodes in the cluster should have MSU. This change simplifies the initialization of MSU by always assigning it a correct value in the constructor of Engine regardless of whether it's a replica or primary. Relates #33842 --- .../elasticsearch/index/engine/Engine.java | 27 ++------------ .../index/engine/InternalEngine.java | 36 ++++++++++--------- .../index/engine/ReadOnlyEngine.java | 16 ++++++--- .../elasticsearch/index/shard/IndexShard.java | 23 +----------- .../index/engine/InternalEngineTests.java | 27 +------------- .../index/engine/ReadOnlyEngineTests.java | 2 -- .../index/shard/IndexShardTests.java | 14 ++------ .../index/shard/RefreshListenersTests.java | 1 - .../index/engine/EngineTestCase.java | 1 - .../ccr/index/engine/FollowingEngine.java | 3 +- .../index/engine/FollowingEngineTests.java | 2 -- 11 files changed, 40 insertions(+), 112 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 9bed93c371696..63659126f8438 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -98,7 +98,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -142,16 +141,6 @@ public abstract class Engine implements Closeable { */ protected volatile long lastWriteNanos = System.nanoTime(); - /* - * This marker tracks the max seq_no of either update operations or delete operations have been processed in this engine. - * An index request is considered as an update if it overwrites existing documents with the same docId in the Lucene index. - * This marker is started uninitialized (-2), and the optimization using seq_no will be disabled if this marker is uninitialized. - * The value of this marker never goes backwards, and is updated/changed differently on primary and replica: - * 1. A primary initializes this marker once using the max_seq_no from its history, then advances when processing an update or delete. - * 2. A replica never advances this marker by itself but only inherits from its primary (via advanceMaxSeqNoOfUpdatesOrDeletes). - */ - private final AtomicLong maxSeqNoOfUpdatesOrDeletes = new AtomicLong(UNASSIGNED_SEQ_NO); - protected Engine(EngineConfig engineConfig) { Objects.requireNonNull(engineConfig.getStore(), "Store must be provided to the engine"); @@ -1961,25 +1950,13 @@ public interface TranslogRecoveryRunner { * Moreover, operations that are optimized using the MSU optimization must not be processed twice as this will create duplicates * in Lucene. To avoid this we check the local checkpoint tracker to see if an operation was already processed. * - * @see #reinitializeMaxSeqNoOfUpdatesOrDeletes() * @see #advanceMaxSeqNoOfUpdatesOrDeletes(long) */ - public final long getMaxSeqNoOfUpdatesOrDeletes() { - return maxSeqNoOfUpdatesOrDeletes.get(); - } - - /** - * A primary shard calls this method to re-initialize the max_seq_no_of_updates marker using the - * max_seq_no from Lucene index and translog before replaying the local translog in its local recovery. - */ - public abstract void reinitializeMaxSeqNoOfUpdatesOrDeletes(); + public abstract long getMaxSeqNoOfUpdatesOrDeletes(); /** * A replica shard receives a new max_seq_no_of_updates from its primary shard, then calls this method * to advance this marker to at least the given sequence number. */ - public final void advanceMaxSeqNoOfUpdatesOrDeletes(long seqNo) { - maxSeqNoOfUpdatesOrDeletes.updateAndGet(curr -> Math.max(curr, seqNo)); - assert maxSeqNoOfUpdatesOrDeletes.get() >= seqNo : maxSeqNoOfUpdatesOrDeletes.get() + " < " + seqNo; - } + public abstract void advanceMaxSeqNoOfUpdatesOrDeletes(long maxSeqNoOfUpdatesOnPrimary); } diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 654d31d22671a..bb301bc4addbb 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -47,7 +47,6 @@ import org.apache.lucene.util.InfoStream; import org.elasticsearch.Assertions; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; @@ -146,6 +145,10 @@ public class InternalEngine extends Engine { private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1); private final AtomicLong maxSeenAutoIdTimestamp = new AtomicLong(-1); private final AtomicLong maxSeqNoOfNonAppendOnlyOperations = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + // max_seq_no_of_updates_or_deletes tracks the max seq_no of update or delete operations that have been processed in this engine. + // An index request is considered as an update if it overwrites existing documents with the same docId in the Lucene index. + // The value of this marker never goes backwards, and is tracked/updated differently on primary and replica. + private final AtomicLong maxSeqNoOfUpdatesOrDeletes; private final CounterMetric numVersionLookups = new CounterMetric(); private final CounterMetric numIndexVersionsLookups = new CounterMetric(); // Lucene operations since this engine was opened - not include operations from existing segments. @@ -228,6 +231,7 @@ public InternalEngine(EngineConfig engineConfig) { () -> acquireSearcher("create_local_checkpoint_tracker", SearcherScope.INTERNAL), localCheckpointTrackerSupplier); this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); + maxSeqNoOfUpdatesOrDeletes = new AtomicLong(SequenceNumbers.max(localCheckpointTracker.getMaxSeqNo(), translog.getMaxSeqNo())); success = true; } finally { if (success == false) { @@ -405,7 +409,6 @@ public InternalEngine recoverFromTranslog(TranslogRecoveryRunner translogRecover flushLock.lock(); try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); - assert getMaxSeqNoOfUpdatesOrDeletes() != SequenceNumbers.UNASSIGNED_SEQ_NO : "max_seq_no_of_updates is uninitialized"; if (pendingTranslogRecovery.get() == false) { throw new IllegalStateException("Engine has already been recovered"); } @@ -874,7 +877,7 @@ public IndexResult index(Index index) throws IOException { final boolean toAppend = plan.indexIntoLucene && plan.useLuceneUpdateDocument == false; if (toAppend == false) { - advanceMaxSeqNoOfUpdatesOrDeletes(index.seqNo()); + advanceMaxSeqNoOfUpdatesOrDeletesOnPrimary(index.seqNo()); } } else { markSeqNoAsSeen(index.seqNo()); @@ -981,7 +984,6 @@ protected IndexingStrategy indexingStrategyForOperation(final Index index) throw protected final IndexingStrategy planIndexingAsPrimary(Index index) throws IOException { assert index.origin() == Operation.Origin.PRIMARY : "planing as primary but origin isn't. got " + index.origin(); - assert getMaxSeqNoOfUpdatesOrDeletes() != SequenceNumbers.UNASSIGNED_SEQ_NO : "max_seq_no_of_updates is not initialized"; final IndexingStrategy plan; // resolve an external operation into an internal one which is safe to replay if (canOptimizeAddDocument(index)) { @@ -1322,7 +1324,6 @@ protected boolean assertNonPrimaryOrigin(final Operation operation) { protected final DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOException { assert delete.origin() == Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); - assert getMaxSeqNoOfUpdatesOrDeletes() != SequenceNumbers.UNASSIGNED_SEQ_NO : "max_seq_no_of_updates is not initialized"; // resolve operation from external to internal final VersionValue versionValue = resolveDocVersion(delete, delete.getIfSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); assert incrementVersionLookup(); @@ -2718,13 +2719,22 @@ private void updateAutoIdTimestamp(long newTimestamp, boolean unsafe) { assert maxUnsafeAutoIdTimestamp.get() <= maxSeenAutoIdTimestamp.get(); } + @Override + public long getMaxSeqNoOfUpdatesOrDeletes() { + return maxSeqNoOfUpdatesOrDeletes.get(); + } + + @Override + public void advanceMaxSeqNoOfUpdatesOrDeletes(long maxSeqNoOfUpdatesOnPrimary) { + if (maxSeqNoOfUpdatesOnPrimary == SequenceNumbers.UNASSIGNED_SEQ_NO) { + assert false : "max_seq_no_of_updates on primary is unassigned"; + throw new IllegalArgumentException("max_seq_no_of_updates on primary is unassigned"); + } + this.maxSeqNoOfUpdatesOrDeletes.updateAndGet(curr -> Math.max(curr, maxSeqNoOfUpdatesOnPrimary)); + } + private boolean assertMaxSeqNoOfUpdatesIsAdvanced(Term id, long seqNo, boolean allowDeleted, boolean relaxIfGapInSeqNo) { final long maxSeqNoOfUpdates = getMaxSeqNoOfUpdatesOrDeletes(); - // If the primary is on an old version which does not replicate msu, we need to relax this assertion for that. - if (maxSeqNoOfUpdates == SequenceNumbers.UNASSIGNED_SEQ_NO) { - assert config().getIndexSettings().getIndexVersionCreated().before(Version.V_6_5_0); - return true; - } // We treat a delete on the tombstones on replicas as a regular document, then use updateDocument (not addDocument). if (allowDeleted) { final VersionValue versionValue = versionMap.getVersionForAssert(id.bytes()); @@ -2742,12 +2752,6 @@ private boolean assertMaxSeqNoOfUpdatesIsAdvanced(Term id, long seqNo, boolean a return true; } - @Override - public void reinitializeMaxSeqNoOfUpdatesOrDeletes() { - final long maxSeqNo = SequenceNumbers.max(localCheckpointTracker.getMaxSeqNo(), translog.getMaxSeqNo()); - advanceMaxSeqNoOfUpdatesOrDeletes(maxSeqNo); - } - private static void trimUnsafeCommits(EngineConfig engineConfig) throws IOException { final Store store = engineConfig.getStore(); final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index 777aff88e9dbc..7b47d60437fe1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -456,11 +456,6 @@ public void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) { } - @Override - public void reinitializeMaxSeqNoOfUpdatesOrDeletes() { - advanceMaxSeqNoOfUpdatesOrDeletes(seqNoStats.getMaxSeqNo()); - } - protected void processReaders(IndexReader reader, IndexReader previousReader) { searcherFactory.processReaders(reader, previousReader); } @@ -487,4 +482,15 @@ public Translog.Operation next() { } }; } + + @Override + public long getMaxSeqNoOfUpdatesOrDeletes() { + return seqNoStats.getMaxSeqNo(); + } + + @Override + public void advanceMaxSeqNoOfUpdatesOrDeletes(long maxSeqNoOfUpdatesOnPrimary) { + assert maxSeqNoOfUpdatesOnPrimary <= getMaxSeqNoOfUpdatesOrDeletes() : + maxSeqNoOfUpdatesOnPrimary + ">" + getMaxSeqNoOfUpdatesOrDeletes(); + } } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 7d6faa73a9413..ee67597efe31a 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -532,14 +532,6 @@ public void updateShardState(final ShardRouting newRouting, * the reverted operations on this shard by replaying the translog to avoid losing acknowledged writes. */ final Engine engine = getEngine(); - if (getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO) { - // If the old primary was on an old version that did not replicate the msu, - // we need to bootstrap it manually from its local history. - assert indexSettings.getIndexVersionCreated().before(Version.V_6_5_0); - engine.advanceMaxSeqNoOfUpdatesOrDeletes(seqNoStats().getMaxSeqNo()); - } - // in case we previously reset engine, we need to forward MSU before replaying translog. - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.restoreLocalHistoryFromTranslog((resettingEngine, snapshot) -> runTranslogRecovery(resettingEngine, snapshot, Engine.Operation.Origin.LOCAL_RESET, () -> {})); /* Rolling the translog generation is not strictly needed here (as we will never have collisions between @@ -1411,9 +1403,7 @@ public void openEngineAndRecoverFromTranslog() throws IOException { translogRecoveryStats::incrementRecoveredOperations); }; innerOpenEngineAndTranslog(); - final Engine engine = getEngine(); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); - engine.recoverFromTranslog(translogRecoveryRunner, Long.MAX_VALUE); + getEngine().recoverFromTranslog(translogRecoveryRunner, Long.MAX_VALUE); } /** @@ -2206,12 +2196,6 @@ public void activateWithPrimaryContext(final ReplicationTracker.PrimaryContext p getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint(); synchronized (mutex) { replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex - if (getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO) { - // If the old primary was on an old version that did not replicate the msu, - // we need to bootstrap it manually from its local history. - assert indexSettings.getIndexVersionCreated().before(Version.V_6_5_0); - getEngine().advanceMaxSeqNoOfUpdatesOrDeletes(seqNoStats().getMaxSeqNo()); - } } } @@ -3138,7 +3122,6 @@ public void close() throws IOException { newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig())); onNewEngine(newEngineReference.get()); } - newEngineReference.get().advanceMaxSeqNoOfUpdatesOrDeletes(globalCheckpoint); final Engine.TranslogRecoveryRunner translogRunner = (engine, snapshot) -> runTranslogRecovery( engine, snapshot, Engine.Operation.Origin.LOCAL_RESET, () -> { // TODO: add a dedicate recovery stats for the reset translog @@ -3185,11 +3168,7 @@ public long getMaxSeqNoOfUpdatesOrDeletes() { * @see RecoveryTarget#indexTranslogOperations(List, int, long, long, RetentionLeases, long, ActionListener) */ public void advanceMaxSeqNoOfUpdatesOrDeletes(long seqNo) { - assert seqNo != UNASSIGNED_SEQ_NO - || getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO : - "replica has max_seq_no_of_updates=" + getMaxSeqNoOfUpdatesOrDeletes() + " but primary does not"; getEngine().advanceMaxSeqNoOfUpdatesOrDeletes(seqNo); - assert seqNo <= getMaxSeqNoOfUpdatesOrDeletes() : getMaxSeqNoOfUpdatesOrDeletes() + " < " + seqNo; } /** diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 23763426b3745..ae11500e54e5e 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -683,7 +683,6 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { engine = new InternalEngine(engine.config()); assertTrue(engine.isRecovering()); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); Engine.Searcher searcher = wrapper.wrap(engine.acquireSearcher("test")); assertThat(counter.get(), equalTo(2)); @@ -700,7 +699,6 @@ public void testFlushIsDisabledDuringTranslogRecovery() throws IOException { engine = new InternalEngine(engine.config()); expectThrows(IllegalStateException.class, () -> engine.flush(true, true)); assertTrue(engine.isRecovering()); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertFalse(engine.isRecovering()); doc = testParsedDocument("2", null, testDocumentWithTextField(), SOURCE, null); @@ -732,7 +730,6 @@ public void testTranslogMultipleOperationsSameDocument() throws IOException { IOUtils.close(engine); } try (Engine recoveringEngine = new InternalEngine(engine.config())) { - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); try (Engine.Searcher searcher = recoveringEngine.acquireSearcher("test")) { final TotalHitCountCollector collector = new TotalHitCountCollector(); @@ -768,7 +765,6 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s } }; assertThat(getTranslog(recoveringEngine).stats().getUncommittedOperations(), equalTo(docs)); - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertTrue(committed.get()); } finally { @@ -802,7 +798,6 @@ public void testTranslogRecoveryWithMultipleGenerations() throws IOException { } initialEngine.close(); recoveringEngine = new InternalEngine(initialEngine.config()); - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); try (Engine.Searcher searcher = recoveringEngine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), docs); @@ -837,14 +832,12 @@ public void testRecoveryFromTranslogUpToSeqNo() throws IOException { engine.syncTranslog(); } try (InternalEngine engine = new InternalEngine(config)) { - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertThat(engine.getLocalCheckpoint(), equalTo(maxSeqNo)); assertThat(engine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(maxSeqNo)); } try (InternalEngine engine = new InternalEngine(config)) { long upToSeqNo = randomLongBetween(globalCheckpoint.get(), maxSeqNo); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, upToSeqNo); assertThat(engine.getLocalCheckpoint(), equalTo(upToSeqNo)); assertThat(engine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(upToSeqNo)); @@ -1260,7 +1253,6 @@ public void testSyncedFlushSurvivesEngineRestart() throws IOException { store.associateIndexWithNewTranslog(translogUUID); } engine = new InternalEngine(config); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); } @@ -1280,7 +1272,6 @@ public void testSyncedFlushVanishesOnReplay() throws IOException { EngineConfig config = engine.config(); engine.close(); engine = new InternalEngine(config); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertNull("Sync ID must be gone since we have a document to replay", engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID)); @@ -2378,7 +2369,6 @@ public void testSeqNoAndCheckpoints() throws IOException { } try (InternalEngine recoveringEngine = new InternalEngine(initialEngine.config())) { - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertEquals(primarySeqNo, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo()); @@ -2733,7 +2723,6 @@ public void testCurrentTranslogIDisCommitted() throws IOException { assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); } assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); @@ -2751,7 +2740,6 @@ public void testCurrentTranslogIDisCommitted() throws IOException { Map userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertEquals(2, engine.getTranslog().currentFileGeneration()); assertEquals(0L, engine.getTranslog().stats().getUncommittedOperations()); @@ -2765,7 +2753,6 @@ public void testCurrentTranslogIDisCommitted() throws IOException { Map userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("no changes - nothing to commit", "1", @@ -2872,7 +2859,6 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s } } }) { - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); @@ -2885,7 +2871,6 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s try (InternalEngine engine = new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier))) { - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertVisibleCount(engine, 1); final long committedGen = Long.valueOf( @@ -2954,7 +2939,6 @@ public void testTranslogReplay() throws IOException { engine.close(); // we need to reuse the engine config unless the parser.mappingModified won't work engine = new InternalEngine(copy(engine.config(), inSyncGlobalCheckpointSupplier)); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertVisibleCount(engine, numDocs, false); @@ -3718,7 +3702,6 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { InternalEngine engine = new InternalEngine(configSupplier.apply(store))) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp()); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertEquals(timestamp1, engine.segmentsStats(false, false).getMaxUnsafeAutoIdTimestamp()); final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), @@ -4085,7 +4068,6 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro IOUtils.close(initialEngine); } try (Engine recoveringEngine = new InternalEngine(initialEngine.config())) { - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); recoveringEngine.fillSeqNoGaps(2); assertThat(recoveringEngine.getLocalCheckpoint(), greaterThanOrEqualTo((long) (docs - 1))); @@ -4197,7 +4179,6 @@ protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); } }; - noOpEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); noOpEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get()); final String reason = "filling gaps"; @@ -4433,7 +4414,6 @@ public void testRestoreLocalHistoryFromTranslog() throws IOException { totalTranslogOps = engine.getTranslog().totalOperations(); } try (InternalEngine engine = new InternalEngine(engineConfig)) { - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, globalCheckpoint.get()); engine.restoreLocalHistoryFromTranslog(translogHandler); assertThat(getDocIds(engine, true), equalTo(prevDocs)); @@ -4480,7 +4460,6 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { assertEquals(checkpointOnReplica, replicaEngine.getLocalCheckpoint()); recoveringEngine = new InternalEngine(copy(replicaEngine.config(), globalCheckpoint::get)); assertEquals(numDocsOnReplica, getTranslog(recoveringEngine).stats().getUncommittedOperations()); - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertEquals(maxSeqIDOnReplica, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo()); assertEquals(checkpointOnReplica, recoveringEngine.getLocalCheckpoint()); @@ -4516,7 +4495,6 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { if (flushed) { assertThat(recoveringEngine.getTranslogStats().getUncommittedOperations(), equalTo(0)); } - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertEquals(maxSeqIDOnReplica, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo()); assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpoint()); @@ -4711,7 +4689,6 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s super.commitIndexWriter(writer, translog, syncId); } }) { - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); int numDocs = scaledRandomIntBetween(10, 100); for (int docId = 0; docId < numDocs; docId++) { @@ -5485,8 +5462,7 @@ public void testTrackMaxSeqNoOfUpdatesOrDeletesOnPrimary() throws Exception { engine.close(); Set liveDocIds = new HashSet<>(); engine = new InternalEngine(engine.config()); - assertThat(engine.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(-2L)); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); + assertThat(engine.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(-1L)); int numOps = between(1, 500); for (int i = 0; i < numOps; i++) { long currentMaxSeqNoOfUpdates = engine.getMaxSeqNoOfUpdatesOrDeletes(); @@ -5556,7 +5532,6 @@ public void testRebuildLocalCheckpointTracker() throws Exception { "seq_no=" + op.seqNo() + " max_seq_no=" + tracker.getMaxSeqNo() + "checkpoint=" + tracker.getCheckpoint(), tracker.contains(op.seqNo()), equalTo(seqNosInSafeCommit.contains(op.seqNo()))); } - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); assertThat(getDocIds(engine, true), equalTo(docs)); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java index b689400601dc6..f9437ac9251bf 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -92,7 +92,6 @@ public void testReadOnlyEngine() throws Exception { } // Close and reopen the main engine try (InternalEngine recoveringEngine = new InternalEngine(config)) { - recoveringEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); // the locked down engine should still point to the previous commit assertThat(readOnlyEngine.getLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint())); @@ -224,7 +223,6 @@ public void testRecoverFromTranslogAppliesNoOperations() throws IOException { } try (ReadOnlyEngine readOnlyEngine = new ReadOnlyEngine(config, null , null, true, Function.identity())) { final TranslogHandler translogHandler = new TranslogHandler(xContentRegistry(), config.getIndexSettings()); - readOnlyEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); readOnlyEngine.recoverFromTranslog(translogHandler, randomNonNegativeLong()); assertThat(translogHandler.appliedOperations(), equalTo(0L)); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index bf1bfa668829e..b34f364bbed2c 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1110,11 +1110,9 @@ public void testRestoreLocalHistoryFromTranslogOnPromotion() throws IOException, indexShard.updateGlobalCheckpointOnReplica(globalCheckpointOnReplica, "test"); final long globalCheckpoint = randomLongBetween(UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); - final long currentMaxSeqNoOfUpdates = indexShard.getMaxSeqNoOfUpdatesOrDeletes(); final long maxSeqNoOfUpdatesOrDeletes = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, maxSeqNo); final Set docsBeforeRollback = getShardDocUIDs(indexShard); final CountDownLatch latch = new CountDownLatch(1); - final boolean shouldRollback = Math.max(globalCheckpointOnReplica, globalCheckpoint) < maxSeqNo; randomReplicaOperationPermitAcquisition(indexShard, indexShard.getPendingPrimaryTerm() + 1, globalCheckpoint, @@ -1133,13 +1131,7 @@ public void onFailure(Exception e) { }, ""); latch.await(); - if (shouldRollback) { - assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(Collections.max( - Arrays.asList(maxSeqNoOfUpdatesOrDeletes, globalCheckpoint, globalCheckpointOnReplica)) - )); - } else { - assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(Math.max(maxSeqNoOfUpdatesOrDeletes, currentMaxSeqNoOfUpdates))); - } + assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(maxSeqNo)); final ShardRouting newRouting = indexShard.routingEntry().moveActiveReplicaToPrimary(); final CountDownLatch resyncLatch = new CountDownLatch(1); indexShard.updateShardState( @@ -1154,7 +1146,6 @@ public void onFailure(Exception e) { assertThat(indexShard.getLocalCheckpoint(), equalTo(maxSeqNo)); assertThat(indexShard.seqNoStats().getMaxSeqNo(), equalTo(maxSeqNo)); assertThat(getShardDocUIDs(indexShard), equalTo(docsBeforeRollback)); - // we conservatively roll MSU forward to maxSeqNo during restoreLocalHistory, ideally it should become just currentMaxSeqNoOfUpdates assertThat(indexShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(maxSeqNo)); closeShard(indexShard, false); } @@ -3653,6 +3644,7 @@ public void testSupplyTombstoneDoc() throws Exception { public void testResetEngine() throws Exception { IndexShard shard = newStartedShard(false); indexOnReplicaWithGaps(shard, between(0, 1000), Math.toIntExact(shard.getLocalCheckpoint())); + long maxSeqNoBeforeRollback = shard.seqNoStats().getMaxSeqNo(); final long globalCheckpoint = randomLongBetween(shard.getGlobalCheckpoint(), shard.getLocalCheckpoint()); shard.updateGlobalCheckpointOnReplica(globalCheckpoint, "test"); Set docBelowGlobalCheckpoint = getShardDocUIDs(shard).stream() @@ -3694,7 +3686,7 @@ public void testResetEngine() throws Exception { assertThat(getShardDocUIDs(shard), equalTo(docBelowGlobalCheckpoint)); assertThat(shard.seqNoStats().getMaxSeqNo(), equalTo(globalCheckpoint)); assertThat(shard.translogStats().estimatedNumberOfOperations(), equalTo(translogStats.estimatedNumberOfOperations())); - assertThat(shard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(globalCheckpoint)); + assertThat(shard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(maxSeqNoBeforeRollback)); done.set(true); thread.join(); closeShard(shard, false); diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 0216756e65a84..e264d33ffed61 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -152,7 +152,6 @@ public void onFailedEngine(String reason, @Nullable Exception e) { () -> primaryTerm, EngineTestCase.tombstoneDocSupplier()); engine = new InternalEngine(config); - engine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); engine.recoverFromTranslog((e, s) -> 0, Long.MAX_VALUE); listeners.setCurrentRefreshLocationSupplier(engine::getTranslogLastWriteLocation); } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 2a5b110795628..afa319af7e1cf 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -515,7 +515,6 @@ protected InternalEngine createEngine(@Nullable IndexWriterFactory indexWriterFa } InternalEngine internalEngine = createInternalEngine(indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, config); - internalEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); internalEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); return internalEngine; } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java index bbb0689a8a7e6..619e0a04baf9a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java @@ -133,7 +133,8 @@ protected long generateSeqNoForOperationOnPrimary(final Operation operation) { @Override protected void advanceMaxSeqNoOfUpdatesOrDeletesOnPrimary(long seqNo) { - // ignore, this is not really a primary + assert getMaxSeqNoOfUpdatesOrDeletes() >= seqNo : seqNo + " < " + getMaxSeqNoOfUpdatesOrDeletes(); + super.advanceMaxSeqNoOfUpdatesOrDeletesOnPrimary(seqNo); // extra safe in production code } @Override diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index e3d997886334b..4a56d6370eb91 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -289,7 +289,6 @@ private FollowingEngine createEngine(Store store, EngineConfig config) throws IO store.associateIndexWithNewTranslog(translogUuid); FollowingEngine followingEngine = new FollowingEngine(config); TranslogHandler translogHandler = new TranslogHandler(xContentRegistry(), config.getIndexSettings()); - followingEngine.reinitializeMaxSeqNoOfUpdatesOrDeletes(); followingEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE); return followingEngine; } @@ -495,7 +494,6 @@ private void runFollowTest(CheckedBiConsumer Date: Tue, 30 Apr 2019 09:45:35 +0100 Subject: [PATCH 236/260] Mute CcrRetentionLeaseIT.testForgetFollower https://github.com/elastic/elasticsearch/issues/39850 --- .../java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 2215526942947..33c1428c7e19e 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -922,6 +922,7 @@ public void onResponseReceived( } } + @AwaitsFix( bugUrl = "https://github.com/elastic/elasticsearch/issues/39850") public void testForgetFollower() throws Exception { final String leaderIndex = "leader"; final String followerIndex = "follower"; From b26fe5d212dd49f5862c9ab20057699aa5b8d5c4 Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Tue, 30 Apr 2019 12:01:08 +0300 Subject: [PATCH 237/260] Security Tokens moved to a new separate index (#40742) This commit introduces the `.security-tokens` and `.security-tokens-7` alias-index pair. Because index snapshotting is at the index level granularity (ie you cannot snapshot a subset of an index) snapshoting .`security` had the undesirable effect of storing ephemeral security tokens. The changes herein address this issue by moving tokens "seamlessly" (without user intervention) to another index, so that a "Security Backup" (ie snapshot of `.security`) would not be bloated by ephemeral data. --- .../index/RestrictedIndicesNames.java | 13 +- ...te.json => security-index-template-7.json} | 2 +- .../security-tokens-index-template-7.json | 96 ++ .../authz/store/ReservedRolesStoreTests.java | 22 +- .../xpack/security/Security.java | 19 +- .../xpack/security/authc/ApiKeyService.java | 12 +- .../security/authc/ExpiredApiKeysRemover.java | 4 +- .../security/authc/ExpiredTokenRemover.java | 49 +- .../xpack/security/authc/TokenService.java | 831 +++++++++++------- .../authc/esnative/NativeUsersStore.java | 26 +- .../mapper/NativeRoleMappingStore.java | 15 +- .../authz/store/NativePrivilegeStore.java | 10 +- .../authz/store/NativeRolesStore.java | 18 +- .../support/SecurityIndexManager.java | 80 +- .../integration/ClearRolesCacheTests.java | 4 +- .../test/NativeRealmIntegTestCase.java | 2 +- .../test/SecurityIntegTestCase.java | 8 +- .../xpack/security/SecurityTests.java | 16 +- ...ansportOpenIdConnectLogoutActionTests.java | 2 +- ...sportSamlInvalidateSessionActionTests.java | 8 +- .../saml/TransportSamlLogoutActionTests.java | 2 +- .../TransportCreateTokenActionTests.java | 6 +- .../security/authc/ApiKeyIntegTests.java | 10 +- .../authc/AuthenticationServiceTests.java | 6 +- .../security/authc/TokenAuthIntegTests.java | 17 +- .../security/authc/TokenServiceTests.java | 143 ++- .../esnative/ESNativeMigrateToolTests.java | 6 +- .../authc/esnative/NativeRealmIntegTests.java | 36 +- .../authc/esnative/NativeRealmTests.java | 5 +- .../authc/esnative/NativeUsersStoreTests.java | 8 +- .../mapper/NativeRoleMappingStoreTests.java | 13 +- .../authz/AuthorizationServiceTests.java | 101 ++- .../authz/AuthorizedIndicesTests.java | 27 +- .../authz/IndicesAndAliasesResolverTests.java | 17 +- .../authz/SnapshotUserRoleIntegTests.java | 14 +- .../accesscontrol/IndicesPermissionTests.java | 14 +- .../authz/store/CompositeRolesStoreTests.java | 13 +- .../store/NativePrivilegeStoreTests.java | 21 +- .../authz/store/NativeRolesStoreTests.java | 8 +- .../support/SecurityIndexManagerTests.java | 95 +- .../security/test/SecurityTestUtils.java | 4 +- .../xpack/security/user/XPackUserTests.java | 3 - .../hidden-index/13_security-tokens_read.yml | 161 ++++ .../14_security-tokens-7_read.yml | 161 ++++ x-pack/qa/rolling-upgrade/build.gradle | 4 + .../TokenBackwardsCompatibilityIT.java | 469 ++++++---- .../test/mixed_cluster/50_token_auth.yml | 140 ++- .../test/old_cluster/50_token_auth.yml | 58 +- .../test/upgraded_cluster/50_token_auth.yml | 59 +- .../ldap/AbstractAdLdapRealmTestCase.java | 2 +- 50 files changed, 2021 insertions(+), 839 deletions(-) rename x-pack/plugin/core/src/main/resources/{security-index-template.json => security-index-template-7.json} (99%) create mode 100644 x-pack/plugin/core/src/main/resources/security-tokens-index-template-7.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/13_security-tokens_read.yml create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/14_security-tokens-7_read.yml diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/index/RestrictedIndicesNames.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/index/RestrictedIndicesNames.java index 439168350fc7e..80c17c484739c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/index/RestrictedIndicesNames.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/index/RestrictedIndicesNames.java @@ -14,12 +14,15 @@ import java.util.Set; public final class RestrictedIndicesNames { - public static final String INTERNAL_SECURITY_INDEX_6 = ".security-6"; - public static final String INTERNAL_SECURITY_INDEX_7 = ".security-7"; - public static final String SECURITY_INDEX_NAME = ".security"; + public static final String INTERNAL_SECURITY_MAIN_INDEX_6 = ".security-6"; + public static final String INTERNAL_SECURITY_MAIN_INDEX_7 = ".security-7"; + public static final String SECURITY_MAIN_ALIAS = ".security"; - public static final Set RESTRICTED_NAMES = Collections.unmodifiableSet( - Sets.newHashSet(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX_6, INTERNAL_SECURITY_INDEX_7)); + public static final String INTERNAL_SECURITY_TOKENS_INDEX_7 = ".security-tokens-7"; + public static final String SECURITY_TOKENS_ALIAS = ".security-tokens"; + + public static final Set RESTRICTED_NAMES = Collections.unmodifiableSet(Sets.newHashSet(SECURITY_MAIN_ALIAS, + INTERNAL_SECURITY_MAIN_INDEX_6, INTERNAL_SECURITY_MAIN_INDEX_7, INTERNAL_SECURITY_TOKENS_INDEX_7, SECURITY_TOKENS_ALIAS)); public static final Automaton NAMES_AUTOMATON = Automatons.patterns(RESTRICTED_NAMES); diff --git a/x-pack/plugin/core/src/main/resources/security-index-template.json b/x-pack/plugin/core/src/main/resources/security-index-template-7.json similarity index 99% rename from x-pack/plugin/core/src/main/resources/security-index-template.json rename to x-pack/plugin/core/src/main/resources/security-index-template-7.json index 8d567df5a514f..ebf6d073cd8a6 100644 --- a/x-pack/plugin/core/src/main/resources/security-index-template.json +++ b/x-pack/plugin/core/src/main/resources/security-index-template-7.json @@ -1,5 +1,5 @@ { - "index_patterns" : [ ".security-*" ], + "index_patterns" : [ ".security-7" ], "order" : 1000, "settings" : { "number_of_shards" : 1, diff --git a/x-pack/plugin/core/src/main/resources/security-tokens-index-template-7.json b/x-pack/plugin/core/src/main/resources/security-tokens-index-template-7.json new file mode 100644 index 0000000000000..e7450d0be9c28 --- /dev/null +++ b/x-pack/plugin/core/src/main/resources/security-tokens-index-template-7.json @@ -0,0 +1,96 @@ +{ + "index_patterns" : [ ".security-tokens-7" ], + "order" : 1000, + "settings" : { + "number_of_shards" : 1, + "number_of_replicas" : 0, + "auto_expand_replicas" : "0-1", + "index.priority": 1000, + "index.format": 7 + }, + "mappings" : { + "_doc" : { + "_meta": { + "security-version": "${security.template.version}" + }, + "dynamic" : "strict", + "properties" : { + "doc_type" : { + "type" : "keyword" + }, + "creation_time" : { + "type" : "date", + "format" : "epoch_millis" + }, + "refresh_token" : { + "type" : "object", + "properties" : { + "token" : { + "type" : "keyword" + }, + "refreshed" : { + "type" : "boolean" + }, + "refresh_time": { + "type": "date", + "format": "epoch_millis" + }, + "superseded_by": { + "type": "keyword" + }, + "invalidated" : { + "type" : "boolean" + }, + "client" : { + "type" : "object", + "properties" : { + "type" : { + "type" : "keyword" + }, + "user" : { + "type" : "keyword" + }, + "realm" : { + "type" : "keyword" + } + } + } + } + }, + "access_token" : { + "type" : "object", + "properties" : { + "user_token" : { + "type" : "object", + "properties" : { + "id" : { + "type" : "keyword" + }, + "expiration_time" : { + "type" : "date", + "format" : "epoch_millis" + }, + "version" : { + "type" : "integer" + }, + "metadata" : { + "type" : "object", + "dynamic" : false + }, + "authentication" : { + "type" : "binary" + } + } + }, + "invalidated" : { + "type" : "boolean" + }, + "realm" : { + "type" : "keyword" + } + } + } + } + } + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java index 625e5ddf47c34..78f9623f4fbb8 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java @@ -601,14 +601,14 @@ public void testRemoteMonitoringCollectorRole() { private void assertMonitoringOnRestrictedIndices(Role role) { final Settings indexSettings = Settings.builder().put("index.version.created", Version.CURRENT).build(); - final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); final MetaData metaData = new MetaData.Builder() .put(new IndexMetaData.Builder(internalSecurityIndex) .settings(indexSettings) .numberOfShards(1) .numberOfReplicas(0) - .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_INDEX_NAME).build()) + .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).build()) .build(), true) .build(); final FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(Settings.EMPTY); @@ -616,10 +616,10 @@ private void assertMonitoringOnRestrictedIndices(Role role) { GetSettingsAction.NAME, IndicesShardStoresAction.NAME, UpgradeStatusAction.NAME, RecoveryAction.NAME); for (final String indexMonitoringActionName : indexMonitoringActionNamesList) { final Map authzMap = role.indices().authorize(indexMonitoringActionName, - Sets.newHashSet(internalSecurityIndex, RestrictedIndicesNames.SECURITY_INDEX_NAME), + Sets.newHashSet(internalSecurityIndex, RestrictedIndicesNames.SECURITY_MAIN_ALIAS), metaData.getAliasAndIndexLookup(), fieldPermissionsCache); assertThat(authzMap.get(internalSecurityIndex).isGranted(), is(true)); - assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_INDEX_NAME).isGranted(), is(true)); + assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).isGranted(), is(true)); } } @@ -713,8 +713,8 @@ public void testSuperuserRole() { assertThat(superuserRole.cluster().check("internal:admin/foo", request), is(false)); final Settings indexSettings = Settings.builder().put("index.version.created", Version.CURRENT).build(); - final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); final MetaData metaData = new MetaData.Builder() .put(new IndexMetaData.Builder("a1").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) .put(new IndexMetaData.Builder("a2").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) @@ -731,7 +731,7 @@ public void testSuperuserRole() { .settings(indexSettings) .numberOfShards(1) .numberOfReplicas(0) - .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_INDEX_NAME).build()) + .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).build()) .build(), true) .build(); @@ -753,8 +753,8 @@ public void testSuperuserRole() { assertThat(authzMap.get("aaaaaa").isGranted(), is(true)); assertThat(authzMap.get("b").isGranted(), is(true)); authzMap = superuserRole.indices().authorize(randomFrom(IndexAction.NAME, DeleteIndexAction.NAME, SearchAction.NAME), - Sets.newHashSet(RestrictedIndicesNames.SECURITY_INDEX_NAME), lookup, fieldPermissionsCache); - assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_INDEX_NAME).isGranted(), is(true)); + Sets.newHashSet(RestrictedIndicesNames.SECURITY_MAIN_ALIAS), lookup, fieldPermissionsCache); + assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).isGranted(), is(true)); assertThat(authzMap.get(internalSecurityIndex).isGranted(), is(true)); assertTrue(superuserRole.indices().check(SearchAction.NAME)); assertFalse(superuserRole.indices().check("unknown")); @@ -762,7 +762,7 @@ public void testSuperuserRole() { assertThat(superuserRole.runAs().check(randomAlphaOfLengthBetween(1, 30)), is(true)); assertThat(superuserRole.indices().allowedIndicesMatcher(randomFrom(IndexAction.NAME, DeleteIndexAction.NAME, SearchAction.NAME)) - .test(RestrictedIndicesNames.SECURITY_INDEX_NAME), is(true)); + .test(RestrictedIndicesNames.SECURITY_MAIN_ALIAS), is(true)); assertThat(superuserRole.indices().allowedIndicesMatcher(randomFrom(IndexAction.NAME, DeleteIndexAction.NAME, SearchAction.NAME)) .test(internalSecurityIndex), is(true)); } 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 51dafd433caf5..815993e343cbc 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 @@ -258,9 +258,9 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_FORMAT_SETTING; import static org.elasticsearch.xpack.core.XPackSettings.API_KEY_SERVICE_ENABLED_SETTING; import static org.elasticsearch.xpack.core.XPackSettings.HTTP_SSL_ENABLED; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_INDEX_FORMAT; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_TEMPLATE_NAME; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7; public class Security extends Plugin implements ActionPlugin, IngestPlugin, NetworkPlugin, ClusterPlugin, DiscoveryPlugin, MapperPlugin, ExtensiblePlugin { @@ -406,9 +406,10 @@ Collection createComponents(Client client, ThreadPool threadPool, Cluste components.add(auditTrailService); this.auditTrailService.set(auditTrailService); - securityIndex.set(SecurityIndexManager.buildSecurityIndexManager(client, clusterService)); + securityIndex.set(SecurityIndexManager.buildSecurityMainIndexManager(client, clusterService)); - final TokenService tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex.get(), clusterService); + final TokenService tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex.get(), + SecurityIndexManager.buildSecurityTokensIndexManager(client, clusterService), clusterService); this.tokenService.set(tokenService); components.add(tokenService); @@ -965,7 +966,7 @@ public List> getExecutorBuilders(final Settings settings) { public UnaryOperator> getIndexTemplateMetaDataUpgrader() { return templates -> { // .security index is not managed by using templates anymore - templates.remove(SECURITY_TEMPLATE_NAME); + templates.remove(SECURITY_MAIN_TEMPLATE_7); templates.remove("security_audit_log"); return templates; }; @@ -1031,9 +1032,9 @@ static final class ValidateUpgradedSecurityIndex implements BiConsumerwrap(response -> { @@ -721,7 +721,7 @@ private void findApiKeys(final BoolQueryBuilder boolQuery, boolean filterOutInva expiredQuery.should(QueryBuilders.boolQuery().mustNot(QueryBuilders.existsQuery("expiration_time"))); boolQuery.filter(expiredQuery); } - final SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) + final SearchRequest request = client.prepareSearch(SECURITY_MAIN_ALIAS) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(boolQuery) .setVersion(false) @@ -795,7 +795,7 @@ private void indexInvalidation(Collection apiKeyIds, ActionListener indicesWithTokens = new ArrayList<>(); + if (securityTokensIndex.isAvailable()) { + indicesWithTokens.add(securityTokensIndex.aliasName()); + } + if (securityMainIndex.isAvailable() && checkMainIndexForExpiredTokens) { + indicesWithTokens.add(securityMainIndex.aliasName()); + } + if (indicesWithTokens.isEmpty()) { + return; + } + DeleteByQueryRequest expiredDbq = new DeleteByQueryRequest(indicesWithTokens.toArray(new String[0])); if (timeout != TimeValue.MINUS_ONE) { expiredDbq.setTimeout(timeout); expiredDbq.getSearchRequest().source().timeout(timeout); @@ -59,12 +80,20 @@ public void doRun() { final Instant now = Instant.now(); expiredDbq .setQuery(QueryBuilders.boolQuery() - .filter(QueryBuilders.termsQuery("doc_type", "token")) - .filter(QueryBuilders.rangeQuery("creation_time").lte(now.minus(24L, ChronoUnit.HOURS).toEpochMilli()))); + .filter(QueryBuilders.termsQuery("doc_type", TokenService.TOKEN_DOC_TYPE)) + .filter(QueryBuilders.rangeQuery("creation_time") + .lte(now.minus(MAXIMUM_TOKEN_LIFETIME_HOURS, ChronoUnit.HOURS).toEpochMilli()))); logger.trace(() -> new ParameterizedMessage("Removing old tokens: [{}]", Strings.toString(expiredDbq))); executeAsyncWithOrigin(client, SECURITY_ORIGIN, DeleteByQueryAction.INSTANCE, expiredDbq, - ActionListener.wrap(r -> { - debugDbqResponse(r); + ActionListener.wrap(bulkResponse -> { + debugDbqResponse(bulkResponse); + // tokens can still linger on the main index for their maximum lifetime after the tokens index has been created, because + // only after the tokens index has been created all nodes will store tokens there and not on the main security index + if (checkMainIndexForExpiredTokens && securityTokensIndex.indexExists() + && securityTokensIndex.getCreationTime().isBefore(now.minus(MAXIMUM_TOKEN_LIFETIME_HOURS, ChronoUnit.HOURS)) + && bulkResponse.getBulkFailures().isEmpty() && bulkResponse.getSearchFailures().isEmpty()) { + checkMainIndexForExpiredTokens = false; + } markComplete(); }, this::onFailure)); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java index 19f7a50471043..10236e1719b5c 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java @@ -69,6 +69,7 @@ import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -120,10 +121,12 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -137,7 +140,6 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; import static org.elasticsearch.threadpool.ThreadPool.Names.GENERIC; /** @@ -174,10 +176,15 @@ public final class TokenService { public static final Setting DELETE_TIMEOUT = Setting.timeSetting("xpack.security.authc.token.delete.timeout", TimeValue.MINUS_ONE, Property.NodeScope); - private static final String TOKEN_DOC_TYPE = "token"; + static final String TOKEN_DOC_TYPE = "token"; private static final String TOKEN_DOC_ID_PREFIX = TOKEN_DOC_TYPE + "_"; static final int MINIMUM_BYTES = VERSION_BYTES + SALT_BYTES + IV_BYTES + 1; static final int MINIMUM_BASE64_BYTES = Double.valueOf(Math.ceil((4 * MINIMUM_BYTES) / 3)).intValue(); + static final Version VERSION_TOKENS_INDEX_INTRODUCED = Version.V_8_0_0; // TODO change upon backport + static final Version VERSION_ACCESS_TOKENS_AS_UUIDS = Version.V_7_1_0; + static final Version VERSION_MULTIPLE_CONCURRENT_REFRESHES = Version.V_7_1_0; + // UUIDs are 16 bytes encoded base64 without padding, therefore the length is (16 / 3) * 4 + ((16 % 3) * 8 + 5) / 6 chars + private static final int TOKEN_ID_LENGTH = 22; private static final Logger logger = LogManager.getLogger(TokenService.class); private final SecureRandom secureRandom = new SecureRandom(); @@ -187,7 +194,8 @@ public final class TokenService { private final TimeValue expirationDelay; private final TimeValue deleteInterval; private final Client client; - private final SecurityIndexManager securityIndex; + private final SecurityIndexManager securityMainIndex; + private final SecurityIndexManager securityTokensIndex; private final ExpiredTokenRemover expiredTokenRemover; private final boolean enabled; private volatile TokenKeys keyCache; @@ -196,13 +204,9 @@ public final class TokenService { /** * Creates a new token service - * - * @param settings the node settings - * @param clock the clock that will be used for comparing timestamps - * @param client the client to use when checking for revocations */ - public TokenService(Settings settings, Clock clock, Client client, - SecurityIndexManager securityIndex, ClusterService clusterService) throws GeneralSecurityException { + public TokenService(Settings settings, Clock clock, Client client, SecurityIndexManager securityMainIndex, + SecurityIndexManager securityTokensIndex, ClusterService clusterService) throws GeneralSecurityException { byte[] saltArr = new byte[SALT_BYTES]; secureRandom.nextBytes(saltArr); final SecureString tokenPassphrase = generateTokenKey(); @@ -210,11 +214,12 @@ public TokenService(Settings settings, Clock clock, Client client, this.clock = clock.withZone(ZoneOffset.UTC); this.expirationDelay = TOKEN_EXPIRATION.get(settings); this.client = client; - this.securityIndex = securityIndex; + this.securityMainIndex = securityMainIndex; + this.securityTokensIndex = securityTokensIndex; this.lastExpirationRunMs = client.threadPool().relativeTimeInMillis(); this.deleteInterval = DELETE_INTERVAL.get(settings); this.enabled = isTokenServiceEnabled(settings); - this.expiredTokenRemover = new ExpiredTokenRemover(settings, client); + this.expiredTokenRemover = new ExpiredTokenRemover(settings, client, securityMainIndex, securityTokensIndex); ensureEncryptionCiphersSupported(); KeyAndCache keyAndCache = new KeyAndCache(new KeyAndTimestamp(tokenPassphrase, createdTimeStamps.incrementAndGet()), new BytesKey(saltArr)); @@ -224,10 +229,6 @@ public TokenService(Settings settings, Clock clock, Client client, getTokenMetaData(); } - public static Boolean isTokenServiceEnabled(Settings settings) { - return XPackSettings.TOKEN_SERVICE_ENABLED_SETTING.get(settings); - } - /** * Creates an access token and optionally a refresh token as well, based on the provided authentication and metadata with an * auto-generated token document id. The created tokens are stored in the security index. @@ -235,16 +236,25 @@ public static Boolean isTokenServiceEnabled(Settings settings) { public void createOAuth2Tokens(Authentication authentication, Authentication originatingClientAuth, Map metadata, boolean includeRefreshToken, ActionListener> listener) { - createOAuth2Tokens(UUIDs.randomBase64UUID(), authentication, originatingClientAuth, metadata, includeRefreshToken, listener); + // the created token is compatible with the oldest node version in the cluster + final Version tokenVersion = getTokenVersionCompatibility(); + // tokens moved to a separate index in newer versions + final SecurityIndexManager tokensIndex = getTokensIndexForVersion(tokenVersion); + // the id of the created tokens ought be unguessable + final String userTokenId = UUIDs.randomBase64UUID(); + createOAuth2Tokens(userTokenId, tokenVersion, tokensIndex, authentication, originatingClientAuth, metadata, includeRefreshToken, + listener); } /** * Create an access token and optionally a refresh token as well, based on the provided authentication and metadata, with the given * token document id. The created tokens are be stored in the security index. */ - private void createOAuth2Tokens(String userTokenId, Authentication authentication, Authentication originatingClientAuth, - Map metadata, boolean includeRefreshToken, - ActionListener> listener) { + private void createOAuth2Tokens(String userTokenId, Version tokenVersion, SecurityIndexManager tokensIndex, + Authentication authentication, Authentication originatingClientAuth, Map metadata, + boolean includeRefreshToken, ActionListener> listener) { + assert userTokenId.length() == TOKEN_ID_LENGTH : "We assume token ids have a fixed length for nodes of a certain version." + + " When changing the token length, be careful that the inferences about its length still hold."; ensureEnabled(); if (authentication == null) { listener.onFailure(traceLog("create token", new IllegalArgumentException("authentication must be provided"))); @@ -252,53 +262,32 @@ private void createOAuth2Tokens(String userTokenId, Authentication authenticatio listener.onFailure(traceLog("create token", new IllegalArgumentException("originating client authentication must be provided"))); } else { - final Instant created = clock.instant(); - final Instant expiration = getExpirationTime(created); - final Version version = clusterService.state().nodes().getMinNodeVersion(); final Authentication tokenAuth = new Authentication(authentication.getUser(), authentication.getAuthenticatedBy(), - authentication.getLookedUpBy(), version, AuthenticationType.TOKEN, authentication.getMetadata()); - final UserToken userToken = new UserToken(userTokenId, version, tokenAuth, expiration, metadata); + authentication.getLookedUpBy(), tokenVersion, AuthenticationType.TOKEN, authentication.getMetadata()); + final UserToken userToken = new UserToken(userTokenId, tokenVersion, tokenAuth, getExpirationTime(), metadata); + final String plainRefreshToken = includeRefreshToken ? UUIDs.randomBase64UUID() : null; + final BytesReference tokenDocument = createTokenDocument(userToken, plainRefreshToken, originatingClientAuth); final String documentId = getTokenDocumentId(userToken); - final String refreshToken = includeRefreshToken ? UUIDs.randomBase64UUID() : null; - - final IndexRequest request; - try (XContentBuilder builder = XContentFactory.jsonBuilder()) { - builder.startObject(); - builder.field("doc_type", TOKEN_DOC_TYPE); - builder.field("creation_time", created.toEpochMilli()); - if (includeRefreshToken) { - builder.startObject("refresh_token") - .field("token", refreshToken) - .field("invalidated", false) - .field("refreshed", false) - .startObject("client") - .field("type", "unassociated_client") - .field("user", originatingClientAuth.getUser().principal()) - .field("realm", originatingClientAuth.getAuthenticatedBy().getName()) - .endObject() - .endObject(); - } - builder.startObject("access_token") - .field("invalidated", false) - .field("user_token", userToken) - .field("realm", authentication.getAuthenticatedBy().getName()) - .endObject(); - builder.endObject(); - request = client.prepareIndex(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, documentId) - .setOpType(OpType.CREATE) - .setSource(builder) - .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) - .request(); - } catch (IOException e) { - // unexpected exception - listener.onFailure(e); - return; - } - securityIndex.prepareIndexIfNeededThenExecute(ex -> listener.onFailure(traceLog("prepare security index", documentId, ex)), - () -> executeAsyncWithOrigin(client, SECURITY_ORIGIN, IndexAction.INSTANCE, request, ActionListener - .wrap(indexResponse -> { + final IndexRequest indexTokenRequest = client.prepareIndex(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, documentId) + .setOpType(OpType.CREATE) + .setSource(tokenDocument, XContentType.JSON) + .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) + .request(); + tokensIndex.prepareIndexIfNeededThenExecute( + ex -> listener.onFailure(traceLog("prepare tokens index [" + tokensIndex.aliasName() + "]", documentId, ex)), + () -> executeAsyncWithOrigin(client, SECURITY_ORIGIN, IndexAction.INSTANCE, indexTokenRequest, + ActionListener.wrap(indexResponse -> { if (indexResponse.getResult() == Result.CREATED) { - listener.onResponse(new Tuple<>(userToken, refreshToken)); + if (tokenVersion.onOrAfter(VERSION_TOKENS_INDEX_INTRODUCED)) { + final String versionedRefreshToken = plainRefreshToken != null + ? prependVersionAndEncode(tokenVersion, plainRefreshToken) + : null; + listener.onResponse(new Tuple<>(userToken, versionedRefreshToken)); + } else { + // prior versions are not version-prepended, as nodes on those versions don't expect it. + // Such nodes might exist in a mixed cluster during a rolling upgrade. + listener.onResponse(new Tuple<>(userToken, plainRefreshToken)); + } } else { listener.onFailure(traceLog("create token", new ElasticsearchException("failed to create token document [{}]", indexResponse))); @@ -349,17 +338,19 @@ public void getAuthenticationAndMetaData(String token, ActionListener listener) { - if (securityIndex.isAvailable() == false) { - logger.warn("failed to get access token [{}] because index is not available", userTokenId); + private void getUserTokenFromId(String userTokenId, Version tokenVersion, ActionListener listener) { + final SecurityIndexManager tokensIndex = getTokensIndexForVersion(tokenVersion); + if (tokensIndex.isAvailable() == false) { + logger.warn("failed to get access token [{}] because index [{}] is not available", userTokenId, tokensIndex.aliasName()); listener.onResponse(null); } else { - securityIndex.checkIndexVersionThenExecute( - ex -> listener.onFailure(traceLog("prepare security index", userTokenId, ex)), + tokensIndex.checkIndexVersionThenExecute( + ex -> listener.onFailure(traceLog("prepare tokens index [" + tokensIndex.aliasName() +"]", userTokenId, ex)), () -> { - final GetRequest getRequest = client.prepareGet(SecurityIndexManager.SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, + final GetRequest getRequest = client.prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, getTokenDocumentId(userTokenId)).request(); Consumer onFailure = ex -> listener.onFailure(traceLog("decode token", userTokenId, ex)); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, @@ -386,7 +377,8 @@ void getUserTokenFromId(String userTokenId, ActionListener listener) // if the index or the shard is not there / available we assume that // the token is not valid if (isShardNotAvailableException(e)) { - logger.warn("failed to get access token [{}] because index is not available", userTokenId); + logger.warn("failed to get access token [{}] because index [{}] is not available", userTokenId, + tokensIndex.aliasName()); listener.onResponse(null); } else { logger.error(new ParameterizedMessage("failed to get access token [{}]", userTokenId), e); @@ -397,24 +389,24 @@ void getUserTokenFromId(String userTokenId, ActionListener listener) } } - /* - * If needed, for tokens that were created in a pre 7.1.0 cluster, it asynchronously decodes the token to get the token document Id. - * The process for this is asynchronous as we may need to compute a key, which can be computationally expensive - * so this should not block the current thread, which is typically a network thread. A second reason for being asynchronous is that - * we can restrain the amount of resources consumed by the key computation to a single thread. - * For tokens created in an after 7.1.0 cluster, the token is just the token document Id so this is used directly without decryption + /** + * If needed, for tokens that were created in a pre {@code #VERSION_ACCESS_TOKENS_UUIDS} cluster, it asynchronously decodes the token to + * get the token document id. The process for this is asynchronous as we may need to compute a key, which can be computationally + * expensive so this should not block the current thread, which is typically a network thread. A second reason for being asynchronous is + * that we can restrain the amount of resources consumed by the key computation to a single thread. For tokens created in an after + * {@code #VERSION_ACCESS_TOKENS_UUIDS} cluster, the token is just the token document Id so this is used directly without decryption */ void decodeToken(String token, ActionListener listener) { final byte[] bytes = token.getBytes(StandardCharsets.UTF_8); try (StreamInput in = new InputStreamStreamInput(Base64.getDecoder().wrap(new ByteArrayInputStream(bytes)), bytes.length)) { final Version version = Version.readVersion(in); in.setVersion(version); - if (version.onOrAfter(Version.V_7_1_0)) { - // The token was created in a > 7.1.0 cluster so it contains the tokenId as a String + if (version.onOrAfter(VERSION_ACCESS_TOKENS_AS_UUIDS)) { + // The token was created in a > VERSION_ACCESS_TOKENS_UUIDS cluster so it contains the tokenId as a String String usedTokenId = in.readString(); - getUserTokenFromId(usedTokenId, listener); + getUserTokenFromId(usedTokenId, version, listener); } else { - // The token was created in a < 7.1.0 cluster so we need to decrypt it to get the tokenId + // The token was created in a < VERSION_ACCESS_TOKENS_UUIDS cluster so we need to decrypt it to get the tokenId if (in.available() < MINIMUM_BASE64_BYTES) { logger.debug("invalid token, smaller than [{}] bytes", MINIMUM_BASE64_BYTES); listener.onResponse(null); @@ -433,7 +425,7 @@ void decodeToken(String token, ActionListener listener) { try { final Cipher cipher = getDecryptionCipher(iv, decodeKey, version, decodedSalt); final String tokenId = decryptTokenId(encryptedTokenId, cipher, version); - getUserTokenFromId(tokenId, listener); + getUserTokenFromId(tokenId, version, listener); } catch (IOException | GeneralSecurityException e) { // could happen with a token that is not ours logger.warn("invalid token", e); @@ -461,30 +453,6 @@ void decodeToken(String token, ActionListener listener) { } } - private void getKeyAsync(BytesKey decodedSalt, KeyAndCache keyAndCache, ActionListener listener) { - final SecretKey decodeKey = keyAndCache.getKey(decodedSalt); - if (decodeKey != null) { - listener.onResponse(decodeKey); - } else { - /* As a measure of protected against DOS, we can pass requests requiring a key - * computation off to a single thread executor. For normal usage, the initial - * request(s) that require a key computation will be delayed and there will be - * some additional latency. - */ - client.threadPool().executor(THREAD_POOL_NAME) - .submit(new KeyComputingRunnable(decodedSalt, keyAndCache, listener)); - } - } - - private static String decryptTokenId(byte[] encryptedTokenId, Cipher cipher, Version version) throws IOException { - try (ByteArrayInputStream bais = new ByteArrayInputStream(encryptedTokenId); - CipherInputStream cis = new CipherInputStream(bais, cipher); - StreamInput decryptedInput = new InputStreamStreamInput(cis)) { - decryptedInput.setVersion(version); - return decryptedInput.readString(); - } - } - /** * This method performs the steps necessary to invalidate an access token so that it may no longer be * used. The process of invalidation involves performing an update to the token document and setting @@ -501,7 +469,7 @@ public void invalidateAccessToken(String accessToken, ActionListener listener) { ensureEnabled(); @@ -520,12 +486,12 @@ public void invalidateAccessToken(UserToken userToken, ActionListener backoff = DEFAULT_BACKOFF.iterator(); - indexInvalidation(Collections.singleton(userToken.getId()), backoff, "access_token", null, listener); + indexInvalidation(Collections.singleton(userToken), backoff, "access_token", null, listener); } } /** - * This method onvalidates a refresh token so that it may no longer be used. Iinvalidation involves performing an update to the token + * This method invalidates a refresh token so that it may no longer be used. Invalidation involves performing an update to the token * document and setting the refresh_token.invalidated field to true * * @param refreshToken The string representation of the refresh token @@ -541,8 +507,8 @@ public void invalidateRefreshToken(String refreshToken, ActionListener backoff = DEFAULT_BACKOFF.iterator(); findTokenFromRefreshToken(refreshToken, backoff, ActionListener.wrap(searchResponse -> { - final String docId = getTokenIdFromDocumentId(searchResponse.getId()); - indexInvalidation(Collections.singletonList(docId), backoff, "refresh_token", null, listener); + final Tuple parsedTokens = parseTokensFromDocument(searchResponse.getSourceAsMap(), null); + indexInvalidation(Collections.singletonList(parsedTokens.v1()), backoff, "refresh_token", null, listener); }, listener::onFailure)); } } @@ -568,7 +534,7 @@ public void invalidateActiveTokensForRealmAndUser(@Nullable String realmName, @N logger.warn("No tokens to invalidate for realm [{}] and username [{}]", realmName, username); listener.onResponse(TokensInvalidationResult.emptyResult()); } else { - invalidateAllTokens(tokenTuples.stream().map(t -> t.v1().getId()).collect(Collectors.toList()), listener); + invalidateAllTokens(tokenTuples.stream().map(t -> t.v1()).collect(Collectors.toList()), listener); } }, listener::onFailure)); } else { @@ -581,7 +547,7 @@ public void invalidateActiveTokensForRealmAndUser(@Nullable String realmName, @N logger.warn("No tokens to invalidate for realm [{}] and username [{}]", realmName, username); listener.onResponse(TokensInvalidationResult.emptyResult()); } else { - invalidateAllTokens(tokenTuples.stream().map(t -> t.v1().getId()).collect(Collectors.toList()), listener); + invalidateAllTokens(tokenTuples.stream().map(t -> t.v1()).collect(Collectors.toList()), listener); } }, listener::onFailure)); } @@ -592,34 +558,64 @@ public void invalidateActiveTokensForRealmAndUser(@Nullable String realmName, @N * Invalidates a collection of access_token and refresh_token that were retrieved by * {@link TokenService#invalidateActiveTokensForRealmAndUser} * - * @param accessTokenIds The ids of the access tokens which should be invalidated (along with the respective refresh_token) + * @param userTokens The user tokens for which access and refresh tokens should be invalidated * @param listener the listener to notify upon completion */ - private void invalidateAllTokens(Collection accessTokenIds, ActionListener listener) { + private void invalidateAllTokens(Collection userTokens, ActionListener listener) { maybeStartTokenRemover(); // Invalidate the refresh tokens first so that they cannot be used to get new // access tokens while we invalidate the access tokens we currently know about final Iterator backoff = DEFAULT_BACKOFF.iterator(); - indexInvalidation(accessTokenIds, backoff, "refresh_token", null, ActionListener.wrap(result -> - indexInvalidation(accessTokenIds, backoff, "access_token", result, listener), + indexInvalidation(userTokens, backoff, "refresh_token", null, ActionListener.wrap(result -> + indexInvalidation(userTokens, backoff, "access_token", result, listener), listener::onFailure)); } + /** + * Invalidates access and/or refresh tokens associated to a user token (coexisting in the same token document) + */ + private void indexInvalidation(Collection userTokens, Iterator backoff, String srcPrefix, + @Nullable TokensInvalidationResult previousResult, ActionListener listener) { + final Set idsOfRecentTokens = new HashSet<>(); + final Set idsOfOlderTokens = new HashSet<>(); + for (UserToken userToken : userTokens) { + if (userToken.getVersion().onOrAfter(VERSION_TOKENS_INDEX_INTRODUCED)) { + idsOfRecentTokens.add(userToken.getId()); + } else { + idsOfOlderTokens.add(userToken.getId()); + } + } + if (false == idsOfOlderTokens.isEmpty()) { + indexInvalidation(idsOfOlderTokens, securityMainIndex, backoff, srcPrefix, previousResult, ActionListener.wrap(newResult -> { + if (false == idsOfRecentTokens.isEmpty()) { + // carry-over result of the invalidation for the tokens security index + indexInvalidation(idsOfRecentTokens, securityTokensIndex, backoff, srcPrefix, newResult, listener); + } else { + listener.onResponse(newResult); + } + }, listener::onFailure)); + } else { + indexInvalidation(idsOfRecentTokens, securityTokensIndex, backoff, srcPrefix, previousResult, listener); + } + } + /** * Performs the actual invalidation of a collection of tokens. In case of recoverable errors ( see * {@link TransportActions#isShardNotAvailableException} ) the UpdateRequests to mark the tokens as invalidated are retried using * an exponential backoff policy. * - * @param tokenIds the tokens to invalidate - * @param backoff the amount of time to delay between attempts - * @param srcPrefix the prefix to use when constructing the doc to update, either refresh_token or access_token depending on - * what type of tokens should be invalidated - * @param previousResult if this not the initial attempt for invalidation, it contains the result of invalidating - * tokens up to the point of the retry. This result is added to the result of the current attempt - * @param listener the listener to notify upon completion + * @param tokenIds the tokens to invalidate + * @param tokensIndexManager the manager for the index where the tokens are stored + * @param backoff the amount of time to delay between attempts + * @param srcPrefix the prefix to use when constructing the doc to update, either refresh_token or access_token depending on + * what type of tokens should be invalidated + * @param previousResult if this not the initial attempt for invalidation, it contains the result of invalidating + * tokens up to the point of the retry. This result is added to the result of the current attempt + * @param listener the listener to notify upon completion */ - private void indexInvalidation(Collection tokenIds, Iterator backoff, String srcPrefix, - @Nullable TokensInvalidationResult previousResult, ActionListener listener) { + private void indexInvalidation(Collection tokenIds, SecurityIndexManager tokensIndexManager, Iterator backoff, + String srcPrefix, @Nullable TokensInvalidationResult previousResult, + ActionListener listener) { if (tokenIds.isEmpty()) { logger.warn("No [{}] tokens provided for invalidation", srcPrefix); listener.onFailure(invalidGrantException("No tokens provided for invalidation")); @@ -627,14 +623,15 @@ private void indexInvalidation(Collection tokenIds, Iterator BulkRequestBuilder bulkRequestBuilder = client.prepareBulk(); for (String tokenId : tokenIds) { UpdateRequest request = client - .prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getTokenDocumentId(tokenId)) + .prepareUpdate(tokensIndexManager.aliasName(), SINGLE_MAPPING_NAME, getTokenDocumentId(tokenId)) .setDoc(srcPrefix, Collections.singletonMap("invalidated", true)) .setFetchSource(srcPrefix, null) .request(); bulkRequestBuilder.add(request); } bulkRequestBuilder.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); - securityIndex.prepareIndexIfNeededThenExecute(ex -> listener.onFailure(traceLog("prepare security index", ex)), + tokensIndexManager.prepareIndexIfNeededThenExecute( + ex -> listener.onFailure(traceLog("prepare index [" + tokensIndexManager.aliasName() + "]", ex)), () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, bulkRequestBuilder.request(), ActionListener.wrap(bulkResponse -> { ArrayList retryTokenDocIds = new ArrayList<>(); @@ -672,8 +669,9 @@ private void indexInvalidation(Collection tokenIds, Iterator retryTokenDocIds.size(), tokenIds.size()); final TokensInvalidationResult incompleteResult = new TokensInvalidationResult(invalidated, previouslyInvalidated, failedRequestResponses); - final Runnable retryWithContextRunnable = client.threadPool().getThreadContext().preserveContext( - () -> indexInvalidation(retryTokenDocIds, backoff, srcPrefix, incompleteResult, listener)); + final Runnable retryWithContextRunnable = client.threadPool().getThreadContext() + .preserveContext(() -> indexInvalidation(retryTokenDocIds, tokensIndexManager, backoff, + srcPrefix, incompleteResult, listener)); client.threadPool().schedule(retryWithContextRunnable, backoff.next(), GENERIC); } else { if (retryTokenDocIds.isEmpty() == false) { @@ -695,7 +693,8 @@ private void indexInvalidation(Collection tokenIds, Iterator if (isShardNotAvailableException(cause) && backoff.hasNext()) { logger.debug("failed to invalidate tokens, retrying "); final Runnable retryWithContextRunnable = client.threadPool().getThreadContext() - .preserveContext(() -> indexInvalidation(tokenIds, backoff, srcPrefix, previousResult, listener)); + .preserveContext(() -> indexInvalidation(tokenIds, tokensIndexManager, backoff, srcPrefix, + previousResult, listener)); client.threadPool().schedule(retryWithContextRunnable, backoff.next(), GENERIC); } else { listener.onFailure(e); @@ -721,38 +720,69 @@ public void refreshToken(String refreshToken, ActionListener backoff, ActionListener listener) { + if (refreshToken.length() == TOKEN_ID_LENGTH) { + // first check if token has the old format before the new version-prepended one + logger.debug("Assuming an unversioned refresh token [{}], generated for node versions" + + " prior to the introduction of the version-header format.", refreshToken); + findTokenFromRefreshToken(refreshToken, securityMainIndex, backoff, listener); + } else { + try { + final Tuple versionAndRefreshTokenTuple = unpackVersionAndPayload(refreshToken); + final Version refreshTokenVersion = versionAndRefreshTokenTuple.v1(); + final String unencodedRefreshToken = versionAndRefreshTokenTuple.v2(); + if (false == refreshTokenVersion.onOrAfter(VERSION_TOKENS_INDEX_INTRODUCED) + || unencodedRefreshToken.length() != TOKEN_ID_LENGTH) { + logger.debug("Decoded refresh token [{}] with version [{}] is invalid.", unencodedRefreshToken, refreshTokenVersion); + listener.onFailure(malformedTokenException()); + } else { + findTokenFromRefreshToken(unencodedRefreshToken, securityTokensIndex, backoff, listener); + } + } catch (IOException e) { + logger.debug("Could not decode refresh token [" + refreshToken + "].", e); + listener.onFailure(malformedTokenException()); + } + } + } + + /** + * Performs an asynchronous search request for the token document that contains the {@code refreshToken} and calls the {@code listener} + * with the resulting {@link SearchResponse}. In case of recoverable errors the {@code SearchRequest} is retried using an exponential + * backoff policy. This method requires the tokens index where the token document, pointed to by the refresh token, resides. + */ + private void findTokenFromRefreshToken(String refreshToken, SecurityIndexManager tokensIndexManager, Iterator backoff, + ActionListener listener) { final Consumer onFailure = ex -> listener.onFailure(traceLog("find token by refresh token", refreshToken, ex)); final Consumer maybeRetryOnFailure = ex -> { if (backoff.hasNext()) { final TimeValue backofTimeValue = backoff.next(); logger.debug("retrying after [" + backofTimeValue + "] back off"); final Runnable retryWithContextRunnable = client.threadPool().getThreadContext() - .preserveContext(() -> findTokenFromRefreshToken(refreshToken, backoff, listener)); + .preserveContext(() -> findTokenFromRefreshToken(refreshToken, tokensIndexManager, backoff, listener)); client.threadPool().schedule(retryWithContextRunnable, backofTimeValue, GENERIC); } else { logger.warn("failed to find token from refresh token after all retries"); onFailure.accept(ex); } }; - final SecurityIndexManager frozenSecurityIndex = securityIndex.freeze(); - if (frozenSecurityIndex.indexExists() == false) { - logger.warn("security index does not exist therefore refresh token cannot be validated"); + final SecurityIndexManager frozenTokensIndex = tokensIndexManager.freeze(); + if (frozenTokensIndex.indexExists() == false) { + logger.warn("index [{}] does not exist therefore refresh token cannot be validated", frozenTokensIndex.aliasName()); listener.onFailure(invalidGrantException("could not refresh the requested token")); - } else if (frozenSecurityIndex.isAvailable() == false) { - logger.debug("security index is not available to find token from refresh token, retrying"); + } else if (frozenTokensIndex.isAvailable() == false) { + logger.debug("index [{}] is not available to find token from refresh token, retrying", frozenTokensIndex.aliasName()); maybeRetryOnFailure.accept(invalidGrantException("could not refresh the requested token")); } else { - final SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) + final SearchRequest request = client.prepareSearch(tokensIndexManager.aliasName()) .setQuery(QueryBuilders.boolQuery() .filter(QueryBuilders.termQuery("doc_type", TOKEN_DOC_TYPE)) .filter(QueryBuilders.termQuery("refresh_token.token", refreshToken))) .seqNoAndPrimaryTerm(true) .request(); - securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> + tokensIndexManager.checkIndexVersionThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, request, ActionListener.wrap(searchResponse -> { if (searchResponse.isTimedOut()) { @@ -805,65 +835,28 @@ private void innerRefresh(String tokenDocId, Map source, long se final RefreshTokenStatus refreshTokenStatus = checkRefreshResult.v1(); if (refreshTokenStatus.isRefreshed()) { logger.debug("Token document [{}] was recently refreshed, when a new token document [{}] was generated. Reusing that result.", - tokenDocId, refreshTokenStatus.getSupersedingDocId()); - getTokenDocAsync(refreshTokenStatus.getSupersedingDocId(), new ActionListener() { - private final Consumer maybeRetryOnFailure = ex -> { - if (backoff.hasNext()) { - final TimeValue backofTimeValue = backoff.next(); - logger.debug("retrying after [" + backofTimeValue + "] back off"); - final Runnable retryWithContextRunnable = client.threadPool().getThreadContext() - .preserveContext(() -> getTokenDocAsync(refreshTokenStatus.getSupersedingDocId(), this)); - client.threadPool().schedule(retryWithContextRunnable, backofTimeValue, GENERIC); - } else { - logger.warn("back off retries exhausted"); - onFailure.accept(ex); - } - }; - - @Override - public void onResponse(GetResponse response) { - if (response.isExists()) { - logger.debug("found superseding token document [{}] for token document [{}]", - refreshTokenStatus.getSupersedingDocId(), tokenDocId); - final Tuple parsedTokens; - try { - parsedTokens = parseTokensFromDocument(response.getSource(), null); - } catch (IllegalStateException | DateTimeException e) { - logger.error("unable to decode existing user token", e); - listener.onFailure(new ElasticsearchSecurityException("could not refresh the requested token", e)); - return; - } - listener.onResponse(parsedTokens); - } else { - // We retry this since the creation of the superseding token document might already be in flight but not - // yet completed, triggered by a refresh request that came a few milliseconds ago - logger.info("could not find superseding token document [{}] for token document [{}], retrying", - refreshTokenStatus.getSupersedingDocId(), tokenDocId); - maybeRetryOnFailure.accept(invalidGrantException("could not refresh the requested token")); - } - } - - @Override - public void onFailure(Exception e) { - if (isShardNotAvailableException(e)) { - logger.info("could not find superseding token document [{}] for refresh, retrying", - refreshTokenStatus.getSupersedingDocId()); - maybeRetryOnFailure.accept(invalidGrantException("could not refresh the requested token")); - } else { - logger.warn("could not find superseding token document [{}] for refresh", refreshTokenStatus.getSupersedingDocId()); - onFailure.accept(invalidGrantException("could not refresh the requested token")); - } - } - }); + tokenDocId, refreshTokenStatus.getSupersededBy()); + getSupersedingTokenDocAsyncWithRetry(refreshTokenStatus, backoff, listener); } else { final String newUserTokenId = UUIDs.randomBase64UUID(); + final Version newTokenVersion = getTokenVersionCompatibility(); final Map updateMap = new HashMap<>(); updateMap.put("refreshed", true); updateMap.put("refresh_time", clock.instant().toEpochMilli()); - updateMap.put("superseded_by", getTokenDocumentId(newUserTokenId)); + if (newTokenVersion.onOrAfter(VERSION_TOKENS_INDEX_INTRODUCED)) { + // the superseding token document reference is formated as "|"; + // for now, only the ".security-tokens|" is a valid reference format + updateMap.put("superseded_by", securityTokensIndex.aliasName() + "|" + getTokenDocumentId(newUserTokenId)); + } else { + // preservers the format of the reference (without the alias prefix) + // so that old nodes in a mixed cluster can still understand it + updateMap.put("superseded_by", getTokenDocumentId(newUserTokenId)); + } assert seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO : "expected an assigned sequence number"; assert primaryTerm != SequenceNumbers.UNASSIGNED_PRIMARY_TERM : "expected an assigned primary term"; - final UpdateRequestBuilder updateRequest = client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, tokenDocId) + final SecurityIndexManager refreshedTokenIndex = getTokensIndexForVersion(refreshTokenStatus.getVersion()); + final UpdateRequestBuilder updateRequest = client + .prepareUpdate(refreshedTokenIndex.aliasName(), SINGLE_MAPPING_NAME, tokenDocId) .setDoc("refresh_token", updateMap) .setFetchSource(true) .setRefreshPolicy(RefreshPolicy.IMMEDIATE) @@ -876,8 +869,8 @@ public void onFailure(Exception e) { updateResponse.getGetResult().sourceAsMap())); final Tuple parsedTokens = parseTokensFromDocument(source, null); final UserToken toRefreshUserToken = parsedTokens.v1(); - createOAuth2Tokens(newUserTokenId, toRefreshUserToken.getAuthentication(), clientAuth, - toRefreshUserToken.getMetadata(), true, listener); + createOAuth2Tokens(newUserTokenId, newTokenVersion, getTokensIndexForVersion(newTokenVersion), + toRefreshUserToken.getAuthentication(), clientAuth, toRefreshUserToken.getMetadata(), true, listener); } else if (backoff.hasNext()) { logger.info("failed to update the original token document [{}], the update result was [{}]. Retrying", tokenDocId, updateResponse.getResult()); @@ -895,7 +888,7 @@ public void onFailure(Exception e) { if (cause instanceof VersionConflictEngineException) { // The document has been updated by another thread, get it again. logger.debug("version conflict while updating document [{}], attempting to get it again", tokenDocId); - getTokenDocAsync(tokenDocId, new ActionListener() { + getTokenDocAsync(tokenDocId, refreshedTokenIndex, new ActionListener() { @Override public void onResponse(GetResponse response) { if (response.isExists()) { @@ -913,7 +906,7 @@ public void onFailure(Exception e) { if (backoff.hasNext()) { logger.info("could not get token document [{}] for refresh, retrying", tokenDocId); final Runnable retryWithContextRunnable = client.threadPool().getThreadContext() - .preserveContext(() -> getTokenDocAsync(tokenDocId, this)); + .preserveContext(() -> getTokenDocAsync(tokenDocId, refreshedTokenIndex, this)); client.threadPool().schedule(retryWithContextRunnable, backoff.next(), GENERIC); } else { logger.warn("could not get token document [{}] for refresh after all retries", tokenDocId); @@ -942,14 +935,91 @@ public void onFailure(Exception e) { } } - private void getTokenDocAsync(String tokenDocId, ActionListener listener) { - final GetRequest getRequest = client.prepareGet(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, tokenDocId).request(); + private void getSupersedingTokenDocAsyncWithRetry(RefreshTokenStatus refreshTokenStatus, Iterator backoff, + ActionListener> listener) { + final Consumer onFailure = ex -> listener + .onFailure(traceLog("get superseding token", refreshTokenStatus.getSupersededBy(), ex)); + getSupersedingTokenDocAsync(refreshTokenStatus, new ActionListener() { + private final Consumer maybeRetryOnFailure = ex -> { + if (backoff.hasNext()) { + final TimeValue backofTimeValue = backoff.next(); + logger.debug("retrying after [" + backofTimeValue + "] back off"); + final Runnable retryWithContextRunnable = client.threadPool().getThreadContext() + .preserveContext(() -> getSupersedingTokenDocAsync(refreshTokenStatus, this)); + client.threadPool().schedule(retryWithContextRunnable, backofTimeValue, GENERIC); + } else { + logger.warn("back off retries exhausted"); + onFailure.accept(ex); + } + }; + + @Override + public void onResponse(GetResponse response) { + if (response.isExists()) { + logger.debug("found superseding token document [{}] in index [{}] by following the [{}] reference", response.getId(), + response.getIndex(), refreshTokenStatus.getSupersededBy()); + final Tuple parsedTokens; + try { + parsedTokens = parseTokensFromDocument(response.getSource(), null); + } catch (IllegalStateException | DateTimeException e) { + logger.error("unable to decode existing user token", e); + listener.onFailure(new ElasticsearchSecurityException("could not refresh the requested token", e)); + return; + } + listener.onResponse(parsedTokens); + } else { + // We retry this since the creation of the superseding token document might already be in flight but not + // yet completed, triggered by a refresh request that came a few milliseconds ago + logger.info("could not find superseding token document from [{}] reference, retrying", + refreshTokenStatus.getSupersededBy()); + maybeRetryOnFailure.accept(invalidGrantException("could not refresh the requested token")); + } + } + + @Override + public void onFailure(Exception e) { + if (isShardNotAvailableException(e)) { + logger.info("could not find superseding token document from reference [{}], retrying", + refreshTokenStatus.getSupersededBy()); + maybeRetryOnFailure.accept(invalidGrantException("could not refresh the requested token")); + } else { + logger.warn("could not find superseding token document from reference [{}]", refreshTokenStatus.getSupersededBy()); + onFailure.accept(invalidGrantException("could not refresh the requested token")); + } + } + }); + } + + private void getSupersedingTokenDocAsync(RefreshTokenStatus refreshTokenStatus, ActionListener listener) { + final String supersedingDocReference = refreshTokenStatus.getSupersededBy(); + if (supersedingDocReference.startsWith(securityTokensIndex.aliasName() + "|")) { + // superseding token doc is stored on the new tokens index, irrespective of where the superseded token doc resides + final String supersedingDocId = supersedingDocReference.substring(securityTokensIndex.aliasName().length() + 1); + getTokenDocAsync(supersedingDocId, securityTokensIndex, listener); + } else { + assert false == supersedingDocReference + .contains("|") : "The superseding doc reference appears to contain an alias name but should not"; + getTokenDocAsync(supersedingDocReference, securityMainIndex, listener); + } + } + + private void getTokenDocAsync(String tokenDocId, SecurityIndexManager tokensIndex, ActionListener listener) { + final GetRequest getRequest = client.prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, tokenDocId).request(); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, listener, client::get); } + private Version getTokenVersionCompatibility() { + // newly minted tokens are compatible with the min node version in the cluster + return clusterService.state().nodes().getMinNodeVersion(); + } + + public static Boolean isTokenServiceEnabled(Settings settings) { + return XPackSettings.TOKEN_SERVICE_ENABLED_SETTING.get(settings); + } + /** - * A refresh token has a hardcoded maximum lifetime of 24h. This checks if the token document represents a valid token wrt this time - * interval. + * A refresh token has a fixed maximum lifetime of {@code ExpiredTokenRemover#MAXIMUM_TOKEN_LIFETIME_HOURS} hours. This checks if the + * token document represents a valid token wrt this time interval. */ private static Optional checkTokenDocumentExpired(Instant now, Map source) { final Long creationEpochMilli = (Long) source.get("creation_time"); @@ -957,7 +1027,7 @@ private static Optional checkTokenDocumentExpire throw new IllegalStateException("token document is missing creation time value"); } else { final Instant creationTime = Instant.ofEpochMilli(creationEpochMilli); - if (now.isAfter(creationTime.plus(24L, ChronoUnit.HOURS))) { + if (now.isAfter(creationTime.plus(ExpiredTokenRemover.MAXIMUM_TOKEN_LIFETIME_HOURS, ChronoUnit.HOURS))) { return Optional.of(invalidGrantException("token document has expired")); } else { return Optional.empty(); @@ -974,12 +1044,13 @@ private static Tuple source) throws IllegalStateException, DateTimeException { final RefreshTokenStatus refreshTokenStatus = RefreshTokenStatus.fromSourceMap(getRefreshTokenSourceMap(source)); final UserToken userToken = UserToken.fromSourceMap(getUserTokenSourceMap(source)); + refreshTokenStatus.setVersion(userToken.getVersion()); final ElasticsearchSecurityException validationException = checkTokenDocumentExpired(now, source).orElseGet(() -> { if (refreshTokenStatus.isInvalidated()) { return invalidGrantException("token has been invalidated"); } else { return checkClientCanRefresh(refreshTokenStatus, clientAuth) - .orElse(checkMultipleRefreshes(now, refreshTokenStatus, userToken).orElse(null)); + .orElse(checkMultipleRefreshes(now, refreshTokenStatus).orElse(null)); } }); return new Tuple<>(refreshTokenStatus, Optional.ofNullable(validationException)); @@ -1032,14 +1103,13 @@ private static Map getUserTokenSourceMap(Map sou * @return An {@code Optional} containing the exception in case this refresh token cannot be reused, or an empty Optional if * refreshing is allowed. */ - private static Optional checkMultipleRefreshes(Instant now, RefreshTokenStatus refreshToken, - UserToken userToken) { - if (refreshToken.isRefreshed()) { - if (userToken.getVersion().onOrAfter(Version.V_7_1_0)) { - if (now.isAfter(refreshToken.getRefreshInstant().plus(30L, ChronoUnit.SECONDS))) { + private static Optional checkMultipleRefreshes(Instant now, RefreshTokenStatus refreshTokenStatus) { + if (refreshTokenStatus.isRefreshed()) { + if (refreshTokenStatus.getVersion().onOrAfter(VERSION_MULTIPLE_CONCURRENT_REFRESHES)) { + if (now.isAfter(refreshTokenStatus.getRefreshInstant().plus(30L, ChronoUnit.SECONDS))) { return Optional.of(invalidGrantException("token has already been refreshed more than 30 seconds in the past")); } - if (now.isBefore(refreshToken.getRefreshInstant().minus(30L, ChronoUnit.SECONDS))) { + if (now.isBefore(refreshTokenStatus.getRefreshInstant().minus(30L, ChronoUnit.SECONDS))) { return Optional .of(invalidGrantException("token has been refreshed more than 30 seconds in the future, clock skew too great")); } @@ -1061,39 +1131,40 @@ private static Optional checkMultipleRefreshes(I public void findActiveTokensForRealm(String realmName, @Nullable Predicate> filter, ActionListener>> listener) { ensureEnabled(); - final SecurityIndexManager frozenSecurityIndex = securityIndex.freeze(); if (Strings.isNullOrEmpty(realmName)) { - listener.onFailure(new IllegalArgumentException("Realm name is required")); - } else if (frozenSecurityIndex.indexExists() == false) { - listener.onResponse(Collections.emptyList()); - } else if (frozenSecurityIndex.isAvailable() == false) { - listener.onFailure(frozenSecurityIndex.getUnavailableReason()); - } else { - final Instant now = clock.instant(); - final BoolQueryBuilder boolQuery = QueryBuilders.boolQuery() - .filter(QueryBuilders.termQuery("doc_type", TOKEN_DOC_TYPE)) - .filter(QueryBuilders.termQuery("access_token.realm", realmName)) - .filter(QueryBuilders.boolQuery() - .should(QueryBuilders.boolQuery() - .must(QueryBuilders.termQuery("access_token.invalidated", false)) - .must(QueryBuilders.rangeQuery("access_token.user_token.expiration_time").gte(now.toEpochMilli())) - ) - .should(QueryBuilders.boolQuery() - .must(QueryBuilders.termQuery("refresh_token.invalidated", false)) - .must(QueryBuilders.rangeQuery("creation_time").gte(now.toEpochMilli() - TimeValue.timeValueHours(24).millis())) - ) - ); - - final SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) - .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) - .setQuery(boolQuery) - .setVersion(false) - .setSize(1000) - .setFetchSource(true) - .request(); - securityIndex.checkIndexVersionThenExecute(listener::onFailure, - () -> ScrollHelper.fetchAllByEntity(client, request, listener, (SearchHit hit) -> filterAndParseHit(hit, filter))); + listener.onFailure(new IllegalArgumentException("realm name is required")); + return; } + sourceIndicesWithTokensAndRun(ActionListener.wrap(indicesWithTokens -> { + if (indicesWithTokens.isEmpty()) { + listener.onResponse(Collections.emptyList()); + } else { + final Instant now = clock.instant(); + final BoolQueryBuilder boolQuery = QueryBuilders.boolQuery() + .filter(QueryBuilders.termQuery("doc_type", TOKEN_DOC_TYPE)) + .filter(QueryBuilders.termQuery("access_token.realm", realmName)) + .filter(QueryBuilders.boolQuery() + .should(QueryBuilders.boolQuery() + .must(QueryBuilders.termQuery("access_token.invalidated", false)) + .must(QueryBuilders.rangeQuery("access_token.user_token.expiration_time").gte(now.toEpochMilli())) + ) + .should(QueryBuilders.boolQuery() + .must(QueryBuilders.termQuery("refresh_token.invalidated", false)) + .must(QueryBuilders.rangeQuery("creation_time").gte(now.toEpochMilli() + - TimeValue.timeValueHours(ExpiredTokenRemover.MAXIMUM_TOKEN_LIFETIME_HOURS).millis())) + ) + ); + final SearchRequest request = client.prepareSearch(indicesWithTokens.toArray(new String[0])) + .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) + .setQuery(boolQuery) + .setVersion(false) + .setSize(1000) + .setFetchSource(true) + .request(); + ScrollHelper.fetchAllByEntity(client, request, listener, (SearchHit hit) -> filterAndParseHit(hit, filter)); + } + }, listener::onFailure)); + } /** @@ -1105,38 +1176,117 @@ public void findActiveTokensForRealm(String realmName, @Nullable Predicate>> listener) { ensureEnabled(); - final SecurityIndexManager frozenSecurityIndex = securityIndex.freeze(); if (Strings.isNullOrEmpty(username)) { listener.onFailure(new IllegalArgumentException("username is required")); - } else if (frozenSecurityIndex.indexExists() == false) { - listener.onResponse(Collections.emptyList()); - } else if (frozenSecurityIndex.isAvailable() == false) { - listener.onFailure(frozenSecurityIndex.getUnavailableReason()); - } else { - final Instant now = clock.instant(); - final BoolQueryBuilder boolQuery = QueryBuilders.boolQuery() - .filter(QueryBuilders.termQuery("doc_type", TOKEN_DOC_TYPE)) - .filter(QueryBuilders.boolQuery() - .should(QueryBuilders.boolQuery() - .must(QueryBuilders.termQuery("access_token.invalidated", false)) - .must(QueryBuilders.rangeQuery("access_token.user_token.expiration_time").gte(now.toEpochMilli())) - ) - .should(QueryBuilders.boolQuery() - .must(QueryBuilders.termQuery("refresh_token.invalidated", false)) - .must(QueryBuilders.rangeQuery("creation_time").gte(now.toEpochMilli() - TimeValue.timeValueHours(24).millis())) - ) - ); - - final SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) - .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) - .setQuery(boolQuery) - .setVersion(false) - .setSize(1000) - .setFetchSource(true) - .request(); - securityIndex.checkIndexVersionThenExecute(listener::onFailure, - () -> ScrollHelper.fetchAllByEntity(client, request, listener, - (SearchHit hit) -> filterAndParseHit(hit, isOfUser(username)))); + return; + } + sourceIndicesWithTokensAndRun(ActionListener.wrap(indicesWithTokens -> { + if (indicesWithTokens.isEmpty()) { + listener.onResponse(Collections.emptyList()); + } else { + final Instant now = clock.instant(); + final BoolQueryBuilder boolQuery = QueryBuilders.boolQuery() + .filter(QueryBuilders.termQuery("doc_type", TOKEN_DOC_TYPE)) + .filter(QueryBuilders.boolQuery() + .should(QueryBuilders.boolQuery() + .must(QueryBuilders.termQuery("access_token.invalidated", false)) + .must(QueryBuilders.rangeQuery("access_token.user_token.expiration_time").gte(now.toEpochMilli())) + ) + .should(QueryBuilders.boolQuery() + .must(QueryBuilders.termQuery("refresh_token.invalidated", false)) + .must(QueryBuilders.rangeQuery("creation_time").gte(now.toEpochMilli() + - TimeValue.timeValueHours(ExpiredTokenRemover.MAXIMUM_TOKEN_LIFETIME_HOURS).millis())) + ) + ); + final SearchRequest request = client.prepareSearch(indicesWithTokens.toArray(new String[0])) + .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) + .setQuery(boolQuery) + .setVersion(false) + .setSize(1000) + .setFetchSource(true) + .request(); + ScrollHelper.fetchAllByEntity(client, request, listener, (SearchHit hit) -> filterAndParseHit(hit, isOfUser(username))); + } + }, listener::onFailure)); + } + + /** + * Security tokens were traditionally stored on the main security index but after version {@code #VERSION_TOKENS_INDEX_INTRODUCED} they + * have been stored on a dedicated separate index. This move has been implemented without requiring user intervention, so the newly + * created tokens started to be created in the new index, while the old tokens were still usable out of the main security index, subject + * to their maximum lifetime of {@code ExpiredTokenRemover#MAXIMUM_TOKEN_LIFETIME_HOURS} hours. Once the dedicated tokens index has been + * automatically created, all the onwards created tokens will be stored inside it. This function returns the list of the indices names + * that might contain tokens. Unless there are availability or version issues, the dedicated tokens index always contains tokens. The + * main security index might contain tokens if the tokens index has not been created yet, or if it has been created recently so + * that there might still be tokens that have not yet exceeded their maximum lifetime. + */ + private void sourceIndicesWithTokensAndRun(ActionListener> listener) { + final List indicesWithTokens = new ArrayList<>(2); + final SecurityIndexManager frozenTokensIndex = securityTokensIndex.freeze(); + if (frozenTokensIndex.indexExists()) { + // an existing tokens index always contains tokens (if available and version allows) + if (false == frozenTokensIndex.isAvailable()) { + listener.onFailure(frozenTokensIndex.getUnavailableReason()); + return; + } + if (false == frozenTokensIndex.isIndexUpToDate()) { + listener.onFailure(new IllegalStateException( + "Index [" + frozenTokensIndex.aliasName() + "] is not on the current version. Features relying on the index" + + " will not be available until the upgrade API is run on the index")); + return; + } + indicesWithTokens.add(frozenTokensIndex.aliasName()); + } + final SecurityIndexManager frozenMainIndex = securityMainIndex.freeze(); + if (frozenMainIndex.indexExists()) { + // main security index _might_ contain tokens if the tokens index has been created recently + if (false == frozenTokensIndex.indexExists() || frozenTokensIndex.getCreationTime() + .isAfter(clock.instant().minus(ExpiredTokenRemover.MAXIMUM_TOKEN_LIFETIME_HOURS, ChronoUnit.HOURS))) { + if (false == frozenMainIndex.isAvailable()) { + listener.onFailure(frozenMainIndex.getUnavailableReason()); + return; + } + if (false == frozenMainIndex.isIndexUpToDate()) { + listener.onFailure(new IllegalStateException( + "Index [" + frozenMainIndex.aliasName() + "] is not on the current version. Features relying on the index" + + " will not be available until the upgrade API is run on the index")); + return; + } + indicesWithTokens.add(frozenMainIndex.aliasName()); + } + } + listener.onResponse(indicesWithTokens); + } + + private BytesReference createTokenDocument(UserToken userToken, @Nullable String refreshToken, + @Nullable Authentication originatingClientAuth) { + assert refreshToken == null || originatingClientAuth != null : "non-null refresh token " + refreshToken + + " requires non-null client authn " + originatingClientAuth; + try (XContentBuilder builder = XContentFactory.jsonBuilder()) { + builder.startObject(); + builder.field("doc_type", TOKEN_DOC_TYPE); + builder.field("creation_time", getCreationTime(userToken.getExpirationTime()).toEpochMilli()); + if (refreshToken != null) { + builder.startObject("refresh_token") + .field("token", refreshToken) + .field("invalidated", false) + .field("refreshed", false) + .startObject("client") + .field("type", "unassociated_client") + .field("user", originatingClientAuth.getUser().principal()) + .field("realm", originatingClientAuth.getAuthenticatedBy().getName()) + .endObject() + .endObject(); + } + builder.startObject("access_token") + .field("invalidated", false) + .field("user_token", userToken) + .field("realm", userToken.getAuthentication().getAuthenticatedBy().getName()) + .endObject(); + builder.endObject(); + return BytesReference.bytes(builder); + } catch (IOException e) { + throw new RuntimeException("Unexpected exception when constructing a JSON document.", e); } } @@ -1174,13 +1324,21 @@ private Tuple filterAndParseHit(SearchHit hit, @Nullable Pred */ private Tuple parseTokensFromDocument(Map source, @Nullable Predicate> filter) throws IllegalStateException, DateTimeException { - final String refreshToken = (String) ((Map) source.get("refresh_token")).get("token"); + final String plainRefreshToken = (String) ((Map) source.get("refresh_token")).get("token"); final Map userTokenSource = (Map) ((Map) source.get("access_token")).get("user_token"); if (null != filter && filter.test(userTokenSource) == false) { return null; } - return new Tuple<>(UserToken.fromSourceMap(userTokenSource), refreshToken); + final UserToken userToken = UserToken.fromSourceMap(userTokenSource); + if (userToken.getVersion().onOrAfter(VERSION_TOKENS_INDEX_INTRODUCED)) { + final String versionedRefreshToken = plainRefreshToken != null ? + prependVersionAndEncode(userToken.getVersion(), plainRefreshToken) : null; + return new Tuple<>(userToken, versionedRefreshToken); + } else { + // do not prepend version to refresh token as the audience node version cannot deal with it + return new Tuple<>(userToken, plainRefreshToken); + } } private static String getTokenDocumentId(UserToken userToken) { @@ -1205,21 +1363,38 @@ private void ensureEnabled() { } } + /** + * In version {@code #VERSION_TOKENS_INDEX_INTRODUCED} security tokens were moved into a separate index, away from the other entities in + * the main security index, due to their ephemeral nature. They moved "seamlessly" - without manual user intervention. In this way, new + * tokens are created in the new index, while the existing ones were left in place - to be accessed from the old index - and due to be + * removed automatically by the {@code ExpiredTokenRemover} periodic job. Therefore, in general, when searching for a token we need to + * consider both the new and the old indices. + */ + private SecurityIndexManager getTokensIndexForVersion(Version version) { + if (version.onOrAfter(VERSION_TOKENS_INDEX_INTRODUCED)) { + return securityTokensIndex; + } else { + return securityMainIndex; + } + } + /** * Checks if the access token has been explicitly invalidated */ private void checkIfTokenIsValid(UserToken userToken, ActionListener listener) { - Instant currentTime = clock.instant(); - if (currentTime.isAfter(userToken.getExpirationTime())) { + if (clock.instant().isAfter(userToken.getExpirationTime())) { listener.onFailure(traceLog("validate token", userToken.getId(), expiredTokenException())); - } else if (securityIndex.indexExists() == false) { + return; + } + final SecurityIndexManager tokensIndex = getTokensIndexForVersion(userToken.getVersion()); + if (tokensIndex.indexExists() == false) { // index doesn't exist so the token is considered invalid as we cannot verify its validity - logger.warn("failed to validate access token because the security index doesn't exist"); + logger.warn("failed to validate access token because the index [" + tokensIndex.aliasName() + "] doesn't exist"); listener.onResponse(null); } else { - securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> { - final GetRequest getRequest = client.prepareGet(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getTokenDocumentId(userToken)) - .request(); + tokensIndex.checkIndexVersionThenExecute(listener::onFailure, () -> { + final GetRequest getRequest = client + .prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, getTokenDocumentId(userToken)).request(); Consumer onFailure = ex -> listener.onFailure(traceLog("check token state", userToken.getId(), ex)); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, ActionListener.wrap(response -> { @@ -1260,16 +1435,18 @@ public TimeValue getExpirationDelay() { return expirationDelay; } - private Instant getExpirationTime(Instant now) { - return now.plusSeconds(expirationDelay.getSeconds()); + private Instant getExpirationTime() { + return clock.instant().plusSeconds(expirationDelay.getSeconds()); + } + + private Instant getCreationTime(Instant expire) { + return expire.minusSeconds(expirationDelay.getSeconds()); } private void maybeStartTokenRemover() { - if (securityIndex.isAvailable()) { - if (client.threadPool().relativeTimeInMillis() - lastExpirationRunMs > deleteInterval.getMillis()) { - expiredTokenRemover.submit(client.threadPool()); - lastExpirationRunMs = client.threadPool().relativeTimeInMillis(); - } + if (client.threadPool().relativeTimeInMillis() - lastExpirationRunMs > deleteInterval.getMillis()) { + expiredTokenRemover.submit(client.threadPool()); + lastExpirationRunMs = client.threadPool().relativeTimeInMillis(); } } @@ -1287,12 +1464,12 @@ private String getFromHeader(ThreadContext threadContext) { } /** - * Serializes a token to a String containing the version of the node that created the token and - * either an encrypted representation of the token id for versions earlier to 7.1.0 or the token ie - * itself for versions after 7.1.0 + * Serializes a token to a String containing the minimum compatible node version for decoding it back and either an encrypted + * representation of the token id for versions earlier to {@code #VERSION_ACCESS_TOKENS_UUIDS} or the token itself for versions after + * {@code #VERSION_ACCESS_TOKENS_UUIDS} */ public String getAccessTokenAsString(UserToken userToken) throws IOException, GeneralSecurityException { - if (clusterService.state().nodes().getMinNodeVersion().onOrAfter(Version.V_7_1_0)) { + if (userToken.getVersion().onOrAfter(VERSION_ACCESS_TOKENS_AS_UUIDS)) { try (ByteArrayOutputStream os = new ByteArrayOutputStream(MINIMUM_BASE64_BYTES); OutputStream base64 = Base64.getEncoder().wrap(os); StreamOutput out = new OutputStreamStreamOutput(base64)) { @@ -1325,6 +1502,33 @@ public String getAccessTokenAsString(UserToken userToken) throws IOException, Ge } } + private static String prependVersionAndEncode(Version version, String payload) { + try (ByteArrayOutputStream os = new ByteArrayOutputStream(); + OutputStream base64 = Base64.getEncoder().wrap(os); + StreamOutput out = new OutputStreamStreamOutput(base64)) { + out.setVersion(version); + Version.writeVersion(version, out); + out.writeString(payload); + return new String(os.toByteArray(), StandardCharsets.UTF_8); + } catch (IOException e) { + throw new RuntimeException("Unexpected exception when working with small in-memory streams", e); + } + } + + // public for testing + /** + * Unpacks a base64 encoded pair of a version tag and String payload. + */ + public static Tuple unpackVersionAndPayload(String encodedPack) throws IOException { + final byte[] bytes = encodedPack.getBytes(StandardCharsets.UTF_8); + try (StreamInput in = new InputStreamStreamInput(Base64.getDecoder().wrap(new ByteArrayInputStream(bytes)), bytes.length)) { + final Version version = Version.readVersion(in); + in.setVersion(version); + final String payload = in.readString(); + return new Tuple(version, payload); + } + } + private void ensureEncryptionCiphersSupported() throws NoSuchPaddingException, NoSuchAlgorithmException { Cipher.getInstance(ENCRYPTION_CIPHER); SecretKeyFactory.getInstance(KDF_ALGORITHM); @@ -1344,6 +1548,30 @@ Cipher getEncryptionCipher(byte[] iv, KeyAndCache keyAndCache, Version version) return cipher; } + private void getKeyAsync(BytesKey decodedSalt, KeyAndCache keyAndCache, ActionListener listener) { + final SecretKey decodeKey = keyAndCache.getKey(decodedSalt); + if (decodeKey != null) { + listener.onResponse(decodeKey); + } else { + /* As a measure of protected against DOS, we can pass requests requiring a key + * computation off to a single thread executor. For normal usage, the initial + * request(s) that require a key computation will be delayed and there will be + * some additional latency. + */ + client.threadPool().executor(THREAD_POOL_NAME) + .submit(new KeyComputingRunnable(decodedSalt, keyAndCache, listener)); + } + } + + private static String decryptTokenId(byte[] encryptedTokenId, Cipher cipher, Version version) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(encryptedTokenId); + CipherInputStream cis = new CipherInputStream(bais, cipher); + StreamInput decryptedInput = new InputStreamStreamInput(cis)) { + decryptedInput.setVersion(version); + return decryptedInput.readString(); + } + } + private Cipher getDecryptionCipher(byte[] iv, SecretKey key, Version version, BytesKey salt) throws GeneralSecurityException { Cipher cipher = Cipher.getInstance(ENCRYPTION_CIPHER); cipher.init(Cipher.DECRYPT_MODE, key, new GCMParameterSpec(128, iv), secureRandom); @@ -1838,16 +2066,17 @@ private static final class RefreshTokenStatus { private final String associatedRealm; private final boolean refreshed; @Nullable private final Instant refreshInstant; - @Nullable private final String supersededByDocId; + @Nullable private final String supersededBy; + private Version version; private RefreshTokenStatus(boolean invalidated, String associatedUser, String associatedRealm, boolean refreshed, - Instant refreshInstant, String supersededByDocId) { + Instant refreshInstant, String supersededBy) { this.invalidated = invalidated; this.associatedUser = associatedUser; this.associatedRealm = associatedRealm; this.refreshed = refreshed; this.refreshInstant = refreshInstant; - this.supersededByDocId = supersededByDocId; + this.supersededBy = supersededBy; } boolean isInvalidated() { @@ -1870,8 +2099,16 @@ boolean isRefreshed() { return refreshInstant; } - @Nullable String getSupersedingDocId() { - return supersededByDocId; + @Nullable String getSupersededBy() { + return supersededBy; + } + + Version getVersion() { + return version; + } + + void setVersion(Version version) { + this.version = version; } static RefreshTokenStatus fromSourceMap(Map refreshTokenSource) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java index a0c579dd881ce..0270e31216c42 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStore.java @@ -66,7 +66,7 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; /** * NativeUsersStore is a store for users that reads from an Elasticsearch index. This store is responsible for fetching the full @@ -146,7 +146,7 @@ public void getUsers(String[] userNames, final ActionListener> } final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { - SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) + SearchRequest request = client.prepareSearch(SECURITY_MAIN_ALIAS) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) .setSize(1000) @@ -171,7 +171,7 @@ void getUserCount(final ActionListener listener) { } else { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareSearch(SECURITY_INDEX_NAME) + client.prepareSearch(SECURITY_MAIN_ALIAS) .setQuery(QueryBuilders.termQuery(Fields.TYPE.getPreferredName(), USER_DOC_TYPE)) .setSize(0) .setTrackTotalHits(true) @@ -205,7 +205,7 @@ private void getUserAndPassword(final String user, final ActionListener executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareGet(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, user)).request(), + client.prepareGet(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, user)).request(), new ActionListener() { @Override public void onResponse(GetResponse response) { @@ -245,7 +245,7 @@ public void changePassword(final ChangePasswordRequest request, final ActionList securityIndex.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(docType, username)) + client.prepareUpdate(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(docType, username)) .setDoc(Requests.INDEX_CONTENT_TYPE, Fields.PASSWORD.getPreferredName(), String.valueOf(request.passwordHash())) .setRefreshPolicy(request.getRefreshPolicy()).request(), @@ -283,7 +283,7 @@ public void onFailure(Exception e) { private void createReservedUser(String username, char[] passwordHash, RefreshPolicy refresh, ActionListener listener) { securityIndex.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareIndex(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(RESERVED_USER_TYPE, username)) + client.prepareIndex(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(RESERVED_USER_TYPE, username)) .setSource(Fields.PASSWORD.getPreferredName(), String.valueOf(passwordHash), Fields.ENABLED.getPreferredName(), true, Fields.TYPE.getPreferredName(), RESERVED_USER_TYPE) .setRefreshPolicy(refresh).request(), @@ -323,7 +323,7 @@ private void updateUserWithoutPassword(final PutUserRequest putUserRequest, fina // We must have an existing document securityIndex.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, putUserRequest.username())) + client.prepareUpdate(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, putUserRequest.username())) .setDoc(Requests.INDEX_CONTENT_TYPE, Fields.USERNAME.getPreferredName(), putUserRequest.username(), Fields.ROLES.getPreferredName(), putUserRequest.roles(), @@ -367,7 +367,7 @@ private void indexUser(final PutUserRequest putUserRequest, final ActionListener assert putUserRequest.passwordHash() != null; securityIndex.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareIndex(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, putUserRequest.username())) + client.prepareIndex(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, putUserRequest.username())) .setSource(Fields.USERNAME.getPreferredName(), putUserRequest.username(), Fields.PASSWORD.getPreferredName(), String.valueOf(putUserRequest.passwordHash()), Fields.ROLES.getPreferredName(), putUserRequest.roles(), @@ -410,7 +410,7 @@ private void setRegularUserEnabled(final String username, final boolean enabled, final ActionListener listener) { securityIndex.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, username)) + client.prepareUpdate(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, username)) .setDoc(Requests.INDEX_CONTENT_TYPE, Fields.ENABLED.getPreferredName(), enabled) .setRefreshPolicy(refreshPolicy) .request(), @@ -444,7 +444,7 @@ private void setReservedUserEnabled(final String username, final boolean enabled boolean clearCache, final ActionListener listener) { securityIndex.prepareIndexIfNeededThenExecute(listener::onFailure, () -> { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(RESERVED_USER_TYPE, username)) + client.prepareUpdate(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(RESERVED_USER_TYPE, username)) .setDoc(Requests.INDEX_CONTENT_TYPE, Fields.ENABLED.getPreferredName(), enabled) .setUpsert(XContentType.JSON, Fields.PASSWORD.getPreferredName(), "", @@ -479,7 +479,7 @@ public void deleteUser(final DeleteUserRequest deleteUserRequest, final ActionLi } else { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> { DeleteRequest request = client - .prepareDelete(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, deleteUserRequest.username())) + .prepareDelete(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(USER_DOC_TYPE, deleteUserRequest.username())) .request(); request.setRefreshPolicy(deleteUserRequest.getRefreshPolicy()); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, request, @@ -526,7 +526,7 @@ void getReservedUserInfo(String username, ActionListener liste } else { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareGet(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForUser(RESERVED_USER_TYPE, username)) + client.prepareGet(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForUser(RESERVED_USER_TYPE, username)) .request(), new ActionListener() { @Override @@ -571,7 +571,7 @@ void getAllReservedUserInfo(ActionListener> listen } else { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareSearch(SECURITY_INDEX_NAME) + client.prepareSearch(SECURITY_MAIN_ALIAS) .setTrackTotalHits(true) .setQuery(QueryBuilders.termQuery(Fields.TYPE.getPreferredName(), RESERVED_USER_TYPE)) .setFetchSource(true).request(), diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java index ab860ed058d91..1b6da7f68ca4e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java @@ -35,6 +35,7 @@ import org.elasticsearch.xpack.core.security.authc.support.mapper.ExpressionRoleMapping; import org.elasticsearch.xpack.core.security.authc.support.mapper.expressiondsl.ExpressionModel; import org.elasticsearch.xpack.core.security.client.SecurityClient; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.security.authc.support.CachingRealm; import org.elasticsearch.xpack.security.authc.support.UserRoleMapper; import org.elasticsearch.xpack.security.support.SecurityIndexManager; @@ -60,13 +61,13 @@ import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING; import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.isIndexDeleted; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.isMoveFromRedToNonRed; /** * This store reads + writes {@link ExpressionRoleMapping role mappings} in an Elasticsearch - * {@link SecurityIndexManager#SECURITY_INDEX_NAME index}. + * {@link RestrictedIndicesNames#SECURITY_MAIN_ALIAS index}. *
    * The store is responsible for all read and write operations as well as * {@link #resolveRoles(UserData, ActionListener) resolving roles}. @@ -131,7 +132,7 @@ protected void loadMappings(ActionListener> listener final QueryBuilder query = QueryBuilders.termQuery(DOC_TYPE_FIELD, DOC_TYPE_ROLE_MAPPING); final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { - SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) + SearchRequest request = client.prepareSearch(SECURITY_MAIN_ALIAS) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) .setSize(1000) @@ -143,7 +144,7 @@ protected void loadMappings(ActionListener> listener listener.onResponse(mappings.stream().filter(Objects::nonNull).collect(Collectors.toList())), ex -> { logger.error(new ParameterizedMessage("failed to load role mappings from index [{}] skipping all mappings.", - SECURITY_INDEX_NAME), ex); + SECURITY_MAIN_ALIAS), ex); listener.onResponse(Collections.emptyList()); })), doc -> buildMapping(getNameFromId(doc.getId()), doc.getSourceRef())); @@ -202,7 +203,7 @@ private void innerPutMapping(PutRoleMappingRequest request, ActionListener { executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareDelete(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForName(request.getName())) + client.prepareDelete(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForName(request.getName())) .setRefreshPolicy(request.getRefreshPolicy()) .request(), new ActionListener() { @@ -286,7 +287,7 @@ private void getMappings(ActionListener> listener) { logger.info("The security index is not yet available - no role mappings can be loaded"); if (logger.isDebugEnabled()) { logger.debug("Security Index [{}] [exists: {}] [available: {}] [mapping up to date: {}]", - SECURITY_INDEX_NAME, + SECURITY_MAIN_ALIAS, securityIndex.indexExists(), securityIndex.isAvailable(), securityIndex.isMappingUpToDate() diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java index 6a29f1de0e18b..7e70f6116cee7 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStore.java @@ -65,7 +65,7 @@ import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; import static org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor.DOC_TYPE_VALUE; import static org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor.Fields.APPLICATION; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; /** * {@code NativePrivilegeStore} is a store that reads/writes {@link ApplicationPrivilegeDescriptor} objects, @@ -129,7 +129,7 @@ public void getPrivileges(Collection applications, Collection na } final Supplier supplier = client.threadPool().getThreadContext().newRestorableContext(false); try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { - SearchRequest request = client.prepareSearch(SECURITY_INDEX_NAME) + SearchRequest request = client.prepareSearch(SECURITY_MAIN_ALIAS) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) .setSize(1000) @@ -201,7 +201,7 @@ void getPrivilege(String application, String name, ActionListener executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareGet(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, toDocId(application, name)) + client.prepareGet(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, toDocId(application, name)) .request(), new ActionListener() { @Override @@ -253,7 +253,7 @@ private void innerPutPrivilege(ApplicationPrivilegeDescriptor privilege, WriteRe final String name = privilege.getName(); final XContentBuilder xContentBuilder = privilege.toXContent(jsonBuilder(), true); ClientHelper.executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareIndex(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, toDocId(privilege.getApplication(), name)) + client.prepareIndex(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, toDocId(privilege.getApplication(), name)) .setSource(xContentBuilder) .setRefreshPolicy(refreshPolicy) .request(), listener, client::index); @@ -284,7 +284,7 @@ public void deletePrivileges(String application, Collection names, Write }, listener::onFailure), names.size()); for (String name : names) { ClientHelper.executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareDelete(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, toDocId(application, name)) + client.prepareDelete(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, toDocId(application, name)) .setRefreshPolicy(refreshPolicy) .request(), groupListener, client::delete); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java index 289a616e254d6..2169f84082b7f 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java @@ -66,7 +66,7 @@ import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; import static org.elasticsearch.xpack.core.security.authz.RoleDescriptor.ROLE_TYPE; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; /** * NativeRolesStore is a {@code RolesStore} that, instead of reading from a @@ -115,7 +115,7 @@ public void getRoleDescriptors(Set names, final ActionListener supplier = client.threadPool().getThreadContext().newRestorableContext(false); try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(SECURITY_ORIGIN)) { - SearchRequest request = client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + SearchRequest request = client.prepareSearch(SECURITY_MAIN_ALIAS) .setScroll(DEFAULT_KEEPALIVE_SETTING.get(settings)) .setQuery(query) .setSize(1000) @@ -133,7 +133,7 @@ public void getRoleDescriptors(Set names, final ActionListener { final String[] roleIds = names.stream().map(NativeRolesStore::getIdForRole).toArray(String[]::new); - MultiGetRequest multiGetRequest = client.prepareMultiGet().add(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, roleIds).request(); + MultiGetRequest multiGetRequest = client.prepareMultiGet().add(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, roleIds).request(); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, multiGetRequest, ActionListener.wrap(mGetResponse -> { final MultiGetItemResponse[] responses = mGetResponse.getResponses(); @@ -170,7 +170,7 @@ public void deleteRole(final DeleteRoleRequest deleteRoleRequest, final ActionLi } else { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> { DeleteRequest request = client - .prepareDelete(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForRole(deleteRoleRequest.name())).request(); + .prepareDelete(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForRole(deleteRoleRequest.name())).request(); request.setRefreshPolicy(deleteRoleRequest.getRefreshPolicy()); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, request, new ActionListener() { @@ -210,7 +210,7 @@ void innerPutRole(final PutRoleRequest request, final RoleDescriptor role, final listener.onFailure(e); return; } - final IndexRequest indexRequest = client.prepareIndex(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForRole(role.getName())) + final IndexRequest indexRequest = client.prepareIndex(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForRole(role.getName())) .setSource(xContentBuilder) .setRefreshPolicy(request.getRefreshPolicy()) .request(); @@ -244,11 +244,11 @@ public void usageStats(ActionListener> listener) { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, client.prepareMultiSearch() - .add(client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + .add(client.prepareSearch(SECURITY_MAIN_ALIAS) .setQuery(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) .setTrackTotalHits(true) .setSize(0)) - .add(client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + .add(client.prepareSearch(SECURITY_MAIN_ALIAS) .setQuery(QueryBuilders.boolQuery() .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) .must(QueryBuilders.boolQuery() @@ -259,7 +259,7 @@ public void usageStats(ActionListener> listener) { .setTrackTotalHits(true) .setSize(0) .setTerminateAfter(1)) - .add(client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + .add(client.prepareSearch(SECURITY_MAIN_ALIAS) .setQuery(QueryBuilders.boolQuery() .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) .filter(existsQuery("indices.query"))) @@ -331,7 +331,7 @@ public void onFailure(Exception e) { private void executeGetRoleRequest(String role, ActionListener listener) { securityIndex.checkIndexVersionThenExecute(listener::onFailure, () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, - client.prepareGet(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, getIdForRole(role)).request(), + client.prepareGet(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, getIdForRole(role)).request(), listener, client::get)); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java index 31cf1f1568f24..0d48cd6b856e1 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java @@ -53,6 +53,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.time.Instant; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -73,16 +74,18 @@ import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; /** - * Manages the lifecycle of a single index, mapping and data upgrades/migrations. + * Manages the lifecycle, mapping and data upgrades/migrations of the {@code RestrictedIndicesNames#SECURITY_MAIN_ALIAS} + * and {@code RestrictedIndicesNames#SECURITY_MAIN_ALIAS} alias-index pair. */ public class SecurityIndexManager implements ClusterStateListener { - public static final String INTERNAL_SECURITY_INDEX = RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7; - public static final int INTERNAL_INDEX_FORMAT = 6; + public static final int INTERNAL_MAIN_INDEX_FORMAT = 6; + public static final int INTERNAL_TOKENS_INDEX_FORMAT = 7; + public static final String SECURITY_MAIN_TEMPLATE_7 = "security-index-template-7"; + public static final String SECURITY_TOKENS_TEMPLATE_7 = "security-tokens-index-template-7"; public static final String SECURITY_VERSION_STRING = "security-version"; public static final String TEMPLATE_VERSION_PATTERN = Pattern.quote("${security.template.version}"); - public static final String SECURITY_TEMPLATE_NAME = "security-index-template"; - public static final String SECURITY_INDEX_NAME = ".security"; + private static final Logger logger = LogManager.getLogger(SecurityIndexManager.class); private final String aliasName; @@ -95,19 +98,26 @@ public class SecurityIndexManager implements ClusterStateListener { private volatile State indexState; - public static SecurityIndexManager buildSecurityIndexManager(Client client, ClusterService clusterService) { - return new SecurityIndexManager(client, SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX, INTERNAL_INDEX_FORMAT, - SecurityIndexManager::readSecurityTemplateAsBytes, clusterService); + public static SecurityIndexManager buildSecurityMainIndexManager(Client client, ClusterService clusterService) { + return new SecurityIndexManager(client, clusterService, RestrictedIndicesNames.SECURITY_MAIN_ALIAS, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7, INTERNAL_MAIN_INDEX_FORMAT, + () -> SecurityIndexManager.readTemplateAsBytes(SECURITY_MAIN_TEMPLATE_7)); } - private SecurityIndexManager(Client client, String aliasName, String internalIndexName, int internalIndexFormat, - Supplier mappingSourceSupplier, ClusterService clusterService) { + public static SecurityIndexManager buildSecurityTokensIndexManager(Client client, ClusterService clusterService) { + return new SecurityIndexManager(client, clusterService, RestrictedIndicesNames.SECURITY_TOKENS_ALIAS, + RestrictedIndicesNames.INTERNAL_SECURITY_TOKENS_INDEX_7, INTERNAL_TOKENS_INDEX_FORMAT, + () -> SecurityIndexManager.readTemplateAsBytes(SECURITY_TOKENS_TEMPLATE_7)); + } + + private SecurityIndexManager(Client client, ClusterService clusterService, String aliasName, String internalIndexName, + int internalIndexFormat, Supplier mappingSourceSupplier) { this(client, aliasName, internalIndexName, internalIndexFormat, mappingSourceSupplier, State.UNRECOVERED_STATE); clusterService.addListener(this); } private SecurityIndexManager(Client client, String aliasName, String internalIndexName, int internalIndexFormat, - Supplier mappingSourceSupplier, State indexState) { + Supplier mappingSourceSupplier, State indexState) { this.aliasName = aliasName; this.internalIndexName = internalIndexName; this.internalIndexFormat = internalIndexFormat; @@ -126,8 +136,16 @@ public boolean checkMappingVersion(Predicate requiredVersion) { return currentIndexState.mappingVersion == null || requiredVersion.test(currentIndexState.mappingVersion); } + public String aliasName() { + return aliasName; + } + public boolean indexExists() { - return this.indexState.indexExists; + return this.indexState.indexExists(); + } + + public Instant getCreationTime() { + return this.indexState.creationTime; } /** @@ -156,7 +174,7 @@ public ElasticsearchException getUnavailableReason() { throw new IllegalStateException("caller must make sure to use a frozen state and check indexAvailable"); } - if (localState.indexExists) { + if (localState.indexExists()) { return new UnavailableShardsException(null, "at least one primary shard for the index [" + localState.concreteIndexName + "] is unavailable"); } else { @@ -183,17 +201,17 @@ public void clusterChanged(ClusterChangedEvent event) { } final State previousState = indexState; final IndexMetaData indexMetaData = resolveConcreteIndex(aliasName, event.state().metaData()); - final boolean indexExists = indexMetaData != null; - final boolean isIndexUpToDate = indexExists == false || + final Instant creationTime = indexMetaData != null ? Instant.ofEpochMilli(indexMetaData.getCreationDate()) : null; + final boolean isIndexUpToDate = indexMetaData == null || INDEX_FORMAT_SETTING.get(indexMetaData.getSettings()).intValue() == internalIndexFormat; final boolean indexAvailable = checkIndexAvailable(event.state()); - final boolean mappingIsUpToDate = indexExists == false || checkIndexMappingUpToDate(event.state()); + final boolean mappingIsUpToDate = indexMetaData == null || checkIndexMappingUpToDate(event.state()); final Version mappingVersion = oldestIndexMappingVersion(event.state()); final ClusterHealthStatus indexStatus = indexMetaData == null ? null : new ClusterIndexHealth(indexMetaData, event.state().getRoutingTable().index(indexMetaData.getIndex())).getStatus(); final String concreteIndexName = indexMetaData == null ? internalIndexName : indexMetaData.getIndex().getName(); - final State newState = new State(indexExists, isIndexUpToDate, indexAvailable, mappingIsUpToDate, mappingVersion, concreteIndexName, - indexStatus); + final State newState = new State(creationTime, isIndexUpToDate, indexAvailable, mappingIsUpToDate, mappingVersion, + concreteIndexName, indexStatus); this.indexState = newState; if (newState.equals(previousState) == false) { @@ -304,7 +322,7 @@ private static Version readMappingVersion(String indexName, MappingMetaData mapp */ public void checkIndexVersionThenExecute(final Consumer consumer, final Runnable andThen) { final State indexState = this.indexState; // use a local copy so all checks execute against the same state! - if (indexState.indexExists && indexState.isIndexUpToDate == false) { + if (indexState.indexExists() && indexState.isIndexUpToDate == false) { consumer.accept(new IllegalStateException( "Index [" + indexState.concreteIndexName + "] is not on the current version. Security features relying on the index" + " will not be available until the upgrade API is run on the index")); @@ -324,11 +342,11 @@ public void prepareIndexIfNeededThenExecute(final Consumer consumer, consumer.accept(new ElasticsearchStatusException( "Cluster state has not been recovered yet, cannot write to the [" + indexState.concreteIndexName + "] index", RestStatus.SERVICE_UNAVAILABLE)); - } else if (indexState.indexExists && indexState.isIndexUpToDate == false) { + } else if (indexState.indexExists() && indexState.isIndexUpToDate == false) { consumer.accept(new IllegalStateException( "Index [" + indexState.concreteIndexName + "] is not on the current version." + "Security features relying on the index will not be available until the upgrade API is run on the index")); - } else if (indexState.indexExists == false) { + } else if (indexState.indexExists() == false) { assert indexState.concreteIndexName != null; logger.info("security index does not exist. Creating [{}] with alias [{}]", indexState.concreteIndexName, this.aliasName); final byte[] mappingSource = mappingSourceSupplier.get(); @@ -396,8 +414,8 @@ public static boolean isIndexDeleted(State previousState, State currentState) { return previousState.indexStatus != null && currentState.indexStatus == null; } - private static byte[] readSecurityTemplateAsBytes() { - return TemplateUtils.loadTemplate("/" + SECURITY_TEMPLATE_NAME + ".json", Version.CURRENT.toString(), + private static byte[] readTemplateAsBytes(String templateName) { + return TemplateUtils.loadTemplate("/" + templateName + ".json", Version.CURRENT.toString(), SecurityIndexManager.TEMPLATE_VERSION_PATTERN).getBytes(StandardCharsets.UTF_8); } @@ -423,8 +441,8 @@ private static Tuple parseMappingAndSettingsFromTemplateBytes( * State of the security index. */ public static class State { - public static final State UNRECOVERED_STATE = new State(false, false, false, false, null, null, null); - public final boolean indexExists; + public static final State UNRECOVERED_STATE = new State(null, false, false, false, null, null, null); + public final Instant creationTime; public final boolean isIndexUpToDate; public final boolean indexAvailable; public final boolean mappingUpToDate; @@ -432,9 +450,9 @@ public static class State { public final String concreteIndexName; public final ClusterHealthStatus indexStatus; - public State(boolean indexExists, boolean isIndexUpToDate, boolean indexAvailable, + public State(Instant creationTime, boolean isIndexUpToDate, boolean indexAvailable, boolean mappingUpToDate, Version mappingVersion, String concreteIndexName, ClusterHealthStatus indexStatus) { - this.indexExists = indexExists; + this.creationTime = creationTime; this.isIndexUpToDate = isIndexUpToDate; this.indexAvailable = indexAvailable; this.mappingUpToDate = mappingUpToDate; @@ -448,7 +466,7 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; State state = (State) o; - return indexExists == state.indexExists && + return Objects.equals(creationTime, state.creationTime) && isIndexUpToDate == state.isIndexUpToDate && indexAvailable == state.indexAvailable && mappingUpToDate == state.mappingUpToDate && @@ -457,9 +475,13 @@ public boolean equals(Object o) { indexStatus == state.indexStatus; } + public boolean indexExists() { + return creationTime != null; + } + @Override public int hashCode() { - return Objects.hash(indexExists, isIndexUpToDate, indexAvailable, mappingUpToDate, mappingVersion, concreteIndexName, + return Objects.hash(creationTime, isIndexUpToDate, indexAvailable, mappingUpToDate, mappingVersion, concreteIndexName, indexStatus); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/ClearRolesCacheTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/ClearRolesCacheTests.java index c7f5123c6a1b7..6d7eacfe26cfa 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/ClearRolesCacheTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/ClearRolesCacheTests.java @@ -13,8 +13,8 @@ import org.elasticsearch.xpack.core.security.action.role.PutRoleResponse; import org.elasticsearch.xpack.core.security.authz.store.RoleRetrievalResult; import org.elasticsearch.xpack.core.security.client.SecurityClient; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.security.authz.store.NativeRolesStore; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.Before; import org.junit.BeforeClass; @@ -56,7 +56,7 @@ public void setupForTests() { logger.debug("--> created role [{}]", role); } - ensureGreen(SecurityIndexManager.SECURITY_INDEX_NAME); + ensureGreen(RestrictedIndicesNames.SECURITY_MAIN_ALIAS); final Set rolesSet = new HashSet<>(Arrays.asList(roles)); // warm up the caches on every node diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/NativeRealmIntegTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/NativeRealmIntegTestCase.java index 63f5ace535229..671a94452fa0a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/NativeRealmIntegTestCase.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/NativeRealmIntegTestCase.java @@ -59,7 +59,7 @@ protected boolean addMockHttpTransport() { @Override public Set excludeTemplates() { Set templates = Sets.newHashSet(super.excludeTemplates()); - templates.add(SecurityIndexManager.SECURITY_TEMPLATE_NAME); // don't remove the security index template + templates.add(SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7); // don't remove the security index template return templates; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java index c3e3bddf10e97..bd9d58e6ea549 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecurityIntegTestCase.java @@ -67,7 +67,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.hamcrest.Matchers.is; import static org.hamcrest.core.IsCollectionContaining.hasItem; @@ -491,7 +491,7 @@ public void assertSecurityIndexActive(TestCluster testCluster) throws Exception XContentBuilder builder = JsonXContent.contentBuilder().prettyPrint().startObject(); assertTrue("security index mapping not sufficient to read:\n" + Strings.toString(clusterState.toXContent(builder, ToXContent.EMPTY_PARAMS).endObject()), - SecurityIndexManager.checkIndexMappingVersionMatches(SECURITY_INDEX_NAME, clusterState, logger, + SecurityIndexManager.checkIndexMappingVersionMatches(SECURITY_MAIN_ALIAS, clusterState, logger, Version.CURRENT.minimumIndexCompatibilityVersion()::onOrBefore)); Index securityIndex = resolveSecurityIndex(clusterState.metaData()); if (securityIndex != null) { @@ -509,7 +509,7 @@ protected void deleteSecurityIndex() { UsernamePasswordToken.basicAuthHeaderValue(SecuritySettingsSource.TEST_SUPERUSER, SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING))); GetIndexRequest getIndexRequest = new GetIndexRequest(); - getIndexRequest.indices(SECURITY_INDEX_NAME); + getIndexRequest.indices(SECURITY_MAIN_ALIAS); getIndexRequest.indicesOptions(IndicesOptions.lenientExpandOpen()); GetIndexResponse getIndexResponse = client.admin().indices().getIndex(getIndexRequest).actionGet(); if (getIndexResponse.getIndices().length > 0) { @@ -520,7 +520,7 @@ protected void deleteSecurityIndex() { } private static Index resolveSecurityIndex(MetaData metaData) { - final AliasOrIndex aliasOrIndex = metaData.getAliasAndIndexLookup().get(SECURITY_INDEX_NAME); + final AliasOrIndex aliasOrIndex = metaData.getAliasAndIndexLookup().get(SECURITY_MAIN_ALIAS); if (aliasOrIndex != null) { return aliasOrIndex.getIndices().get(0).getIndex(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index 08f1149572faa..02db3a1e11a46 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -67,8 +67,8 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_FORMAT_SETTING; import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_INDEX_FORMAT; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -296,8 +296,8 @@ public void testIndexJoinValidator_Old_And_Rolling() throws Exception { BiConsumer joinValidator = security.getJoinValidator(); assertNotNull(joinValidator); DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT); - IndexMetaData indexMetaData = IndexMetaData.builder(SECURITY_INDEX_NAME) - .settings(settings(Version.V_6_1_0).put(INDEX_FORMAT_SETTING.getKey(), INTERNAL_INDEX_FORMAT - 1)) + IndexMetaData indexMetaData = IndexMetaData.builder(SECURITY_MAIN_ALIAS) + .settings(settings(Version.V_6_1_0).put(INDEX_FORMAT_SETTING.getKey(), INTERNAL_MAIN_INDEX_FORMAT - 1)) .numberOfShards(1).numberOfReplicas(0) .build(); DiscoveryNode existingOtherNode = new DiscoveryNode("bar", buildNewFakeTransportAddress(), Version.V_6_1_0); @@ -316,8 +316,8 @@ public void testIndexJoinValidator_FullyCurrentCluster() throws Exception { BiConsumer joinValidator = security.getJoinValidator(); assertNotNull(joinValidator); DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT); - int indexFormat = randomBoolean() ? INTERNAL_INDEX_FORMAT : INTERNAL_INDEX_FORMAT - 1; - IndexMetaData indexMetaData = IndexMetaData.builder(SECURITY_INDEX_NAME) + int indexFormat = randomBoolean() ? INTERNAL_MAIN_INDEX_FORMAT : INTERNAL_MAIN_INDEX_FORMAT - 1; + IndexMetaData indexMetaData = IndexMetaData.builder(SECURITY_MAIN_ALIAS) .settings(settings(Version.V_6_1_0).put(INDEX_FORMAT_SETTING.getKey(), indexFormat)) .numberOfShards(1).numberOfReplicas(0) .build(); @@ -335,8 +335,8 @@ public void testIndexUpgradeValidatorWithUpToDateIndex() throws Exception { assertNotNull(joinValidator); Version version = randomBoolean() ? Version.CURRENT : Version.V_6_1_0; DiscoveryNode node = new DiscoveryNode("foo", buildNewFakeTransportAddress(), Version.CURRENT); - IndexMetaData indexMetaData = IndexMetaData.builder(SECURITY_INDEX_NAME) - .settings(settings(version).put(INDEX_FORMAT_SETTING.getKey(), INTERNAL_INDEX_FORMAT)) + IndexMetaData indexMetaData = IndexMetaData.builder(SECURITY_MAIN_ALIAS) + .settings(settings(version).put(INDEX_FORMAT_SETTING.getKey(), INTERNAL_MAIN_INDEX_FORMAT)) .numberOfShards(1).numberOfReplicas(0) .build(); DiscoveryNode existingOtherNode = new DiscoveryNode("bar", buildNewFakeTransportAddress(), version); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/oidc/TransportOpenIdConnectLogoutActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/oidc/TransportOpenIdConnectLogoutActionTests.java index ddf1742109915..e31ccc6733290 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/oidc/TransportOpenIdConnectLogoutActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/oidc/TransportOpenIdConnectLogoutActionTests.java @@ -167,7 +167,7 @@ public void setup() throws Exception { when(securityIndex.isAvailable()).thenReturn(true); final ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); - tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, clusterService); + tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, securityIndex, clusterService); final TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java index de0463ffcaab3..63c58c5ce10e8 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java @@ -195,10 +195,13 @@ void doExecute(Action action, Request request, ActionListener null, null, Collections.emptySet()); @@ -308,7 +311,8 @@ public void testInvalidateCorrectTokensFromLogoutRequest() throws Exception { assertThat(filter1.get(1), instanceOf(TermQueryBuilder.class)); assertThat(((TermQueryBuilder) filter1.get(1)).fieldName(), equalTo("refresh_token.token")); - assertThat(((TermQueryBuilder) filter1.get(1)).value(), equalTo(tokenToInvalidate1.v2())); + assertThat(((TermQueryBuilder) filter1.get(1)).value(), + equalTo(TokenService.unpackVersionAndPayload(tokenToInvalidate1.v2()).v2())); assertThat(bulkRequests.size(), equalTo(4)); // 4 updates (refresh-token + access-token) // Invalidate refresh token 1 diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java index 22b9fb7a11127..bd1a20db2f196 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlLogoutActionTests.java @@ -203,7 +203,7 @@ public void setup() throws Exception { when(securityIndex.isAvailable()).thenReturn(true); final ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); - tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, clusterService); + tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, securityIndex, clusterService); final TransportService transportService = new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/token/TransportCreateTokenActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/token/TransportCreateTokenActionTests.java index fb1399220474d..cea1a532e13c6 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/token/TransportCreateTokenActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/token/TransportCreateTokenActionTests.java @@ -147,7 +147,8 @@ public void stopThreadPool() throws Exception { } public void testClientCredentialsCreatesWithoutRefreshToken() throws Exception { - final TokenService tokenService = new TokenService(SETTINGS, Clock.systemUTC(), client, securityIndex, clusterService); + final TokenService tokenService = new TokenService(SETTINGS, Clock.systemUTC(), client, securityIndex, securityIndex, + clusterService); Authentication authentication = new Authentication(new User("joe"), new Authentication.RealmRef("realm", "type", "node"), null); authentication.writeToContext(threadPool.getThreadContext()); @@ -171,7 +172,8 @@ public void testClientCredentialsCreatesWithoutRefreshToken() throws Exception { } public void testPasswordGrantTypeCreatesWithRefreshToken() throws Exception { - final TokenService tokenService = new TokenService(SETTINGS, Clock.systemUTC(), client, securityIndex, clusterService); + final TokenService tokenService = new TokenService(SETTINGS, Clock.systemUTC(), client, securityIndex, securityIndex, + clusterService); Authentication authentication = new Authentication(new User("joe"), new Authentication.RealmRef("realm", "type", "node"), null); authentication.writeToContext(threadPool.getThreadContext()); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java index 256bf6d9df532..f6849cae4c1cd 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import org.elasticsearch.xpack.core.security.client.SecurityClient; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.security.transport.filter.IPFilter; import org.junit.After; import org.junit.Before; @@ -53,7 +52,7 @@ import java.util.stream.Collectors; import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -336,7 +335,7 @@ public void testExpiredApiKeysBehaviorWhenKeysExpired1WeekBeforeAnd1DayBefore() Instant dayBefore = created.minus(1L, ChronoUnit.DAYS); assertTrue(Instant.now().isAfter(dayBefore)); UpdateResponse expirationDateUpdatedResponse = client - .prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, createdApiKeys.get(0).getId()) + .prepareUpdate(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, createdApiKeys.get(0).getId()) .setDoc("expiration_time", dayBefore.toEpochMilli()) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); @@ -346,7 +345,7 @@ public void testExpiredApiKeysBehaviorWhenKeysExpired1WeekBeforeAnd1DayBefore() // hack doc to modify the expiration time to the week before Instant weekBefore = created.minus(8L, ChronoUnit.DAYS); assertTrue(Instant.now().isAfter(weekBefore)); - expirationDateUpdatedResponse = client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, createdApiKeys.get(1).getId()) + expirationDateUpdatedResponse = client.prepareUpdate(SECURITY_MAIN_ALIAS, SINGLE_MAPPING_NAME, createdApiKeys.get(1).getId()) .setDoc("expiration_time", weekBefore.toEpochMilli()) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); @@ -390,8 +389,7 @@ public void testExpiredApiKeysBehaviorWhenKeysExpired1WeekBeforeAnd1DayBefore() private void refreshSecurityIndex() throws Exception { assertBusy(() -> { - final RefreshResponse refreshResponse = client().admin().indices().prepareRefresh(SecurityIndexManager.SECURITY_INDEX_NAME) - .get(); + final RefreshResponse refreshResponse = client().admin().indices().prepareRefresh(SECURITY_MAIN_ALIAS).get(); assertThat(refreshResponse.getFailedShards(), is(0)); }); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java index 63e8578e8c2fc..a86edb98251ff 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java @@ -154,7 +154,7 @@ public class AuthenticationServiceTests extends ESTestCase { @SuppressForbidden(reason = "Allow accessing localhost") public void init() throws Exception { concreteSecurityIndexName = randomFrom( - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); token = mock(AuthenticationToken.class); when(token.principal()).thenReturn(randomAlphaOfLength(5)); @@ -220,7 +220,7 @@ licenseState, threadContext, mock(ReservedRealm.class), Arrays.asList(firstRealm }).when(securityIndex).checkIndexVersionThenExecute(any(Consumer.class), any(Runnable.class)); ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); apiKeyService = new ApiKeyService(settings, Clock.systemUTC(), client, securityIndex, clusterService, threadPool); - tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, clusterService); + tokenService = new TokenService(settings, Clock.systemUTC(), client, securityIndex, securityIndex, clusterService); service = new AuthenticationService(settings, realms, auditTrail, new DefaultAuthenticationFailureHandler(Collections.emptyMap()), threadPool, new AnonymousUser(settings), tokenService, apiKeyService); } @@ -1394,6 +1394,6 @@ private void setCompletedToTrue(AtomicBoolean completed) { } private SecurityIndexManager.State dummyState(ClusterHealthStatus indexStatus) { - return new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, indexStatus); + return new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, indexStatus); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenAuthIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenAuthIntegTests.java index 6bcdfc94e9957..7bac18cfcfb67 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenAuthIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenAuthIntegTests.java @@ -36,7 +36,7 @@ import org.elasticsearch.xpack.core.security.authc.TokenMetaData; import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; import org.elasticsearch.xpack.core.security.client.SecurityClient; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.junit.After; import org.junit.Before; @@ -55,7 +55,6 @@ import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; import static org.hamcrest.Matchers.equalTo; @TestLogging("org.elasticsearch.xpack.security.authz.store.FileRolesStore:DEBUG") @@ -161,7 +160,7 @@ public void testExpiredTokensDeletedAfterExpiration() throws Exception { assertThat(invalidateResponse.getResult().getErrors().size(), equalTo(0)); AtomicReference docId = new AtomicReference<>(); assertBusy(() -> { - SearchResponse searchResponse = client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + SearchResponse searchResponse = client.prepareSearch(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS) .setSource(SearchSourceBuilder.searchSource() .query(QueryBuilders.termQuery("doc_type", "token"))) .setSize(1) @@ -174,7 +173,7 @@ public void testExpiredTokensDeletedAfterExpiration() throws Exception { // hack doc to modify the creation time to the day before Instant yesterday = created.minus(36L, ChronoUnit.HOURS); assertTrue(Instant.now().isAfter(yesterday)); - client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, docId.get()) + client.prepareUpdate(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS, SINGLE_MAPPING_NAME, docId.get()) .setDoc("creation_time", yesterday.toEpochMilli()) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); @@ -192,8 +191,8 @@ public void testExpiredTokensDeletedAfterExpiration() throws Exception { assertEquals("token malformed", e.getMessage()); } } - client.admin().indices().prepareRefresh(SecurityIndexManager.SECURITY_INDEX_NAME).get(); - SearchResponse searchResponse = client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + client.admin().indices().prepareRefresh(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS).get(); + SearchResponse searchResponse = client.prepareSearch(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS) .setSource(SearchSourceBuilder.searchSource() .query(QueryBuilders.termQuery("doc_type", "token"))) .setTerminateAfter(1) @@ -358,10 +357,10 @@ public void testRefreshingMultipleTimesFails() throws Exception { // We now have two documents, the original(now refreshed) token doc and the new one with the new access doc AtomicReference docId = new AtomicReference<>(); assertBusy(() -> { - SearchResponse searchResponse = client.prepareSearch(SecurityIndexManager.SECURITY_INDEX_NAME) + SearchResponse searchResponse = client.prepareSearch(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS) .setSource(SearchSourceBuilder.searchSource() .query(QueryBuilders.boolQuery() - .must(QueryBuilders.termQuery("doc_type", "token")) + .must(QueryBuilders.termQuery("doc_type", TokenService.TOKEN_DOC_TYPE)) .must(QueryBuilders.termQuery("refresh_token.refreshed", "true")))) .setSize(1) .setTerminateAfter(1) @@ -374,7 +373,7 @@ public void testRefreshingMultipleTimesFails() throws Exception { Instant refreshed = Instant.now(); Instant aWhileAgo = refreshed.minus(50L, ChronoUnit.SECONDS); assertTrue(Instant.now().isAfter(aWhileAgo)); - UpdateResponse updateResponse = client.prepareUpdate(SECURITY_INDEX_NAME, SINGLE_MAPPING_NAME, docId.get()) + UpdateResponse updateResponse = client.prepareUpdate(RestrictedIndicesNames.SECURITY_TOKENS_ALIAS, SINGLE_MAPPING_NAME, docId.get()) .setDoc("refresh_token", Collections.singletonMap("refresh_time", aWhileAgo.toEpochMilli())) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .setFetchSource("refresh_token", Strings.EMPTY_STRING) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java index 3d050bd2af79c..494b8070c57d9 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java @@ -21,6 +21,9 @@ import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; @@ -51,6 +54,7 @@ import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.core.watcher.watch.ClockMock; import org.elasticsearch.xpack.security.support.SecurityIndexManager; +import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -74,6 +78,7 @@ import static java.time.Clock.systemUTC; import static org.elasticsearch.repositories.ESBlobStoreTestCase.randomBytes; +import static org.elasticsearch.test.ClusterServiceUtils.setState; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; @@ -92,8 +97,10 @@ public class TokenServiceTests extends ESTestCase { .put(XPackSettings.TOKEN_SERVICE_ENABLED_SETTING.getKey(), true).build(); private Client client; - private SecurityIndexManager securityIndex; + private SecurityIndexManager securityMainIndex; + private SecurityIndexManager securityTokensIndex; private ClusterService clusterService; + private DiscoveryNode oldNode; private Settings tokenServiceEnabledSettings = Settings.builder() .put(XPackSettings.TOKEN_SERVICE_ENABLED_SETTING.getKey(), true).build(); @@ -121,20 +128,21 @@ public void setupClient() { }).when(client).execute(eq(IndexAction.INSTANCE), any(IndexRequest.class), any(ActionListener.class)); // setup lifecycle service - securityIndex = mock(SecurityIndexManager.class); - doAnswer(invocationOnMock -> { - Runnable runnable = (Runnable) invocationOnMock.getArguments()[1]; - runnable.run(); - return null; - }).when(securityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); - doAnswer(invocationOnMock -> { - Runnable runnable = (Runnable) invocationOnMock.getArguments()[1]; - runnable.run(); - return null; - }).when(securityIndex).checkIndexVersionThenExecute(any(Consumer.class), any(Runnable.class)); - when(securityIndex.indexExists()).thenReturn(true); - when(securityIndex.isAvailable()).thenReturn(true); + this.securityMainIndex = mockSecurityManager(); + this.securityTokensIndex = mockSecurityManager(); this.clusterService = ClusterServiceUtils.createClusterService(threadPool); + // version 7.1 was an "inflection" point in the Token Service development (access_tokens as UUIDS, multiple concurrent refreshes, + // tokens docs on a separate index), let's test the TokenService works in a mixed cluster with nodes with versions prior to these + // developments + if (randomBoolean()) { + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_0_0)); + } + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + clusterService.close(); } @BeforeClass @@ -151,7 +159,8 @@ public static void shutdownThreadpool() throws InterruptedException { } public void testAttachAndGetToken() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -172,8 +181,8 @@ public void testAttachAndGetToken() throws Exception { try (ThreadContext.StoredContext ignore = requestContext.newStoredContext(true)) { // verify a second separate token service with its own salt can also verify - TokenService anotherService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex - , clusterService); + TokenService anotherService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); anotherService.refreshMetaData(tokenService.getTokenMetaData()); PlainActionFuture future = new PlainActionFuture<>(); anotherService.getAndValidateToken(requestContext, future); @@ -183,7 +192,8 @@ public void testAttachAndGetToken() throws Exception { } public void testInvalidAuthorizationHeader() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); ThreadContext requestContext = new ThreadContext(Settings.EMPTY); String token = randomFrom("", " "); String authScheme = randomFrom("Bearer ", "BEARER ", "bearer ", "Basic "); @@ -198,7 +208,8 @@ public void testInvalidAuthorizationHeader() throws Exception { } public void testRotateKey() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -251,12 +262,14 @@ private void rotateKeys(TokenService tokenService) { } public void testKeyExchange() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); int numRotations = randomIntBetween(1, 5); for (int i = 0; i < numRotations; i++) { rotateKeys(tokenService); } - TokenService otherTokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService otherTokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); otherTokenService.refreshMetaData(tokenService.getTokenMetaData()); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); @@ -272,7 +285,7 @@ public void testKeyExchange() throws Exception { PlainActionFuture future = new PlainActionFuture<>(); otherTokenService.getAndValidateToken(requestContext, future); UserToken serialized = future.get(); - assertAuthentication(authentication, serialized.getAuthentication()); + assertEquals(authentication, serialized.getAuthentication()); } rotateKeys(tokenService); @@ -288,7 +301,8 @@ public void testKeyExchange() throws Exception { } public void testPruneKeys() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -350,7 +364,8 @@ public void testPruneKeys() throws Exception { } public void testPassphraseWorks() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -371,8 +386,8 @@ public void testPassphraseWorks() throws Exception { try (ThreadContext.StoredContext ignore = requestContext.newStoredContext(true)) { // verify a second separate token service with its own passphrase cannot verify - TokenService anotherService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, - clusterService); + TokenService anotherService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); PlainActionFuture future = new PlainActionFuture<>(); anotherService.getAndValidateToken(requestContext, future); assertNull(future.get()); @@ -380,7 +395,8 @@ public void testPassphraseWorks() throws Exception { } public void testGetTokenWhenKeyCacheHasExpired() throws Exception { - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); @@ -393,9 +409,9 @@ public void testGetTokenWhenKeyCacheHasExpired() throws Exception { } public void testInvalidatedToken() throws Exception { - when(securityIndex.indexExists()).thenReturn(true); - TokenService tokenService = - new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + when(securityMainIndex.indexExists()).thenReturn(true); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -449,7 +465,8 @@ public void testTokenExpiryConfig() { public void testTokenExpiry() throws Exception { ClockMock clock = ClockMock.frozen(); - TokenService tokenService = new TokenService(tokenServiceEnabledSettings, clock, client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, clock, client, securityMainIndex, securityTokensIndex, + clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -502,7 +519,7 @@ public void testTokenServiceDisabled() throws Exception { TokenService tokenService = new TokenService(Settings.builder() .put(XPackSettings.TOKEN_SERVICE_ENABLED_SETTING.getKey(), false) .build(), - Clock.systemUTC(), client, securityIndex, clusterService); + Clock.systemUTC(), client, securityMainIndex, securityTokensIndex, clusterService); IllegalStateException e = expectThrows(IllegalStateException.class, () -> tokenService.createOAuth2Tokens(null, null, null, true, null)); assertEquals("tokens are not enabled", e.getMessage()); @@ -545,7 +562,8 @@ public void testMalformedToken() throws Exception { final int numBytes = randomIntBetween(1, TokenService.MINIMUM_BYTES + 32); final byte[] randomBytes = new byte[numBytes]; random().nextBytes(randomBytes); - TokenService tokenService = new TokenService(Settings.EMPTY, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(Settings.EMPTY, systemUTC(), client, securityMainIndex, securityTokensIndex, + clusterService); ThreadContext requestContext = new ThreadContext(Settings.EMPTY); requestContext.putHeader("Authorization", "Bearer " + Base64.getEncoder().encodeToString(randomBytes)); @@ -558,8 +576,8 @@ public void testMalformedToken() throws Exception { } public void testIndexNotAvailable() throws Exception { - TokenService tokenService = - new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); tokenService.createOAuth2Tokens(authentication, authentication, Collections.emptyMap(), true, tokenFuture); @@ -576,34 +594,44 @@ public void testIndexNotAvailable() throws Exception { return Void.TYPE; }).when(client).get(any(GetRequest.class), any(ActionListener.class)); + final SecurityIndexManager tokensIndex; + if (oldNode != null) { + tokensIndex = securityMainIndex; + when(securityTokensIndex.isAvailable()).thenReturn(false); + when(securityTokensIndex.indexExists()).thenReturn(false); + } else { + tokensIndex = securityTokensIndex; + when(securityMainIndex.isAvailable()).thenReturn(false); + when(securityMainIndex.indexExists()).thenReturn(false); + } try (ThreadContext.StoredContext ignore = requestContext.newStoredContext(true)) { PlainActionFuture future = new PlainActionFuture<>(); tokenService.getAndValidateToken(requestContext, future); assertNull(future.get()); - when(securityIndex.isAvailable()).thenReturn(false); - when(securityIndex.indexExists()).thenReturn(true); + when(tokensIndex.isAvailable()).thenReturn(false); + when(tokensIndex.indexExists()).thenReturn(true); future = new PlainActionFuture<>(); tokenService.getAndValidateToken(requestContext, future); assertNull(future.get()); - when(securityIndex.indexExists()).thenReturn(false); + when(tokensIndex.indexExists()).thenReturn(false); future = new PlainActionFuture<>(); tokenService.getAndValidateToken(requestContext, future); assertNull(future.get()); - when(securityIndex.isAvailable()).thenReturn(true); - when(securityIndex.indexExists()).thenReturn(true); + when(tokensIndex.isAvailable()).thenReturn(true); + when(tokensIndex.indexExists()).thenReturn(true); mockGetTokenFromId(token, false); future = new PlainActionFuture<>(); tokenService.getAndValidateToken(requestContext, future); - assertEquals(token.getAuthentication(), future.get().getAuthentication()); + assertEquals(future.get().getAuthentication(), token.getAuthentication()); } } public void testGetAuthenticationWorksWithExpiredUserToken() throws Exception { - TokenService tokenService = - new TokenService(tokenServiceEnabledSettings, Clock.systemUTC(), client, securityIndex, clusterService); + TokenService tokenService = new TokenService(tokenServiceEnabledSettings, Clock.systemUTC(), client, securityMainIndex, + securityTokensIndex, clusterService); Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); UserToken expired = new UserToken(authentication, Instant.now().minus(3L, ChronoUnit.DAYS)); mockGetTokenFromId(expired, false); @@ -672,4 +700,33 @@ protected String getDeprecatedAccessTokenString(TokenService tokenService, UserT } } + private SecurityIndexManager mockSecurityManager() { + SecurityIndexManager mockSecurityIndex = mock(SecurityIndexManager.class); + doAnswer(invocationOnMock -> { + Runnable runnable = (Runnable) invocationOnMock.getArguments()[1]; + runnable.run(); + return null; + }).when(mockSecurityIndex).prepareIndexIfNeededThenExecute(any(Consumer.class), any(Runnable.class)); + doAnswer(invocationOnMock -> { + Runnable runnable = (Runnable) invocationOnMock.getArguments()[1]; + runnable.run(); + return null; + }).when(mockSecurityIndex).checkIndexVersionThenExecute(any(Consumer.class), any(Runnable.class)); + when(mockSecurityIndex.indexExists()).thenReturn(true); + when(mockSecurityIndex.isAvailable()).thenReturn(true); + return mockSecurityIndex; + } + + private DiscoveryNode addAnotherDataNodeWithVersion(ClusterService clusterService, Version version) { + final ClusterState currentState = clusterService.state(); + final DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(currentState.getNodes()); + final DiscoveryNode anotherDataNode = new DiscoveryNode("another_data_node#" + version, buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.singleton(DiscoveryNode.Role.DATA), version); + discoBuilder.add(anotherDataNode); + final ClusterState.Builder newStateBuilder = ClusterState.builder(currentState); + newStateBuilder.nodes(discoBuilder); + setState(clusterService, newStateBuilder.build()); + return anotherDataNode; + } + } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java index 650ccc55e41b3..a73fc93f32e45 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java @@ -15,7 +15,7 @@ import org.elasticsearch.test.NativeRealmIntegTestCase; import org.elasticsearch.common.CharArrays; import org.elasticsearch.xpack.core.security.client.SecurityClient; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.junit.BeforeClass; import java.nio.charset.StandardCharsets; @@ -85,7 +85,7 @@ public void testRetrieveUsers() throws Exception { addedUsers.add(uname); } logger.error("--> waiting for .security index"); - ensureGreen(SecurityIndexManager.SECURITY_INDEX_NAME); + ensureGreen(RestrictedIndicesNames.SECURITY_MAIN_ALIAS); MockTerminal t = new MockTerminal(); String username = nodeClientUsername(); @@ -136,7 +136,7 @@ public void testRetrieveRoles() throws Exception { addedRoles.add(rname); } logger.error("--> waiting for .security index"); - ensureGreen(SecurityIndexManager.SECURITY_INDEX_NAME); + ensureGreen(RestrictedIndicesNames.SECURITY_MAIN_ALIAS); MockTerminal t = new MockTerminal(); String username = nodeClientUsername(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java index 2d5aaa73f2c58..7021873a02878 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmIntegTests.java @@ -44,13 +44,13 @@ import org.elasticsearch.xpack.core.security.authz.permission.Role; import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; import org.elasticsearch.xpack.core.security.client.SecurityClient; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.core.security.user.AnonymousUser; import org.elasticsearch.xpack.core.security.user.ElasticUser; import org.elasticsearch.xpack.core.security.user.KibanaUser; import org.elasticsearch.xpack.core.security.user.SystemUser; import org.elasticsearch.xpack.core.security.user.User; import org.elasticsearch.xpack.security.authz.store.NativeRolesStore; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.junit.Before; import org.junit.BeforeClass; @@ -66,8 +66,8 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_SECURITY_INDEX; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; @@ -145,7 +145,7 @@ public void testAddAndGetUser() throws Exception { logger.error("--> creating user"); c.preparePutUser("joe", "s3kirt".toCharArray(), hasher, "role1", "user").get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); logger.info("--> retrieving user"); GetUsersResponse resp = c.prepareGetUsers("joe").get(); assertTrue("user should exist", resp.hasUsers()); @@ -200,7 +200,7 @@ public void testAddAndGetRole() throws Exception { .metadata(metadata) .get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); logger.info("--> retrieving role"); GetRolesResponse resp = c.prepareGetRoles().names("test_role").get(); assertTrue("role should exist", resp.hasRoles()); @@ -251,7 +251,7 @@ public void testAddUserAndRoleThenAuth() throws Exception { logger.error("--> creating user"); c.preparePutUser("joe", "s3krit".toCharArray(), hasher, "test_role").get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); logger.info("--> retrieving user"); GetUsersResponse resp = c.prepareGetUsers("joe").get(); assertTrue("user should exist", resp.hasUsers()); @@ -272,7 +272,7 @@ public void testUpdatingUserAndAuthentication() throws Exception { logger.error("--> creating user"); c.preparePutUser("joe", "s3krit".toCharArray(), hasher, SecuritySettingsSource.TEST_ROLE).get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); logger.info("--> retrieving user"); GetUsersResponse resp = c.prepareGetUsers("joe").get(); assertTrue("user should exist", resp.hasUsers()); @@ -308,7 +308,7 @@ public void testCreateDeleteAuthenticate() { c.preparePutUser("joe", "s3krit".toCharArray(), hasher, SecuritySettingsSource.TEST_ROLE).get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); logger.info("--> retrieving user"); GetUsersResponse resp = c.prepareGetUsers("joe").get(); assertTrue("user should exist", resp.hasUsers()); @@ -346,7 +346,7 @@ public void testCreateAndUpdateRole() { logger.error("--> creating user"); c.preparePutUser("joe", "s3krit".toCharArray(), hasher, "test_role").get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); if (authenticate) { final String token = basicAuthHeaderValue("joe", new SecureString("s3krit".toCharArray())); @@ -395,7 +395,7 @@ public void testSnapshotDeleteRestore() { logger.error("--> creating user"); securityClient().preparePutUser("joe", "s3krit".toCharArray(), hasher, "test_role", "snapshot_user").get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); logger.info("--> creating repository"); assertAcked(client().admin().cluster() .preparePutRepository("test-repo") @@ -409,10 +409,10 @@ public void testSnapshotDeleteRestore() { .prepareCreateSnapshot("test-repo", "test-snap-1") .setWaitForCompletion(true) .setIncludeGlobalState(false) - .setIndices(SECURITY_INDEX_NAME) + .setIndices(SECURITY_MAIN_ALIAS) .get().getSnapshotInfo(); assertThat(snapshotInfo.state(), is(SnapshotState.SUCCESS)); - assertThat(snapshotInfo.indices(), contains(SecurityIndexManager.INTERNAL_SECURITY_INDEX)); + assertThat(snapshotInfo.indices(), contains(INTERNAL_SECURITY_MAIN_INDEX_7)); deleteSecurityIndex(); // the realm cache should clear itself but we don't wish to race it securityClient().prepareClearRealmCache().get(); @@ -429,7 +429,7 @@ public void testSnapshotDeleteRestore() { RestoreSnapshotResponse response = client().admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap-1") .setWaitForCompletion(true).setIncludeAliases(true).get(); assertThat(response.status(), equalTo(RestStatus.OK)); - assertThat(response.getRestoreInfo().indices(), contains(SecurityIndexManager.INTERNAL_SECURITY_INDEX)); + assertThat(response.getRestoreInfo().indices(), contains(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7)); // the realm cache should clear itself but we don't wish to race it securityClient().prepareClearRealmCache().get(); // users and roles are retrievable @@ -459,7 +459,7 @@ public void testAuthenticateWithDeletedRole() { .get(); c.preparePutUser("joe", "s3krit".toCharArray(), hasher, "test_role").get(); logger.error("--> waiting for .security index"); - ensureGreen(SECURITY_INDEX_NAME); + ensureGreen(SECURITY_MAIN_ALIAS); final String token = basicAuthHeaderValue("joe", new SecureString("s3krit".toCharArray())); ClusterHealthResponse response = client().filterWithHeader(Collections.singletonMap("Authorization", token)).admin().cluster() @@ -590,12 +590,12 @@ public void testUsersAndRolesDoNotInterfereWithIndicesStats() throws Exception { .get(); } - IndicesStatsResponse response = client().admin().indices().prepareStats("foo", SECURITY_INDEX_NAME).get(); + IndicesStatsResponse response = client().admin().indices().prepareStats("foo", SECURITY_MAIN_ALIAS).get(); assertThat(response.getFailedShards(), is(0)); assertThat(response.getIndices().size(), is(2)); - assertThat(response.getIndices().get(INTERNAL_SECURITY_INDEX), notNullValue()); - assertThat(response.getIndices().get(INTERNAL_SECURITY_INDEX).getIndex(), - is(INTERNAL_SECURITY_INDEX)); + assertThat(response.getIndices().get(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7), notNullValue()); + assertThat(response.getIndices().get(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7).getIndex(), + is(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7)); } public void testOperationsOnReservedUsers() throws Exception { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java index fb0d55c75cd32..28625f20627e1 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.security.support.SecurityIndexManager; +import java.time.Instant; import java.util.concurrent.atomic.AtomicInteger; import static org.mockito.Mockito.mock; @@ -23,10 +24,10 @@ public class NativeRealmTests extends ESTestCase { private final String concreteSecurityIndexName = randomFrom( - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); private SecurityIndexManager.State dummyState(ClusterHealthStatus indexStatus) { - return new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, indexStatus); + return new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, indexStatus); } public void testCacheClearOnIndexHealthChange() { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java index ab82d18f2e0c7..4cbf5307d3ed6 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.support.Hasher; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.core.security.user.APMSystemUser; import org.elasticsearch.xpack.core.security.user.BeatsSystemUser; import org.elasticsearch.xpack.core.security.user.ElasticUser; @@ -111,7 +112,7 @@ public void testBlankPasswordInIndexImpliesDefaultPassword() throws Exception { values.put(PASSWORD_FIELD, BLANK_PASSWORD); final GetResult result = new GetResult( - SecurityIndexManager.SECURITY_INDEX_NAME, + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, MapperService.SINGLE_MAPPING_NAME, NativeUsersStore.getIdForUser(NativeUsersStore.RESERVED_USER_TYPE, randomAlphaOfLength(12)), 0, 1, 1L, @@ -180,7 +181,7 @@ public void testVerifyNonExistentUser() throws Exception { nativeUsersStore.verifyPassword(username, password, future); final GetResult getResult = new GetResult( - SecurityIndexManager.SECURITY_INDEX_NAME, + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, MapperService.SINGLE_MAPPING_NAME, NativeUsersStore.getIdForUser(NativeUsersStore.USER_DOC_TYPE, username), UNASSIGNED_SEQ_NO, 0, 1L, @@ -222,7 +223,7 @@ private void respondToGetUserRequest(String username, SecureString password, Str values.put(User.Fields.TYPE.getPreferredName(), NativeUsersStore.USER_DOC_TYPE); final BytesReference source = BytesReference.bytes(jsonBuilder().map(values)); final GetResult getResult = new GetResult( - SecurityIndexManager.SECURITY_INDEX_NAME, + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, MapperService.SINGLE_MAPPING_NAME, NativeUsersStore.getIdForUser(NativeUsersStore.USER_DOC_TYPE, username), 0, 1, 1L, @@ -230,7 +231,6 @@ private void respondToGetUserRequest(String username, SecureString password, Str source, Collections.emptyMap()); - actionRespond(GetRequest.class, new GetResponse(getResult)); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java index e96284ba1549a..6bb6e0c7b5854 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.hamcrest.Matchers; +import java.time.Instant; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -53,7 +54,7 @@ public class NativeRoleMappingStoreTests extends ESTestCase { private final String concreteSecurityIndexName = randomFrom( - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); public void testResolveRoles() throws Exception { // Does match DN @@ -137,7 +138,7 @@ private String randomiseDn(String dn) { } private SecurityIndexManager.State dummyState(ClusterHealthStatus indexStatus) { - return new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, indexStatus); + return new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, indexStatus); } public void testCacheClearOnIndexHealthChange() { @@ -182,13 +183,13 @@ public void testCacheClearOnIndexOutOfDateChange() { final NativeRoleMappingStore store = buildRoleMappingStoreForInvalidationTesting(numInvalidation); store.onSecurityIndexStateChange( - new SecurityIndexManager.State(true, false, true, true, null, concreteSecurityIndexName, null), - new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, null)); + new SecurityIndexManager.State(Instant.now(), false, true, true, null, concreteSecurityIndexName, null), + new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, null)); assertEquals(1, numInvalidation.get()); store.onSecurityIndexStateChange( - new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, null), - new SecurityIndexManager.State(true, false, true, true, null, concreteSecurityIndexName, null)); + new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, null), + new SecurityIndexManager.State(Instant.now(), false, true, true, null, concreteSecurityIndexName, null)); assertEquals(2, numInvalidation.get()); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java index 29d02326cd214..b370c8e2b6b0f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java @@ -156,8 +156,8 @@ import static org.elasticsearch.test.SecurityTestsUtils.assertThrowsAuthorizationException; import static org.elasticsearch.test.SecurityTestsUtils.assertThrowsAuthorizationExceptionRunAs; import static org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail.PRINCIPAL_ROLES_FIELD_NAME; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_SECURITY_INDEX; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.endsWith; @@ -864,8 +864,8 @@ public void testGrantAllRestrictedUserCannotExecuteOperationAgainstSecurityIndic ClusterState state = mock(ClusterState.class); when(clusterService.state()).thenReturn(state); when(state.metaData()).thenReturn(MetaData.builder() - .put(new IndexMetaData.Builder(INTERNAL_SECURITY_INDEX) - .putAlias(new AliasMetaData.Builder(SECURITY_INDEX_NAME).build()) + .put(new IndexMetaData.Builder(INTERNAL_SECURITY_MAIN_INDEX_7) + .putAlias(new AliasMetaData.Builder(SECURITY_MAIN_ALIAS).build()) .settings(Settings.builder().put("index.version.created", Version.CURRENT).build()) .numberOfShards(1) .numberOfReplicas(0) @@ -874,31 +874,33 @@ public void testGrantAllRestrictedUserCannotExecuteOperationAgainstSecurityIndic final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); List> requests = new ArrayList<>(); + requests.add(new Tuple<>(BulkAction.NAME + "[s]", + new DeleteRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "id"))); + requests.add( + new Tuple<>(UpdateAction.NAME, new UpdateRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "id"))); requests.add( - new Tuple<>(BulkAction.NAME + "[s]", new DeleteRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "id"))); - requests.add(new Tuple<>(UpdateAction.NAME, new UpdateRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "id"))); - requests.add(new Tuple<>(BulkAction.NAME + "[s]", new IndexRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); - requests.add(new Tuple<>(SearchAction.NAME, new SearchRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new Tuple<>(BulkAction.NAME + "[s]", new IndexRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); + requests.add(new Tuple<>(SearchAction.NAME, new SearchRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(TermVectorsAction.NAME, - new TermVectorsRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "type", "id"))); - requests.add(new Tuple<>(GetAction.NAME, new GetRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "id"))); + new TermVectorsRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "type", "id"))); + requests.add(new Tuple<>(GetAction.NAME, new GetRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "id"))); requests.add(new Tuple<>(IndicesAliasesAction.NAME, new IndicesAliasesRequest() - .addAliasAction(AliasActions.add().alias("security_alias").index(INTERNAL_SECURITY_INDEX)))); + .addAliasAction(AliasActions.add().alias("security_alias").index(INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(UpdateSettingsAction.NAME, - new UpdateSettingsRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new UpdateSettingsRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); // cannot execute monitor operations requests.add(new Tuple<>(IndicesStatsAction.NAME, - new IndicesStatsRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); - requests.add( - new Tuple<>(RecoveryAction.NAME, new RecoveryRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new IndicesStatsRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); + requests.add(new Tuple<>(RecoveryAction.NAME, + new RecoveryRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(IndicesSegmentsAction.NAME, - new IndicesSegmentsRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new IndicesSegmentsRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(GetSettingsAction.NAME, - new GetSettingsRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new GetSettingsRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(IndicesShardStoresAction.NAME, - new IndicesShardStoresRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new IndicesShardStoresRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(UpgradeStatusAction.NAME, - new UpgradeStatusRequest().indices(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new UpgradeStatusRequest().indices(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); for (Tuple requestTuple : requests) { String action = requestTuple.v1(); @@ -912,13 +914,13 @@ public void testGrantAllRestrictedUserCannotExecuteOperationAgainstSecurityIndic } // we should allow waiting for the health of the index or any index if the user has this permission - ClusterHealthRequest request = new ClusterHealthRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)); + ClusterHealthRequest request = new ClusterHealthRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)); authorize(authentication, ClusterHealthAction.NAME, request); verify(auditTrail).accessGranted(eq(requestId), eq(authentication), eq(ClusterHealthAction.NAME), eq(request), authzInfoRoles(new String[]{role.getName()})); // multiple indices - request = new ClusterHealthRequest(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX, "foo", "bar"); + request = new ClusterHealthRequest(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7, "foo", "bar"); authorize(authentication, ClusterHealthAction.NAME, request); verify(auditTrail).accessGranted(eq(requestId), eq(authentication), eq(ClusterHealthAction.NAME), eq(request), authzInfoRoles(new String[]{role.getName()})); @@ -940,8 +942,8 @@ public void testMonitoringOperationsAgainstSecurityIndexRequireAllowRestricted() ClusterState state = mock(ClusterState.class); when(clusterService.state()).thenReturn(state); when(state.metaData()).thenReturn(MetaData.builder() - .put(new IndexMetaData.Builder(INTERNAL_SECURITY_INDEX) - .putAlias(new AliasMetaData.Builder(SECURITY_INDEX_NAME).build()) + .put(new IndexMetaData.Builder(INTERNAL_SECURITY_MAIN_INDEX_7) + .putAlias(new AliasMetaData.Builder(SECURITY_MAIN_ALIAS).build()) .settings(Settings.builder().put("index.version.created", Version.CURRENT).build()) .numberOfShards(1) .numberOfReplicas(0) @@ -949,12 +951,12 @@ public void testMonitoringOperationsAgainstSecurityIndexRequireAllowRestricted() .build()); List> requests = new ArrayList<>(); - requests.add(new Tuple<>(IndicesStatsAction.NAME, new IndicesStatsRequest().indices(SECURITY_INDEX_NAME))); - requests.add(new Tuple<>(RecoveryAction.NAME, new RecoveryRequest().indices(SECURITY_INDEX_NAME))); - requests.add(new Tuple<>(IndicesSegmentsAction.NAME, new IndicesSegmentsRequest().indices(SECURITY_INDEX_NAME))); - requests.add(new Tuple<>(GetSettingsAction.NAME, new GetSettingsRequest().indices(SECURITY_INDEX_NAME))); - requests.add(new Tuple<>(IndicesShardStoresAction.NAME, new IndicesShardStoresRequest().indices(SECURITY_INDEX_NAME))); - requests.add(new Tuple<>(UpgradeStatusAction.NAME, new UpgradeStatusRequest().indices(SECURITY_INDEX_NAME))); + requests.add(new Tuple<>(IndicesStatsAction.NAME, new IndicesStatsRequest().indices(SECURITY_MAIN_ALIAS))); + requests.add(new Tuple<>(RecoveryAction.NAME, new RecoveryRequest().indices(SECURITY_MAIN_ALIAS))); + requests.add(new Tuple<>(IndicesSegmentsAction.NAME, new IndicesSegmentsRequest().indices(SECURITY_MAIN_ALIAS))); + requests.add(new Tuple<>(GetSettingsAction.NAME, new GetSettingsRequest().indices(SECURITY_MAIN_ALIAS))); + requests.add(new Tuple<>(IndicesShardStoresAction.NAME, new IndicesShardStoresRequest().indices(SECURITY_MAIN_ALIAS))); + requests.add(new Tuple<>(UpgradeStatusAction.NAME, new UpgradeStatusRequest().indices(SECURITY_MAIN_ALIAS))); for (final Tuple requestTuple : requests) { final String action = requestTuple.v1(); @@ -984,8 +986,8 @@ public void testSuperusersCanExecuteOperationAgainstSecurityIndex() throws IOExc ClusterState state = mock(ClusterState.class); when(clusterService.state()).thenReturn(state); when(state.metaData()).thenReturn(MetaData.builder() - .put(new IndexMetaData.Builder(INTERNAL_SECURITY_INDEX) - .putAlias(new AliasMetaData.Builder(SECURITY_INDEX_NAME).build()) + .put(new IndexMetaData.Builder(INTERNAL_SECURITY_MAIN_INDEX_7) + .putAlias(new AliasMetaData.Builder(SECURITY_MAIN_ALIAS).build()) .settings(Settings.builder().put("index.version.created", Version.CURRENT).build()) .numberOfShards(1) .numberOfReplicas(0) @@ -994,25 +996,28 @@ public void testSuperusersCanExecuteOperationAgainstSecurityIndex() throws IOExc final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); List> requests = new ArrayList<>(); - requests.add(new Tuple<>(DeleteAction.NAME, new DeleteRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "id"))); + requests.add( + new Tuple<>(DeleteAction.NAME, new DeleteRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "id"))); requests.add(new Tuple<>(BulkAction.NAME + "[s]", - createBulkShardRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), DeleteRequest::new))); - requests.add(new Tuple<>(UpdateAction.NAME, new UpdateRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "id"))); - requests.add(new Tuple<>(IndexAction.NAME, new IndexRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + createBulkShardRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), DeleteRequest::new))); + requests.add( + new Tuple<>(UpdateAction.NAME, new UpdateRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "id"))); + requests.add(new Tuple<>(IndexAction.NAME, new IndexRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(BulkAction.NAME + "[s]", - createBulkShardRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), IndexRequest::new))); - requests.add(new Tuple<>(SearchAction.NAME, new SearchRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + createBulkShardRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), IndexRequest::new))); + requests.add(new Tuple<>(SearchAction.NAME, new SearchRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(TermVectorsAction.NAME, - new TermVectorsRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "type", "id"))); - requests.add(new Tuple<>(GetAction.NAME, new GetRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "type", "id"))); - requests.add(new Tuple<>(TermVectorsAction.NAME, - new TermVectorsRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "type", "id"))); - requests.add(new Tuple<>(IndicesAliasesAction.NAME, - new IndicesAliasesRequest().addAliasAction(AliasActions.add().alias("security_alias").index(INTERNAL_SECURITY_INDEX)))); + new TermVectorsRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "type", "id"))); requests.add( - new Tuple<>(ClusterHealthAction.NAME, new ClusterHealthRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX)))); + new Tuple<>(GetAction.NAME, new GetRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "type", "id"))); + requests.add(new Tuple<>(TermVectorsAction.NAME, + new TermVectorsRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "type", "id"))); + requests.add(new Tuple<>(IndicesAliasesAction.NAME, new IndicesAliasesRequest() + .addAliasAction(AliasActions.add().alias("security_alias").index(INTERNAL_SECURITY_MAIN_INDEX_7)))); + requests.add(new Tuple<>(ClusterHealthAction.NAME, + new ClusterHealthRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); requests.add(new Tuple<>(ClusterHealthAction.NAME, - new ClusterHealthRequest(randomFrom(SECURITY_INDEX_NAME, INTERNAL_SECURITY_INDEX), "foo", "bar"))); + new ClusterHealthRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7), "foo", "bar"))); for (final Tuple requestTuple : requests) { final String action = requestTuple.v1(); @@ -1033,8 +1038,8 @@ public void testSuperusersCanExecuteOperationAgainstSecurityIndexWithWildcard() ClusterState state = mock(ClusterState.class); when(clusterService.state()).thenReturn(state); when(state.metaData()).thenReturn(MetaData.builder() - .put(new IndexMetaData.Builder(INTERNAL_SECURITY_INDEX) - .putAlias(new AliasMetaData.Builder(SECURITY_INDEX_NAME).build()) + .put(new IndexMetaData.Builder(INTERNAL_SECURITY_MAIN_INDEX_7) + .putAlias(new AliasMetaData.Builder(SECURITY_MAIN_ALIAS).build()) .settings(Settings.builder().put("index.version.created", Version.CURRENT).build()) .numberOfShards(1) .numberOfReplicas(0) @@ -1046,7 +1051,7 @@ public void testSuperusersCanExecuteOperationAgainstSecurityIndexWithWildcard() SearchRequest request = new SearchRequest("_all"); authorize(authentication, action, request); verify(auditTrail).accessGranted(eq(requestId), eq(authentication), eq(action), eq(request), authzInfoRoles(superuser.roles())); - assertThat(request.indices(), arrayContainingInAnyOrder(INTERNAL_SECURITY_INDEX, SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContainingInAnyOrder(INTERNAL_SECURITY_MAIN_INDEX_7, SECURITY_MAIN_ALIAS)); } public void testCompositeActionsAreImmediatelyRejected() { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java index 202c9cb715f58..51dba4e4c2334 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.security.authz.store.CompositeRolesStore; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; import java.util.List; import java.util.Set; @@ -47,8 +46,8 @@ public void testAuthorizedIndicesUserWithSomeRoles() { RoleDescriptor bRole = new RoleDescriptor("b", null, new IndicesPrivileges[] { IndicesPrivileges.builder().indices("b").privileges("READ").build() }, null); Settings indexSettings = Settings.builder().put("index.version.created", Version.CURRENT).build(); - final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); MetaData metaData = MetaData.builder() .put(new IndexMetaData.Builder("a1").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) .put(new IndexMetaData.Builder("a2").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) @@ -65,7 +64,7 @@ public void testAuthorizedIndicesUserWithSomeRoles() { .settings(indexSettings) .numberOfShards(1) .numberOfReplicas(0) - .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_INDEX_NAME).build()) + .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).build()) .build(), true) .build(); final PlainActionFuture future = new PlainActionFuture<>(); @@ -78,7 +77,7 @@ public void testAuthorizedIndicesUserWithSomeRoles() { assertFalse(list.contains("bbbbb")); assertFalse(list.contains("ba")); assertThat(list, not(contains(internalSecurityIndex))); - assertThat(list, not(contains(RestrictedIndicesNames.SECURITY_INDEX_NAME))); + assertThat(list, not(contains(RestrictedIndicesNames.SECURITY_MAIN_ALIAS))); } public void testAuthorizedIndicesUserWithSomeRolesEmptyMetaData() { @@ -101,8 +100,8 @@ public void testSecurityIndicesAreRestrictedForDefaultRole() { .cluster(ClusterPrivilege.ALL) .build(); Settings indexSettings = Settings.builder().put("index.version.created", Version.CURRENT).build(); - final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); MetaData metaData = MetaData.builder() .put(new IndexMetaData.Builder("an-index").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) .put(new IndexMetaData.Builder("another-index").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) @@ -111,7 +110,7 @@ public void testSecurityIndicesAreRestrictedForDefaultRole() { .settings(indexSettings) .numberOfShards(1) .numberOfReplicas(0) - .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_INDEX_NAME).build()) + .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).build()) .build(), true) .build(); @@ -119,7 +118,7 @@ public void testSecurityIndicesAreRestrictedForDefaultRole() { RBACEngine.resolveAuthorizedIndicesFromRole(role, SearchAction.NAME, metaData.getAliasAndIndexLookup()); assertThat(authorizedIndices, containsInAnyOrder("an-index", "another-index")); assertThat(authorizedIndices, not(contains(internalSecurityIndex))); - assertThat(authorizedIndices, not(contains(RestrictedIndicesNames.SECURITY_INDEX_NAME))); + assertThat(authorizedIndices, not(contains(RestrictedIndicesNames.SECURITY_MAIN_ALIAS))); } public void testSecurityIndicesAreNotRemovedFromUnrestrictedRole() { @@ -128,8 +127,8 @@ public void testSecurityIndicesAreNotRemovedFromUnrestrictedRole() { .cluster(ClusterPrivilege.ALL) .build(); Settings indexSettings = Settings.builder().put("index.version.created", Version.CURRENT).build(); - final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); MetaData metaData = MetaData.builder() .put(new IndexMetaData.Builder("an-index").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) .put(new IndexMetaData.Builder("another-index").settings(indexSettings).numberOfShards(1).numberOfReplicas(0).build(), true) @@ -137,18 +136,18 @@ public void testSecurityIndicesAreNotRemovedFromUnrestrictedRole() { .settings(indexSettings) .numberOfShards(1) .numberOfReplicas(0) - .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_INDEX_NAME).build()) + .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).build()) .build(), true) .build(); List authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole(role, SearchAction.NAME, metaData.getAliasAndIndexLookup()); assertThat(authorizedIndices, containsInAnyOrder( - "an-index", "another-index", SecurityIndexManager.SECURITY_INDEX_NAME, internalSecurityIndex)); + "an-index", "another-index", RestrictedIndicesNames.SECURITY_MAIN_ALIAS, internalSecurityIndex)); List authorizedIndicesSuperUser = RBACEngine.resolveAuthorizedIndicesFromRole(role, SearchAction.NAME, metaData.getAliasAndIndexLookup()); assertThat(authorizedIndicesSuperUser, containsInAnyOrder( - "an-index", "another-index", SecurityIndexManager.SECURITY_INDEX_NAME, internalSecurityIndex)); + "an-index", "another-index", RestrictedIndicesNames.SECURITY_MAIN_ALIAS, internalSecurityIndex)); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index dc32580980e02..665b70a8881b2 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -64,7 +64,6 @@ import org.elasticsearch.xpack.core.security.user.XPackSecurityUser; import org.elasticsearch.xpack.core.security.user.XPackUser; import org.elasticsearch.xpack.security.authz.store.CompositeRolesStore; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.security.test.SecurityTestUtils; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -79,7 +78,7 @@ import java.util.Map; import java.util.Set; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; import static org.hamcrest.Matchers.contains; @@ -119,7 +118,7 @@ public void setup() { indexNameExpressionResolver = new IndexNameExpressionResolver(); final boolean withAlias = randomBoolean(); - final String securityIndexName = SECURITY_INDEX_NAME + (withAlias ? "-" + randomAlphaOfLength(5) : ""); + final String securityIndexName = SECURITY_MAIN_ALIAS + (withAlias ? "-" + randomAlphaOfLength(5) : ""); MetaData metaData = MetaData.builder() .put(indexBuilder("foo").putAlias(AliasMetaData.builder("foofoobar")) .putAlias(AliasMetaData.builder("foounauthorized")).settings(settings)) @@ -1222,14 +1221,14 @@ public void testXPackSecurityUserHasAccessToSecurityIndex() { { final List authorizedIndices = buildAuthorizedIndices(XPackSecurityUser.INSTANCE, SearchAction.NAME); List indices = resolveIndices(request, authorizedIndices).getLocal(); - assertThat(indices, hasItem(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(indices, hasItem(SECURITY_MAIN_ALIAS)); } { IndicesAliasesRequest aliasesRequest = new IndicesAliasesRequest(); - aliasesRequest.addAliasAction(AliasActions.add().alias("security_alias").index(SECURITY_INDEX_NAME)); + aliasesRequest.addAliasAction(AliasActions.add().alias("security_alias").index(SECURITY_MAIN_ALIAS)); final List authorizedIndices = buildAuthorizedIndices(XPackSecurityUser.INSTANCE, IndicesAliasesAction.NAME); List indices = resolveIndices(aliasesRequest, authorizedIndices).getLocal(); - assertThat(indices, hasItem(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(indices, hasItem(SECURITY_MAIN_ALIAS)); } } @@ -1237,7 +1236,7 @@ public void testXPackUserDoesNotHaveAccessToSecurityIndex() { SearchRequest request = new SearchRequest(); final List authorizedIndices = buildAuthorizedIndices(XPackUser.INSTANCE, SearchAction.NAME); List indices = resolveIndices(request, authorizedIndices).getLocal(); - assertThat(indices, not(hasItem(SecurityIndexManager.SECURITY_INDEX_NAME))); + assertThat(indices, not(hasItem(SECURITY_MAIN_ALIAS))); } public void testNonXPackUserAccessingSecurityIndex() { @@ -1249,7 +1248,7 @@ public void testNonXPackUserAccessingSecurityIndex() { SearchRequest request = new SearchRequest(); final List authorizedIndices = buildAuthorizedIndices(allAccessUser, SearchAction.NAME); List indices = resolveIndices(request, authorizedIndices).getLocal(); - assertThat(indices, not(hasItem(SecurityIndexManager.SECURITY_INDEX_NAME))); + assertThat(indices, not(hasItem(SECURITY_MAIN_ALIAS))); } { @@ -1257,7 +1256,7 @@ public void testNonXPackUserAccessingSecurityIndex() { aliasesRequest.addAliasAction(AliasActions.add().alias("security_alias1").index("*")); final List authorizedIndices = buildAuthorizedIndices(allAccessUser, IndicesAliasesAction.NAME); List indices = resolveIndices(aliasesRequest, authorizedIndices).getLocal(); - assertThat(indices, not(hasItem(SecurityIndexManager.SECURITY_INDEX_NAME))); + assertThat(indices, not(hasItem(SECURITY_MAIN_ALIAS))); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/SnapshotUserRoleIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/SnapshotUserRoleIntegTests.java index 8a79bc86f6702..66ea23b518c29 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/SnapshotUserRoleIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/SnapshotUserRoleIntegTests.java @@ -22,8 +22,8 @@ import java.util.Collections; import java.util.Locale; -import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7; -import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; @@ -56,7 +56,7 @@ public void setupClusterBeforeSnapshot() { final String snapshotUserToken = basicAuthHeaderValue(user, new SecureString(password)); client = client().filterWithHeader(Collections.singletonMap("Authorization", snapshotUserToken)); securityClient().preparePutUser(user, password, Hasher.BCRYPT, "snapshot_user").get(); - ensureGreen(INTERNAL_SECURITY_INDEX_7); + ensureGreen(INTERNAL_SECURITY_MAIN_INDEX_7); } public void testSnapshotUserRoleCanSnapshotAndSeeAllIndices() { @@ -67,17 +67,17 @@ public void testSnapshotUserRoleCanSnapshotAndSeeAllIndices() { assertThat(getRepositoriesResponse.repositories().get(0).name(), is("repo")); // view all indices, including restricted ones final GetIndexResponse getIndexResponse = client.admin().indices().prepareGetIndex().setIndices(randomFrom("_all", "*")).get(); - assertThat(Arrays.asList(getIndexResponse.indices()), containsInAnyOrder(INTERNAL_SECURITY_INDEX_7, ordinaryIndex)); + assertThat(Arrays.asList(getIndexResponse.indices()), containsInAnyOrder(INTERNAL_SECURITY_MAIN_INDEX_7, ordinaryIndex)); // create snapshot that includes restricted indices final CreateSnapshotResponse snapshotResponse = client.admin().cluster().prepareCreateSnapshot("repo", "snap") .setIndices(randomFrom("_all", "*")).setWaitForCompletion(true).get(); assertThat(snapshotResponse.getSnapshotInfo().state(), is(SnapshotState.SUCCESS)); - assertThat(snapshotResponse.getSnapshotInfo().indices(), containsInAnyOrder(INTERNAL_SECURITY_INDEX_7, ordinaryIndex)); + assertThat(snapshotResponse.getSnapshotInfo().indices(), containsInAnyOrder(INTERNAL_SECURITY_MAIN_INDEX_7, ordinaryIndex)); // view snapshots for repo final GetSnapshotsResponse getSnapshotResponse = client.admin().cluster().prepareGetSnapshots("repo").get(); assertThat(getSnapshotResponse.getSnapshots().size(), is(1)); assertThat(getSnapshotResponse.getSnapshots().get(0).snapshotId().getName(), is("snap")); - assertThat(getSnapshotResponse.getSnapshots().get(0).indices(), containsInAnyOrder(INTERNAL_SECURITY_INDEX_7, ordinaryIndex)); + assertThat(getSnapshotResponse.getSnapshots().get(0).indices(), containsInAnyOrder(INTERNAL_SECURITY_MAIN_INDEX_7, ordinaryIndex)); } public void testSnapshotUserRoleIsReserved() { @@ -112,7 +112,7 @@ public void testSnapshotUserRoleUnathorizedForDestructiveActions() { () -> client.admin().cluster().prepareDeleteSnapshot("repo", randomAlphaOfLength(4).toLowerCase(Locale.ROOT)).get(), "cluster:admin/snapshot/delete", "snapshot_user"); // try destructive/revealing actions on all indices - for (final String indexToTest : Arrays.asList(INTERNAL_SECURITY_INDEX_7, SECURITY_INDEX_NAME, ordinaryIndex)) { + for (final String indexToTest : Arrays.asList(INTERNAL_SECURITY_MAIN_INDEX_7, SECURITY_MAIN_ALIAS, ordinaryIndex)) { assertThrowsAuthorizationException(() -> client.prepareSearch(indexToTest).get(), "indices:data/read/search", "snapshot_user"); assertThrowsAuthorizationException(() -> client.prepareGet(indexToTest, "doc", "1").get(), "indices:data/read/get", "snapshot_user"); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java index 8f3e69815750d..9bd69d3eb1a77 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java @@ -293,14 +293,14 @@ public void testErrorMessageIfIndexPatternIsTooComplex() { public void testSecurityIndicesPermissions() { final Settings indexSettings = Settings.builder().put("index.version.created", Version.CURRENT).build(); - final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + final String internalSecurityIndex = randomFrom(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); final MetaData metaData = new MetaData.Builder() .put(new IndexMetaData.Builder(internalSecurityIndex) .settings(indexSettings) .numberOfShards(1) .numberOfReplicas(0) - .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_INDEX_NAME).build()) + .putAlias(new AliasMetaData.Builder(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).build()) .build(), true) .build(); FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(Settings.EMPTY); @@ -309,18 +309,18 @@ public void testSecurityIndicesPermissions() { // allow_restricted_indices: false IndicesPermission.Group group = new IndicesPermission.Group(IndexPrivilege.ALL, new FieldPermissions(), null, false, "*"); Map authzMap = new IndicesPermission(group).authorize(SearchAction.NAME, - Sets.newHashSet(internalSecurityIndex, RestrictedIndicesNames.SECURITY_INDEX_NAME), lookup, + Sets.newHashSet(internalSecurityIndex, RestrictedIndicesNames.SECURITY_MAIN_ALIAS), lookup, fieldPermissionsCache); assertThat(authzMap.get(internalSecurityIndex).isGranted(), is(false)); - assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_INDEX_NAME).isGranted(), is(false)); + assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).isGranted(), is(false)); // allow_restricted_indices: true group = new IndicesPermission.Group(IndexPrivilege.ALL, new FieldPermissions(), null, true, "*"); authzMap = new IndicesPermission(group).authorize(SearchAction.NAME, - Sets.newHashSet(internalSecurityIndex, RestrictedIndicesNames.SECURITY_INDEX_NAME), lookup, + Sets.newHashSet(internalSecurityIndex, RestrictedIndicesNames.SECURITY_MAIN_ALIAS), lookup, fieldPermissionsCache); assertThat(authzMap.get(internalSecurityIndex).isGranted(), is(true)); - assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_INDEX_NAME).isGranted(), is(true)); + assertThat(authzMap.get(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).isGranted(), is(true)); } private static FieldPermissionsDefinition fieldPermissionDef(String[] granted, String[] denied) { 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 ba2ba455a505c..a39545f3a9b3a 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 @@ -57,6 +57,7 @@ import org.elasticsearch.xpack.security.support.SecurityIndexManager; import java.io.IOException; +import java.time.Instant; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -98,7 +99,7 @@ public class CompositeRolesStoreTests extends ESTestCase { private final FieldPermissionsCache cache = new FieldPermissionsCache(Settings.EMPTY); private final String concreteSecurityIndexName = randomFrom( - RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_INDEX_7); + RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_6, RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7); public void testRolesWhenDlsFlsUnlicensed() throws IOException { XPackLicenseState licenseState = mock(XPackLicenseState.class); @@ -762,7 +763,7 @@ Settings.EMPTY, fileRolesStore, nativeRolesStore, reservedRolesStore, mock(Nativ } private SecurityIndexManager.State dummyState(ClusterHealthStatus indexStatus) { - return new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, indexStatus); + return new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, indexStatus); } public void testCacheClearOnIndexHealthChange() { @@ -837,13 +838,13 @@ public void invalidateAll() { }; compositeRolesStore.onSecurityIndexStateChange( - new SecurityIndexManager.State(true, false, true, true, null, concreteSecurityIndexName, null), - new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, null)); + new SecurityIndexManager.State(Instant.now(), false, true, true, null, concreteSecurityIndexName, null), + new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, null)); assertEquals(1, numInvalidation.get()); compositeRolesStore.onSecurityIndexStateChange( - new SecurityIndexManager.State(true, true, true, true, null, concreteSecurityIndexName, null), - new SecurityIndexManager.State(true, false, true, true, null, concreteSecurityIndexName, null)); + new SecurityIndexManager.State(Instant.now(), true, true, true, null, concreteSecurityIndexName, null), + new SecurityIndexManager.State(Instant.now(), false, true, true, null, concreteSecurityIndexName, null)); assertEquals(2, numInvalidation.get()); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java index a833748854943..7f7a262131bb2 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java @@ -39,6 +39,7 @@ import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.xpack.core.security.action.role.ClearRolesCacheRequest; import org.elasticsearch.xpack.core.security.authz.privilege.ApplicationPrivilegeDescriptor; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.hamcrest.Matchers; import org.junit.After; @@ -125,7 +126,7 @@ public void testGetSinglePrivilegeByName() throws Exception { assertThat(requests, iterableWithSize(1)); assertThat(requests.get(0), instanceOf(GetRequest.class)); GetRequest request = (GetRequest) requests.get(0); - assertThat(request.index(), equalTo(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.index(), equalTo(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); assertThat(request.type(), equalTo(MapperService.SINGLE_MAPPING_NAME)); assertThat(request.id(), equalTo("application-privilege_myapp:admin")); @@ -143,7 +144,7 @@ public void testGetMissingPrivilege() throws Exception { assertThat(requests, iterableWithSize(1)); assertThat(requests.get(0), instanceOf(GetRequest.class)); GetRequest request = (GetRequest) requests.get(0); - assertThat(request.index(), equalTo(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.index(), equalTo(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); assertThat(request.type(), equalTo(MapperService.SINGLE_MAPPING_NAME)); assertThat(request.id(), equalTo("application-privilege_myapp:admin")); @@ -166,7 +167,7 @@ public void testGetPrivilegesByApplicationName() throws Exception { assertThat(requests, iterableWithSize(1)); assertThat(requests.get(0), instanceOf(SearchRequest.class)); SearchRequest request = (SearchRequest) requests.get(0); - assertThat(request.indices(), arrayContaining(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContaining(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); final String query = Strings.toString(request.source().query()); assertThat(query, containsString("{\"terms\":{\"application\":[\"myapp\",\"yourapp\"]")); @@ -187,7 +188,7 @@ public void testGetPrivilegesByWildcardApplicationName() throws Exception { assertThat(requests, iterableWithSize(1)); assertThat(requests.get(0), instanceOf(SearchRequest.class)); SearchRequest request = (SearchRequest) requests.get(0); - assertThat(request.indices(), arrayContaining(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContaining(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); final String query = Strings.toString(request.source().query()); assertThat(query, containsString("{\"bool\":{\"filter\":[{\"terms\":{\"application\":[\"yourapp\"]")); @@ -207,7 +208,7 @@ public void testGetPrivilegesByStarApplicationName() throws Exception { assertThat(requests, iterableWithSize(1)); assertThat(requests.get(0), instanceOf(SearchRequest.class)); SearchRequest request = (SearchRequest) requests.get(0); - assertThat(request.indices(), arrayContaining(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContaining(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); final String query = Strings.toString(request.source().query()); assertThat(query, containsString("{\"exists\":{\"field\":\"application\"")); @@ -232,7 +233,7 @@ public void testGetAllPrivileges() throws Exception { assertThat(requests, iterableWithSize(1)); assertThat(requests.get(0), instanceOf(SearchRequest.class)); SearchRequest request = (SearchRequest) requests.get(0); - assertThat(request.indices(), arrayContaining(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContaining(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); final String query = Strings.toString(request.source().query()); assertThat(query, containsString("{\"term\":{\"type\":{\"value\":\"application-privilege\"")); @@ -268,7 +269,7 @@ public void testPutPrivileges() throws Exception { for (int i = 0; i < putPrivileges.size(); i++) { ApplicationPrivilegeDescriptor privilege = putPrivileges.get(i); IndexRequest request = indexRequests.get(i); - assertThat(request.indices(), arrayContaining(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContaining(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); assertThat(request.type(), equalTo(MapperService.SINGLE_MAPPING_NAME)); assertThat(request.id(), equalTo( "application-privilege_" + privilege.getApplication() + ":" + privilege.getName() @@ -277,7 +278,7 @@ public void testPutPrivileges() throws Exception { assertThat(request.source(), equalTo(BytesReference.bytes(builder))); final boolean created = privilege.getName().equals("user") == false; indexListener.onResponse(new IndexResponse( - new ShardId(SecurityIndexManager.SECURITY_INDEX_NAME, uuid, i), + new ShardId(RestrictedIndicesNames.SECURITY_MAIN_ALIAS, uuid, i), request.type(), request.id(), 1, 1, 1, created )); } @@ -313,12 +314,12 @@ public void testDeletePrivileges() throws Exception { for (int i = 0; i < privilegeNames.size(); i++) { String name = privilegeNames.get(i); DeleteRequest request = deletes.get(i); - assertThat(request.indices(), arrayContaining(SecurityIndexManager.SECURITY_INDEX_NAME)); + assertThat(request.indices(), arrayContaining(RestrictedIndicesNames.SECURITY_MAIN_ALIAS)); assertThat(request.type(), equalTo(MapperService.SINGLE_MAPPING_NAME)); assertThat(request.id(), equalTo("application-privilege_app1:" + name)); final boolean found = name.equals("p2") == false; deleteListener.onResponse(new DeleteResponse( - new ShardId(SecurityIndexManager.SECURITY_INDEX_NAME, uuid, i), + new ShardId(RestrictedIndicesNames.SECURITY_MAIN_ALIAS, uuid, i), request.type(), request.id(), 1, 1, 1, found )); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStoreTests.java index d204085a41f79..39c65fb87822e 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStoreTests.java @@ -56,7 +56,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; @@ -187,7 +187,7 @@ public void testPutOfRoleWithFlsDlsUnlicensed() throws IOException { final ClusterService clusterService = mock(ClusterService.class); final XPackLicenseState licenseState = mock(XPackLicenseState.class); final AtomicBoolean methodCalled = new AtomicBoolean(false); - final SecurityIndexManager securityIndex = SecurityIndexManager.buildSecurityIndexManager(client, clusterService); + final SecurityIndexManager securityIndex = SecurityIndexManager.buildSecurityMainIndexManager(client, clusterService); final NativeRolesStore rolesStore = new NativeRolesStore(Settings.EMPTY, client, licenseState, securityIndex) { @Override void innerPutRole(final PutRoleRequest request, final RoleDescriptor role, final ActionListener listener) { @@ -247,7 +247,7 @@ void innerPutRole(final PutRoleRequest request, final RoleDescriptor role, final private ClusterState getClusterStateWithSecurityIndex() { final boolean withAlias = randomBoolean(); - final String securityIndexName = SECURITY_INDEX_NAME + (withAlias ? "-" + randomAlphaOfLength(5) : ""); + final String securityIndexName = SECURITY_MAIN_ALIAS + (withAlias ? "-" + randomAlphaOfLength(5) : ""); Settings settings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) @@ -256,7 +256,7 @@ private ClusterState getClusterStateWithSecurityIndex() { .build(); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder(securityIndexName).settings(settings)) - .put(new IndexTemplateMetaData(SecurityIndexManager.SECURITY_TEMPLATE_NAME, 0, 0, + .put(new IndexTemplateMetaData(SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7, 0, 0, Collections.singletonList(securityIndexName), Settings.EMPTY, ImmutableOpenMap.of(), ImmutableOpenMap.of())) .build(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java index 95c5dc96a5059..3dd5395b1fea0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -52,12 +53,12 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.security.test.SecurityTestUtils; +import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.core.template.TemplateUtils; import org.hamcrest.Matchers; import org.junit.Before; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_TEMPLATE_NAME; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7; import static org.elasticsearch.xpack.security.support.SecurityIndexManager.TEMPLATE_VERSION_PATTERN; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -71,7 +72,6 @@ public class SecurityIndexManagerTests extends ESTestCase { private static final ClusterName CLUSTER_NAME = new ClusterName("security-index-manager-tests"); private static final ClusterState EMPTY_CLUSTER_STATE = new ClusterState.Builder(CLUSTER_NAME).build(); - private static final String INDEX_NAME = ".security"; private static final String TEMPLATE_NAME = "SecurityIndexManagerTests-template"; private SecurityIndexManager manager; private Map, Map>> actions; @@ -96,13 +96,14 @@ void doExecute(Action action, Request request, ActionListener { @@ -255,8 +258,8 @@ public void testListeneredNotCalledBeforeStateNotRecovered() throws Exception { assertThat(manager.isStateRecovered(), is(false)); assertThat(listenerCalled.get(), is(false)); // state recovered with index - ClusterState.Builder clusterStateBuilder = createClusterState(INDEX_NAME, TEMPLATE_NAME, - SecurityIndexManager.INTERNAL_INDEX_FORMAT); + ClusterState.Builder clusterStateBuilder = createClusterState(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7, + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, TEMPLATE_NAME, SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT); markShardsAvailable(clusterStateBuilder); manager.clusterChanged(event(clusterStateBuilder)); assertThat(manager.isStateRecovered(), is(true)); @@ -278,8 +281,8 @@ public void testIndexOutOfDateListeners() throws Exception { assertTrue(manager.isIndexUpToDate()); // index doesn't exist and now exists with wrong format - ClusterState.Builder clusterStateBuilder = createClusterState(INDEX_NAME, TEMPLATE_NAME, - SecurityIndexManager.INTERNAL_INDEX_FORMAT - 1); + ClusterState.Builder clusterStateBuilder = createClusterState(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7, + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, TEMPLATE_NAME, SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT - 1); markShardsAvailable(clusterStateBuilder); manager.clusterChanged(event(clusterStateBuilder)); assertTrue(listenerCalled.get()); @@ -295,7 +298,8 @@ public void testIndexOutOfDateListeners() throws Exception { listenerCalled.set(false); // index doesn't exist and now exists with correct format - clusterStateBuilder = createClusterState(INDEX_NAME, TEMPLATE_NAME, SecurityIndexManager.INTERNAL_INDEX_FORMAT); + clusterStateBuilder = createClusterState(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7, + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, TEMPLATE_NAME, SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT); markShardsAvailable(clusterStateBuilder); manager.clusterChanged(event(clusterStateBuilder)); assertTrue(listenerCalled.get()); @@ -317,18 +321,19 @@ private void assertIndexUpToDateButNotAvailable() { assertThat(manager.isStateRecovered(), Matchers.equalTo(true)); } - public static ClusterState.Builder createClusterState(String indexName, String templateName) throws IOException { - return createClusterState(indexName, templateName, templateName, SecurityIndexManager.INTERNAL_INDEX_FORMAT); + public static ClusterState.Builder createClusterState(String indexName, String aliasName, String templateName) throws IOException { + return createClusterState(indexName, aliasName, templateName, templateName, SecurityIndexManager.INTERNAL_MAIN_INDEX_FORMAT); } - public static ClusterState.Builder createClusterState(String indexName, String templateName, int format) throws IOException { - return createClusterState(indexName, templateName, templateName, format); + public static ClusterState.Builder createClusterState(String indexName, String aliasName, String templateName, int format) + throws IOException { + return createClusterState(indexName, aliasName, templateName, templateName, format); } - private static ClusterState.Builder createClusterState(String indexName, String templateName, String buildMappingFrom, int format) - throws IOException { + private static ClusterState.Builder createClusterState(String indexName, String aliasName, String templateName, String buildMappingFrom, + int format) throws IOException { IndexTemplateMetaData.Builder templateBuilder = getIndexTemplateMetaData(templateName); - IndexMetaData.Builder indexMeta = getIndexMetadata(indexName, buildMappingFrom, format); + IndexMetaData.Builder indexMeta = getIndexMetadata(indexName, aliasName, buildMappingFrom, format); MetaData.Builder metaDataBuilder = new MetaData.Builder(); metaDataBuilder.put(templateBuilder); @@ -338,7 +343,7 @@ private static ClusterState.Builder createClusterState(String indexName, String } private void markShardsAvailable(ClusterState.Builder clusterStateBuilder) { - clusterStateBuilder.routingTable(SecurityTestUtils.buildIndexRoutingTable(INDEX_NAME)); + clusterStateBuilder.routingTable(SecurityTestUtils.buildIndexRoutingTable(RestrictedIndicesNames.INTERNAL_SECURITY_MAIN_INDEX_7)); } private static ClusterState state() { @@ -349,7 +354,8 @@ private static ClusterState state() { .build(); } - private static IndexMetaData.Builder getIndexMetadata(String indexName, String templateName, int format) throws IOException { + private static IndexMetaData.Builder getIndexMetadata(String indexName, String aliasName, String templateName, int format) + throws IOException { IndexMetaData.Builder indexMetaData = IndexMetaData.builder(indexName); indexMetaData.settings(Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) @@ -357,7 +363,7 @@ private static IndexMetaData.Builder getIndexMetadata(String indexName, String t .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.INDEX_FORMAT_SETTING.getKey(), format) .build()); - + indexMetaData.putAlias(AliasMetaData.builder(aliasName).build()); final Map mappings = getTemplateMappings(templateName); for (Map.Entry entry : mappings.entrySet()) { indexMetaData.putMapping(entry.getKey(), entry.getValue()); @@ -389,7 +395,7 @@ private static String loadTemplate(String templateName) { } public void testMappingVersionMatching() throws IOException { - String templateString = "/" + SECURITY_TEMPLATE_NAME + ".json"; + String templateString = "/" + SECURITY_MAIN_TEMPLATE_7 + ".json"; ClusterState.Builder clusterStateBuilder = createClusterStateWithMappingAndTemplate(templateString); manager.clusterChanged(new ClusterChangedEvent("test-event", clusterStateBuilder.build(), EMPTY_CLUSTER_STATE)); assertTrue(manager.checkMappingVersion(Version.CURRENT.minimumIndexCompatibilityVersion()::before)); @@ -397,17 +403,19 @@ public void testMappingVersionMatching() throws IOException { } public void testMissingVersionMappingThrowsError() throws IOException { - String templateString = "/missing-version-" + SECURITY_TEMPLATE_NAME + ".json"; + String templateString = "/missing-version-security-index-template.json"; ClusterState.Builder clusterStateBuilder = createClusterStateWithMappingAndTemplate(templateString); final ClusterState clusterState = clusterStateBuilder.build(); IllegalStateException exception = expectThrows(IllegalStateException.class, - () -> SecurityIndexManager.checkIndexMappingVersionMatches(SECURITY_INDEX_NAME, clusterState, logger, Version.CURRENT::equals)); - assertEquals("Cannot read security-version string in index " + SECURITY_INDEX_NAME, exception.getMessage()); + () -> SecurityIndexManager.checkIndexMappingVersionMatches(RestrictedIndicesNames.SECURITY_MAIN_ALIAS, + clusterState, logger, Version.CURRENT::equals)); + assertEquals("Cannot read security-version string in index " + RestrictedIndicesNames.SECURITY_MAIN_ALIAS, + exception.getMessage()); } public void testIndexTemplateIsIdentifiedAsUpToDate() throws IOException { ClusterState.Builder clusterStateBuilder = createClusterStateWithTemplate( - "/" + SECURITY_TEMPLATE_NAME + ".json" + "/" + SECURITY_MAIN_TEMPLATE_7 + ".json" ); manager.clusterChanged(new ClusterChangedEvent("test-event", clusterStateBuilder.build(), EMPTY_CLUSTER_STATE)); // No upgrade actions run @@ -415,20 +423,20 @@ public void testIndexTemplateIsIdentifiedAsUpToDate() throws IOException { } public void testIndexTemplateVersionMatching() throws Exception { - String templateString = "/" + SECURITY_TEMPLATE_NAME + ".json"; + String templateString = "/" + SECURITY_MAIN_TEMPLATE_7 + ".json"; ClusterState.Builder clusterStateBuilder = createClusterStateWithTemplate(templateString); final ClusterState clusterState = clusterStateBuilder.build(); assertTrue(SecurityIndexManager.checkTemplateExistsAndVersionMatches( - SecurityIndexManager.SECURITY_TEMPLATE_NAME, clusterState, logger, + SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7, clusterState, logger, Version.V_6_0_0::before)); assertFalse(SecurityIndexManager.checkTemplateExistsAndVersionMatches( - SecurityIndexManager.SECURITY_TEMPLATE_NAME, clusterState, logger, + SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7, clusterState, logger, Version.V_6_0_0::after)); } public void testUpToDateMappingsAreIdentifiedAsUpToDate() throws IOException { - String securityTemplateString = "/" + SECURITY_TEMPLATE_NAME + ".json"; + String securityTemplateString = "/" + SECURITY_MAIN_TEMPLATE_7 + ".json"; ClusterState.Builder clusterStateBuilder = createClusterStateWithMappingAndTemplate(securityTemplateString); manager.clusterChanged(new ClusterChangedEvent("test-event", clusterStateBuilder.build(), EMPTY_CLUSTER_STATE)); @@ -438,8 +446,8 @@ public void testUpToDateMappingsAreIdentifiedAsUpToDate() throws IOException { public void testMissingIndexIsIdentifiedAsUpToDate() throws IOException { final ClusterName clusterName = new ClusterName("test-cluster"); final ClusterState.Builder clusterStateBuilder = ClusterState.builder(clusterName); - String mappingString = "/" + SECURITY_TEMPLATE_NAME + ".json"; - IndexTemplateMetaData.Builder templateMeta = getIndexTemplateMetaData(SECURITY_TEMPLATE_NAME, mappingString); + String mappingString = "/" + SECURITY_MAIN_TEMPLATE_7 + ".json"; + IndexTemplateMetaData.Builder templateMeta = getIndexTemplateMetaData(SECURITY_MAIN_TEMPLATE_7, mappingString); MetaData.Builder builder = new MetaData.Builder(clusterStateBuilder.build().getMetaData()); builder.put(templateMeta); clusterStateBuilder.metaData(builder); @@ -450,24 +458,24 @@ public void testMissingIndexIsIdentifiedAsUpToDate() throws IOException { private ClusterState.Builder createClusterStateWithTemplate(String securityTemplateString) throws IOException { // add the correct mapping no matter what the template - ClusterState clusterState = createClusterStateWithIndex("/" + SECURITY_TEMPLATE_NAME + ".json").build(); + ClusterState clusterState = createClusterStateWithIndex("/" + SECURITY_MAIN_TEMPLATE_7 + ".json").build(); final MetaData.Builder metaDataBuilder = new MetaData.Builder(clusterState.metaData()); - metaDataBuilder.put(getIndexTemplateMetaData(SECURITY_TEMPLATE_NAME, securityTemplateString)); + metaDataBuilder.put(getIndexTemplateMetaData(SECURITY_MAIN_TEMPLATE_7, securityTemplateString)); return ClusterState.builder(clusterState).metaData(metaDataBuilder); } private ClusterState.Builder createClusterStateWithMapping(String securityTemplateString) throws IOException { final ClusterState clusterState = createClusterStateWithIndex(securityTemplateString).build(); final String indexName = clusterState.metaData().getAliasAndIndexLookup() - .get(SECURITY_INDEX_NAME).getIndices().get(0).getIndex().getName(); + .get(RestrictedIndicesNames.SECURITY_MAIN_ALIAS).getIndices().get(0).getIndex().getName(); return ClusterState.builder(clusterState).routingTable(SecurityTestUtils.buildIndexRoutingTable(indexName)); } private ClusterState.Builder createClusterStateWithMappingAndTemplate(String securityTemplateString) throws IOException { ClusterState.Builder clusterStateBuilder = createClusterStateWithMapping(securityTemplateString); MetaData.Builder metaDataBuilder = new MetaData.Builder(clusterStateBuilder.build().metaData()); - String securityMappingString = "/" + SECURITY_TEMPLATE_NAME + ".json"; - IndexTemplateMetaData.Builder securityTemplateMeta = getIndexTemplateMetaData(SECURITY_TEMPLATE_NAME, securityMappingString); + String securityMappingString = "/" + SECURITY_MAIN_TEMPLATE_7 + ".json"; + IndexTemplateMetaData.Builder securityTemplateMeta = getIndexTemplateMetaData(SECURITY_MAIN_TEMPLATE_7, securityMappingString); metaDataBuilder.put(securityTemplateMeta); return clusterStateBuilder.metaData(metaDataBuilder); } @@ -493,7 +501,8 @@ private static IndexMetaData.Builder createIndexMetadata(String indexName, Strin private ClusterState.Builder createClusterStateWithIndex(String securityTemplate) throws IOException { final MetaData.Builder metaDataBuilder = new MetaData.Builder(); final boolean withAlias = randomBoolean(); - final String securityIndexName = SECURITY_INDEX_NAME + (withAlias ? "-" + randomAlphaOfLength(5) : ""); + final String securityIndexName = RestrictedIndicesNames.SECURITY_MAIN_ALIAS + + (withAlias ? "-" + randomAlphaOfLength(5) : ""); metaDataBuilder.put(createIndexMetadata(securityIndexName, securityTemplate)); ClusterState.Builder clusterStateBuilder = ClusterState.builder(state()); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/test/SecurityTestUtils.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/test/SecurityTestUtils.java index 12474b7a04d59..0ee5ff21ee117 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/test/SecurityTestUtils.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/test/SecurityTestUtils.java @@ -35,7 +35,7 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING; import static java.nio.file.StandardOpenOption.WRITE; -import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME; +import static org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames.SECURITY_MAIN_ALIAS; public class SecurityTestUtils { @@ -83,7 +83,7 @@ public static RoutingTable buildIndexRoutingTable(String indexName) { * Adds the index alias {@code .security} to the underlying concrete index. */ public static MetaData addAliasToMetaData(MetaData metaData, String indexName) { - AliasMetaData aliasMetaData = AliasMetaData.newAliasMetaDataBuilder(SECURITY_INDEX_NAME).build(); + AliasMetaData aliasMetaData = AliasMetaData.newAliasMetaDataBuilder(SECURITY_MAIN_ALIAS).build(); MetaData.Builder metaDataBuilder = new MetaData.Builder(metaData); IndexMetaData indexMetaData = metaData.index(indexName); metaDataBuilder.put(IndexMetaData.builder(indexMetaData).putAlias(aliasMetaData)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/user/XPackUserTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/user/XPackUserTests.java index a295e47b6d704..c8b20e1b4604c 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/user/XPackUserTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/user/XPackUserTests.java @@ -14,7 +14,6 @@ import org.elasticsearch.xpack.core.security.index.RestrictedIndicesNames; import org.elasticsearch.xpack.core.security.user.XPackUser; import org.elasticsearch.xpack.security.audit.index.IndexNameResolver; -import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.hamcrest.Matchers; import org.joda.time.DateTime; @@ -32,8 +31,6 @@ public void testXPackUserCanAccessNonSecurityIndices() { public void testXPackUserCannotAccessRestrictedIndices() { final String action = randomFrom(GetAction.NAME, SearchAction.NAME, IndexAction.NAME); final Predicate predicate = XPackUser.ROLE.indices().allowedIndicesMatcher(action); - assertThat(predicate.test(SecurityIndexManager.SECURITY_INDEX_NAME), Matchers.is(false)); - assertThat(predicate.test(SecurityIndexManager.INTERNAL_SECURITY_INDEX), Matchers.is(false)); for (String index : RestrictedIndicesNames.RESTRICTED_NAMES) { assertThat(predicate.test(index), Matchers.is(false)); } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/13_security-tokens_read.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/13_security-tokens_read.yml new file mode 100644 index 0000000000000..183731e1ba839 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/13_security-tokens_read.yml @@ -0,0 +1,161 @@ +--- +setup: + - skip: + features: headers + + - do: + cluster.health: + wait_for_status: yellow + + - do: + security.put_role: + name: "all_access" + body: > + { + "cluster": [ "all" ], + "indices": [ + { "names": ["*"], "privileges": ["all"] } + ] + } + + - do: + security.put_user: + username: "test_user" + body: > + { + "password" : "x-pack-test-password", + "roles" : [ "all_access" ], + "full_name" : "user with all possible privileges (but not superuser)" + } + +--- +teardown: + - do: + security.delete_user: + username: "test_user" + ignore: 404 + + - do: + security.delete_role: + name: "all_access" + ignore: 404 + +--- +"Test get security tokens index metadata": + + - do: + security.get_token: + body: + grant_type: "password" + username: "test_user" + password: "x-pack-test-password" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - match: { expires_in: 1200 } + - is_false: scope + + - do: + catch: forbidden + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + indices.get: + index: ".security-tokens" + + - do: + catch: forbidden + headers: + Authorization: Bearer ${token} + indices.get: + index: ".security-tokens" + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + indices.get: + index: ".secu*rity*tokens" + - length: { $body: 0 } + + - do: + headers: + Authorization: Bearer ${token} + indices.get: + index: ".secu*rity*tokens" + - length: { $body: 0 } + +--- +"Test get security document": + + - do: + security.get_token: + body: + grant_type: "password" + username: "test_user" + password: "x-pack-test-password" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - match: { expires_in: 1200 } + - is_false: scope + + - do: + catch: forbidden + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + get: + index: ".security-tokens" + id: token_${token} + + - do: + catch: forbidden + headers: + Authorization: Bearer ${token} + get: + index: ".security-tokens" + id: token_${token} + +--- +"Test search security tokens index": + + - do: + security.get_token: + body: + grant_type: "password" + username: "test_user" + password: "x-pack-test-password" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - match: { expires_in: 1200 } + - is_false: scope + + - do: + catch: forbidden + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + search: + rest_total_hits_as_int: true + index: ".security-tokens" + + - do: + catch: forbidden + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: ".security-tokens" + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + search: + rest_total_hits_as_int: true + index: ".secu*rity*tokens" + - match: { hits.total: 0 } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: ".secu*rity*tokens" + - match: { hits.total: 0 } + diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/14_security-tokens-7_read.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/14_security-tokens-7_read.yml new file mode 100644 index 0000000000000..97cff4ba98035 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/hidden-index/14_security-tokens-7_read.yml @@ -0,0 +1,161 @@ +--- +setup: + - skip: + features: headers + + - do: + cluster.health: + wait_for_status: yellow + + - do: + security.put_role: + name: "all_access" + body: > + { + "cluster": [ "all" ], + "indices": [ + { "names": ["*"], "privileges": ["all"] } + ] + } + + - do: + security.put_user: + username: "test_user" + body: > + { + "password" : "x-pack-test-password", + "roles" : [ "all_access" ], + "full_name" : "user with all possible privileges (but not superuser)" + } + +--- +teardown: + - do: + security.delete_user: + username: "test_user" + ignore: 404 + + - do: + security.delete_role: + name: "all_access" + ignore: 404 + +--- +"Test get security tokens index metadata": + + - do: + security.get_token: + body: + grant_type: "password" + username: "test_user" + password: "x-pack-test-password" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - match: { expires_in: 1200 } + - is_false: scope + + - do: + catch: forbidden + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + indices.get: + index: ".security-tokens-7" + + - do: + catch: forbidden + headers: + Authorization: Bearer ${token} + indices.get: + index: ".security-tokens-7" + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + indices.get: + index: ".secu*rity*tokens-7" + - length: { $body: 0 } + + - do: + headers: + Authorization: Bearer ${token} + indices.get: + index: ".secu*rity*tokens-7" + - length: { $body: 0 } + +--- +"Test get security document": + + - do: + security.get_token: + body: + grant_type: "password" + username: "test_user" + password: "x-pack-test-password" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - match: { expires_in: 1200 } + - is_false: scope + + - do: + catch: forbidden + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + get: + index: ".security-tokens-7" + id: token_${token} + + - do: + catch: forbidden + headers: + Authorization: Bearer ${token} + get: + index: ".security-tokens-7" + id: token_${token} + +--- +"Test search security tokens index": + + - do: + security.get_token: + body: + grant_type: "password" + username: "test_user" + password: "x-pack-test-password" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - match: { expires_in: 1200 } + - is_false: scope + + - do: + catch: forbidden + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + search: + rest_total_hits_as_int: true + index: ".security-tokens-7" + + - do: + catch: forbidden + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: ".security-tokens-7" + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + search: + rest_total_hits_as_int: true + index: ".secu*rity*tokens-7" + - match: { hits.total: 0 } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: ".secu*rity*tokens-7" + - match: { hits.total: 0 } + diff --git a/x-pack/qa/rolling-upgrade/build.gradle b/x-pack/qa/rolling-upgrade/build.gradle index 53810b89386f1..85bcc0a1fc8bb 100644 --- a/x-pack/qa/rolling-upgrade/build.gradle +++ b/x-pack/qa/rolling-upgrade/build.gradle @@ -119,6 +119,8 @@ for (Version version : bwcVersions.wireCompatible) { 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 'logger.org.elasticsearch.xpack.security.authc.TokenService', 'trace' setting 'xpack.security.audit.enabled', 'true' if (project.inFipsJvm) { setting 'xpack.security.transport.ssl.key', 'testnode.pem' @@ -182,6 +184,8 @@ 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.timeout', '60m' + setting 'logger.org.elasticsearch.xpack.security.authc.TokenService', 'trace' if (project.inFipsJvm) { setting 'xpack.security.transport.ssl.key', 'testnode.pem' setting 'xpack.security.transport.ssl.certificate', 'testnode.crt' diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java index 0dca61eba5721..69c515d80a3d2 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java @@ -14,197 +14,366 @@ import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.test.rest.yaml.ObjectPath; +import org.junit.After; +import org.junit.Before; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; public class TokenBackwardsCompatibilityIT extends AbstractUpgradeTestCase { - public void testGeneratingTokenInOldCluster() throws Exception { + private Collection twoClients = null; + + @Before + private void collectClientsByVersion() throws IOException { + Map clientsByVersion = getRestClientByVersion(); + if (clientsByVersion.size() == 2) { + // usual case, clients have different versions + twoClients = clientsByVersion.values(); + } else { + assert clientsByVersion.size() == 1 : "A rolling upgrade has a maximum of two distinct node versions, found: " + + clientsByVersion.keySet(); + // tests assumes exactly two clients to simplify some logic + twoClients = new ArrayList<>(); + twoClients.add(clientsByVersion.values().iterator().next()); + twoClients.add(clientsByVersion.values().iterator().next()); + } + } + + @After + private void closeClientsByVersion() throws IOException { + for (RestClient client : twoClients) { + client.close(); + } + twoClients = null; + } + + public void testGeneratingTokensInOldCluster() throws Exception { assumeTrue("this test should only run against the old cluster", CLUSTER_TYPE == ClusterType.OLD); - // Create a couple of tokens and store them in the token_backwards_compatibility_it index to be used for tests in the mixed/upgraded - // clusters - Request createTokenRequest = new Request("POST", "/_security/oauth2/token"); - createTokenRequest.setJsonEntity( - "{\n" + - " \"username\": \"test_user\",\n" + - " \"password\": \"x-pack-test-password\",\n" + - " \"grant_type\": \"password\"\n" + - "}"); - Response response = client().performRequest(createTokenRequest); - assertOK(response); - Map responseMap = entityAsMap(response); - String token = (String) responseMap.get("access_token"); - assertNotNull(token); - assertTokenWorks(token); + // Creates two access and refresh tokens and stores them in the token_backwards_compatibility_it index to be used for tests in the + // mixed/upgraded clusters + Map responseMap = createTokens(client(), "test_user", "x-pack-test-password"); + String accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + String refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); - Request indexRequest1 = new Request("PUT", "token_backwards_compatibility_it/_doc/old_cluster_token1"); - indexRequest1.setJsonEntity( - "{\n" + - " \"token\": \"" + token + "\"\n" + - "}"); - Response indexResponse1 = client().performRequest(indexRequest1); - assertOK(indexResponse1); - Request createSecondTokenRequest = new Request("POST", "/_security/oauth2/token"); - createSecondTokenRequest.setEntity(createTokenRequest.getEntity()); - response = client().performRequest(createSecondTokenRequest); - responseMap = entityAsMap(response); - token = (String) responseMap.get("access_token"); - assertNotNull(token); - assertTokenWorks(token); - Request indexRequest2 = new Request("PUT", "token_backwards_compatibility_it/_doc/old_cluster_token2"); - indexRequest2.setJsonEntity( - "{\n" + - " \"token\": \"" + token + "\"\n" + - "}"); - Response indexResponse2 = client().performRequest(indexRequest2); - assertOK(indexResponse2); + storeTokens(client(), 1, accessToken, refreshToken); + + responseMap = createTokens(client(), "test_user", "x-pack-test-password"); + accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + + storeTokens(client(), 2, accessToken, refreshToken); + } + + public void testRefreshingTokensInOldCluster() throws Exception { + assumeTrue("this test should only run against the old cluster", CLUSTER_TYPE == ClusterType.OLD); + // Creates access and refresh tokens and uses the refresh token. The new resulting tokens are used in different phases + Map responseMap = createTokens(client(), "test_user", "x-pack-test-password"); + String accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + String refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + + storeTokens(client(), 3, accessToken, refreshToken); + + // refresh the token just created. The old token is invalid (tested further) and the new refresh token is tested in the upgraded + // cluster + Map refreshResponseMap = refreshToken(client(), refreshToken); + String refreshedAccessToken = (String) refreshResponseMap.get("access_token"); + String refreshedRefreshToken = (String) refreshResponseMap.get("refresh_token"); + assertNotNull(refreshedAccessToken); + assertNotNull(refreshedRefreshToken); + assertAccessTokenWorks(refreshedAccessToken); + // assert previous access token still works + assertAccessTokenWorks(accessToken); + + storeTokens(client(), 4, refreshedAccessToken, refreshedRefreshToken); } - public void testTokenWorksInMixedCluster() throws Exception { + public void testInvalidatingTokensInOldCluster() throws Exception { + assumeTrue("this test should only run against the old cluster", CLUSTER_TYPE == ClusterType.OLD); + // Creates access and refresh tokens and tries to use the access tokens several times + Map responseMap = createTokens(client(), "test_user", "x-pack-test-password"); + String accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + String refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + + storeTokens(client(), 5, accessToken, refreshToken); + + // invalidate access token + invalidateAccessToken(client(), accessToken); + assertAccessTokenDoesNotWork(accessToken); + // invalidate refresh token + invalidateRefreshToken(client(), refreshToken); + assertRefreshTokenInvalidated(refreshToken); + } + + public void testAccessTokensWorkInMixedCluster() throws Exception { // Verify that an old token continues to work during all stages of the rolling upgrade assumeTrue("this test should only run against the mixed cluster", CLUSTER_TYPE == ClusterType.MIXED); - Request getRequest = new Request("GET", "token_backwards_compatibility_it/_doc/old_cluster_token1"); - Response getResponse = client().performRequest(getRequest); - assertOK(getResponse); - Map source = (Map) entityAsMap(getResponse).get("_source"); - assertTokenWorks((String) source.get("token")); + for (int tokenIdx : Arrays.asList(1, 3, 4)) { // 2 is invalidated in another mixed-cluster test, 5 is invalidated in the old cluster + Map source = retrieveStoredTokens(client(), tokenIdx); + assertAccessTokenWorks((String) source.get("token")); + } } - public void testInvalidatingTokenInMixedCluster() throws Exception { - // Verify that we can invalidate a token in a mixed cluster + public void testTokensStayInvalidatedInMixedCluster() throws Exception { + // Verify that an old, invalidated token remains invalidated during all stages of the rolling upgrade assumeTrue("this test should only run against the mixed cluster", CLUSTER_TYPE == ClusterType.MIXED); - Request getRequest = new Request("GET", "token_backwards_compatibility_it/_doc/old_cluster_token2"); - Response getResponse = client().performRequest(getRequest); - assertOK(getResponse); - Map source = (Map) entityAsMap(getResponse).get("_source"); - String token = (String) source.get("token"); - // The token might be already invalidated by running testInvalidatingTokenInMixedCluster in a previous stage - // we don't try to assert it works before invalidating. This case is handled by testTokenWorksInMixedCluster - Request invalidateRequest = new Request("DELETE", "/_security/oauth2/token"); - invalidateRequest.setJsonEntity("{\"token\": \"" + token + "\"}"); - invalidateRequest.addParameter("error_trace", "true"); - client().performRequest(invalidateRequest); - assertTokenDoesNotWork(token); + Map source = retrieveStoredTokens(client(), 5); + assertAccessTokenDoesNotWork((String) source.get("token")); + assertRefreshTokenInvalidated((String) source.get("refresh_token")); } - public void testMixedClusterWithUpgradedMaster() throws Exception { + public void testGeneratingTokensInMixedCluster() throws Exception { assumeTrue("this test should only run against the mixed cluster", CLUSTER_TYPE == ClusterType.MIXED); - assumeTrue("the master must be on the latest version before we can write", isMasterOnLatestVersion()); - - // create token and refresh on version that supports it - Request createTokenRequest = new Request("POST", "/_security/oauth2/token"); - createTokenRequest.setJsonEntity( - "{\n" + - " \"username\": \"test_user\",\n" + - " \"password\": \"x-pack-test-password\",\n" + - " \"grant_type\": \"password\"\n" + - "}"); - try (RestClient client = getRestClientForCurrentVersionNodesOnly()) { - Response response = client.performRequest(createTokenRequest); - Map responseMap = entityAsMap(response); + // Creates two access and refresh tokens and stores them in the token_backwards_compatibility_it index to be used for tests in the + // mixed/upgraded clusters + int generatedTokenIdxDuringMixed = 10; + for (RestClient client : twoClients) { + Map responseMap = createTokens(client, "test_user", "x-pack-test-password"); String accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); String refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + + storeTokens(client(), generatedTokenIdxDuringMixed++, accessToken, refreshToken); + + responseMap = createTokens(client, "test_user", "x-pack-test-password"); + accessToken = (String) responseMap.get("access_token"); assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + refreshToken = (String) responseMap.get("refresh_token"); assertNotNull(refreshToken); - assertTokenWorks(accessToken); - Request tokenRefreshRequest = new Request("POST", "/_security/oauth2/token"); - tokenRefreshRequest.setJsonEntity( - "{\n" + - " \"refresh_token\": \"" + refreshToken + "\",\n" + - " \"grant_type\": \"refresh_token\"\n" + - "}"); - response = client.performRequest(tokenRefreshRequest); - responseMap = entityAsMap(response); - String updatedAccessToken = (String) responseMap.get("access_token"); - String updatedRefreshToken = (String) responseMap.get("refresh_token"); - assertNotNull(updatedAccessToken); - assertNotNull(updatedRefreshToken); - assertTokenWorks(updatedAccessToken); - assertTokenWorks(accessToken); - assertNotEquals(accessToken, updatedAccessToken); - assertNotEquals(refreshToken, updatedRefreshToken); - // Invalidate the new access token and ensure that it no longer works - Request invalidateTokenRequest = new Request("DELETE", "/_security/oauth2/token"); - invalidateTokenRequest.setJsonEntity( - "{\n" + - " \"token\": \"" + updatedAccessToken + "\"\n" + - "}"); - Response invalidateTokenResponse = client.performRequest(invalidateTokenRequest); - assertOK(invalidateTokenResponse); - assertTokenDoesNotWork(updatedAccessToken); + storeTokens(client(), generatedTokenIdxDuringMixed++, accessToken, refreshToken); + } + } + + public void testRefreshingTokensInMixedCluster() throws Exception { + // verify new nodes can refresh tokens created by old nodes and vice versa + assumeTrue("this test should only run against the mixed cluster", CLUSTER_TYPE == ClusterType.MIXED); + for (RestClient client1 : twoClients) { + Map responseMap = createTokens(client1, "test_user", "x-pack-test-password"); + String accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + String refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + for (RestClient client2 : twoClients) { + responseMap = refreshToken(client2, refreshToken); + accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + } } } - public void testUpgradedCluster() throws Exception { + public void testInvalidatingTokensInMixedCluster() throws Exception { + // Verify that we can invalidate an access and refresh token in a mixed cluster + assumeTrue("this test should only run against the mixed cluster", CLUSTER_TYPE == ClusterType.MIXED); + Map source = retrieveStoredTokens(client(), 2); + String accessToken = (String) source.get("token"); + String refreshToken = (String) source.get("refresh_token"); + // The token might be already invalidated by running testInvalidatingTokenInMixedCluster in a previous stage + // we don't try to assert it works before invalidating. This case is handled by testTokenWorksInMixedCluster + invalidateAccessToken(client(), accessToken); + assertAccessTokenDoesNotWork(accessToken); + // invalidate refresh token + invalidateRefreshToken(client(), refreshToken); + assertRefreshTokenInvalidated(refreshToken); + } + + public void testTokensStayInvalidatedInUpgradedCluster() throws Exception { assumeTrue("this test should only run against the upgraded cluster", CLUSTER_TYPE == ClusterType.UPGRADED); + for (int tokenIdx : Arrays.asList(2, 5)) { + Map source = retrieveStoredTokens(client(), tokenIdx); + assertAccessTokenDoesNotWork((String) source.get("token")); + assertRefreshTokenInvalidated((String) source.get("refresh_token")); + } + } - // Use an old token to authenticate, then invalidate it and verify that it can no longer be used - Request getRequest = new Request("GET", "token_backwards_compatibility_it/_doc/old_cluster_token1"); - Response getResponse = client().performRequest(getRequest); - assertOK(getResponse); - Map source = (Map) entityAsMap(getResponse).get("_source"); - final String token = (String) source.get("token"); + public void testAccessTokensWorkInUpgradedCluster() throws Exception { + assumeTrue("this test should only run against the upgraded cluster", CLUSTER_TYPE == ClusterType.UPGRADED); + for (int tokenIdx : Arrays.asList(3, 4, 10, 12)) { + Map source = retrieveStoredTokens(client(), tokenIdx); + assertAccessTokenWorks((String) source.get("token")); + } + } - Request invalidateRequest = new Request("DELETE", "/_security/oauth2/token"); - invalidateRequest.setJsonEntity("{\"token\": \"" + token + "\"}"); - invalidateRequest.addParameter("error_trace", "true"); - Response invalidationResponse = client().performRequest(invalidateRequest); - assertOK(invalidationResponse); - assertTokenDoesNotWork(token); - } - - private void assertTokenWorks(String token) throws IOException { - Request request = new Request("GET", "/_security/_authenticate"); - RequestOptions.Builder options = request.getOptions().toBuilder(); - options.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + token); - request.setOptions(options); - Response authenticateResponse = client().performRequest(request); - assertOK(authenticateResponse); - assertEquals("test_user", entityAsMap(authenticateResponse).get("username")); - } - - private void assertTokenDoesNotWork(String token) { - Request request = new Request("GET", "/_security/_authenticate"); - RequestOptions.Builder options = request.getOptions().toBuilder(); - options.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + token); - request.setOptions(options); - ResponseException e = expectThrows(ResponseException.class, () -> client().performRequest(request)); - assertEquals(401, e.getResponse().getStatusLine().getStatusCode()); - Response response = e.getResponse(); - assertEquals("Bearer realm=\"security\", error=\"invalid_token\", error_description=\"The access token expired\"", - response.getHeader("WWW-Authenticate")); - } - - private boolean isMasterOnLatestVersion() throws Exception { - Response response = client().performRequest(new Request("GET", "_cluster/state")); - assertOK(response); - final String masterNodeId = ObjectPath.createFromResponse(response).evaluate("master_node"); - response = client().performRequest(new Request("GET", "_nodes")); - assertOK(response); - ObjectPath objectPath = ObjectPath.createFromResponse(response); - logger.info("Master node is on version: " + objectPath.evaluate("nodes." + masterNodeId + ".version")); - return Version.CURRENT.equals(Version.fromString(objectPath.evaluate("nodes." + masterNodeId + ".version"))); + public void testGeneratingTokensInUpgradedCluster() throws Exception { + assumeTrue("this test should only run against the upgraded cluster", CLUSTER_TYPE == ClusterType.UPGRADED); + Map responseMap = createTokens(client(), "test_user", "x-pack-test-password"); + String accessToken = (String) responseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + String refreshToken = (String) responseMap.get("refresh_token"); + assertNotNull(refreshToken); + } + + public void testRefreshingTokensInUpgradedCluster() throws Exception { + assumeTrue("this test should only run against the upgraded cluster", CLUSTER_TYPE == ClusterType.UPGRADED); + for (int tokenIdx : Arrays.asList(4, 10, 12)) { + Map source = retrieveStoredTokens(client(), tokenIdx); + Map refreshedResponseMap = refreshToken(client(), (String) source.get("refresh_token")); + String accessToken = (String) refreshedResponseMap.get("access_token"); + assertNotNull(accessToken); + assertAccessTokenWorks(accessToken); + String refreshToken = (String) refreshedResponseMap.get("refresh_token"); + assertNotNull(refreshToken); + } + } + + public void testInvalidatingTokensInUpgradedCluster() throws Exception { + assumeTrue("this test should only run against the upgraded cluster", CLUSTER_TYPE == ClusterType.UPGRADED); + for (int tokenIdx : Arrays.asList(1, 11, 13)) { + Map source = retrieveStoredTokens(client(), tokenIdx); + String accessToken = (String) source.get("token"); + String refreshToken = (String) source.get("refresh_token"); + // invalidate access token + invalidateAccessToken(client(), accessToken); + assertAccessTokenDoesNotWork(accessToken); + // invalidate refresh token + invalidateRefreshToken(client(), refreshToken); + assertRefreshTokenInvalidated(refreshToken); + } + } + + private void assertAccessTokenWorks(String token) throws IOException { + for (RestClient client : twoClients) { + Request request = new Request("GET", "/_security/_authenticate"); + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + token); + request.setOptions(options); + Response authenticateResponse = client.performRequest(request); + assertOK(authenticateResponse); + assertEquals("test_user", entityAsMap(authenticateResponse).get("username")); + } + } + + private void assertAccessTokenDoesNotWork(String token) throws IOException { + for (RestClient client : twoClients) { + Request request = new Request("GET", "/_security/_authenticate"); + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + token); + request.setOptions(options); + ResponseException e = expectThrows(ResponseException.class, () -> client.performRequest(request)); + assertEquals(401, e.getResponse().getStatusLine().getStatusCode()); + Response response = e.getResponse(); + assertEquals("Bearer realm=\"security\", error=\"invalid_token\", error_description=\"The access token expired\"", + response.getHeader("WWW-Authenticate")); + } + } + + private void assertRefreshTokenInvalidated(String refreshToken) throws IOException { + for (RestClient client : twoClients) { + Request refreshTokenRequest = new Request("POST", "/_security/oauth2/token"); + refreshTokenRequest.setJsonEntity( + "{\n" + + " \"refresh_token\": \"" + refreshToken + "\",\n" + + " \"grant_type\": \"refresh_token\"\n" + + "}"); + ResponseException e = expectThrows(ResponseException.class, () -> client.performRequest(refreshTokenRequest)); + assertEquals(400, e.getResponse().getStatusLine().getStatusCode()); + Response response = e.getResponse(); + Map responseMap = entityAsMap(response); + assertEquals("invalid_grant", responseMap.get("error")); + assertEquals("token has been invalidated", responseMap.get("error_description")); + } } - private RestClient getRestClientForCurrentVersionNodesOnly() throws IOException { + private Map getRestClientByVersion() throws IOException { Response response = client().performRequest(new Request("GET", "_nodes")); assertOK(response); ObjectPath objectPath = ObjectPath.createFromResponse(response); Map nodesAsMap = objectPath.evaluate("nodes"); - List hosts = new ArrayList<>(); + Map> hostsByVersion = new HashMap<>(); for (Map.Entry entry : nodesAsMap.entrySet()) { Map nodeDetails = (Map) entry.getValue(); Version version = Version.fromString((String) nodeDetails.get("version")); - if (Version.CURRENT.equals(version)) { - Map httpInfo = (Map) nodeDetails.get("http"); - hosts.add(HttpHost.create((String) httpInfo.get("publish_address"))); - } + Map httpInfo = (Map) nodeDetails.get("http"); + hostsByVersion.computeIfAbsent(version, k -> new ArrayList<>()).add(HttpHost.create((String) httpInfo.get("publish_address"))); + } + Map clientsByVersion = new HashMap<>(); + for (Map.Entry> entry : hostsByVersion.entrySet()) { + clientsByVersion.put(entry.getKey(), buildClient(restClientSettings(), entry.getValue().toArray(new HttpHost[0]))); } + return clientsByVersion; + } - return buildClient(restClientSettings(), hosts.toArray(new HttpHost[0])); + private Map createTokens(RestClient client, String username, String password) throws IOException { + final Request createTokenRequest = new Request("POST", "/_security/oauth2/token"); + createTokenRequest.setJsonEntity( + "{\n" + + " \"username\": \"" + username + "\",\n" + + " \"password\": \"" + password + "\",\n" + + " \"grant_type\": \"password\"\n" + + "}"); + Response response = client().performRequest(createTokenRequest); + assertOK(response); + return entityAsMap(response); + } + + private void storeTokens(RestClient client, int idx, String accessToken, String refreshToken) throws IOException { + final Request indexRequest = new Request("PUT", "token_backwards_compatibility_it/_doc/old_cluster_token" + idx); + indexRequest.setJsonEntity( + "{\n" + + " \"token\": \"" + accessToken + "\",\n" + + " \"refresh_token\": \"" + refreshToken + "\"\n" + + "}"); + Response indexResponse1 = client.performRequest(indexRequest); + assertOK(indexResponse1); + } + + private Map retrieveStoredTokens(RestClient client, int tokenIdx) throws IOException { + Request getRequest = new Request("GET", "token_backwards_compatibility_it/_doc/old_cluster_token" + tokenIdx); + Response getResponse = client().performRequest(getRequest); + assertOK(getResponse); + return (Map) entityAsMap(getResponse).get("_source"); + } + + private Map refreshToken(RestClient client, String refreshToken) throws IOException { + final Request refreshTokenRequest = new Request("POST", "/_security/oauth2/token"); + refreshTokenRequest.setJsonEntity( + "{\n" + + " \"refresh_token\": \"" + refreshToken + "\",\n" + + " \"grant_type\": \"refresh_token\"\n" + + "}"); + Response refreshResponse = client.performRequest(refreshTokenRequest); + assertOK(refreshResponse); + return entityAsMap(refreshResponse); + } + + private void invalidateAccessToken(RestClient client, String accessToken) throws IOException { + Request invalidateRequest = new Request("DELETE", "/_security/oauth2/token"); + invalidateRequest.setJsonEntity("{\"token\": \"" + accessToken + "\"}"); + invalidateRequest.addParameter("error_trace", "true"); + Response invalidateResponse = client.performRequest(invalidateRequest); + assertOK(invalidateResponse); + } + + private void invalidateRefreshToken(RestClient client, String refreshToken) throws IOException { + Request invalidateRequest = new Request("DELETE", "/_security/oauth2/token"); + invalidateRequest.setJsonEntity("{\"refresh_token\": \"" + refreshToken + "\"}"); + invalidateRequest.addParameter("error_trace", "true"); + Response invalidateResponse = client.performRequest(invalidateRequest); + assertOK(invalidateResponse); } } diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml index aae5f30859743..f426d9b2525b4 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml @@ -3,14 +3,17 @@ - skip: features: headers + - do: + cluster.health: + wait_for_status: yellow + - do: get: index: token_index - type: doc id: "6" - match: { _index: token_index } - - match: { _type: doc } + - match: { _type: _doc } - match: { _id: "6" } - is_true: _source.token - set: { _source.token : token } @@ -24,6 +27,125 @@ - match: { roles.0: "superuser" } - match: { full_name: "Token User" } + # call three times because the client rotates the nodes + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } + +--- +"Get the indexed refreshed access token and use if to authenticate": + - skip: + features: headers + + - do: + get: + index: token_index + id: "7" + + - match: { _index: token_index } + - match: { _type: _doc } + - match: { _id: "7" } + - is_true: _source.token + - set: { _source.token : token } + + - do: + headers: + Authorization: Bearer ${token} + security.authenticate: {} + + - match: { username: "token_user" } + - match: { roles.0: "superuser" } + - match: { full_name: "Token User" } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } + +--- +"Get the indexed refresh token and use it to get another access token and authenticate": + - skip: + features: headers + + - do: + get: + index: token_index + id: "8" + + - match: { _index: token_index } + - match: { _type: _doc } + - match: { _id: "8" } + - is_true: _source.token + - set: { _source.token : refresh_token } + + - do: + security.get_token: + body: + grant_type: "refresh_token" + refresh_token: "${refresh_token}" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - is_true: refresh_token + - set: { refresh_token: refresh_token } + - match: { expires_in: 3600 } + - is_false: scope + + - do: + headers: + Authorization: Bearer ${token} + security.authenticate: {} + + - match: { username: "token_user" } + - match: { roles.0: "superuser" } + - match: { full_name: "Token User" } + - do: headers: Authorization: Bearer ${token} @@ -31,7 +153,7 @@ rest_total_hits_as_int: true index: token_index - - match: { hits.total: 6 } + - match: { hits.total: 8 } - do: headers: @@ -40,7 +162,7 @@ rest_total_hits_as_int: true index: token_index - - match: { hits.total: 6 } + - match: { hits.total: 8 } - do: headers: @@ -49,5 +171,13 @@ rest_total_hits_as_int: true index: token_index - - match: { hits.total: 6 } + - match: { hits.total: 8 } + # overwrite the used refresh token with the new one + - do: + headers: + Authorization: Bearer ${token} + index: + index: token_index + id: "8" + body: { "token" : "${refresh_token}"} diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml index 02fa0f31ce37d..e4d0eb8757f85 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml @@ -27,7 +27,9 @@ - match: { type: "Bearer" } - is_true: access_token - set: { access_token: token } - - match: { expires_in: 1200 } + - is_true: refresh_token + - set: { refresh_token: refresh_token } + - match: { expires_in: 3600 } - is_false: scope - do: @@ -54,15 +56,15 @@ bulk: refresh: true body: - - '{"index": {"_index": "token_index", "_type": "doc", "_id" : "1"}}' + - '{"index": {"_index": "token_index", "_type": "_doc", "_id" : "1"}}' - '{"f1": "v1_old", "f2": 0}' - - '{"index": {"_index": "token_index", "_type": "doc", "_id" : "2"}}' + - '{"index": {"_index": "token_index", "_type": "_doc", "_id" : "2"}}' - '{"f1": "v2_old", "f2": 1}' - - '{"index": {"_index": "token_index", "_type": "doc", "_id" : "3"}}' + - '{"index": {"_index": "token_index", "_type": "_doc", "_id" : "3"}}' - '{"f1": "v3_old", "f2": 2}' - - '{"index": {"_index": "token_index", "_type": "doc", "_id" : "4"}}' + - '{"index": {"_index": "token_index", "_type": "_doc", "_id" : "4"}}' - '{"f1": "v4_old", "f2": 3}' - - '{"index": {"_index": "token_index", "_type": "doc", "_id" : "5"}}' + - '{"index": {"_index": "token_index", "_type": "_doc", "_id" : "5"}}' - '{"f1": "v5_old", "f2": 4}' - do: @@ -81,6 +83,48 @@ Authorization: Bearer ${token} index: index: token_index - type: doc id: "6" body: { "token" : "${token}"} + + # refresh token and store it as well + - do: + security.get_token: + body: + grant_type: "refresh_token" + refresh_token: "${refresh_token}" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: refreshed_access_token } + - is_true: refresh_token + - set: { refresh_token: refreshed_refresh_token } + - match: { expires_in: 3600 } + - is_false: scope + + # test refresh token (use it) + - do: + headers: + Authorization: Bearer ${refreshed_access_token} + security.authenticate: {} + + - match: { username: "token_user" } + - match: { roles.0: "superuser" } + - match: { full_name: "Token User" } + + # store the new refreshed access token + - do: + headers: + Authorization: Bearer ${refreshed_access_token} + index: + index: token_index + id: "7" + body: { "token" : "${refreshed_access_token}"} + + # store the refresh token + - do: + headers: + Authorization: Bearer ${refreshed_access_token} + index: + index: token_index + id: "8" + body: { "token" : "${refreshed_refresh_token}"} diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml index f892c9f2f9833..430f94c1064d6 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml @@ -2,15 +2,18 @@ "Get the indexed token and use if to authenticate": - skip: features: headers - + + - do: + cluster.health: + wait_for_status: yellow + - do: get: index: token_index - type: doc id: "6" - match: { _index: token_index } - - match: { _type: doc } + - match: { _type: _doc } - match: { _id: "6" } - is_true: _source.token - set: { _source.token : token } @@ -31,7 +34,7 @@ rest_total_hits_as_int: true index: token_index - - match: { hits.total: 6 } + - match: { hits.total: 8 } # counter example that we are really checking this - do: @@ -41,3 +44,51 @@ search: rest_total_hits_as_int: true index: token_index + +--- +"Get the indexed refresh token and use if to get another access token and authenticate": + - skip: + features: headers + + - do: + get: + index: token_index + id: "8" + + - match: { _index: token_index } + - match: { _type: _doc } + - match: { _id: "8" } + - is_true: _source.token + - set: { _source.token : refresh_token } + + - do: + security.get_token: + body: + grant_type: "refresh_token" + refresh_token: "${refresh_token}" + + - match: { type: "Bearer" } + - is_true: access_token + - set: { access_token: token } + - is_true: refresh_token + - set: { refresh_token: refresh_token } + - match: { expires_in: 3600 } + - is_false: scope + + - do: + headers: + Authorization: Bearer ${token} + security.authenticate: {} + + - match: { username: "token_user" } + - match: { roles.0: "superuser" } + - match: { full_name: "Token User" } + + - do: + headers: + Authorization: Bearer ${token} + search: + rest_total_hits_as_int: true + index: token_index + + - match: { hits.total: 8 } diff --git a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java index bfec6d100a984..86d411053afe9 100644 --- a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java +++ b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java @@ -199,7 +199,7 @@ public void cleanupSecurityIndex() throws Exception { @Override public Set excludeTemplates() { Set templates = Sets.newHashSet(super.excludeTemplates()); - templates.add(SecurityIndexManager.SECURITY_TEMPLATE_NAME); // don't remove the security index template + templates.add(SecurityIndexManager.SECURITY_MAIN_TEMPLATE_7); // don't remove the security index template return templates; } From d184056bf5cf9968db95c16cb27f6f2ec505bc3e Mon Sep 17 00:00:00 2001 From: David Roberts Date: Tue, 23 Apr 2019 15:11:38 +0100 Subject: [PATCH 238/260] [TEST] Mute RemoteClusterServiceTests.testCollectNodes Due to https://github.com/elastic/elasticsearch/issues/41067 --- .../org/elasticsearch/transport/RemoteClusterServiceTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java index f2630e15ce95d..d2c476571c927 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java @@ -575,6 +575,7 @@ private ActionListener connectionListener(final CountDownLatch latch) { return ActionListener.wrap(x -> latch.countDown(), x -> fail()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/41067") public void testCollectNodes() throws InterruptedException, IOException { final Settings settings = Settings.EMPTY; final List knownNodes_c1 = new CopyOnWriteArrayList<>(); From b63318209460ade94d78b2394eefc4d04e125c13 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 30 Apr 2019 12:38:22 +0200 Subject: [PATCH 239/260] Implement Bulk Deletes for GCS Repository (#41368) * Implement Bulk Deletes for GCS Repository * Just like #40322 for AWS * We already had a bulk delete API but weren't using it from the blob container implementation, now we are using it * Made the bulk delete API also compliant with our interface that only suppresses errors about non existent blobs by stating failed deletes (I didn't use any bulk stat action here since having to stat here should be the exception anyway and it would make error handling a lot more complex) * Fixed bulk delete API to limit its batch size to 100 in line with GCS recommendations --- .../gcs/GoogleCloudStorageBlobContainer.java | 9 +++- .../gcs/GoogleCloudStorageBlobStore.java | 53 ++++++++++++------- .../repositories/gcs/MockStorage.java | 27 +++++++++- 3 files changed, 69 insertions(+), 20 deletions(-) diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java index 8ad9b453a9092..fb81a5c90039f 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java @@ -26,7 +26,9 @@ import java.io.IOException; import java.io.InputStream; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; class GoogleCloudStorageBlobContainer extends AbstractBlobContainer { @@ -78,7 +80,12 @@ public void deleteBlob(String blobName) throws IOException { blobStore.deleteBlob(buildKey(blobName)); } - protected String buildKey(String blobName) { + @Override + public void deleteBlobsIgnoringIfNotExists(List blobNames) throws IOException { + blobStore.deleteBlobsIgnoringIfNotExists(blobNames.stream().map(this::buildKey).collect(Collectors.toList())); + } + + private String buildKey(String blobName) { assert blobName != null; return path + blobName; } diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java index 84184660159a4..d873a5cd29074 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java @@ -19,6 +19,7 @@ package org.elasticsearch.repositories.gcs; +import com.google.cloud.BatchResult; import com.google.cloud.ReadChannel; import com.google.cloud.WriteChannel; import com.google.cloud.storage.Blob; @@ -27,10 +28,9 @@ import com.google.cloud.storage.Bucket; import com.google.cloud.storage.Storage; import com.google.cloud.storage.Storage.BlobListOption; +import com.google.cloud.storage.StorageBatch; import com.google.cloud.storage.StorageException; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; @@ -50,17 +50,18 @@ import java.nio.channels.WritableByteChannel; import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.net.HttpURLConnection.HTTP_PRECON_FAILED; class GoogleCloudStorageBlobStore implements BlobStore { - - private static final Logger logger = LogManager.getLogger(GoogleCloudStorageBlobStore.class); // The recommended maximum size of a blob that should be uploaded in a single // request. Larger files should be uploaded over multiple requests (this is @@ -105,7 +106,7 @@ public void close() { * @param bucketName name of the bucket * @return true iff the bucket exists */ - boolean doesBucketExist(String bucketName) { + private boolean doesBucketExist(String bucketName) { try { final Bucket bucket = SocketAccess.doPrivilegedIOException(() -> client().get(bucketName)); return bucket != null; @@ -295,8 +296,8 @@ void deleteBlob(String blobName) throws IOException { * * @param prefix prefix of the blobs to delete */ - void deleteBlobsByPrefix(String prefix) throws IOException { - deleteBlobs(listBlobsByPrefix("", prefix).keySet()); + private void deleteBlobsByPrefix(String prefix) throws IOException { + deleteBlobsIgnoringIfNotExists(listBlobsByPrefix("", prefix).keySet()); } /** @@ -304,7 +305,7 @@ void deleteBlobsByPrefix(String prefix) throws IOException { * * @param blobNames names of the blobs to delete */ - void deleteBlobs(Collection blobNames) throws IOException { + void deleteBlobsIgnoringIfNotExists(Collection blobNames) throws IOException { if (blobNames.isEmpty()) { return; } @@ -314,17 +315,33 @@ void deleteBlobs(Collection blobNames) throws IOException { return; } final List blobIdsToDelete = blobNames.stream().map(blob -> BlobId.of(bucketName, blob)).collect(Collectors.toList()); - final List deletedStatuses = SocketAccess.doPrivilegedIOException(() -> client().delete(blobIdsToDelete)); - assert blobIdsToDelete.size() == deletedStatuses.size(); - boolean failed = false; - for (int i = 0; i < blobIdsToDelete.size(); i++) { - if (deletedStatuses.get(i) == false) { - logger.error("Failed to delete blob [{}] in bucket [{}]", blobIdsToDelete.get(i).getName(), bucketName); - failed = true; + final List failedBlobs = Collections.synchronizedList(new ArrayList<>()); + final StorageException e = SocketAccess.doPrivilegedIOException(() -> { + final AtomicReference ioe = new AtomicReference<>(); + final StorageBatch batch = client().batch(); + for (BlobId blob : blobIdsToDelete) { + batch.delete(blob).notify( + new BatchResult.Callback<>() { + @Override + public void success(Boolean result) { + } + + @Override + public void error(StorageException exception) { + if (exception.getCode() != HTTP_NOT_FOUND) { + failedBlobs.add(blob); + if (ioe.compareAndSet(null, exception) == false) { + ioe.get().addSuppressed(exception); + } + } + } + }); } - } - if (failed) { - throw new IOException("Failed to delete all [" + blobIdsToDelete.size() + "] blobs"); + batch.submit(); + return ioe.get(); + }); + if (e != null) { + throw new IOException("Exception when deleting blobs [" + failedBlobs + "]", e); } } diff --git a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java index 97c7e2ab76bd2..eddf2a9f78082 100644 --- a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java +++ b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/MockStorage.java @@ -20,6 +20,7 @@ package org.elasticsearch.repositories.gcs; import com.google.api.gax.paging.Page; +import com.google.cloud.BatchResult; import com.google.cloud.Policy; import com.google.cloud.ReadChannel; import com.google.cloud.RestorableState; @@ -34,11 +35,13 @@ import com.google.cloud.storage.ServiceAccount; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageBatch; +import com.google.cloud.storage.StorageBatchResult; import com.google.cloud.storage.StorageException; import com.google.cloud.storage.StorageOptions; import com.google.cloud.storage.StorageRpcOptionUtils; import com.google.cloud.storage.StorageTestUtils; import org.elasticsearch.core.internal.io.IOUtils; +import org.mockito.stubbing.Answer; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -57,6 +60,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyVararg; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; + /** * {@link MockStorage} mocks a {@link Storage} client by storing all the blobs * in a given concurrent map. @@ -356,8 +364,25 @@ public byte[] readAllBytes(BlobId blob, BlobSourceOption... options) { } @Override + @SuppressWarnings("unchecked") public StorageBatch batch() { - return null; + final Answer throwOnMissingMock = invocationOnMock -> { + throw new AssertionError("Did not expect call to method [" + invocationOnMock.getMethod().getName() + ']'); + }; + final StorageBatch batch = mock(StorageBatch.class, throwOnMissingMock); + StorageBatchResult result = mock(StorageBatchResult.class, throwOnMissingMock); + doAnswer(answer -> { + BatchResult.Callback callback = (BatchResult.Callback) answer.getArguments()[0]; + callback.success(true); + return null; + }).when(result).notify(any(BatchResult.Callback.class)); + doAnswer(invocation -> { + final BlobId blobId = (BlobId) invocation.getArguments()[0]; + delete(blobId); + return result; + }).when(batch).delete(any(BlobId.class), anyVararg()); + doAnswer(invocation -> null).when(batch).submit(); + return batch; } @Override From 0946e08e632c33fb58017cdefa149f528ed76a26 Mon Sep 17 00:00:00 2001 From: Tom Veasey Date: Tue, 30 Apr 2019 13:08:11 +0100 Subject: [PATCH 240/260] [ML] Update for model selection change and disable temporarily (#41482) --- .../xpack/ml/integration/DeleteExpiredDataIT.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java index a62d0f6f43d2f..eb89c15e146fe 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java @@ -91,6 +91,7 @@ public void testDeleteExpiredDataGivenNothingToDelete() throws Exception { client().execute(DeleteExpiredDataAction.INSTANCE, new DeleteExpiredDataAction.Request()).get(); } + @AwaitsFix(bugUrl = "https://github.com/elastic/ml-cpp/pulls/468") public void testDeleteExpiredData() throws Exception { // Index some unused state documents (more than 10K to test scrolling works) BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); @@ -137,7 +138,7 @@ public void testDeleteExpiredData() throws Exception { for (Job.Builder job : getJobs()) { assertThat(getBuckets(job.getId()).size(), is(greaterThanOrEqualTo(47))); - assertThat(getRecords(job.getId()).size(), equalTo(1)); + assertThat(getRecords(job.getId()).size(), equalTo(2)); List modelSnapshots = getModelSnapshots(job.getId()); assertThat(modelSnapshots.size(), equalTo(1)); String snapshotDocId = ModelSnapshot.documentId(modelSnapshots.get(0)); @@ -173,7 +174,7 @@ public void testDeleteExpiredData() throws Exception { startDatafeed(job.getId() + "-feed", 0, now); waitUntilJobIsClosed(job.getId()); assertThat(getBuckets(job.getId()).size(), is(greaterThanOrEqualTo(70))); - assertThat(getRecords(job.getId()).size(), equalTo(1)); + assertThat(getRecords(job.getId()).size(), equalTo(2)); List modelSnapshots = getModelSnapshots(job.getId()); assertThat(modelSnapshots.size(), equalTo(2)); } @@ -205,7 +206,7 @@ public void testDeleteExpiredData() throws Exception { // no-retention job should have kept all data assertThat(getBuckets("no-retention").size(), is(greaterThanOrEqualTo(70))); - assertThat(getRecords("no-retention").size(), equalTo(1)); + assertThat(getRecords("no-retention").size(), equalTo(2)); assertThat(getModelSnapshots("no-retention").size(), equalTo(2)); List buckets = getBuckets("results-retention"); @@ -216,11 +217,11 @@ public void testDeleteExpiredData() throws Exception { assertThat(getModelSnapshots("results-retention").size(), equalTo(2)); assertThat(getBuckets("snapshots-retention").size(), is(greaterThanOrEqualTo(70))); - assertThat(getRecords("snapshots-retention").size(), equalTo(1)); + assertThat(getRecords("snapshots-retention").size(), equalTo(2)); assertThat(getModelSnapshots("snapshots-retention").size(), equalTo(1)); assertThat(getBuckets("snapshots-retention-with-retain").size(), is(greaterThanOrEqualTo(70))); - assertThat(getRecords("snapshots-retention-with-retain").size(), equalTo(1)); + assertThat(getRecords("snapshots-retention-with-retain").size(), equalTo(2)); assertThat(getModelSnapshots("snapshots-retention-with-retain").size(), equalTo(2)); buckets = getBuckets("results-and-snapshots-retention"); From 6351e712515c91016cd61d04dae0e89215a4b78e Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 09:25:04 -0400 Subject: [PATCH 241/260] [DOCS] Add soft line breaks to maintain nested definition list (#41617) --- .../settings/notification-settings.asciidoc | 36 ++++++++++++------- 1 file changed, 23 insertions(+), 13 deletions(-) diff --git a/docs/reference/settings/notification-settings.asciidoc b/docs/reference/settings/notification-settings.asciidoc index f08d4e1ff9951..ee99c07d91dad 100644 --- a/docs/reference/settings/notification-settings.asciidoc +++ b/docs/reference/settings/notification-settings.asciidoc @@ -90,8 +90,10 @@ via email, see {xpack-ref}/actions-email.html#configuring-email-actions[Configur `xpack.notification.email.account`:: Specifies account information for sending notifications via email. You can specify the following email account attributes: - ++ +-- [[email-account-attributes]] + `profile` (<>);; The {xpack-ref}/actions-email.html#configuring-email[email profile] to use to build the MIME messages that are sent from the account. Valid values: `standard`, `gmail` and @@ -157,14 +159,17 @@ can specify the following email account attributes: `smtp.wait_on_quit` (<>);; If set to false the QUIT command is sent and the connection closed. If set to true, the QUIT command is sent and a reply is waited for. True by default. +-- `xpack.notification.email.html.sanitization.allow`:: Specifies the HTML elements that are allowed in email notifications. For more information, see {xpack-ref}/actions-email.html#email-html-sanitization[Configuring HTML Sanitization Options]. You can specify individual HTML elements and the following HTML feature groups: - ++ +-- [[html-feature-groups]] + `_tables`;; All table related elements: ``, `` and `
    `, `
    `. @@ -196,6 +201,7 @@ and the following HTML feature groups: `img:embedded`;; Only embedded images. Embedded images can only use the `cid:` URL protocol in their `src` attribute. +-- `xpack.notification.email.html.sanitization.disallow`:: Specifies the HTML elements that are NOT allowed in email notifications. @@ -216,7 +222,8 @@ via Slack, see {xpack-ref}/actions-slack.html#configuring-slack-actions[Configu `xpack.notification.slack` :: Specifies account information for sending notifications via Slack. You can specify the following Slack account attributes: - ++ +-- [[slack-account-attributes]] `secure_url` (<>);; @@ -244,7 +251,7 @@ via Slack. You can specify the following Slack account attributes: Specified as an array as defined in the https://api.slack.com/docs/attachments[ Slack attachments documentation]. - +-- [float] [[jira-notification-settings]] @@ -256,7 +263,8 @@ to create issues in Jira, see {xpack-ref}/actions-jira.html#configuring-jira-ac `xpack.notification.jira` :: Specifies account information for using notifications to create issues in Jira. You can specify the following Jira account attributes: - ++ +-- [[jira-account-attributes]] `secure_url` (<>);; @@ -272,7 +280,7 @@ issues in Jira. You can specify the following Jira account attributes: Default fields values for the issue created in Jira. See {xpack-ref}/actions-jira.html#jira-action-attributes[Jira Action Attributes] for more information. Optional. - +-- [float] [[pagerduty-notification-settings]] @@ -286,7 +294,8 @@ via PagerDuty, see {xpack-ref}/actions-pagerduty.html#configuring-pagerduty-act `xpack.notification.pagerduty`:: Specifies account information for sending notifications via PagerDuty. You can specify the following PagerDuty account attributes: - ++ +-- `name`;; A name for the PagerDuty account associated with the API key you are using to access PagerDuty. Required. @@ -299,25 +308,26 @@ via PagerDuty. You can specify the following PagerDuty account attributes: `event_defaults`;; Default values for {xpack-ref}/actions-pagerduty.html#pagerduty-event-trigger-incident-attributes[ PagerDuty event attributes]. Optional. - + + `description`:: A string that contains the default description for PagerDuty events. If no default is configured, each PagerDuty action must specify a `description`. - + + `incident_key`:: A string that contains the default incident key to use when sending PagerDuty events. - + + `client`:: A string that specifies the default monitoring client. - + + `client_url`:: The URL of the default monitoring client. - + + `event_type`:: The default event type. Valid values: `trigger`,`resolve`, `acknowledge`. - + + `attach_payload`:: Whether or not to provide the watch payload as context for the event by default. Valid values: `true`, `false`. +-- \ No newline at end of file From 217f5b936f1ad3912b0e78b51161e4af0e7dd462 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 09:58:09 -0400 Subject: [PATCH 242/260] [DOCS] Remove line break from deprecated[] macro (#41616) --- docs/reference/settings/monitoring-settings.asciidoc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/reference/settings/monitoring-settings.asciidoc b/docs/reference/settings/monitoring-settings.asciidoc index 53084d4098b84..c48b7d8764d2c 100644 --- a/docs/reference/settings/monitoring-settings.asciidoc +++ b/docs/reference/settings/monitoring-settings.asciidoc @@ -55,8 +55,7 @@ ignored. `xpack.monitoring.collection.interval` (<>):: Setting to `-1` to disable data collection is no longer supported beginning with -7.0.0. deprecated[6.3.0, Use `xpack.monitoring.collection.enabled` set to -`false` instead.] +7.0.0. deprecated[6.3.0, Use `xpack.monitoring.collection.enabled` set to `false` instead.] + Controls how often data samples are collected. Defaults to `10s`. If you modify the collection interval, set the `xpack.monitoring.min_interval_seconds` From adf67053f4f0e8ba9f84ba7a2843e4df074e3b80 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 10:19:09 -0400 Subject: [PATCH 243/260] [DOCS] Add anchors for Asciidoctor migration (#41648) --- .../diversified-sampler-aggregation.asciidoc | 2 ++ .../bucket/filters-aggregation.asciidoc | 1 + .../bucket/sampler-aggregation.asciidoc | 1 + .../significantterms-aggregation.asciidoc | 1 + .../significanttext-aggregation.asciidoc | 4 ++-- .../metrics/weighted-avg-aggregation.asciidoc | 6 ++++++ .../pipeline/avg-bucket-aggregation.asciidoc | 3 +++ .../bucket-script-aggregation.asciidoc | 4 +++- .../bucket-selector-aggregation.asciidoc | 3 ++- .../pipeline/bucket-sort-aggregation.asciidoc | 2 ++ .../cumulative-sum-aggregation.asciidoc | 2 ++ .../pipeline/derivative-aggregation.asciidoc | 2 ++ ...extended-stats-bucket-aggregation.asciidoc | 2 ++ .../pipeline/max-bucket-aggregation.asciidoc | 2 ++ .../pipeline/min-bucket-aggregation.asciidoc | 2 ++ .../pipeline/movfn-aggregation.asciidoc | 20 +++++++++++++++++++ .../percentiles-bucket-aggregation.asciidoc | 2 ++ .../pipeline/serial-diff-aggregation.asciidoc | 2 ++ .../stats-bucket-aggregation.asciidoc | 2 ++ .../pipeline/sum-bucket-aggregation.asciidoc | 2 ++ .../synonym-graph-tokenfilter.asciidoc | 1 + .../tokenfilters/synonym-tokenfilter.asciidoc | 1 + docs/reference/docs/refresh.asciidoc | 1 + docs/reference/docs/update.asciidoc | 2 ++ docs/reference/getting-started.asciidoc | 1 + docs/reference/how-to/disk-usage.asciidoc | 3 +++ docs/reference/how-to/indexing-speed.asciidoc | 1 + docs/reference/how-to/search-speed.asciidoc | 2 ++ docs/reference/ilm/start-stop-ilm.asciidoc | 3 ++- .../ilm/using-policies-rollover.asciidoc | 2 ++ docs/reference/index-modules.asciidoc | 1 + .../reference/indices/rollover-index.asciidoc | 2 ++ docs/reference/indices/split-index.asciidoc | 1 + .../ingest/processors/dissect.asciidoc | 6 ++++++ .../mapping/dynamic/templates.asciidoc | 1 + .../mapping/fields/field-names-field.asciidoc | 1 + .../mapping/fields/source-field.asciidoc | 1 + .../mapping/params/fielddata.asciidoc | 2 ++ .../mapping/params/ignore-malformed.asciidoc | 1 + .../mapping/removal_of_types.asciidoc | 1 + .../mapping/types/geo-shape.asciidoc | 6 ++++++ docs/reference/mapping/types/ip.asciidoc | 1 + docs/reference/mapping/types/nested.asciidoc | 3 ++- .../mapping/types/percolator.asciidoc | 1 + .../migration/migrate_8_0/analysis.asciidoc | 1 + .../migration/migrate_8_0/security.asciidoc | 2 ++ .../modules/scripting/expression.asciidoc | 1 + docs/reference/modules/snapshots.asciidoc | 1 + docs/reference/modules/threadpool.asciidoc | 3 +++ docs/reference/query-dsl/bool-query.asciidoc | 1 + .../reference/query-dsl/exists-query.asciidoc | 2 ++ .../query-dsl/has-child-query.asciidoc | 1 + docs/reference/query-dsl/ids-query.asciidoc | 1 + .../query-dsl/multi-match-query.asciidoc | 3 +++ .../query-dsl/script-score-query.asciidoc | 9 ++++++--- .../rollup/rollup-search-limitations.asciidoc | 1 + docs/reference/search/profile.asciidoc | 5 ++++- docs/reference/search/rank-eval.asciidoc | 1 + docs/reference/settings/ssl-settings.asciidoc | 1 - docs/reference/setup/install/deb.asciidoc | 1 + docs/reference/setup/install/docker.asciidoc | 1 + docs/reference/setup/install/rpm.asciidoc | 1 + docs/reference/setup/starting.asciidoc | 4 ++++ docs/reference/sql/endpoints/jdbc.asciidoc | 2 ++ docs/reference/sql/limitations.asciidoc | 4 ++++ docs/reference/sql/security.asciidoc | 1 + 66 files changed, 146 insertions(+), 11 deletions(-) diff --git a/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc b/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc index 07d8261d200b4..edd6c6163da85 100644 --- a/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/diversified-sampler-aggregation.asciidoc @@ -186,6 +186,7 @@ Please note that Elasticsearch will ignore the choice of execution hint if it is ==== Limitations +[[div-sampler-breadth-first-nested-agg]] ===== Cannot be nested under `breadth_first` aggregations Being a quality-based filter the diversified_sampler aggregation needs access to the relevance score produced for each document. It therefore cannot be nested under a `terms` aggregation which has the `collect_mode` switched from the default `depth_first` mode to `breadth_first` as this discards scores. @@ -194,6 +195,7 @@ In this situation an error will be thrown. ===== Limited de-dup logic. The de-duplication logic applies only at a shard level so will not apply across shards. +[[spec-syntax-geo-date-fields]] ===== No specialized syntax for geo/date fields Currently the syntax for defining the diversifying values is defined by a choice of `field` or `script` - there is no added syntactical sugar for expressing geo or date units such as "7d" (7 diff --git a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc index 94b91654f0c7f..4e83c14a18ba7 100644 --- a/docs/reference/aggregations/bucket/filters-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/filters-aggregation.asciidoc @@ -118,6 +118,7 @@ request. The response for this example would be: // TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] // TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] +[[other-bucket]] ==== `Other` Bucket The `other_bucket` parameter can be set to add a bucket to the response which will contain all documents that do diff --git a/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc b/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc index c5ac91e9d3ad8..d3a255be1d5fe 100644 --- a/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/sampler-aggregation.asciidoc @@ -155,6 +155,7 @@ The default value is 100. ==== Limitations +[[sampler-breadth-first-nested-agg]] ===== Cannot be nested under `breadth_first` aggregations Being a quality-based filter the sampler aggregation needs access to the relevance score produced for each document. It therefore cannot be nested under a `terms` aggregation which has the `collect_mode` switched from the default `depth_first` mode to `breadth_first` as this discards scores. diff --git a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc index 5766cb1a73e58..50c238edb93c5 100644 --- a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc @@ -436,6 +436,7 @@ Available parameters in the script are `_subset_size`:: Number of documents in the subset. `_superset_size`:: Number of documents in the superset. +[[sig-terms-shard-size]] ===== Size & Shard Size The `size` parameter can be set to define how many term buckets should be returned out of the overall terms list. By diff --git a/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc b/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc index 429c822d3623d..3a9221c3c4372 100644 --- a/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significanttext-aggregation.asciidoc @@ -92,7 +92,7 @@ It only occurs 5 times in our index as a whole (see the `bg_count`) and yet 4 of were lucky enough to appear in our 100 document sample of "bird flu" results. That suggests a significant word and one which the user can potentially add to their search. - +[[filter-duplicate-text-noisy-data]] ==== Dealing with noisy data using `filter_duplicate_text` Free-text fields often contain a mix of original content and mechanical copies of text (cut-and-paste biographies, email reply chains, retweets, boilerplate headers/footers, page navigation menus, sidebar news links, copyright notices, standard disclaimers, addresses). @@ -353,7 +353,7 @@ However, the `size` and `shard size` settings covered in the next section provid This aggregation supports the same scoring heuristics (JLH, mutual_information, gnd, chi_square etc) as the <> aggregation - +[[sig-text-shard-size]] ===== Size & Shard Size The `size` parameter can be set to define how many term buckets should be returned out of the overall terms list. By diff --git a/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc b/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc index 252728a6db367..70804a7904f1f 100644 --- a/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/weighted-avg-aggregation.asciidoc @@ -12,7 +12,9 @@ As a formula, a weighted average is the `∑(value * weight) / ∑(weight)` A regular average can be thought of as a weighted average where every value has an implicit weight of `1`. +[[weighted-avg-params]] .`weighted_avg` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`value` | The configuration for the field or script that provides the values |Required | @@ -23,7 +25,9 @@ A regular average can be thought of as a weighted average where every value has The `value` and `weight` objects have per-field specific configuration: +[[value-params]] .`value` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`field` | The field that values should be extracted from |Required | @@ -31,7 +35,9 @@ The `value` and `weight` objects have per-field specific configuration: |`script` | A script which provides the values for the document. This is mutually exclusive with `field` |Optional |=== +[[weight-params]] .`weight` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`field` | The field that weights should be extracted from |Required | diff --git a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc index 274efcbce62fc..ea150b4ab6d8b 100644 --- a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc @@ -4,6 +4,7 @@ A sibling pipeline aggregation which calculates the (mean) average value of a specified metric in a sibling aggregation. The specified metric must be numeric and the sibling aggregation must be a multi-bucket aggregation. +[[avg-bucket-agg-syntax]] ==== Syntax An `avg_bucket` aggregation looks like this in isolation: @@ -18,7 +19,9 @@ An `avg_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[avg-bucket-params]] .`avg_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the average for (see <> for more diff --git a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc index 1825b37f0c734..7dc99ba7719cd 100644 --- a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc @@ -4,6 +4,7 @@ A parent pipeline aggregation which executes a script which can perform per bucket computations on specified metrics in the parent multi-bucket aggregation. The specified metric must be numeric and the script must return a numeric value. +[[bucket-script-agg-syntax]] ==== Syntax A `bucket_script` aggregation looks like this in isolation: @@ -24,8 +25,9 @@ A `bucket_script` aggregation looks like this in isolation: <1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to the metrics to use for that variable. - +[[bucket-script-params]] .`bucket_script` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> diff --git a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc index 4cc532c99c5d2..7ec19174a06e3 100644 --- a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc @@ -29,8 +29,9 @@ A `bucket_selector` aggregation looks like this in isolation: <1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to the metrics to use for that variable. - +[[bucket-selector-params]] .`bucket_selector` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> diff --git a/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc index 633175dbf2825..a136a6ee4d578 100644 --- a/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-sort-aggregation.asciidoc @@ -33,7 +33,9 @@ A `bucket_sort` aggregation looks like this in isolation: <1> Here, `sort_field_1` is the bucket path to the variable to be used as the primary sort and its order is ascending. +[[bucket-sort-params]] .`bucket_sort` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`sort` |The list of fields to sort on. See <> for more details. |Optional | diff --git a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc index 748946f8bd671..a6dff7fa84651 100644 --- a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc @@ -19,7 +19,9 @@ A `cumulative_sum` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[cumulative-sum-params]] .`cumulative_sum` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the cumulative sum for (see <> for more diff --git a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc index 8479d1f45aea1..f40ace7432d57 100644 --- a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc @@ -17,7 +17,9 @@ A `derivative` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[derivative-params]] .`derivative` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the derivative for (see <> for more diff --git a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc index eeef705a6468d..c35223885fce0 100644 --- a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc @@ -20,7 +20,9 @@ A `extended_stats_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[extended-stats-bucket-params]] .`extended_stats_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to calculate stats for (see <> for more diff --git a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc index 8881315f50ab4..53a3aaa28f7e5 100644 --- a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc @@ -19,7 +19,9 @@ A `max_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[max-bucket-params]] .`max_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the maximum for (see <> for more diff --git a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc index ad6aaa28c90dd..620cf02c714fa 100644 --- a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc @@ -19,7 +19,9 @@ A `min_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[min-bucket-params]] .`min_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the minimum for (see <> for more diff --git a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc index 5745527bddd6f..f6fb25c76f662 100644 --- a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc @@ -24,7 +24,9 @@ A `moving_fn` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[moving-avg-params]] .`moving_avg` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |Path to the metric of interest (see <> for more details |Required | @@ -188,7 +190,9 @@ The functions are available from the `MovingFunctions` namespace. E.g. `MovingF This function accepts a collection of doubles and returns the maximum value in that window. `null` and `NaN` values are ignored; the maximum is only calculated over the real values. If the window is empty, or all values are `null`/`NaN`, `NaN` is returned as the result. +[[max-params]] .`max(double[] values)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the maximum @@ -229,7 +233,9 @@ POST /_search This function accepts a collection of doubles and returns the minimum value in that window. `null` and `NaN` values are ignored; the minimum is only calculated over the real values. If the window is empty, or all values are `null`/`NaN`, `NaN` is returned as the result. +[[min-params]] .`min(double[] values)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the minimum @@ -270,7 +276,9 @@ POST /_search This function accepts a collection of doubles and returns the sum of the values in that window. `null` and `NaN` values are ignored; the sum is only calculated over the real values. If the window is empty, or all values are `null`/`NaN`, `0.0` is returned as the result. +[[sum-params]] .`sum(double[] values)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the sum of @@ -312,7 +320,9 @@ This function accepts a collection of doubles and average, then returns the stan `null` and `NaN` values are ignored; the sum is only calculated over the real values. If the window is empty, or all values are `null`/`NaN`, `0.0` is returned as the result. +[[stddev-params]] .`stdDev(double[] values)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the standard deviation of @@ -363,7 +373,9 @@ the values from a `simple` moving average tend to "lag" behind the real data. `null`/`NaN`, `NaN` is returned as the result. This means that the count used in the average calculation is count of non-`null`,non-`NaN` values. +[[unweightedavg-params]] .`unweightedAvg(double[] values)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the sum of @@ -407,7 +419,9 @@ the "lag" behind the data's mean, since older points have less influence. If the window is empty, or all values are `null`/`NaN`, `NaN` is returned as the result. +[[linearweightedavg-params]] .`linearWeightedAvg(double[] values)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the sum of @@ -456,7 +470,9 @@ moving average. This tends to make the moving average track the data more close `null`/`NaN`, `NaN` is returned as the result. This means that the count used in the average calculation is count of non-`null`,non-`NaN` values. +[[ewma-params]] .`ewma(double[] values, double alpha)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the sum of @@ -511,7 +527,9 @@ Values are produced by multiplying the level and trend components. `null`/`NaN`, `NaN` is returned as the result. This means that the count used in the average calculation is count of non-`null`,non-`NaN` values. +[[holt-params]] .`holt(double[] values, double alpha)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the sum of @@ -572,7 +590,9 @@ for future enhancements. `null`/`NaN`, `NaN` is returned as the result. This means that the count used in the average calculation is count of non-`null`,non-`NaN` values. +[[holtwinters-params]] .`holtWinters(double[] values, double alpha)` Parameters +[options="header"] |=== |Parameter Name |Description |`values` |The window of values to find the sum of diff --git a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc index 032b6ef4e419d..456a4046c0624 100644 --- a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc @@ -18,7 +18,9 @@ A `percentiles_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[percentiles-bucket-params]] .`percentiles_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the percentiles for (see <> for more diff --git a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc index 70aea68f88c34..1506e39685845 100644 --- a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc @@ -46,7 +46,9 @@ A `serial_diff` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[serial-diff-params]] .`serial_diff` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |Path to the metric of interest (see <> for more details |Required | diff --git a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc index b9c52ae981f75..2c9f585ebea3a 100644 --- a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc @@ -18,7 +18,9 @@ A `stats_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[stats-bucket-params]] .`stats_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to calculate stats for (see <> for more diff --git a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc index b39cf472323c2..83e0e32135040 100644 --- a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc @@ -18,7 +18,9 @@ A `sum_bucket` aggregation looks like this in isolation: -------------------------------------------------- // NOTCONSOLE +[[sum-bucket-params]] .`sum_bucket` Parameters +[options="header"] |=== |Parameter Name |Description |Required |Default Value |`buckets_path` |The path to the buckets we wish to find the sum for (see <> for more diff --git a/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc index 785eb77f2c65e..2285c6f6e8989 100644 --- a/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc @@ -93,6 +93,7 @@ set to `false` no mapping would get added as when `expand=false` the target mapp stop word. [float] +[[synonym-graph-tokenizer-ignore_case-deprecated]] ==== `tokenizer` and `ignore_case` are deprecated The `tokenizer` parameter controls the tokenizers that will be used to diff --git a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc index 87c99f6f38683..1107236194655 100644 --- a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc @@ -83,6 +83,7 @@ stop word. [float] +[[synonym-tokenizer-ignore_case-deprecated]] ==== `tokenizer` and `ignore_case` are deprecated The `tokenizer` parameter controls the tokenizers that will be used to diff --git a/docs/reference/docs/refresh.asciidoc b/docs/reference/docs/refresh.asciidoc index e5051497ecda3..121e0c494d828 100644 --- a/docs/reference/docs/refresh.asciidoc +++ b/docs/reference/docs/refresh.asciidoc @@ -63,6 +63,7 @@ general, if you have a running system you don't wish to disturb then `refresh=wait_for` is a smaller modification. [float] +[[refresh_wait_for-force-refresh]] === `refresh=wait_for` Can Force a Refresh If a `refresh=wait_for` request comes in when there are already diff --git a/docs/reference/docs/update.asciidoc b/docs/reference/docs/update.asciidoc index 66e5783e7eb9b..00cd66232190f 100644 --- a/docs/reference/docs/update.asciidoc +++ b/docs/reference/docs/update.asciidoc @@ -243,6 +243,7 @@ POST test/_update/1 // TEST[continued] [float] +[[scripted_upsert]] ==== `scripted_upsert` If you would like your script to run regardless of whether the document exists @@ -272,6 +273,7 @@ POST sessions/_update/dh3sgudg8gsrgl // TEST[continued] [float] +[[doc_as_upsert]] ==== `doc_as_upsert` Instead of sending a partial `doc` plus an `upsert` doc, setting diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index 6e112559f8771..b81d2b284371d 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -505,6 +505,7 @@ This REST access pattern is so pervasive throughout all the API commands that if Elasticsearch provides data manipulation and search capabilities in near real time. By default, you can expect a one second delay (refresh interval) from the time you index/update/delete your data until the time that it appears in your search results. This is an important distinction from other platforms like SQL wherein data is immediately available after a transaction is completed. [float] +[[indexing-replacing-documents]] === Indexing/Replacing Documents We've previously seen how we can index a single document. Let's recall that command again: diff --git a/docs/reference/how-to/disk-usage.asciidoc b/docs/reference/how-to/disk-usage.asciidoc index 713e03d188ad9..a475122b99ec2 100644 --- a/docs/reference/how-to/disk-usage.asciidoc +++ b/docs/reference/how-to/disk-usage.asciidoc @@ -91,6 +91,7 @@ PUT index // CONSOLE [float] +[[default-dynamic-string-mapping]] === Don't use default dynamic string mappings The default <> will index string fields @@ -133,11 +134,13 @@ Larger shards are going to be more efficient at storing data. To increase the si Keep in mind that large shard sizes come with drawbacks, such as long full recovery times. [float] +[[disable-source]] === Disable `_source` The <> field stores the original JSON body of the document. If you don’t need access to it you can disable it. However, APIs that needs access to `_source` such as update and reindex won’t work. [float] +[[best-compression]] === Use `best_compression` The `_source` and stored fields can easily take a non negligible amount of disk diff --git a/docs/reference/how-to/indexing-speed.asciidoc b/docs/reference/how-to/indexing-speed.asciidoc index 1c8b989779d6c..147d3a557c830 100644 --- a/docs/reference/how-to/indexing-speed.asciidoc +++ b/docs/reference/how-to/indexing-speed.asciidoc @@ -17,6 +17,7 @@ it is advisable to avoid going beyond a couple tens of megabytes per request even if larger requests seem to perform better. [float] +[[multiple-workers-threads]] === Use multiple workers/threads to send data to Elasticsearch A single thread sending bulk requests is unlikely to be able to max out the diff --git a/docs/reference/how-to/search-speed.asciidoc b/docs/reference/how-to/search-speed.asciidoc index 017dd74afa084..e09b18df20c5d 100644 --- a/docs/reference/how-to/search-speed.asciidoc +++ b/docs/reference/how-to/search-speed.asciidoc @@ -161,6 +161,7 @@ GET index/_search // TEST[continued] [float] +[[map-ids-as-keyword]] === Consider mapping identifiers as `keyword` The fact that some data is numeric does not mean it should always be mapped as a @@ -354,6 +355,7 @@ conjunctions faster at the cost of slightly slower indexing. Read more about it in the <>. [float] +[[preference-cache-optimization]] === Use `preference` to optimize cache utilization There are multiple caches that can help with search performance, such as the diff --git a/docs/reference/ilm/start-stop-ilm.asciidoc b/docs/reference/ilm/start-stop-ilm.asciidoc index e5366f028a9c7..22ca0ae48fd98 100644 --- a/docs/reference/ilm/start-stop-ilm.asciidoc +++ b/docs/reference/ilm/start-stop-ilm.asciidoc @@ -68,8 +68,9 @@ If the request does not encounter errors, you receive the following result: The operating modes of ILM: - +[[ilm-operating-modes]] .ILM Operating Modes +[options="header"] |=== |Name |Description |RUNNING |Normal operation where all policies are executed as normal diff --git a/docs/reference/ilm/using-policies-rollover.asciidoc b/docs/reference/ilm/using-policies-rollover.asciidoc index dfb89eb26d121..11be1bed85a16 100644 --- a/docs/reference/ilm/using-policies-rollover.asciidoc +++ b/docs/reference/ilm/using-policies-rollover.asciidoc @@ -28,7 +28,9 @@ new index. The rollover action takes the following parameters: +[[rollover-action-params]] .`rollover` Action Parameters +[options="header"] |=== |Name |Description |max_size |The maximum estimated size the primary shard of the index is allowed diff --git a/docs/reference/index-modules.asciidoc b/docs/reference/index-modules.asciidoc index 96ccc9ab6a8a4..ef7c40b11d048 100644 --- a/docs/reference/index-modules.asciidoc +++ b/docs/reference/index-modules.asciidoc @@ -277,6 +277,7 @@ Other index settings are available in index modules: Control over the transaction log and background flush operations. [float] +[[x-pack-index-settings]] === [xpack]#{xpack} index settings# <>:: diff --git a/docs/reference/indices/rollover-index.asciidoc b/docs/reference/indices/rollover-index.asciidoc index 7f6c3d0452a2d..b5037d0a94233 100644 --- a/docs/reference/indices/rollover-index.asciidoc +++ b/docs/reference/indices/rollover-index.asciidoc @@ -26,7 +26,9 @@ will now have the rollover alias pointing to it as the write index with `is_writ The available conditions are: +[[index-rollover-conditions]] .`conditions` parameters +[options="header"] |=== | Name | Description | max_age | The maximum age of the index diff --git a/docs/reference/indices/split-index.asciidoc b/docs/reference/indices/split-index.asciidoc index d79c0b0e9e250..4b9aaa09a94b6 100644 --- a/docs/reference/indices/split-index.asciidoc +++ b/docs/reference/indices/split-index.asciidoc @@ -50,6 +50,7 @@ Splitting works as follows: had just been re-opened. [float] +[[incremental-resharding]] === Why doesn't Elasticsearch support incremental resharding? Going from `N` shards to `N+1` shards, aka. incremental resharding, is indeed a diff --git a/docs/reference/ingest/processors/dissect.asciidoc b/docs/reference/ingest/processors/dissect.asciidoc index aa54cb4411f3e..f2a7a42b9d668 100644 --- a/docs/reference/ingest/processors/dissect.asciidoc +++ b/docs/reference/ingest/processors/dissect.asciidoc @@ -78,12 +78,14 @@ include::common-options.asciidoc[] } -------------------------------------------------- // NOTCONSOLE + [[dissect-key-modifiers]] ==== Dissect key modifiers Key modifiers can change the default behavior for dissection. Key modifiers may be found on the left or right of the `%{keyname}` always inside the `%{` and `}`. For example `%{+keyname ->}` has the append and right padding modifiers. +[[dissect-key-modifiers-table]] .Dissect Key Modifiers [options="header"] |====== @@ -132,6 +134,7 @@ Right padding modifier with empty key example * level = WARN |====== +[[append-modifier]] ===== Append modifier (`+`) [[dissect-modifier-append-key]] Dissect supports appending two or more results together for the output. @@ -146,6 +149,7 @@ Append modifier example * name = john jacob jingleheimer schmidt |====== +[[append-order-modifier]] ===== Append with order modifier (`+` and `/n`) [[dissect-modifier-append-key-with-order]] Dissect supports appending two or more results together for the output. @@ -160,6 +164,7 @@ Append with order modifier example * name = schmidt,john,jingleheimer,jacob |====== +[[named-skip-key]] ===== Named skip key (`?`) [[dissect-modifier-named-skip-key]] Dissect supports ignoring matches in the final result. This can be done with an empty key `%{}`, but for readability @@ -174,6 +179,7 @@ Named skip key modifier example * @timestamp = 30/Apr/1998:22:00:52 +0000 |====== +[[reference-keys]] ===== Reference keys (`*` and `&`) [[dissect-modifier-reference-keys]] Dissect support using parsed values as the key/value pairings for the structured content. Imagine a system that diff --git a/docs/reference/mapping/dynamic/templates.asciidoc b/docs/reference/mapping/dynamic/templates.asciidoc index 8598eab412e79..a55fd98d91e67 100644 --- a/docs/reference/mapping/dynamic/templates.asciidoc +++ b/docs/reference/mapping/dynamic/templates.asciidoc @@ -286,6 +286,7 @@ PUT my_index -------------------------------------------------- // CONSOLE +[[text-only-mappings-strings]] ===== `text`-only mappings for strings On the contrary to the previous example, if the only thing that you care about diff --git a/docs/reference/mapping/fields/field-names-field.asciidoc b/docs/reference/mapping/fields/field-names-field.asciidoc index c455c55f5ea7f..1ae4ab4c8fc89 100644 --- a/docs/reference/mapping/fields/field-names-field.asciidoc +++ b/docs/reference/mapping/fields/field-names-field.asciidoc @@ -11,6 +11,7 @@ Now the `_field_names` field only indexes the names of fields that have or `norm` enabled the <> query will still be available but will not use the `_field_names` field. +[[disable-field-names]] ==== Disabling `_field_names` Disabling `_field_names` is often not necessary because it no longer diff --git a/docs/reference/mapping/fields/source-field.asciidoc b/docs/reference/mapping/fields/source-field.asciidoc index 757fc0fa5b662..c9fd2cf186909 100644 --- a/docs/reference/mapping/fields/source-field.asciidoc +++ b/docs/reference/mapping/fields/source-field.asciidoc @@ -6,6 +6,7 @@ at index time. The `_source` field itself is not indexed (and thus is not searchable), but it is stored so that it can be returned when executing _fetch_ requests, like <> or <>. +[[disable-source-field]] ==== Disabling the `_source` field Though very handy to have around, the source field does incur storage overhead diff --git a/docs/reference/mapping/params/fielddata.asciidoc b/docs/reference/mapping/params/fielddata.asciidoc index 42f02b7ee28ea..8e46dd0370245 100644 --- a/docs/reference/mapping/params/fielddata.asciidoc +++ b/docs/reference/mapping/params/fielddata.asciidoc @@ -19,6 +19,7 @@ reading the entire inverted index for each segment from disk, inverting the term ↔︎ document relationship, and storing the result in memory, in the JVM heap. +[[fielddata-disabled-text-fields]] ==== Fielddata is disabled on `text` fields by default Fielddata can consume a *lot* of heap space, especially when loading high @@ -75,6 +76,7 @@ PUT my_index <1> Use the `my_field` field for searches. <2> Use the `my_field.keyword` field for aggregations, sorting, or in scripts. +[[enable-fielddata-text-fields]] ==== Enabling fielddata on `text` fields You can enable fielddata on an existing `text` field using the diff --git a/docs/reference/mapping/params/ignore-malformed.asciidoc b/docs/reference/mapping/params/ignore-malformed.asciidoc index f8f9a4591f947..8c91bb48ee7cd 100644 --- a/docs/reference/mapping/params/ignore-malformed.asciidoc +++ b/docs/reference/mapping/params/ignore-malformed.asciidoc @@ -91,6 +91,7 @@ become meaningless. Elasticsearch makes it easy to check how many documents have malformed fields by using `exist` or `term` queries on the special <> field. +[[json-object-limits]] ==== Limits for JSON Objects You can't use `ignore_malformed` with the following datatypes: diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index 8e78f3f140596..4d26f30ada256 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -216,6 +216,7 @@ GET twitter/_search <1> The explicit `type` field takes the place of the implicit `_type` field. [float] +[[parent-child-mapping-types]] ==== Parent/Child without mapping types Previously, a parent-child relationship was represented by making one mapping diff --git a/docs/reference/mapping/types/geo-shape.asciidoc b/docs/reference/mapping/types/geo-shape.asciidoc index a46b8a3f8a87c..1cf85e305a95d 100644 --- a/docs/reference/mapping/types/geo-shape.asciidoc +++ b/docs/reference/mapping/types/geo-shape.asciidoc @@ -323,6 +323,7 @@ POST /example/_doc // CONSOLE [float] +[[linestring]] ===== http://geojson.org/geojson-spec.html#id3[LineString] A `linestring` defined by an array of two or more positions. By @@ -357,6 +358,7 @@ The above `linestring` would draw a straight line starting at the White House to the US Capitol Building. [float] +[[polygon]] ===== http://www.geojson.org/geojson-spec.html#id4[Polygon] A polygon is defined by a list of a list of points. The first and last @@ -473,6 +475,7 @@ POST /example/_doc // CONSOLE [float] +[[multipoint]] ===== http://www.geojson.org/geojson-spec.html#id5[MultiPoint] The following is an example of a list of geojson points: @@ -503,6 +506,7 @@ POST /example/_doc // CONSOLE [float] +[[multilinestring]] ===== http://www.geojson.org/geojson-spec.html#id6[MultiLineString] The following is an example of a list of geojson linestrings: @@ -535,6 +539,7 @@ POST /example/_doc // CONSOLE [float] +[[multipolygon]] ===== http://www.geojson.org/geojson-spec.html#id7[MultiPolygon] The following is an example of a list of geojson polygons (second polygon contains a hole): @@ -567,6 +572,7 @@ POST /example/_doc // CONSOLE [float] +[[geometry_collection]] ===== http://geojson.org/geojson-spec.html#geometrycollection[Geometry Collection] The following is an example of a collection of geojson geometry objects: diff --git a/docs/reference/mapping/types/ip.asciidoc b/docs/reference/mapping/types/ip.asciidoc index 7f3f5f57d7077..46ca444c9558d 100644 --- a/docs/reference/mapping/types/ip.asciidoc +++ b/docs/reference/mapping/types/ip.asciidoc @@ -69,6 +69,7 @@ The following parameters are accepted by `ip` fields: the <> field. Accepts `true` or `false` (default). +[[query-ip-fields]] ==== Querying `ip` fields The most common way to query ip addresses is to use the diff --git a/docs/reference/mapping/types/nested.asciidoc b/docs/reference/mapping/types/nested.asciidoc index f420e680c8590..5969dcfd6956b 100644 --- a/docs/reference/mapping/types/nested.asciidoc +++ b/docs/reference/mapping/types/nested.asciidoc @@ -65,6 +65,7 @@ GET my_index/_search // CONSOLE // TEST[continued] +[[nested-fields-array-objects]] ==== Using `nested` fields for arrays of objects If you need to index arrays of objects and to maintain the independence of @@ -200,7 +201,7 @@ document is indexed as a separate document. To safeguard against ill-defined map the number of nested fields that can be defined per index has been limited to 50. See <>. - +[[limit-nested-json-objects-number]] ==== Limiting the number of `nested` json objects Indexing a document with an array of 100 objects within a nested field, will actually create 101 documents, as each nested object will be indexed as a separate document. diff --git a/docs/reference/mapping/types/percolator.asciidoc b/docs/reference/mapping/types/percolator.asciidoc index ce840e1379de3..00270fff38538 100644 --- a/docs/reference/mapping/types/percolator.asciidoc +++ b/docs/reference/mapping/types/percolator.asciidoc @@ -720,6 +720,7 @@ fail. ==== Limitations [float] +[[parent-child]] ===== Parent/child Because the `percolate` query is processing one document at a time, it doesn't support queries and filters that run diff --git a/docs/reference/migration/migrate_8_0/analysis.asciidoc b/docs/reference/migration/migrate_8_0/analysis.asciidoc index f9a4bb76fd841..dd4b4ae73e920 100644 --- a/docs/reference/migration/migrate_8_0/analysis.asciidoc +++ b/docs/reference/migration/migrate_8_0/analysis.asciidoc @@ -10,6 +10,7 @@ // end::notable-breaking-changes[] [float] +[[ngram-edgengram-filter-names-removed]] ==== The `nGram` and `edgeNGram` token filter names have been removed The `nGram` and `edgeNGram` token filter names that have been deprecated since diff --git a/docs/reference/migration/migrate_8_0/security.asciidoc b/docs/reference/migration/migrate_8_0/security.asciidoc index e4caca1e8fe43..fcc0a5b22168a 100644 --- a/docs/reference/migration/migrate_8_0/security.asciidoc +++ b/docs/reference/migration/migrate_8_0/security.asciidoc @@ -10,12 +10,14 @@ // end::notable-breaking-changes[] [float] +[[accept-default-password-removed]] ==== The `accept_default_password` setting has been removed The `xpack.security.authc.accept_default_password` setting has not had any affect since the 6.0 release of {es}. It has been removed and cannot be used. [float] +[[roles-index-cache-removed]] ==== The `roles.index.cache.*` settings have been removed The `xpack.security.authz.store.roles.index.cache.max_size` and diff --git a/docs/reference/modules/scripting/expression.asciidoc b/docs/reference/modules/scripting/expression.asciidoc index fb7739261ef4a..de3704a899e52 100644 --- a/docs/reference/modules/scripting/expression.asciidoc +++ b/docs/reference/modules/scripting/expression.asciidoc @@ -112,6 +112,7 @@ The following example shows the difference in years between the `date` fields da `doc['date1'].date.year - doc['date0'].date.year` [float] +[[geo-point-field-api]] === `geo_point` field API [cols="<,<",options="header",] |======================================================================= diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index b206b72fb31a9..ec7916d5a3445 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -694,6 +694,7 @@ GET /_snapshot/my_backup/snapshot_1,snapshot_2/_status // TEST[continued] [float] +[[monitor-snapshot-restore-progress]] === Monitoring snapshot/restore progress There are several ways to monitor the progress of the snapshot and restores processes while they are running. Both diff --git a/docs/reference/modules/threadpool.asciidoc b/docs/reference/modules/threadpool.asciidoc index 38f8d40e67f23..d0f68e37730b8 100644 --- a/docs/reference/modules/threadpool.asciidoc +++ b/docs/reference/modules/threadpool.asciidoc @@ -69,6 +69,7 @@ thread_pool: The following are the types of thread pools and their respective parameters: [float] +[[fixed]] ==== `fixed` The `fixed` thread pool holds a fixed size of threads to handle the @@ -92,6 +93,7 @@ thread_pool: -------------------------------------------------- [float] +[[fixed-auto-queue-size]] ==== `fixed_auto_queue_size` experimental[] @@ -138,6 +140,7 @@ thread_pool: -------------------------------------------------- [float] +[[scaling]] ==== `scaling` The `scaling` thread pool holds a dynamic number of threads. This diff --git a/docs/reference/query-dsl/bool-query.asciidoc b/docs/reference/query-dsl/bool-query.asciidoc index d4b5919454836..49dc4e2364efb 100644 --- a/docs/reference/query-dsl/bool-query.asciidoc +++ b/docs/reference/query-dsl/bool-query.asciidoc @@ -58,6 +58,7 @@ POST _search -------------------------------------------------- // CONSOLE +[[score-bool-filter]] ==== Scoring with `bool.filter` Queries specified under the `filter` element have no effect on scoring -- diff --git a/docs/reference/query-dsl/exists-query.asciidoc b/docs/reference/query-dsl/exists-query.asciidoc index b2e27c76e494d..dc624c19039ef 100644 --- a/docs/reference/query-dsl/exists-query.asciidoc +++ b/docs/reference/query-dsl/exists-query.asciidoc @@ -44,6 +44,7 @@ These documents would *not* match the above query: <3> The `user` field is missing completely. [float] +[[null-value-mapping]] ==== `null_value` mapping If the field mapping includes the <> setting @@ -86,6 +87,7 @@ no values in the `user` field and thus would not match the `exists` filter: -------------------------------------------------- // NOTCONSOLE +[[missing-query]] ==== `missing` query There isn't a `missing` query. Instead use the `exists` query inside a diff --git a/docs/reference/query-dsl/has-child-query.asciidoc b/docs/reference/query-dsl/has-child-query.asciidoc index ca23afb0fae26..b75c06c99c566 100644 --- a/docs/reference/query-dsl/has-child-query.asciidoc +++ b/docs/reference/query-dsl/has-child-query.asciidoc @@ -62,6 +62,7 @@ GET /_search // CONSOLE [float] +[[min-max-children]] ==== Min/Max Children The `has_child` query allows you to specify that a minimum and/or maximum diff --git a/docs/reference/query-dsl/ids-query.asciidoc b/docs/reference/query-dsl/ids-query.asciidoc index 6056c24702c91..70554e1acbf1b 100644 --- a/docs/reference/query-dsl/ids-query.asciidoc +++ b/docs/reference/query-dsl/ids-query.asciidoc @@ -18,6 +18,7 @@ GET /_search -------------------------------------------------- // CONSOLE +[[ids-query-top-level-parameters]] ==== Top-level parameters for `ids` [cols="v,v",options="header"] diff --git a/docs/reference/query-dsl/multi-match-query.asciidoc b/docs/reference/query-dsl/multi-match-query.asciidoc index 703abd42ee621..9f574ed814d3c 100644 --- a/docs/reference/query-dsl/multi-match-query.asciidoc +++ b/docs/reference/query-dsl/multi-match-query.asciidoc @@ -21,6 +21,7 @@ GET /_search <2> The fields to be queried. [float] +[[field-boost]] ==== `fields` and per-field boosting Fields can be specified with wildcards, eg: @@ -391,6 +392,7 @@ Also, accepts `analyzer`, `boost`, `operator`, `minimum_should_match`, `lenient`, `zero_terms_query` and `cutoff_frequency`, as explained in <>. +[[cross-field-analysis]] ===== `cross_field` and analysis The `cross_field` type can only work in term-centric mode on fields that have @@ -499,6 +501,7 @@ which will be executed as: blended("will", fields: [first, first.edge, last.edge, last]) blended("smith", fields: [first, first.edge, last.edge, last]) +[[tie-breaker]] ===== `tie_breaker` By default, each per-term `blended` query will use the best score returned by diff --git a/docs/reference/query-dsl/script-score-query.asciidoc b/docs/reference/query-dsl/script-score-query.asciidoc index e6418d408d7f2..42d4a7b1517e3 100644 --- a/docs/reference/query-dsl/script-score-query.asciidoc +++ b/docs/reference/query-dsl/script-score-query.asciidoc @@ -233,7 +233,7 @@ field, whose only drawback is that scores will change if the document is updated since update operations also update the value of the `_seq_no` field. -[[decay-functions]] +[[decay-functions-numeric-fields]] ===== Decay functions for numeric fields You can read more about decay functions {ref}/query-dsl-function-score-query.html#function-decay[here]. @@ -323,10 +323,12 @@ Script Score Query will be a substitute for it. Here we describe how Function Score Query's functions can be equivalently implemented in Script Score Query: +[[script-score]] ===== `script_score` What you used in `script_score` of the Function Score query, you can copy into the Script Score query. No changes here. +[[weight]] ===== `weight` `weight` function can be implemented in the Script Score query through the following script: @@ -342,12 +344,13 @@ the following script: -------------------------------------------------- // NOTCONSOLE +[[random-score]] ===== `random_score` Use `randomScore` function as described in <>. - +[[field-value-factor]] ===== `field_value_factor` `field_value_factor` function can be easily implemented through script: @@ -397,7 +400,7 @@ through a script: | `reciprocal` | `1.0 / doc['f'].value` |======================================================================= - +[[decay-functions]] ===== `decay functions` Script Score query has equivalent <> that can be used in script. diff --git a/docs/reference/rollup/rollup-search-limitations.asciidoc b/docs/reference/rollup/rollup-search-limitations.asciidoc index c8a736450bde0..81cb162bd3c11 100644 --- a/docs/reference/rollup/rollup-search-limitations.asciidoc +++ b/docs/reference/rollup/rollup-search-limitations.asciidoc @@ -32,6 +32,7 @@ To help simplify the problem, we have limited search to just one rollup index at may be able to open this up to multiple rollup jobs. [float] +[[aggregate-stored-only]] === Can only aggregate what's been stored A perhaps obvious limitation, but rollups can only aggregate on data that has been stored in the rollups. If you don't configure the diff --git a/docs/reference/search/profile.asciidoc b/docs/reference/search/profile.asciidoc index 9a6779defab39..e5ba1b851cdc8 100644 --- a/docs/reference/search/profile.asciidoc +++ b/docs/reference/search/profile.asciidoc @@ -243,6 +243,7 @@ sufficient to see that a particular component of a query is slow, and not necess the `advance` phase of that query is the cause, for example. ======================================= +[[query-section]] ==== `query` Section The `query` section contains detailed timing of the query tree executed by Lucene on a particular shard. @@ -399,6 +400,7 @@ The meaning of the stats are as follows: means the `nextDoc()` method was called on two different documents. This can be used to help judge how selective queries are, by comparing counts between different query components. +[[collectors-section]] ==== `collectors` Section The Collectors portion of the response shows high-level execution details. Lucene works by defining a "Collector" @@ -485,7 +487,7 @@ For reference, the various collector reasons are: match_all query (which you will see added to the Query section) to collect your entire dataset - +[[rewrite-section]] ==== `rewrite` Section All queries in Lucene undergo a "rewriting" process. A query (and its sub-queries) may be rewritten one or @@ -694,6 +696,7 @@ Hopefully this will be fixed in future iterations, but it is a tricky problem to [[search-profile-aggregations]] === Profiling Aggregations +[[agg-section]] ==== `aggregations` Section diff --git a/docs/reference/search/rank-eval.asciidoc b/docs/reference/search/rank-eval.asciidoc index 29176f15f8cd4..b25048fec90e6 100644 --- a/docs/reference/search/rank-eval.asciidoc +++ b/docs/reference/search/rank-eval.asciidoc @@ -136,6 +136,7 @@ The `metric` section determines which of the available evaluation metrics is goi Currently, the following metrics are supported: [float] +[[k-precision]] ==== Precision at K (P@k) This metric measures the number of relevant results in the top k search results. Its a form of the well known https://en.wikipedia.org/wiki/Information_retrieval#Precision[Precision] metric that only looks at the top k documents. It is the fraction of relevant documents in those first k diff --git a/docs/reference/settings/ssl-settings.asciidoc b/docs/reference/settings/ssl-settings.asciidoc index a4422b8fb2d3c..f392d0c2fb816 100644 --- a/docs/reference/settings/ssl-settings.asciidoc +++ b/docs/reference/settings/ssl-settings.asciidoc @@ -39,7 +39,6 @@ endif::verifies[] Supported cipher suites can be found in Oracle's http://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html[ Java Cryptography Architecture documentation]. Defaults to ``. - ===== {component} TLS/SSL Key and Trusted Certificate Settings The following settings are used to specify a private key, certificate, and the diff --git a/docs/reference/setup/install/deb.asciidoc b/docs/reference/setup/install/deb.asciidoc index 192737581bd57..529c6b12ce520 100644 --- a/docs/reference/setup/install/deb.asciidoc +++ b/docs/reference/setup/install/deb.asciidoc @@ -173,6 +173,7 @@ include::xpack-indices.asciidoc[] endif::include-xpack[] +[[deb-sysv-init-vs-systemd]] ==== SysV `init` vs `systemd` include::init-systemd.asciidoc[] diff --git a/docs/reference/setup/install/docker.asciidoc b/docs/reference/setup/install/docker.asciidoc index 76112f220dc3a..172c7c1f17c2f 100644 --- a/docs/reference/setup/install/docker.asciidoc +++ b/docs/reference/setup/install/docker.asciidoc @@ -287,6 +287,7 @@ comfortable with them adding the `--batch` flag to the plugin install command. See {plugins}/_other_command_line_parameters.html[Plugin Management documentation] for more details. +[[override-image-default]] ===== D. Override the image's default https://docs.docker.com/engine/reference/run/#cmd-default-command-or-options[CMD] Options can be passed as command-line options to the {es} process by diff --git a/docs/reference/setup/install/rpm.asciidoc b/docs/reference/setup/install/rpm.asciidoc index c6f5647d08e26..041784bb96862 100644 --- a/docs/reference/setup/install/rpm.asciidoc +++ b/docs/reference/setup/install/rpm.asciidoc @@ -160,6 +160,7 @@ include::xpack-indices.asciidoc[] endif::include-xpack[] +[[rpm-sysv-init-vs-systemd]] ==== SysV `init` vs `systemd` include::init-systemd.asciidoc[] diff --git a/docs/reference/setup/starting.asciidoc b/docs/reference/setup/starting.asciidoc index 4474c247361db..c2ab6b1404aa2 100644 --- a/docs/reference/setup/starting.asciidoc +++ b/docs/reference/setup/starting.asciidoc @@ -34,9 +34,11 @@ include::install/zip-windows-start.asciidoc[] include::install/init-systemd.asciidoc[] [float] +[[start-es-deb-init]] include::install/deb-init.asciidoc[] [float] +[[start-es-deb-systemd]] include::install/systemd.asciidoc[] [float] @@ -66,7 +68,9 @@ include::install/msi-windows-start.asciidoc[] include::install/init-systemd.asciidoc[] [float] +[[start-es-rpm-init]] include::install/rpm-init.asciidoc[] [float] +[[start-es-rpm-systemd]] include::install/systemd.asciidoc[] diff --git a/docs/reference/sql/endpoints/jdbc.asciidoc b/docs/reference/sql/endpoints/jdbc.asciidoc index 9014d3f5719b5..33f130a891896 100644 --- a/docs/reference/sql/endpoints/jdbc.asciidoc +++ b/docs/reference/sql/endpoints/jdbc.asciidoc @@ -156,6 +156,7 @@ Opens up a {es-sql} connection to `server` on port `3456`, setting the JDBC conn One can use JDBC through the official `java.sql` and `javax.sql` packages: +[[java-sql]] ==== `java.sql` The former through `java.sql.Driver` and `DriverManager`: @@ -168,6 +169,7 @@ HTTP traffic. The port is by default 9200. <2> Properties for connecting to Elasticsearch. An empty `Properties` instance is fine for unsecured Elasticsearch. +[[javax-sql]] ==== `javax.sql` Accessible through the `javax.sql.DataSource` API: diff --git a/docs/reference/sql/limitations.asciidoc b/docs/reference/sql/limitations.asciidoc index e2a538cd08571..8f7868c892e52 100644 --- a/docs/reference/sql/limitations.asciidoc +++ b/docs/reference/sql/limitations.asciidoc @@ -4,6 +4,7 @@ == SQL Limitations [float] +[[sys-columns-describe-table-nested-fields]] === Nested fields in `SYS COLUMNS` and `DESCRIBE TABLE` {es} has a special type of relationship fields called `nested` fields. In {es-sql} they can be used by referencing their inner @@ -51,6 +52,7 @@ This is because of the way nested queries work in {es}: the root nested field wi pagination taking place on the **root nested document and not on its inner hits**. [float] +[[normalized-keyword-fields]] === Normalized `keyword` fields `keyword` fields in {es} can be normalized by defining a `normalizer`. Such fields are not supported in {es-sql}. @@ -108,6 +110,7 @@ But, if the sub-select would include a `GROUP BY` or `HAVING` or the enclosing ` FROM (SELECT ...) WHERE [simple_condition]`, this is currently **un-supported**. [float] +[[first-last-agg-functions-having-clause]] === Using <>/<> aggregation functions in `HAVING` clause Using `FIRST` and `LAST` in the `HAVING` clause is not supported. The same applies to @@ -115,6 +118,7 @@ Using `FIRST` and `LAST` in the `HAVING` clause is not supported. The same appli is of type <> as they are internally translated to `FIRST` and `LAST`. [float] +[[group-by-time]] === Using TIME data type in GROUP BY or <> Using `TIME` data type as a grouping key is currently not supported. For example: diff --git a/docs/reference/sql/security.asciidoc b/docs/reference/sql/security.asciidoc index a317355866b8b..ad946c33e2d98 100644 --- a/docs/reference/sql/security.asciidoc +++ b/docs/reference/sql/security.asciidoc @@ -7,6 +7,7 @@ In such a scenario, {es-sql} supports both security at the transport layer (by encrypting the communication between the consumer and the server) and authentication (for the access layer). [float] +[[ssl-tls-config]] ==== SSL/TLS configuration In case of an encrypted transport, the SSL/TLS support needs to be enabled in {es-sql} to properly establish communication with {es}. This is done by setting the `ssl` property to `true` or by using the `https` prefix in the URL. + From c1b643549aa071fee2981ae7b70a40abb502d9f0 Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Tue, 30 Apr 2019 20:42:53 +0300 Subject: [PATCH 244/260] Mute tests for backport #41673 (#41698) Mutes security tokens BWC tests for the #41673 backport. --- .../elasticsearch/xpack/security/authc/TokenService.java | 2 +- .../upgrades/TokenBackwardsCompatibilityIT.java | 2 ++ .../rest-api-spec/test/mixed_cluster/50_token_auth.yml | 6 ++++++ .../rest-api-spec/test/old_cluster/50_token_auth.yml | 2 ++ .../rest-api-spec/test/upgraded_cluster/50_token_auth.yml | 4 ++++ 5 files changed, 15 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java index 10236e1719b5c..79c5c84acdaec 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java @@ -180,7 +180,7 @@ public final class TokenService { private static final String TOKEN_DOC_ID_PREFIX = TOKEN_DOC_TYPE + "_"; static final int MINIMUM_BYTES = VERSION_BYTES + SALT_BYTES + IV_BYTES + 1; static final int MINIMUM_BASE64_BYTES = Double.valueOf(Math.ceil((4 * MINIMUM_BYTES) / 3)).intValue(); - static final Version VERSION_TOKENS_INDEX_INTRODUCED = Version.V_8_0_0; // TODO change upon backport + static final Version VERSION_TOKENS_INDEX_INTRODUCED = Version.V_7_1_0; static final Version VERSION_ACCESS_TOKENS_AS_UUIDS = Version.V_7_1_0; static final Version VERSION_MULTIPLE_CONCURRENT_REFRESHES = Version.V_7_1_0; // UUIDs are 16 bytes encoded base64 without padding, therefore the length is (16 / 3) * 4 + ((16 % 3) * 8 + 5) / 6 chars diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java index 69c515d80a3d2..983209c22c0a1 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java @@ -7,6 +7,7 @@ import org.apache.http.HttpHeaders; import org.apache.http.HttpHost; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; @@ -25,6 +26,7 @@ import java.util.List; import java.util.Map; +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/41673") public class TokenBackwardsCompatibilityIT extends AbstractUpgradeTestCase { private Collection twoClients = null; diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml index f426d9b2525b4..f5fb752ca6019 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/50_token_auth.yml @@ -2,6 +2,8 @@ "Get the indexed token and use if to authenticate": - skip: features: headers + version: "all" + reason: "Backport https://github.com/elastic/elasticsearch/pull/41673" - do: cluster.health: @@ -59,6 +61,8 @@ "Get the indexed refreshed access token and use if to authenticate": - skip: features: headers + version: "all" + reason: "Backport https://github.com/elastic/elasticsearch/pull/41673" - do: get: @@ -111,6 +115,8 @@ "Get the indexed refresh token and use it to get another access token and authenticate": - skip: features: headers + version: "all" + reason: "Backport https://github.com/elastic/elasticsearch/pull/41673" - do: get: diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml index e4d0eb8757f85..06e7d5cdc8c9c 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/50_token_auth.yml @@ -2,6 +2,8 @@ "Create a token and reuse it across the upgrade": - skip: features: headers + version: "all" + reason: "Backport https://github.com/elastic/elasticsearch/pull/41673" - do: cluster.health: diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml index 430f94c1064d6..fa717dc0c2f13 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/50_token_auth.yml @@ -2,6 +2,8 @@ "Get the indexed token and use if to authenticate": - skip: features: headers + version: "all" + reason: "Backport https://github.com/elastic/elasticsearch/pull/41673" - do: cluster.health: @@ -49,6 +51,8 @@ "Get the indexed refresh token and use if to get another access token and authenticate": - skip: features: headers + version: "all" + reason: "Backport https://github.com/elastic/elasticsearch/pull/41673" - do: get: From f87a1fde9e81ade28a6a5c3dad354bd048c6a31b Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 30 Apr 2019 12:45:19 -0500 Subject: [PATCH 245/260] [ML] data frame, adding builder classes for complex config classes (#41638) * [ML] data frame, adding builder classes for complex config classes * Addressing PR comments, adding some java docs * cleaning up constructor * fixing indentation * change constructors to be package-private --- .../transforms/DataFrameTransformConfig.java | 52 +++++- .../dataframe/transforms/SourceConfig.java | 45 +++++- .../pivot/DateHistogramGroupSource.java | 152 +++++++++++++----- .../transforms/pivot/GroupConfig.java | 29 +++- .../pivot/HistogramGroupSource.java | 39 ++++- .../transforms/pivot/PivotConfig.java | 49 +++++- .../transforms/pivot/TermsGroupSource.java | 25 ++- .../client/DataFrameTransformIT.java | 41 +++-- ...PreviewDataFrameTransformRequestTests.java | 2 +- .../PutDataFrameTransformRequestTests.java | 2 +- .../pivot/DateHistogramGroupSourceTests.java | 21 +-- .../DataFrameTransformDocumentationIT.java | 131 ++++++++++----- 12 files changed, 462 insertions(+), 126 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java index 4c7f29afc8678..8465ae8342827 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/DataFrameTransformConfig.java @@ -87,11 +87,11 @@ public static DataFrameTransformConfig forPreview(final SourceConfig source, fin return new DataFrameTransformConfig(null, source, null, pivotConfig, null); } - public DataFrameTransformConfig(final String id, - final SourceConfig source, - final DestConfig dest, - final PivotConfig pivotConfig, - final String description) { + DataFrameTransformConfig(final String id, + final SourceConfig source, + final DestConfig dest, + final PivotConfig pivotConfig, + final String description) { this.id = id; this.source = source; this.dest = dest; @@ -170,4 +170,46 @@ public int hashCode() { public String toString() { return Strings.toString(this, true, true); } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private String id; + private SourceConfig source; + private DestConfig dest; + private PivotConfig pivotConfig; + private String description; + + public Builder setId(String id) { + this.id = id; + return this; + } + + public Builder setSource(SourceConfig source) { + this.source = source; + return this; + } + + public Builder setDest(DestConfig dest) { + this.dest = dest; + return this; + } + + public Builder setPivotConfig(PivotConfig pivotConfig) { + this.pivotConfig = pivotConfig; + return this; + } + + public Builder setDescription(String description) { + this.description = description; + return this; + } + + public DataFrameTransformConfig build() { + return new DataFrameTransformConfig(id, source, dest, pivotConfig, description); + } + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SourceConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SourceConfig.java index 19c099ba601dc..7cafa5a890559 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SourceConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/SourceConfig.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.query.QueryBuilder; import java.io.IOException; import java.util.Arrays; @@ -76,7 +77,7 @@ public SourceConfig(String... index) { * @param index Any number of indices. At least one non-null, non-empty, index should be provided * @param queryConfig A QueryConfig object that contains the desired query. Defaults to MatchAll query. */ - public SourceConfig(String[] index, QueryConfig queryConfig) { + SourceConfig(String[] index, QueryConfig queryConfig) { this.index = index; this.queryConfig = queryConfig; } @@ -121,4 +122,46 @@ public int hashCode(){ int hash = Arrays.hashCode(index); return 31 * hash + (queryConfig == null ? 0 : queryConfig.hashCode()); } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private String[] index; + private QueryConfig queryConfig; + + /** + * Sets what indices from which to fetch data + * @param index The indices from which to fetch data + * @return The {@link Builder} with indices set + */ + public Builder setIndex(String... index) { + this.index = index; + return this; + } + + /** + * Sets the {@link QueryConfig} object that references the desired query to use when fetching the data + * @param queryConfig The {@link QueryConfig} to use when fetching data + * @return The {@link Builder} with queryConfig set + */ + public Builder setQueryConfig(QueryConfig queryConfig) { + this.queryConfig = queryConfig; + return this; + } + + /** + * Sets the query to use when fetching the data. Convenience method for {@link #setQueryConfig(QueryConfig)} + * @param query The {@link QueryBuilder} to use when fetch data (overwrites the {@link QueryConfig}) + * @return The {@link Builder} with queryConfig set + */ + public Builder setQuery(QueryBuilder query) { + return this.setQueryConfig(new QueryConfig(query)); + } + + public SourceConfig build() { + return new SourceConfig(index, queryConfig); + } + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java index 461b525934823..71e7e258c5c8b 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSource.java @@ -20,6 +20,7 @@ package org.elasticsearch.client.dataframe.transforms.pivot; import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; @@ -34,30 +35,41 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; +/** + * A grouping via a date histogram aggregation referencing a timefield + */ public class DateHistogramGroupSource extends SingleGroupSource implements ToXContentObject { private static final ParseField TIME_ZONE = new ParseField("time_zone"); private static final ParseField FORMAT = new ParseField("format"); private static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>("date_histogram_group_source", true, (args) -> new DateHistogramGroupSource((String) args[0])); + new ConstructingObjectParser<>("date_histogram_group_source", + true, + (args) -> { + String field = (String)args[0]; + long interval = 0; + DateHistogramInterval dateHistogramInterval = null; + if (args[1] instanceof Long) { + interval = (Long)args[1]; + } else { + dateHistogramInterval = (DateHistogramInterval) args[1]; + } + ZoneId zoneId = (ZoneId) args[2]; + String format = (String) args[3]; + return new DateHistogramGroupSource(field, interval, dateHistogramInterval, format, zoneId); + }); static { PARSER.declareString(optionalConstructorArg(), FIELD); - PARSER.declareField((histogram, interval) -> { - if (interval instanceof Long) { - histogram.setInterval((long) interval); - } else { - histogram.setDateHistogramInterval((DateHistogramInterval) interval); - } - }, p -> { + PARSER.declareField(optionalConstructorArg(), p -> { if (p.currentToken() == XContentParser.Token.VALUE_NUMBER) { return p.longValue(); } else { return new DateHistogramInterval(p.text()); } }, HistogramGroupSource.INTERVAL, ObjectParser.ValueType.LONG); - PARSER.declareField(DateHistogramGroupSource::setTimeZone, p -> { + PARSER.declareField(optionalConstructorArg(), p -> { if (p.currentToken() == XContentParser.Token.VALUE_STRING) { return ZoneId.of(p.text()); } else { @@ -65,20 +77,24 @@ public class DateHistogramGroupSource extends SingleGroupSource implements ToXCo } }, TIME_ZONE, ObjectParser.ValueType.LONG); - PARSER.declareString(DateHistogramGroupSource::setFormat, FORMAT); + PARSER.declareString(optionalConstructorArg(), FORMAT); } public static DateHistogramGroupSource fromXContent(final XContentParser parser) { return PARSER.apply(parser, null); } - private long interval = 0; - private DateHistogramInterval dateHistogramInterval; - private String format; - private ZoneId timeZone; + private final long interval; + private final DateHistogramInterval dateHistogramInterval; + private final String format; + private final ZoneId timeZone; - public DateHistogramGroupSource(String field) { + DateHistogramGroupSource(String field, long interval, DateHistogramInterval dateHistogramInterval, String format, ZoneId timeZone) { super(field); + this.interval = interval; + this.dateHistogramInterval = dateHistogramInterval; + this.format = format; + this.timeZone = timeZone; } @Override @@ -90,40 +106,18 @@ public long getInterval() { return interval; } - public void setInterval(long interval) { - if (interval < 1) { - throw new IllegalArgumentException("[interval] must be greater than or equal to 1."); - } - this.interval = interval; - } - public DateHistogramInterval getDateHistogramInterval() { return dateHistogramInterval; } - public void setDateHistogramInterval(DateHistogramInterval dateHistogramInterval) { - if (dateHistogramInterval == null) { - throw new IllegalArgumentException("[dateHistogramInterval] must not be null"); - } - this.dateHistogramInterval = dateHistogramInterval; - } - public String getFormat() { return format; } - public void setFormat(String format) { - this.format = format; - } - public ZoneId getTimeZone() { return timeZone; } - public void setTimeZone(ZoneId timeZone) { - this.timeZone = timeZone; - } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); @@ -168,4 +162,88 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(field, interval, dateHistogramInterval, timeZone, format); } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private String field; + private long interval = 0; + private DateHistogramInterval dateHistogramInterval; + private String format; + private ZoneId timeZone; + + /** + * The field with which to construct the date histogram grouping + * @param field The field name + * @return The {@link Builder} with the field set. + */ + public Builder setField(String field) { + this.field = field; + return this; + } + + /** + * Set the interval for the DateHistogram grouping + * @param interval the time interval in milliseconds + * @return the {@link Builder} with the interval set. + */ + public Builder setInterval(long interval) { + if (interval < 1) { + throw new IllegalArgumentException("[interval] must be greater than or equal to 1."); + } + this.interval = interval; + return this; + } + + /** + * Set the interval for the DateHistogram grouping + * @param timeValue The time value to use as the interval + * @return the {@link Builder} with the interval set. + */ + public Builder setInterval(TimeValue timeValue) { + return setInterval(timeValue.getMillis()); + } + + /** + * Sets the interval of the DateHistogram grouping + * + * If this DateHistogramInterval is set, it supersedes the #{@link DateHistogramGroupSource#getInterval()} + * @param dateHistogramInterval the DateHistogramInterval to set + * @return The {@link Builder} with the dateHistogramInterval set. + */ + public Builder setDateHistgramInterval(DateHistogramInterval dateHistogramInterval) { + if (dateHistogramInterval == null) { + throw new IllegalArgumentException("[dateHistogramInterval] must not be null"); + } + this.dateHistogramInterval = dateHistogramInterval; + return this; + } + + /** + * Set the optional String formatting for the time interval. + * @param format The format of the output for the time interval key + * @return The {@link Builder} with the format set. + */ + public Builder setFormat(String format) { + this.format = format; + return this; + } + + /** + * Sets the time zone to use for this aggregation + * @param timeZone The zoneId for the timeZone + * @return The {@link Builder} with the timeZone set. + */ + public Builder setTimeZone(ZoneId timeZone) { + this.timeZone = timeZone; + return this; + } + + public DateHistogramGroupSource build() { + return new DateHistogramGroupSource(field, interval, dateHistogramInterval, format, timeZone); + } + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/GroupConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/GroupConfig.java index e9c2a578a3c3f..ef1dd0f64e38e 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/GroupConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/GroupConfig.java @@ -26,12 +26,16 @@ import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.Objects; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +/** + * Class describing how to group data + */ public class GroupConfig implements ToXContentObject { private final Map groups; @@ -126,7 +130,7 @@ private static void consumeUntilEndObject(XContentParser parser, int endObjectCo } while (endObjectCount != 0); } - public GroupConfig(Map groups) { + GroupConfig(Map groups) { this.groups = groups; } @@ -174,4 +178,27 @@ public int hashCode() { public String toString() { return Strings.toString(this, true, true); } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private final Map groups = new HashMap<>(); + + /** + * Add a new grouping to the builder + * @param name The name of the resulting grouped field + * @param group The type of grouping referenced + * @return The {@link Builder} with a new grouping entry added + */ + public Builder groupBy(String name, SingleGroupSource group) { + groups.put(name, group); + return this; + } + + public GroupConfig build() { + return new GroupConfig(groups); + } + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/HistogramGroupSource.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/HistogramGroupSource.java index 764d9c15c6074..390e4c8b64a20 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/HistogramGroupSource.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/HistogramGroupSource.java @@ -31,6 +31,9 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; +/** + * A grouping via a histogram aggregation referencing a numeric field + */ public class HistogramGroupSource extends SingleGroupSource implements ToXContentObject { protected static final ParseField INTERVAL = new ParseField("interval"); @@ -49,7 +52,7 @@ public static HistogramGroupSource fromXContent(final XContentParser parser) { private final double interval; - public HistogramGroupSource(String field, double interval) { + HistogramGroupSource(String field, double interval) { super(field); if (interval <= 0) { throw new IllegalArgumentException("[interval] must be greater than 0."); @@ -97,4 +100,38 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(field, interval); } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private String field; + private double interval; + + /** + * The field to reference in the histogram grouping + * @param field The numeric field name to use in the histogram grouping + * @return The {@link Builder} with the field set. + */ + public Builder setField(String field) { + this.field = field; + return this; + } + + /** + * Set the interval for the histogram aggregation + * @param interval The numeric interval for the histogram grouping + * @return The {@link Builder} with the interval set. + */ + public Builder setInterval(double interval) { + this.interval = interval; + return this; + } + + public HistogramGroupSource build() { + return new HistogramGroupSource(field, interval); + } + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfig.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfig.java index 11362743bcc18..83b434d88bac7 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfig.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/PivotConfig.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.AggregatorFactories; import java.io.IOException; import java.util.Objects; @@ -31,6 +32,9 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; +/** + * Class describing how to pivot data via {@link GroupConfig} and {@link AggregationConfig} objects + */ public class PivotConfig implements ToXContentObject { private static final ParseField GROUP_BY = new ParseField("group_by"); @@ -51,7 +55,7 @@ public static PivotConfig fromXContent(final XContentParser parser) { return PARSER.apply(parser, null); } - public PivotConfig(GroupConfig groups, final AggregationConfig aggregationConfig) { + PivotConfig(GroupConfig groups, final AggregationConfig aggregationConfig) { this.groups = groups; this.aggregationConfig = aggregationConfig; } @@ -96,4 +100,47 @@ public int hashCode() { public boolean isValid() { return groups.isValid() && aggregationConfig.isValid(); } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private GroupConfig groups; + private AggregationConfig aggregationConfig; + + /** + * Set how to group the source data + * @param groups The configuration describing how to group and pivot the source data + * @return the {@link Builder} with the interval set. + */ + public Builder setGroups(GroupConfig groups) { + this.groups = groups; + return this; + } + + /** + * Set the aggregated fields to include in the pivot config + * @param aggregationConfig The configuration describing the aggregated fields + * @return the {@link Builder} with the aggregations set. + */ + public Builder setAggregationConfig(AggregationConfig aggregationConfig) { + this.aggregationConfig = aggregationConfig; + return this; + } + + /** + * Set the aggregated fields to include in the pivot config + * @param aggregations The aggregated field builders + * @return the {@link Builder} with the aggregations set. + */ + public Builder setAggregations(AggregatorFactories.Builder aggregations) { + this.aggregationConfig = new AggregationConfig(aggregations); + return this; + } + + public PivotConfig build() { + return new PivotConfig(groups, aggregationConfig); + } + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/TermsGroupSource.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/TermsGroupSource.java index 02a0caa59968e..d1bacdbc0a645 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/TermsGroupSource.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/dataframe/transforms/pivot/TermsGroupSource.java @@ -42,7 +42,7 @@ public static TermsGroupSource fromXContent(final XContentParser parser) { return PARSER.apply(parser, null); } - public TermsGroupSource(final String field) { + TermsGroupSource(final String field) { super(field); } @@ -60,4 +60,27 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par builder.endObject(); return builder; } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private String field; + + /** + * The field with which to construct the date histogram grouping + * @param field The field name + * @return The {@link Builder} with the field set. + */ + public Builder setField(String field) { + this.field = field; + return this; + } + + public TermsGroupSource build() { + return new TermsGroupSource(field); + } + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java index 9cfa42da1a800..bb1924b8b4957 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/DataFrameTransformIT.java @@ -43,9 +43,7 @@ import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats; import org.elasticsearch.client.dataframe.transforms.DataFrameTransformTaskState; import org.elasticsearch.client.dataframe.transforms.DestConfig; -import org.elasticsearch.client.dataframe.transforms.QueryConfig; import org.elasticsearch.client.dataframe.transforms.SourceConfig; -import org.elasticsearch.client.dataframe.transforms.pivot.AggregationConfig; import org.elasticsearch.client.dataframe.transforms.pivot.GroupConfig; import org.elasticsearch.client.dataframe.transforms.pivot.PivotConfig; import org.elasticsearch.client.dataframe.transforms.pivot.TermsGroupSource; @@ -61,7 +59,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -300,20 +297,21 @@ public void testPreview() throws IOException { } private DataFrameTransformConfig validDataFrameTransformConfig(String id, String source, String destination) { - QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); - AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregations(aggBuilder).build(); DestConfig destConfig = (destination != null) ? new DestConfig(destination) : null; - return new DataFrameTransformConfig(id, - new SourceConfig(new String[]{source}, queryConfig), - destConfig, - pivotConfig, - "this is a test transform"); + return DataFrameTransformConfig.builder() + .setId(id) + .setSource(SourceConfig.builder().setIndex(source).setQuery(new MatchAllQueryBuilder()).build()) + .setDest(destConfig) + .setPivotConfig(pivotConfig) + .setDescription("this is a test transform") + .build(); } public void testGetStats() throws Exception { @@ -321,19 +319,20 @@ public void testGetStats() throws Exception { createIndex(sourceIndex); indexData(sourceIndex); - QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); - AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregations(aggBuilder).build(); String id = "test-get-stats"; - DataFrameTransformConfig transform = new DataFrameTransformConfig(id, - new SourceConfig(new String[]{sourceIndex}, queryConfig), - new DestConfig("pivot-dest"), - pivotConfig, - "transform for testing stats"); + DataFrameTransformConfig transform = DataFrameTransformConfig.builder() + .setId(id) + .setSource(SourceConfig.builder().setIndex(sourceIndex).setQuery(new MatchAllQueryBuilder()).build()) + .setDest(new DestConfig("pivot-dest")) + .setPivotConfig(pivotConfig) + .setDescription("transform for testing stats") + .build(); DataFrameClient client = highLevelClient().dataFrame(); AcknowledgedResponse ack = execute(new PutDataFrameTransformRequest(transform), client::putDataFrameTransform, diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java index 690e93697b54b..c91e1cbb1dd91 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PreviewDataFrameTransformRequestTests.java @@ -70,7 +70,7 @@ public void testValidate() { assertFalse(new PreviewDataFrameTransformRequest(config).validate().isPresent()); // null source is not valid - config = new DataFrameTransformConfig(null, null, null, PivotConfigTests.randomPivotConfig(), null); + config = DataFrameTransformConfig.builder().setPivotConfig(PivotConfigTests.randomPivotConfig()).build(); Optional error = new PreviewDataFrameTransformRequest(config).validate(); assertTrue(error.isPresent()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java index 245e8e94e5599..28fd92dcf913f 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/PutDataFrameTransformRequestTests.java @@ -40,7 +40,7 @@ public class PutDataFrameTransformRequestTests extends AbstractXContentTestCase< public void testValidate() { assertFalse(createTestInstance().validate().isPresent()); - DataFrameTransformConfig config = new DataFrameTransformConfig(null, null, null, PivotConfigTests.randomPivotConfig(), null); + DataFrameTransformConfig config = DataFrameTransformConfig.builder().setPivotConfig(PivotConfigTests.randomPivotConfig()).build(); Optional error = new PutDataFrameTransformRequest(config).validate(); assertTrue(error.isPresent()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java index 91cf13e16e2e6..c6a160d9b8b8d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/dataframe/transforms/pivot/DateHistogramGroupSourceTests.java @@ -29,20 +29,13 @@ public class DateHistogramGroupSourceTests extends AbstractXContentTestCase - new TermsGroupSource("user_id"))); // <2> + GroupConfig groupConfig = GroupConfig.builder() + .groupBy("reviewer", // <1> + TermsGroupSource.builder().setField("user_id").build()) // <2> + .build(); // end::put-data-frame-transform-group-config // tag::put-data-frame-transform-agg-config AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); @@ -134,15 +135,20 @@ public void testPutDataFrameTransform() throws IOException, InterruptedException AggregationConfig aggConfig = new AggregationConfig(aggBuilder); // end::put-data-frame-transform-agg-config // tag::put-data-frame-transform-pivot-config - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); + PivotConfig pivotConfig = PivotConfig.builder() + .setGroups(groupConfig) + .setAggregationConfig(aggConfig) + .build(); // end::put-data-frame-transform-pivot-config // tag::put-data-frame-transform-config - DataFrameTransformConfig transformConfig = - new DataFrameTransformConfig("reviewer-avg-rating", // <1> - sourceConfig, // <2> - new DestConfig("pivot-destination"), // <3> - pivotConfig, // <4> - "This is my test transform"); // <5> + DataFrameTransformConfig transformConfig = DataFrameTransformConfig + .builder() + .setId("reviewer-avg-rating") // <1> + .setSource(sourceConfig) // <2> + .setDest(new DestConfig("pivot-destination")) // <3> + .setPivotConfig(pivotConfig) // <4> + .setDescription("This is my test transform") // <5> + .build(); // end::put-data-frame-transform-config { @@ -160,9 +166,12 @@ public void testPutDataFrameTransform() throws IOException, InterruptedException assertTrue(response.isAcknowledged()); } { - DataFrameTransformConfig configWithDifferentId = new DataFrameTransformConfig("reviewer-avg-rating2", - transformConfig.getSource(), transformConfig.getDestination(), - transformConfig.getPivotConfig(), null); + DataFrameTransformConfig configWithDifferentId = DataFrameTransformConfig.builder() + .setId("reviewer-avg-rating2") + .setSource(transformConfig.getSource()) + .setDest(transformConfig.getDestination()) + .setPivotConfig(transformConfig.getPivotConfig()) + .build(); PutDataFrameTransformRequest request = new PutDataFrameTransformRequest(configWithDifferentId); // tag::put-data-frame-transform-execute-listener @@ -199,14 +208,19 @@ public void testStartStop() throws IOException, InterruptedException { RestHighLevelClient client = highLevelClient(); QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregationConfig(aggConfig).build(); - DataFrameTransformConfig transformConfig = new DataFrameTransformConfig("mega-transform", - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest"), pivotConfig, null); + DataFrameTransformConfig transformConfig = DataFrameTransformConfig.builder() + .setId("mega-transform") + .setSource(SourceConfig.builder().setIndex("source-data").setQueryConfig(queryConfig).build()) + .setDest(new DestConfig("pivot-dest")) + .setPivotConfig(pivotConfig) + .build(); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig), RequestOptions.DEFAULT); transformsToClean.add(transformConfig.getId()); @@ -313,17 +327,31 @@ public void testDeleteDataFrameTransform() throws IOException, InterruptedExcept RestHighLevelClient client = highLevelClient(); - QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); - - DataFrameTransformConfig transformConfig1 = new DataFrameTransformConfig("mega-transform", - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest"), pivotConfig, null); - DataFrameTransformConfig transformConfig2 = new DataFrameTransformConfig("mega-transform2", - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("pivot-dest2"), pivotConfig, null); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregationConfig(aggConfig).build(); + + DataFrameTransformConfig transformConfig1 = DataFrameTransformConfig.builder() + .setId("mega-transform") + .setSource(SourceConfig.builder() + .setIndex("source-data") + .setQuery(new MatchAllQueryBuilder()) + .build()) + .setDest(new DestConfig("pivot-dest")) + .setPivotConfig(pivotConfig) + .build(); + DataFrameTransformConfig transformConfig2 = DataFrameTransformConfig.builder() + .setId("mega-transform2") + .setSource(SourceConfig.builder() + .setIndex("source-data") + .setQuery(new MatchAllQueryBuilder()) + .build()) + .setDest(new DestConfig("pivot-dest2")) + .setPivotConfig(pivotConfig) + .build(); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig1), RequestOptions.DEFAULT); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig2), RequestOptions.DEFAULT); @@ -379,16 +407,20 @@ public void testPreview() throws IOException, InterruptedException { RestHighLevelClient client = highLevelClient(); QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregationConfig(aggConfig).build(); // tag::preview-data-frame-transform-request DataFrameTransformConfig transformConfig = DataFrameTransformConfig.forPreview( - new SourceConfig(new String[]{"source-data"}, queryConfig), // <1> + SourceConfig.builder() + .setIndex("source-data") + .setQueryConfig(queryConfig) + .build(), // <1> pivotConfig); // <2> PreviewDataFrameTransformRequest request = new PreviewDataFrameTransformRequest(transformConfig); // <3> @@ -438,15 +470,23 @@ public void testGetStats() throws IOException, InterruptedException { RestHighLevelClient client = highLevelClient(); QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregationConfig(aggConfig).build(); String id = "statisitcal-transform"; - DataFrameTransformConfig transformConfig = new DataFrameTransformConfig(id, - new SourceConfig(new String[]{"source-data"}, queryConfig), new DestConfig("dest"), pivotConfig, null); + DataFrameTransformConfig transformConfig = DataFrameTransformConfig.builder() + .setId(id) + .setSource(SourceConfig.builder() + .setIndex("source-data") + .setQuery(new MatchAllQueryBuilder()) + .build()) + .setDest(new DestConfig("pivot-dest")) + .setPivotConfig(pivotConfig) + .build(); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(transformConfig), RequestOptions.DEFAULT); // tag::get-data-frame-transform-stats-request @@ -516,16 +556,23 @@ public void testGetDataFrameTransform() throws IOException, InterruptedException createIndex("source-data"); QueryConfig queryConfig = new QueryConfig(new MatchAllQueryBuilder()); - GroupConfig groupConfig = new GroupConfig(Collections.singletonMap("reviewer", new TermsGroupSource("user_id"))); + GroupConfig groupConfig = GroupConfig.builder().groupBy("reviewer", + TermsGroupSource.builder().setField("user_id").build()).build(); AggregatorFactories.Builder aggBuilder = new AggregatorFactories.Builder(); aggBuilder.addAggregator(AggregationBuilders.avg("avg_rating").field("stars")); AggregationConfig aggConfig = new AggregationConfig(aggBuilder); - PivotConfig pivotConfig = new PivotConfig(groupConfig, aggConfig); - - - DataFrameTransformConfig putTransformConfig = new DataFrameTransformConfig("mega-transform", - new SourceConfig(new String[]{"source-data"}, queryConfig), - new DestConfig("pivot-dest"), pivotConfig, null); + PivotConfig pivotConfig = PivotConfig.builder().setGroups(groupConfig).setAggregationConfig(aggConfig).build(); + + + DataFrameTransformConfig putTransformConfig = DataFrameTransformConfig.builder() + .setId("mega-transform") + .setSource(SourceConfig.builder() + .setIndex("source-data") + .setQuery(new MatchAllQueryBuilder()) + .build()) + .setDest(new DestConfig("pivot-dest")) + .setPivotConfig(pivotConfig) + .build(); RestHighLevelClient client = highLevelClient(); client.dataFrame().putDataFrameTransform(new PutDataFrameTransformRequest(putTransformConfig), RequestOptions.DEFAULT); From ba6135f0c76a0cdc45ec690feb1073ac4e677897 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 13:46:13 -0400 Subject: [PATCH 246/260] [DOCS] Allow attribute substitution in titleabbrevs for Asciidoctor migration (#41574) * [DOCS] Replace attributes in titleabbrevs for Asciidoctor migration * [DOCS] Add [subs="attributes"] so attributes render in Asciidoctor * Revert "[DOCS] Replace attributes in titleabbrevs for Asciidoctor migration" This reverts commit 98f130257a7c71e9f6cddf5157af7886418338d8. * [DOCS] Fix merge conflict --- docs/reference/data-frames/apis/delete-transform.asciidoc | 2 ++ docs/reference/data-frames/apis/get-transform-stats.asciidoc | 2 ++ docs/reference/data-frames/apis/get-transform.asciidoc | 2 ++ docs/reference/data-frames/apis/preview-transform.asciidoc | 2 ++ docs/reference/data-frames/apis/put-transform.asciidoc | 2 ++ docs/reference/data-frames/apis/start-transform.asciidoc | 2 ++ docs/reference/data-frames/apis/stop-transform.asciidoc | 2 ++ docs/reference/ilm/apis/get-status.asciidoc | 2 ++ docs/reference/ilm/apis/start.asciidoc | 2 ++ docs/reference/ilm/apis/stop.asciidoc | 2 ++ docs/reference/ml/apis/delete-datafeed.asciidoc | 2 ++ docs/reference/ml/apis/get-datafeed-stats.asciidoc | 2 ++ docs/reference/ml/apis/get-datafeed.asciidoc | 2 ++ docs/reference/ml/apis/get-ml-info.asciidoc | 2 ++ docs/reference/ml/apis/preview-datafeed.asciidoc | 2 ++ docs/reference/ml/apis/put-datafeed.asciidoc | 2 ++ docs/reference/ml/apis/start-datafeed.asciidoc | 2 ++ docs/reference/ml/apis/stop-datafeed.asciidoc | 2 ++ docs/reference/ml/apis/update-datafeed.asciidoc | 2 ++ docs/reference/monitoring/configuring-metricbeat.asciidoc | 2 ++ docs/reference/settings/notification-settings.asciidoc | 2 ++ 21 files changed, 42 insertions(+) diff --git a/docs/reference/data-frames/apis/delete-transform.asciidoc b/docs/reference/data-frames/apis/delete-transform.asciidoc index 327f55d4bc36b..9fb3d8558c0ea 100644 --- a/docs/reference/data-frames/apis/delete-transform.asciidoc +++ b/docs/reference/data-frames/apis/delete-transform.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[delete-data-frame-transform]] === Delete {dataframe-transforms} API + +[subs="attributes"] ++++ Delete {dataframe-transforms} ++++ diff --git a/docs/reference/data-frames/apis/get-transform-stats.asciidoc b/docs/reference/data-frames/apis/get-transform-stats.asciidoc index badb1b665f65f..85e5001b13a9a 100644 --- a/docs/reference/data-frames/apis/get-transform-stats.asciidoc +++ b/docs/reference/data-frames/apis/get-transform-stats.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[get-data-frame-transform-stats]] === Get {dataframe-transform} statistics API + +[subs="attributes"] ++++ Get {dataframe-transform} statistics ++++ diff --git a/docs/reference/data-frames/apis/get-transform.asciidoc b/docs/reference/data-frames/apis/get-transform.asciidoc index 6c0252ff4ad55..85e56aa21cdd1 100644 --- a/docs/reference/data-frames/apis/get-transform.asciidoc +++ b/docs/reference/data-frames/apis/get-transform.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[get-data-frame-transform]] === Get {dataframe-transforms} API + +[subs="attributes"] ++++ Get {dataframe-transforms} ++++ diff --git a/docs/reference/data-frames/apis/preview-transform.asciidoc b/docs/reference/data-frames/apis/preview-transform.asciidoc index 2a7bbf01fdfb8..cfa1763df0802 100644 --- a/docs/reference/data-frames/apis/preview-transform.asciidoc +++ b/docs/reference/data-frames/apis/preview-transform.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[preview-data-frame-transform]] === Preview {dataframe-transforms} API + +[subs="attributes"] ++++ Preview {dataframe-transforms} ++++ diff --git a/docs/reference/data-frames/apis/put-transform.asciidoc b/docs/reference/data-frames/apis/put-transform.asciidoc index 7158af959fc22..222d93dfe4256 100644 --- a/docs/reference/data-frames/apis/put-transform.asciidoc +++ b/docs/reference/data-frames/apis/put-transform.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[put-data-frame-transform]] === Create {dataframe-transforms} API + +[subs="attributes"] ++++ Create {dataframe-transforms} ++++ diff --git a/docs/reference/data-frames/apis/start-transform.asciidoc b/docs/reference/data-frames/apis/start-transform.asciidoc index bed0680df7416..d200ef34f587d 100644 --- a/docs/reference/data-frames/apis/start-transform.asciidoc +++ b/docs/reference/data-frames/apis/start-transform.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[start-data-frame-transform]] === Start {dataframe-transforms} API + +[subs="attributes"] ++++ Start {dataframe-transforms} ++++ diff --git a/docs/reference/data-frames/apis/stop-transform.asciidoc b/docs/reference/data-frames/apis/stop-transform.asciidoc index 2eb5a2d641bd4..4e89579b52ec1 100644 --- a/docs/reference/data-frames/apis/stop-transform.asciidoc +++ b/docs/reference/data-frames/apis/stop-transform.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[stop-data-frame-transform]] === Stop {dataframe-transforms} API + +[subs="attributes"] ++++ Stop {dataframe-transforms} ++++ diff --git a/docs/reference/ilm/apis/get-status.asciidoc b/docs/reference/ilm/apis/get-status.asciidoc index 5e67e26cb160c..af8a4deffcbc1 100644 --- a/docs/reference/ilm/apis/get-status.asciidoc +++ b/docs/reference/ilm/apis/get-status.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[ilm-get-status]] === Get {ilm} status API + +[subs="attributes"] ++++ Get {ilm} status ++++ diff --git a/docs/reference/ilm/apis/start.asciidoc b/docs/reference/ilm/apis/start.asciidoc index 92ca2a9806379..c13f5edcf11cc 100644 --- a/docs/reference/ilm/apis/start.asciidoc +++ b/docs/reference/ilm/apis/start.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[ilm-start]] === Start {ilm} API + +[subs="attributes"] ++++ Start {ilm} ++++ diff --git a/docs/reference/ilm/apis/stop.asciidoc b/docs/reference/ilm/apis/stop.asciidoc index dee1cf4fa3579..1629990c5fd86 100644 --- a/docs/reference/ilm/apis/stop.asciidoc +++ b/docs/reference/ilm/apis/stop.asciidoc @@ -2,6 +2,8 @@ [testenv="basic"] [[ilm-stop]] === Stop {ilm} API + +[subs="attributes"] ++++ Stop {ilm} ++++ diff --git a/docs/reference/ml/apis/delete-datafeed.asciidoc b/docs/reference/ml/apis/delete-datafeed.asciidoc index 5940d5c70f044..d2a7845be7584 100644 --- a/docs/reference/ml/apis/delete-datafeed.asciidoc +++ b/docs/reference/ml/apis/delete-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-delete-datafeed]] === Delete {dfeeds} API + +[subs="attributes"] ++++ Delete {dfeeds} ++++ diff --git a/docs/reference/ml/apis/get-datafeed-stats.asciidoc b/docs/reference/ml/apis/get-datafeed-stats.asciidoc index cb2ca19a60b43..d1b842509b408 100644 --- a/docs/reference/ml/apis/get-datafeed-stats.asciidoc +++ b/docs/reference/ml/apis/get-datafeed-stats.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-get-datafeed-stats]] === Get {dfeed} statistics API + +[subs="attributes"] ++++ Get {dfeed} statistics ++++ diff --git a/docs/reference/ml/apis/get-datafeed.asciidoc b/docs/reference/ml/apis/get-datafeed.asciidoc index 402838742dfa6..2c9ef7e3aec74 100644 --- a/docs/reference/ml/apis/get-datafeed.asciidoc +++ b/docs/reference/ml/apis/get-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-get-datafeed]] === Get {dfeeds} API + +[subs="attributes"] ++++ Get {dfeeds} ++++ diff --git a/docs/reference/ml/apis/get-ml-info.asciidoc b/docs/reference/ml/apis/get-ml-info.asciidoc index fb9212a59afbc..00c57c8cf7f5b 100644 --- a/docs/reference/ml/apis/get-ml-info.asciidoc +++ b/docs/reference/ml/apis/get-ml-info.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[get-ml-info]] === Get machine learning info API + +[subs="attributes"] ++++ Get {ml} info ++++ diff --git a/docs/reference/ml/apis/preview-datafeed.asciidoc b/docs/reference/ml/apis/preview-datafeed.asciidoc index 7eca456c981dd..83af6a78057cc 100644 --- a/docs/reference/ml/apis/preview-datafeed.asciidoc +++ b/docs/reference/ml/apis/preview-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-preview-datafeed]] === Preview {dfeeds} API + +[subs="attributes"] ++++ Preview {dfeeds} ++++ diff --git a/docs/reference/ml/apis/put-datafeed.asciidoc b/docs/reference/ml/apis/put-datafeed.asciidoc index 05e02ce3615df..52728dd093da5 100644 --- a/docs/reference/ml/apis/put-datafeed.asciidoc +++ b/docs/reference/ml/apis/put-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-put-datafeed]] === Create {dfeeds} API + +[subs="attributes"] ++++ Create {dfeeds} ++++ diff --git a/docs/reference/ml/apis/start-datafeed.asciidoc b/docs/reference/ml/apis/start-datafeed.asciidoc index 2ae92288a2666..aee237b72c837 100644 --- a/docs/reference/ml/apis/start-datafeed.asciidoc +++ b/docs/reference/ml/apis/start-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-start-datafeed]] === Start {dfeeds} API + +[subs="attributes"] ++++ Start {dfeeds} ++++ diff --git a/docs/reference/ml/apis/stop-datafeed.asciidoc b/docs/reference/ml/apis/stop-datafeed.asciidoc index c021d9ad18d62..1489137b9db07 100644 --- a/docs/reference/ml/apis/stop-datafeed.asciidoc +++ b/docs/reference/ml/apis/stop-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-stop-datafeed]] === Stop {dfeeds} API + +[subs="attributes"] ++++ Stop {dfeeds} ++++ diff --git a/docs/reference/ml/apis/update-datafeed.asciidoc b/docs/reference/ml/apis/update-datafeed.asciidoc index 1e888f823ff72..a370c1acef9d7 100644 --- a/docs/reference/ml/apis/update-datafeed.asciidoc +++ b/docs/reference/ml/apis/update-datafeed.asciidoc @@ -2,6 +2,8 @@ [testenv="platinum"] [[ml-update-datafeed]] === Update {dfeeds} API + +[subs="attributes"] ++++ Update {dfeeds} ++++ diff --git a/docs/reference/monitoring/configuring-metricbeat.asciidoc b/docs/reference/monitoring/configuring-metricbeat.asciidoc index f41e02b0095f1..df578e88da614 100644 --- a/docs/reference/monitoring/configuring-metricbeat.asciidoc +++ b/docs/reference/monitoring/configuring-metricbeat.asciidoc @@ -2,6 +2,8 @@ [testenv="gold"] [[configuring-metricbeat]] === Collecting {es} monitoring data with {metricbeat} + +[subs="attributes"] ++++ Collecting monitoring data with {metricbeat} ++++ diff --git a/docs/reference/settings/notification-settings.asciidoc b/docs/reference/settings/notification-settings.asciidoc index ee99c07d91dad..736e3cae8d68d 100644 --- a/docs/reference/settings/notification-settings.asciidoc +++ b/docs/reference/settings/notification-settings.asciidoc @@ -1,6 +1,8 @@ [role="xpack"] [[notification-settings]] === {watcher} settings in Elasticsearch + +[subs="attributes"] ++++ {watcher} settings ++++ From b33b5fc1222d4efa3bf727bf7953b3a27996eed2 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 14:11:24 -0400 Subject: [PATCH 247/260] [DOCS] Add attribute to escape minimal pt token link in Asciidoctor (#41613) --- .../analysis/tokenfilters/stemmer-tokenfilter.asciidoc | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc index f59e2f3f2cf88..b5d5426ff2710 100644 --- a/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc @@ -1,6 +1,12 @@ [[analysis-stemmer-tokenfilter]] === Stemmer Token Filter +// Adds attribute for the 'minimal_portuguese' stemmer values link. +// This link contains ~, which is converted to subscript. +// This attribute prevents that substitution. +// See https://github.com/asciidoctor/asciidoctor/wiki/How-to-prevent-URLs-containing-formatting-characters-from-getting-mangled +:min-pt-stemmer-values-url: http://www.inf.ufrgs.br/~buriol/papers/Orengo_CLEF07.pdf + A filter that provides access to (almost) all of the available stemming token filters through a single unified interface. For example: @@ -158,7 +164,7 @@ Portuguese:: http://snowball.tartarus.org/algorithms/portuguese/stemmer.html[`portuguese`], http://dl.acm.org/citation.cfm?id=1141523&dl=ACM&coll=DL&CFID=179095584&CFTOKEN=80067181[*`light_portuguese`*], -http://www.inf.ufrgs.br/\~buriol/papers/Orengo_CLEF07.pdf[`minimal_portuguese`], +{min-pt-stemmer-values-url}[`minimal_portuguese`], http://www.inf.ufrgs.br/\~viviane/rslp/index.htm[`portuguese_rslp`] Romanian:: From ddafdb8ca1dd25fa16c6dbd9acd9763de7fc4738 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 15:18:49 -0400 Subject: [PATCH 248/260] [DOCS] Escape depcreated[] note for Asciidoctor migration (#41608) --- docs/reference/commands/certgen.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/commands/certgen.asciidoc b/docs/reference/commands/certgen.asciidoc index 2ca489b3a60a9..6087fe8440a0e 100644 --- a/docs/reference/commands/certgen.asciidoc +++ b/docs/reference/commands/certgen.asciidoc @@ -3,7 +3,7 @@ [[certgen]] == elasticsearch-certgen -deprecated[6.1,Replaced by <>.] +deprecated[6.1,"Replaced by <>."] The `elasticsearch-certgen` command simplifies the creation of certificate authorities (CA), certificate signing requests (CSR), and signed certificates From 8e3ff6793fe82a134449f56f37e1a6779f5cd699 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 15:50:52 -0400 Subject: [PATCH 249/260] [DOCS] Escape commas in deprecated[] for Asciidoctor migration (#41598) --- docs/reference/mapping/params/boost.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/mapping/params/boost.asciidoc b/docs/reference/mapping/params/boost.asciidoc index 7da03a66ac44e..6dfe564ed7c58 100644 --- a/docs/reference/mapping/params/boost.asciidoc +++ b/docs/reference/mapping/params/boost.asciidoc @@ -64,7 +64,7 @@ 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.] +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."] [WARNING] .Why index time boosting is a bad idea ================================================== From 9d94c6dc47369e3c0c201c4201d9f80b41675002 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 16:02:59 -0400 Subject: [PATCH 250/260] [DOCS] Move beta[] tag for Asciidoctor migration (#41599) --- docs/reference/setup/install.asciidoc | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/reference/setup/install.asciidoc b/docs/reference/setup/install.asciidoc index f5dd418155d55..800cecb40652a 100644 --- a/docs/reference/setup/install.asciidoc +++ b/docs/reference/setup/install.asciidoc @@ -46,9 +46,7 @@ Elasticsearch website or from our RPM repository. `msi`:: -beta[] -+ -The `msi` package is suitable for installation on Windows 64-bit systems with at least +beta[] The `msi` package is suitable for installation on Windows 64-bit systems with at least .NET 4.5 framework installed, and is the easiest choice for getting started with Elasticsearch on Windows. MSIs may be downloaded from the Elasticsearch website. + From 9a8d976970d30d8541a1ff60c1234cc53b08bff9 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 16:22:15 -0400 Subject: [PATCH 251/260] [DOCS] Remove soft line breaks (+) for Asciidoctor migration (#41600) --- docs/reference/monitoring/collectors.asciidoc | 91 +++++++++---------- 1 file changed, 42 insertions(+), 49 deletions(-) diff --git a/docs/reference/monitoring/collectors.asciidoc b/docs/reference/monitoring/collectors.asciidoc index bc77480561a8d..64d56e81cdc87 100644 --- a/docs/reference/monitoring/collectors.asciidoc +++ b/docs/reference/monitoring/collectors.asciidoc @@ -24,66 +24,59 @@ avoid many unnecessary calls. |======================= | Collector | Data Types | Description | Cluster Stats | `cluster_stats` -| Gathers details about the cluster state, including parts of -the actual cluster state (for example `GET /_cluster/state`) and statistics -about it (for example, `GET /_cluster/stats`). This produces a single document -type. In versions prior to X-Pack 5.5, this was actually three separate collectors -that resulted in three separate types: `cluster_stats`, `cluster_state`, and -`cluster_info`. In 5.5 and later, all three are combined into `cluster_stats`. -+ -This only runs on the _elected_ master node and the data collected -(`cluster_stats`) largely controls the UI. When this data is not present, it -indicates either a misconfiguration on the elected master node, timeouts related -to the collection of the data, or issues with storing the data. Only a single -document is produced per collection. +| Gathers details about the cluster state, including parts of the actual cluster +state (for example `GET /_cluster/state`) and statistics about it (for example, +`GET /_cluster/stats`). This produces a single document type. In versions prior +to X-Pack 5.5, this was actually three separate collectors that resulted in +three separate types: `cluster_stats`, `cluster_state`, and `cluster_info`. In +5.5 and later, all three are combined into `cluster_stats`. This only runs on +the _elected_ master node and the data collected (`cluster_stats`) largely +controls the UI. When this data is not present, it indicates either a +misconfiguration on the elected master node, timeouts related to the collection +of the data, or issues with storing the data. Only a single document is produced +per collection. | Index Stats | `indices_stats`, `index_stats` | Gathers details about the indices in the cluster, both in summary and individually. This creates many documents that represent parts of the index -statistics output (for example, `GET /_stats`). -+ -This information only needs to be collected once, so it is collected on the -_elected_ master node. The most common failure for this collector relates to an -extreme number of indices -- and therefore time to gather them -- resulting in -timeouts. One summary `indices_stats` document is produced per collection and one -`index_stats` document is produced per index, per collection. +statistics output (for example, `GET /_stats`). This information only needs to +be collected once, so it is collected on the _elected_ master node. The most +common failure for this collector relates to an extreme number of indices -- and +therefore time to gather them -- resulting in timeouts. One summary +`indices_stats` document is produced per collection and one `index_stats` +document is produced per index, per collection. | Index Recovery | `index_recovery` | Gathers details about index recovery in the cluster. Index recovery represents the assignment of _shards_ at the cluster level. If an index is not recovered, -it is not usable. This also corresponds to shard restoration via snapshots. -+ -This information only needs to be collected once, so it is collected on the -_elected_ master node. The most common failure for this collector relates to an -extreme number of shards -- and therefore time to gather them -- resulting in -timeouts. This creates a single document that contains all recoveries by default, -which can be quite large, but it gives the most accurate picture of recovery in -the production cluster. +it is not usable. This also corresponds to shard restoration via snapshots. This +information only needs to be collected once, so it is collected on the _elected_ +master node. The most common failure for this collector relates to an extreme +number of shards -- and therefore time to gather them -- resulting in timeouts. +This creates a single document that contains all recoveries by default, which +can be quite large, but it gives the most accurate picture of recovery in the +production cluster. | Shards | `shards` | Gathers details about all _allocated_ shards for all indices, particularly -including what node the shard is allocated to. -+ -This information only needs to be collected once, so it is collected on the -_elected_ master node. The collector uses the local cluster state to get the -routing table without any network timeout issues unlike most other collectors. -Each shard is represented by a separate monitoring document. +including what node the shard is allocated to. This information only needs to be +collected once, so it is collected on the _elected_ master node. The collector +uses the local cluster state to get the routing table without any network +timeout issues unlike most other collectors. Each shard is represented by a +separate monitoring document. | Jobs | `job_stats` -| Gathers details about all machine learning job statistics (for example, -`GET /_ml/anomaly_detectors/_stats`). -+ -This information only needs to be collected once, so it is collected on the -_elected_ master node. However, for the master node to be able to perform the -collection, the master node must have `xpack.ml.enabled` set to true (default) -and a license level that supports {ml}. +| Gathers details about all machine learning job statistics (for example, `GET +/_ml/anomaly_detectors/_stats`). This information only needs to be collected +once, so it is collected on the _elected_ master node. However, for the master +node to be able to perform the collection, the master node must have +`xpack.ml.enabled` set to true (default) and a license level that supports {ml}. | Node Stats | `node_stats` | Gathers details about the running node, such as memory utilization and CPU -usage (for example, `GET /_nodes/_local/stats`). -+ -This runs on _every_ node with {monitoring} enabled. One common failure -results in the timeout of the node stats request due to too many segment files. -As a result, the collector spends too much time waiting for the file system -stats to be calculated until it finally times out. A single `node_stats` -document is created per collection. This is collected per node to help to -discover issues with nodes communicating with each other, but not with the -monitoring cluster (for example, intermittent network issues or memory pressure). +usage (for example, `GET /_nodes/_local/stats`). This runs on _every_ node with +{monitoring} enabled. One common failure results in the timeout of the node +stats request due to too many segment files. As a result, the collector spends +too much time waiting for the file system stats to be calculated until it +finally times out. A single `node_stats` document is created per collection. +This is collected per node to help to discover issues with nodes communicating +with each other, but not with the monitoring cluster (for example, intermittent +network issues or memory pressure). |======================= {monitoring} uses a single threaded scheduler to run the collection of {es} From 737b359b943b5ddf461c72b263b2f03f2c645d43 Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 16:30:58 -0400 Subject: [PATCH 252/260] [DOCS] Escape quotes to avoid smart quotes in Asciidoctor (#41603) --- docs/reference/ingest/processors/date-index-name.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/ingest/processors/date-index-name.asciidoc b/docs/reference/ingest/processors/date-index-name.asciidoc index 7f42bf09040e7..fa749fd342056 100644 --- a/docs/reference/ingest/processors/date-index-name.asciidoc +++ b/docs/reference/ingest/processors/date-index-name.asciidoc @@ -137,7 +137,7 @@ understands this to mean `2016-04-01` as is explained in the <>. | `date_rounding` | yes | - | How to round the date when formatting the date into the index name. Valid values are: `y` (year), `M` (month), `w` (week), `d` (day), `h` (hour), `m` (minute) and `s` (second). Supports <>. -| `date_formats` | no | yyyy-MM-dd'T'HH:mm:ss.SSSXX | An array of the expected date formats for parsing dates / timestamps in the document being preprocessed. Can be a java time pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N. +| `date_formats` | no | yyyy-MM-dd+++'T'+++HH:mm:ss.SSSXX | An array of the expected date formats for parsing dates / timestamps in the document being preprocessed. Can be a java time pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N. | `timezone` | no | UTC | The timezone to use when parsing the date and when date math index supports resolves expressions into concrete index names. | `locale` | no | ENGLISH | The locale to use when parsing the date from the document being preprocessed, relevant when parsing month names or week days. | `index_name_format` | no | yyyy-MM-dd | The format to be used when printing the parsed date into the index name. An valid java time pattern is expected here. Supports <>. From 8e04f36caf82ffbf16b661d28cb3c058ca6c4cee Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Tue, 30 Apr 2019 16:40:26 -0400 Subject: [PATCH 253/260] [DOCS] Move block delimiter and anchor in 'Configuring Security' for Asciidoctor migration (#41604) --- x-pack/docs/en/security/configuring-es.asciidoc | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/x-pack/docs/en/security/configuring-es.asciidoc b/x-pack/docs/en/security/configuring-es.asciidoc index 74ace407fd528..b34e6e0c0e9f2 100644 --- a/x-pack/docs/en/security/configuring-es.asciidoc +++ b/x-pack/docs/en/security/configuring-es.asciidoc @@ -82,12 +82,13 @@ user API. ** <>. . Set up roles and users to control access to {es}. ++ +-- For example, to grant _John Doe_ full access to all indices that match the pattern `events*` and enable him to create visualizations and dashboards for those indices in {kib}, you could create an `events_admin` role and assign the role to a new `johndoe` user. -+ --- + [source,shell] ---------------------------------------------------------- curl -XPOST -u elastic 'localhost:9200/_security/role/events_admin' -H "Content-Type: application/json" -d '{ @@ -113,8 +114,7 @@ curl -XPOST -u elastic 'localhost:9200/_security/user/johndoe' -H "Content-Type: // NOTCONSOLE -- -[[enable-auditing]] -. Enable auditing to keep track of attempted and successful interactions with +. [[enable-auditing]]Enable auditing to keep track of attempted and successful interactions with your {es} cluster: + -- From 2154a97a16e6fe3d73d4c36b244605ab87e43b12 Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Wed, 1 May 2019 10:19:00 +0300 Subject: [PATCH 254/260] Amend `prepareIndexIfNeededThenExecute` for token refresh (#41697) This fixes a low level bug that manifests, in certain circumstances, by the failure of the refresh operation. Version 7.1 added a new `superseded_by` field to the `.security` index mapping. This field is used when indexing a refresh operation (a document update). Because the document update was not guarded by the obligatory `prepareIndexIfNeededThenExecute` the refresh operation would fail if it were the first operation when the cluster was upgraded from a version < 7.1 . This failure was catched (and fails reliably) in the backport #41673 . --- .../xpack/security/authc/TokenService.java | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java index 79c5c84acdaec..0481867b594f2 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java @@ -347,13 +347,12 @@ private void getUserTokenFromId(String userTokenId, Version tokenVersion, Action logger.warn("failed to get access token [{}] because index [{}] is not available", userTokenId, tokensIndex.aliasName()); listener.onResponse(null); } else { + final GetRequest getRequest = client.prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, + getTokenDocumentId(userTokenId)).request(); + final Consumer onFailure = ex -> listener.onFailure(traceLog("decode token", userTokenId, ex)); tokensIndex.checkIndexVersionThenExecute( ex -> listener.onFailure(traceLog("prepare tokens index [" + tokensIndex.aliasName() +"]", userTokenId, ex)), - () -> { - final GetRequest getRequest = client.prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, - getTokenDocumentId(userTokenId)).request(); - Consumer onFailure = ex -> listener.onFailure(traceLog("decode token", userTokenId, ex)); - executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, + () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, ActionListener.wrap(response -> { if (response.isExists()) { Map accessTokenSource = @@ -384,8 +383,8 @@ private void getUserTokenFromId(String userTokenId, Version tokenVersion, Action logger.error(new ParameterizedMessage("failed to get access token [{}]", userTokenId), e); listener.onFailure(e); } - }), client::get); - }); + }), client::get) + ); } } @@ -862,7 +861,9 @@ private void innerRefresh(String tokenDocId, Map source, long se .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setIfSeqNo(seqNo) .setIfPrimaryTerm(primaryTerm); - executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, updateRequest.request(), + refreshedTokenIndex.prepareIndexIfNeededThenExecute( + ex -> listener.onFailure(traceLog("prepare index [" + refreshedTokenIndex.aliasName() + "]", ex)), + () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, updateRequest.request(), ActionListener.wrap(updateResponse -> { if (updateResponse.getResult() == DocWriteResponse.Result.UPDATED) { logger.debug(() -> new ParameterizedMessage("updated the original token document to {}", @@ -931,7 +932,7 @@ public void onFailure(Exception e) { } else { onFailure.accept(e); } - }), client::update); + }), client::update)); } } @@ -1005,7 +1006,9 @@ private void getSupersedingTokenDocAsync(RefreshTokenStatus refreshTokenStatus, private void getTokenDocAsync(String tokenDocId, SecurityIndexManager tokensIndex, ActionListener listener) { final GetRequest getRequest = client.prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, tokenDocId).request(); - executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, listener, client::get); + tokensIndex.checkIndexVersionThenExecute( + ex -> listener.onFailure(traceLog("prepare tokens index [" + tokensIndex.aliasName() + "]", tokenDocId, ex)), + () -> executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, listener, client::get)); } private Version getTokenVersionCompatibility() { @@ -1392,10 +1395,10 @@ private void checkIfTokenIsValid(UserToken userToken, ActionListener logger.warn("failed to validate access token because the index [" + tokensIndex.aliasName() + "] doesn't exist"); listener.onResponse(null); } else { + final GetRequest getRequest = client + .prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, getTokenDocumentId(userToken)).request(); + Consumer onFailure = ex -> listener.onFailure(traceLog("check token state", userToken.getId(), ex)); tokensIndex.checkIndexVersionThenExecute(listener::onFailure, () -> { - final GetRequest getRequest = client - .prepareGet(tokensIndex.aliasName(), SINGLE_MAPPING_NAME, getTokenDocumentId(userToken)).request(); - Consumer onFailure = ex -> listener.onFailure(traceLog("check token state", userToken.getId(), ex)); executeAsyncWithOrigin(client.threadPool().getThreadContext(), SECURITY_ORIGIN, getRequest, ActionListener.wrap(response -> { if (response.isExists()) { From 850879586d9f211f265739d045eb7764b3ff79ff Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 1 May 2019 10:41:52 +0200 Subject: [PATCH 255/260] Support empty first line in msearch request body (#41011) With #41442 we have deprecated support for empty line before any action metadata in msearch API. With this commit we remove support for such empty line, in place of parsing it as empty action metadata, which was previously not supported although the following items could have an empty line representing their corresponding action metadata. This way we make all times equal. Relates to #39841 --- .../action/search/MultiSearchRequest.java | 6 ------ .../action/search/MultiSearchRequestTests.java | 10 ++++++++++ .../action/search/msearch-no-metadata.json | 8 ++++++++ 3 files changed, 18 insertions(+), 6 deletions(-) create mode 100644 server/src/test/resources/org/elasticsearch/action/search/msearch-no-metadata.json diff --git a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java index 195e146ab7ca1..528be0369166e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java @@ -183,12 +183,6 @@ public static void readMultiLineFormat(BytesReference data, if (nextMarker == -1) { break; } - // support first line with \n - if (nextMarker == 0) { - from = nextMarker + 1; - continue; - } - SearchRequest searchRequest = new SearchRequest(); if (indices != null) { searchRequest.indices(indices); diff --git a/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java b/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java index da22ce4c96c1a..afe957e2bf3ee 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java @@ -48,6 +48,7 @@ import static org.elasticsearch.search.RandomSearchRequestGenerator.randomSearchRequest; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; @@ -180,6 +181,15 @@ public void testSimpleAdd4() throws Exception { assertThat(request.requests().get(2).routing(), equalTo("123")); } + public void testNoMetadata() throws Exception { + MultiSearchRequest request = parseMultiSearchRequest("/org/elasticsearch/action/search/msearch-no-metadata.json"); + assertThat(request.requests().size(), equalTo(4)); + for (SearchRequest searchRequest : request.requests()) { + assertThat(searchRequest.indices().length, equalTo(0)); + assertThat(searchRequest.source().query(), instanceOf(MatchAllQueryBuilder.class)); + } + } + public void testResponseErrorToXContent() { long tookInMillis = randomIntBetween(1, 1000); MultiSearchResponse response = new MultiSearchResponse( diff --git a/server/src/test/resources/org/elasticsearch/action/search/msearch-no-metadata.json b/server/src/test/resources/org/elasticsearch/action/search/msearch-no-metadata.json new file mode 100644 index 0000000000000..a13fa283eeccf --- /dev/null +++ b/server/src/test/resources/org/elasticsearch/action/search/msearch-no-metadata.json @@ -0,0 +1,8 @@ + +{ "query": {"match_all": {}}} + +{ "query": {"match_all": {}}} + +{ "query": {"match_all": {}}} + +{ "query": {"match_all": {}}} From e0342defae2030d14de68177dffb2f72aed3d93c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 1 May 2019 06:46:31 -0400 Subject: [PATCH 256/260] Drop distinction in entries for keystore (#41701) Today we allow adding entries from a file or from a string, yet we internally maintain this distinction such that if you try to add a value from a file for a setting that expects a string or add a value from a string for a setting that expects a file, you will have a bad time. This causes a pain for operators such that for each setting they need to know this difference. Yet, we do not need to maintain this distinction internally as they are bytes after all. This commit removes that distinction and includes logic to upgrade legacy keystores. --- .../precommit/ForbiddenPatternsTask.java | 1 + .../common/settings/KeyStoreWrapperTests.java | 83 ++++++++++++--- .../format-v3-elasticsearch.keystore | Bin 0 -> 281 bytes .../common/settings/KeyStoreWrapper.java | 97 ++++++++---------- .../FIPS140SecureSettingsBootstrapCheck.java | 2 +- 5 files changed, 115 insertions(+), 68 deletions(-) create mode 100644 distribution/tools/keystore-cli/src/test/resources/format-v3-elasticsearch.keystore diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenPatternsTask.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenPatternsTask.java index 7361b78ad0725..f858ec26fc158 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenPatternsTask.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/ForbiddenPatternsTask.java @@ -66,6 +66,7 @@ public class ForbiddenPatternsTask extends DefaultTask { .exclude("**/*.zip") .exclude("**/*.jks") .exclude("**/*.crt") + .exclude("**/*.keystore") .exclude("**/*.png"); /* diff --git a/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java index bb2b1df7f8c03..f68a731edf8f7 100644 --- a/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java +++ b/distribution/tools/keystore-cli/src/test/java/org/elasticsearch/common/settings/KeyStoreWrapperTests.java @@ -19,6 +19,18 @@ package org.elasticsearch.common.settings; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.SimpleFSDirectory; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.env.Environment; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; + import javax.crypto.Cipher; import javax.crypto.CipherOutputStream; import javax.crypto.SecretKey; @@ -26,36 +38,29 @@ import javax.crypto.spec.GCMParameterSpec; import javax.crypto.spec.PBEKeySpec; import javax.crypto.spec.SecretKeySpec; + import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.EOFException; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.nio.file.FileSystem; +import java.nio.file.Files; import java.nio.file.Path; +import java.security.GeneralSecurityException; import java.security.KeyStore; import java.security.SecureRandom; import java.util.ArrayList; import java.util.Base64; import java.util.List; - -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.SimpleFSDirectory; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.env.Environment; -import org.elasticsearch.test.ESTestCase; -import org.hamcrest.Matchers; -import org.junit.After; -import org.junit.Before; +import java.util.Set; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.notNullValue; public class KeyStoreWrapperTests extends ESTestCase { @@ -386,4 +391,56 @@ public void testBackcompatV2() throws Exception { assertEquals(-1, fileInput.read()); } } + + public void testStringAndFileDistinction() throws Exception { + final KeyStoreWrapper wrapper = KeyStoreWrapper.create(); + wrapper.setString("string_setting", "string_value".toCharArray()); + final Path temp = createTempDir(); + Files.writeString(temp.resolve("file_setting"), "file_value", StandardCharsets.UTF_8); + wrapper.setFile("file_setting", Files.readAllBytes(temp.resolve("file_setting"))); + wrapper.save(env.configFile(), new char[0]); + wrapper.close(); + + final KeyStoreWrapper afterSave = KeyStoreWrapper.load(env.configFile()); + assertNotNull(afterSave); + afterSave.decrypt(new char[0]); + assertThat(afterSave.getSettingNames(), equalTo(Set.of("keystore.seed", "string_setting", "file_setting"))); + assertThat(afterSave.getString("string_setting"), equalTo("string_value")); + assertThat(toByteArray(afterSave.getFile("string_setting")), equalTo("string_value".getBytes(StandardCharsets.UTF_8))); + assertThat(afterSave.getString("file_setting"), equalTo("file_value")); + assertThat(toByteArray(afterSave.getFile("file_setting")), equalTo("file_value".getBytes(StandardCharsets.UTF_8))); + } + + public void testLegacyV3() throws GeneralSecurityException, IOException { + final Path configDir = createTempDir(); + final Path keystore = configDir.resolve("elasticsearch.keystore"); + try (InputStream is = KeyStoreWrapperTests.class.getResourceAsStream("/format-v3-elasticsearch.keystore"); + OutputStream os = Files.newOutputStream(keystore)) { + final byte[] buffer = new byte[4096]; + int readBytes; + while ((readBytes = is.read(buffer)) > 0) { + os.write(buffer, 0, readBytes); + } + } + final KeyStoreWrapper wrapper = KeyStoreWrapper.load(configDir); + assertNotNull(wrapper); + wrapper.decrypt(new char[0]); + assertThat(wrapper.getFormatVersion(), equalTo(3)); + assertThat(wrapper.getSettingNames(), equalTo(Set.of("keystore.seed", "string_setting", "file_setting"))); + assertThat(wrapper.getString("string_setting"), equalTo("string_value")); + assertThat(toByteArray(wrapper.getFile("string_setting")), equalTo("string_value".getBytes(StandardCharsets.UTF_8))); + assertThat(wrapper.getString("file_setting"), equalTo("file_value")); + assertThat(toByteArray(wrapper.getFile("file_setting")), equalTo("file_value".getBytes(StandardCharsets.UTF_8))); + } + + private byte[] toByteArray(final InputStream is) throws IOException { + final ByteArrayOutputStream os = new ByteArrayOutputStream(); + final byte[] buffer = new byte[1024]; + int readBytes; + while ((readBytes = is.read(buffer)) > 0) { + os.write(buffer, 0, readBytes); + } + return os.toByteArray(); + } + } diff --git a/distribution/tools/keystore-cli/src/test/resources/format-v3-elasticsearch.keystore b/distribution/tools/keystore-cli/src/test/resources/format-v3-elasticsearch.keystore new file mode 100644 index 0000000000000000000000000000000000000000..6b845c7e9d6ff6cbabedde30752099b2b994f989 GIT binary patch literal 281 zcmV+!0p|Wc*K8LSWo%({bZKLAWnpq-XfA7Id2@7ca%BJj00RI30089x002Ntv-qD1 zmkfRi))WE=kY>vC6t{Qw7$S{60IQ~!vy&FM%i#Q5KN>fk zGo&SID$<_-000d3ysvhOr@n6v%jWz50051=^)9`Ak`~r$zP%VRNr;O>-V%u8pDZsa zoLT^J(>XVVr`OeSRlv-`lBc-nx#SWMxZ>;|?cFr&?6#MG+x3gP1wek@+bg^kYARw) z0zMwoL2e6gB*M(=lBFO8vgFUYYHhu+4%!a1YPdTtr09)c6q0kWiAL-tbPwM~iH7^4 fu{|@Y?>nU$?$G;0z$laG0000000000+(hJ`?L&c9 literal 0 HcmV?d00001 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 e3fbf30a47ab5..2ae90a868af1b 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java +++ b/server/src/main/java/org/elasticsearch/common/settings/KeyStoreWrapper.java @@ -19,6 +19,18 @@ package org.elasticsearch.common.settings; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.BufferedChecksumIndexInput; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.SimpleFSDirectory; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.cli.ExitCodes; +import org.elasticsearch.cli.UserException; +import org.elasticsearch.common.Randomness; + import javax.crypto.Cipher; import javax.crypto.CipherInputStream; import javax.crypto.CipherOutputStream; @@ -27,6 +39,7 @@ import javax.crypto.spec.GCMParameterSpec; import javax.crypto.spec.PBEKeySpec; import javax.crypto.spec.SecretKeySpec; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -56,18 +69,6 @@ import java.util.Set; import java.util.regex.Pattern; -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.store.BufferedChecksumIndexInput; -import org.apache.lucene.store.ChecksumIndexInput; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.SimpleFSDirectory; -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.cli.ExitCodes; -import org.elasticsearch.cli.UserException; -import org.elasticsearch.common.Randomness; - /** * A disk based container for sensitive settings in Elasticsearch. * @@ -84,17 +85,6 @@ private enum EntryType { FILE } - /** An entry in the keystore. The bytes are opaque and interpreted based on the entry type. */ - private static class Entry { - final EntryType type; - final byte[] bytes; - - Entry(EntryType type, byte[] bytes) { - this.type = type; - this.bytes = bytes; - } - } - /** * A regex for the valid characters that a setting name in the keystore may use. */ @@ -110,7 +100,7 @@ private static class Entry { private static final String KEYSTORE_FILENAME = "elasticsearch.keystore"; /** The version of the metadata written before the keystore data. */ - private static final int FORMAT_VERSION = 3; + private static final int FORMAT_VERSION = 4; /** The oldest metadata format version that can be read. */ private static final int MIN_FORMAT_VERSION = 1; @@ -146,6 +136,7 @@ private static class Entry { // 1: initial version, ES 5.3 // 2: file setting, ES 5.4 // 3: FIPS compliant algos, ES 6.3 + // 4: remove distinction between string/files, ES 6.8/7.1 /** The metadata format version used to read the current keystore wrapper. */ private final int formatVersion; @@ -157,7 +148,7 @@ private static class Entry { private final byte[] dataBytes; /** The decrypted secret data. See {@link #decrypt(char[])}. */ - private final SetOnce> entries = new SetOnce<>(); + private final SetOnce> entries = new SetOnce<>(); private volatile boolean closed; private KeyStoreWrapper(int formatVersion, boolean hasPassword, byte[] dataBytes) { @@ -273,11 +264,13 @@ public static KeyStoreWrapper load(Path configDir) throws IOException { /** Upgrades the format of the keystore, if necessary. */ public static void upgrade(KeyStoreWrapper wrapper, Path configDir, char[] password) throws Exception { - // ensure keystore.seed exists - if (wrapper.getSettingNames().contains(SEED_SETTING.getKey())) { + if (wrapper.getFormatVersion() == FORMAT_VERSION && wrapper.getSettingNames().contains(SEED_SETTING.getKey())) { return; } - addBootstrapSeed(wrapper); + // add keystore.seed if necessary + if (wrapper.getSettingNames().contains(SEED_SETTING.getKey()) == false) { + addBootstrapSeed(wrapper); + } wrapper.save(configDir, password); } @@ -350,11 +343,14 @@ public void decrypt(char[] password) throws GeneralSecurityException, IOExceptio int numEntries = input.readInt(); while (numEntries-- > 0) { String setting = input.readUTF(); - EntryType entryType = EntryType.valueOf(input.readUTF()); + if (formatVersion == 3) { + // legacy, the keystore format would previously store the entry type + input.readUTF(); + } int entrySize = input.readInt(); byte[] entryBytes = new byte[entrySize]; input.readFully(entryBytes); - entries.get().put(setting, new Entry(entryType, entryBytes)); + entries.get().put(setting, entryBytes); } if (input.read() != -1) { throw new SecurityException("Keystore has been corrupted or tampered with"); @@ -373,12 +369,11 @@ private byte[] encrypt(char[] password, byte[] salt, byte[] iv) throws GeneralSe try (CipherOutputStream cipherStream = new CipherOutputStream(bytes, cipher); DataOutputStream output = new DataOutputStream(cipherStream)) { output.writeInt(entries.get().size()); - for (Map.Entry mapEntry : entries.get().entrySet()) { + for (Map.Entry mapEntry : entries.get().entrySet()) { output.writeUTF(mapEntry.getKey()); - Entry entry = mapEntry.getValue(); - output.writeUTF(entry.type.name()); - output.writeInt(entry.bytes.length); - output.write(entry.bytes); + byte[] entry = mapEntry.getValue(); + output.writeInt(entry.length); + output.write(entry); } } return bytes.toByteArray(); @@ -453,7 +448,7 @@ private void decryptLegacyEntries() throws GeneralSecurityException, IOException } Arrays.fill(chars, '\0'); - entries.get().put(setting, new Entry(settingType, bytes)); + entries.get().put(setting, bytes); } } @@ -526,11 +521,8 @@ public Set getSettingNames() { @Override public synchronized SecureString getString(String setting) { ensureOpen(); - Entry entry = entries.get().get(setting); - if (entry == null || entry.type != EntryType.STRING) { - throw new IllegalArgumentException("Secret setting " + setting + " is not a string"); - } - ByteBuffer byteBuffer = ByteBuffer.wrap(entry.bytes); + byte[] entry = entries.get().get(setting); + ByteBuffer byteBuffer = ByteBuffer.wrap(entry); CharBuffer charBuffer = StandardCharsets.UTF_8.decode(byteBuffer); return new SecureString(Arrays.copyOfRange(charBuffer.array(), charBuffer.position(), charBuffer.limit())); } @@ -538,11 +530,8 @@ public synchronized SecureString getString(String setting) { @Override public synchronized InputStream getFile(String setting) { ensureOpen(); - Entry entry = entries.get().get(setting); - if (entry == null || entry.type != EntryType.FILE) { - throw new IllegalArgumentException("Secret setting " + setting + " is not a file"); - } - return new ByteArrayInputStream(entry.bytes); + byte[] entry = entries.get().get(setting); + return new ByteArrayInputStream(entry); } /** @@ -564,9 +553,9 @@ synchronized void setString(String setting, char[] value) { ByteBuffer byteBuffer = StandardCharsets.UTF_8.encode(CharBuffer.wrap(value)); byte[] bytes = Arrays.copyOfRange(byteBuffer.array(), byteBuffer.position(), byteBuffer.limit()); - Entry oldEntry = entries.get().put(setting, new Entry(EntryType.STRING, bytes)); + byte[] oldEntry = entries.get().put(setting, bytes); if (oldEntry != null) { - Arrays.fill(oldEntry.bytes, (byte)0); + Arrays.fill(oldEntry, (byte)0); } } @@ -575,18 +564,18 @@ synchronized void setFile(String setting, byte[] bytes) { ensureOpen(); validateSettingName(setting); - Entry oldEntry = entries.get().put(setting, new Entry(EntryType.FILE, Arrays.copyOf(bytes, bytes.length))); + byte[] oldEntry = entries.get().put(setting, Arrays.copyOf(bytes, bytes.length)); if (oldEntry != null) { - Arrays.fill(oldEntry.bytes, (byte)0); + Arrays.fill(oldEntry, (byte)0); } } /** Remove the given setting from the keystore. */ void remove(String setting) { ensureOpen(); - Entry oldEntry = entries.get().remove(setting); + byte[] oldEntry = entries.get().remove(setting); if (oldEntry != null) { - Arrays.fill(oldEntry.bytes, (byte)0); + Arrays.fill(oldEntry, (byte)0); } } @@ -601,8 +590,8 @@ private void ensureOpen() { public synchronized void close() { this.closed = true; if (null != entries.get() && entries.get().isEmpty() == false) { - for (Entry entry : entries.get().values()) { - Arrays.fill(entry.bytes, (byte) 0); + for (byte[] entry : entries.get().values()) { + Arrays.fill(entry, (byte) 0); } } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/FIPS140SecureSettingsBootstrapCheck.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/FIPS140SecureSettingsBootstrapCheck.java index a8cb32d545e8d..82a58b94a83fe 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/FIPS140SecureSettingsBootstrapCheck.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/FIPS140SecureSettingsBootstrapCheck.java @@ -36,7 +36,7 @@ public BootstrapCheckResult check(BootstrapContext context) { if (fipsModeEnabled) { try (KeyStoreWrapper secureSettings = KeyStoreWrapper.load(environment.configFile())) { if (secureSettings != null && secureSettings.getFormatVersion() < 3) { - return BootstrapCheckResult.failure("Secure settings store is not of the latest version. Please use " + + return BootstrapCheckResult.failure("Secure settings store is not of the appropriate version. Please use " + "bin/elasticsearch-keystore create to generate a new secure settings store and migrate the secure settings there."); } } catch (IOException e) { From 7639e6a39862cc49bb34c8d20b1bbe22424c9c6e Mon Sep 17 00:00:00 2001 From: James Rodewig Date: Wed, 1 May 2019 09:28:12 -0400 Subject: [PATCH 257/260] [DOCS] Reword `type` query deprecation note for Asciidoctor migration (#41595) --- docs/reference/query-dsl/type-query.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/query-dsl/type-query.asciidoc b/docs/reference/query-dsl/type-query.asciidoc index 9825c68c74f73..4364d1e14e90d 100644 --- a/docs/reference/query-dsl/type-query.asciidoc +++ b/docs/reference/query-dsl/type-query.asciidoc @@ -1,7 +1,7 @@ [[query-dsl-type-query]] === Type Query -deprecated[7.0.0, Types are being removed, prefer filtering on a field instead. For more information, please see <>.] +deprecated[7.0.0,Types and the `type` query are deprecated and in the process of being removed. See <>.] Filters documents matching the provided document / mapping type. From 80f89432e5d3d02a8f834a67b4751b1183abec46 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 1 May 2019 10:22:35 -0500 Subject: [PATCH 258/260] [ML] fix array oob in IDGenerator and adjust format for mapping (#41703) * [ML] fix array oob in IDGenerator and adjust format for mapping * Update DataFramePivotRestIT.java --- .../integration/DataFramePivotRestIT.java | 9 +++---- .../integration/DataFrameRestTestCase.java | 12 ++++++--- .../dataframe/persistence/DataframeIndex.java | 27 ++++++++++++++++--- .../dataframe/transforms/IDGenerator.java | 5 +++- .../transforms/IDGeneratorTests.java | 3 +++ 5 files changed, 44 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java index 3a273cde27e19..75e179e5dee4c 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFramePivotRestIT.java @@ -214,9 +214,9 @@ public void testDateHistogramPivot() throws Exception { config += " \"pivot\": {" + " \"group_by\": {" - + " \"by_day\": {" + + " \"by_hr\": {" + " \"date_histogram\": {" - + " \"interval\": \"1d\",\"field\":\"timestamp\",\"format\":\"yyyy-MM-DD\"" + + " \"interval\": \"1h\",\"field\":\"timestamp\",\"format\":\"yyyy-MM-DD_HH\"" + " } } }," + " \"aggregations\": {" + " \"avg_rating\": {" @@ -232,10 +232,9 @@ public void testDateHistogramPivot() throws Exception { startAndWaitForTransform(transformId, dataFrameIndex, BASIC_AUTH_VALUE_DATA_FRAME_ADMIN_WITH_SOME_DATA_ACCESS); assertTrue(indexExists(dataFrameIndex)); - // we expect 21 documents as there shall be 21 days worth of docs Map indexStats = getAsMap(dataFrameIndex + "/_stats"); - assertEquals(21, XContentMapValues.extractValue("_all.total.docs.count", indexStats)); - assertOnePivotValue(dataFrameIndex + "/_search?q=by_day:2017-01-15", 3.82); + assertEquals(104, XContentMapValues.extractValue("_all.total.docs.count", indexStats)); + assertOnePivotValue(dataFrameIndex + "/_search?q=by_hr:1484499600000", 4.0833333333); } @SuppressWarnings("unchecked") diff --git a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java index c0e6c97fd6915..89047219f401d 100644 --- a/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java +++ b/x-pack/plugin/data-frame/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/dataframe/integration/DataFrameRestTestCase.java @@ -89,14 +89,20 @@ protected void createReviewsIndex() throws IOException { // create index final StringBuilder bulk = new StringBuilder(); int day = 10; + int hour = 10; + int min = 10; for (int i = 0; i < numDocs; i++) { bulk.append("{\"index\":{\"_index\":\"" + REVIEWS_INDEX_NAME + "\"}}\n"); long user = Math.round(Math.pow(i * 31 % 1000, distributionTable[i % distributionTable.length]) % 27); int stars = distributionTable[(i * 33) % distributionTable.length]; long business = Math.round(Math.pow(user * stars, distributionTable[i % distributionTable.length]) % 13); - int hour = randomIntBetween(10, 20); - int min = randomIntBetween(30, 59); - int sec = randomIntBetween(30, 59); + if (i % 12 == 0) { + hour = 10 + (i % 13); + } + if (i % 5 == 0) { + min = 10 + (i % 49); + } + int sec = 10 + (i % 49); String date_string = "2017-01-" + day + "T" + hour + ":" + min + ":" + sec + "Z"; bulk.append("{\"user_id\":\"") diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataframeIndex.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataframeIndex.java index 9e05807974657..1c4c9ee0158a5 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataframeIndex.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/persistence/DataframeIndex.java @@ -18,6 +18,8 @@ import org.elasticsearch.xpack.core.dataframe.DataFrameField; import org.elasticsearch.xpack.core.dataframe.DataFrameMessages; import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.DateHistogramGroupSource; +import org.elasticsearch.xpack.core.dataframe.transforms.pivot.SingleGroupSource; import java.io.IOException; import java.util.Map; @@ -31,7 +33,9 @@ public final class DataframeIndex { private static final String PROPERTIES = "properties"; private static final String TYPE = "type"; + private static final String FORMAT = "format"; private static final String META = "_meta"; + private static final String DEFAULT_TIME_FORMAT = "strict_date_optional_time||epoch_millis"; private DataframeIndex() { } @@ -44,7 +48,9 @@ public static void createDestinationIndex(Client client, DataFrameTransformConfi request.settings(Settings.builder() // <1> .put("index.number_of_shards", 1).put("index.number_of_replicas", 0)); - request.mapping(SINGLE_MAPPING_NAME, createMappingXContent(mappings, transformConfig.getId())); + request.mapping(SINGLE_MAPPING_NAME, createMappingXContent(mappings, + transformConfig.getPivotConfig().getGroupConfig().getGroups(), + transformConfig.getId())); client.execute(CreateIndexAction.INSTANCE, request, ActionListener.wrap(createIndexResponse -> { listener.onResponse(true); @@ -56,14 +62,29 @@ public static void createDestinationIndex(Client client, DataFrameTransformConfi })); } - private static XContentBuilder createMappingXContent(Map mappings, String id) { + private static XContentBuilder createMappingXContent(Map mappings, + Map groupSources, + String id) { try { XContentBuilder builder = jsonBuilder().startObject(); builder.startObject(SINGLE_MAPPING_NAME); addMetaData(builder, id); builder.startObject(PROPERTIES); for (Entry field : mappings.entrySet()) { - builder.startObject(field.getKey()).field(TYPE, field.getValue()).endObject(); + String fieldName = field.getKey(); + String fieldType = field.getValue(); + + builder.startObject(fieldName); + builder.field(TYPE, fieldType); + + SingleGroupSource groupSource = groupSources.get(fieldName); + if (groupSource instanceof DateHistogramGroupSource) { + String format = ((DateHistogramGroupSource) groupSource).getFormat(); + if (format != null) { + builder.field(FORMAT, DEFAULT_TIME_FORMAT + "||" + format); + } + } + builder.endObject(); } builder.endObject(); // properties builder.endObject(); // _doc type diff --git a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/IDGenerator.java b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/IDGenerator.java index d9223fe90dd4f..6d6f8455851fe 100644 --- a/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/IDGenerator.java +++ b/x-pack/plugin/data-frame/src/main/java/org/elasticsearch/xpack/dataframe/transforms/IDGenerator.java @@ -21,6 +21,7 @@ */ public final class IDGenerator { private static final byte[] NULL_VALUE = "__NULL_VALUE__".getBytes(StandardCharsets.UTF_8); + private static final byte[] EMPTY_VALUE = "__EMPTY_VALUE__".getBytes(StandardCharsets.UTF_8); private static final byte DELIM = '$'; private static final long SEED = 19; private static final int MAX_FIRST_BYTES = 5; @@ -57,7 +58,9 @@ public String getID() { for (Object value : objectsForIDGeneration.values()) { byte[] v = getBytes(value); - + if (v.length == 0) { + v = EMPTY_VALUE; + } buffer.append(v, 0, v.length); buffer.append(DELIM); diff --git a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/IDGeneratorTests.java b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/IDGeneratorTests.java index fd378a2c4c171..3ce5dd81558f9 100644 --- a/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/IDGeneratorTests.java +++ b/x-pack/plugin/data-frame/src/test/java/org/elasticsearch/xpack/dataframe/transforms/IDGeneratorTests.java @@ -27,6 +27,9 @@ public void testSupportedTypes() { assertNotEquals(id, idGen.getID()); idGen.add("key6", 13); assertNotEquals(id, idGen.getID()); + id = idGen.getID(); + idGen.add("key7", ""); + assertNotEquals(id, idGen.getID()); } public void testOrderIndependence() { From 5309c6cd3e3346bc2d47babc1ccb410ca4367524 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 1 May 2019 09:39:11 -0600 Subject: [PATCH 259/260] Support http read timeouts for transport-nio (#41466) This is related to #27260. Currently there is a setting http.read_timeout that allows users to define a read timeout for the http transport. This commit implements support for this functionality with the transport-nio plugin. The behavior here is that a repeating task will be scheduled for the interval defined. If there have been no requests received since the last run and there are no inflight requests, the channel will be closed. --- .../elasticsearch/nio/BytesWriteHandler.java | 3 + .../elasticsearch/nio/ReadWriteHandler.java | 5 + .../nio/SocketChannelContext.java | 1 + .../org/elasticsearch/nio/TaskScheduler.java | 2 +- .../netty4/Netty4HttpServerTransport.java | 9 +- .../Netty4HttpServerTransportTests.java | 8 +- .../http/nio/HttpReadWriteHandler.java | 47 +++++++- .../http/nio/NioHttpServerTransport.java | 2 +- .../http/nio/HttpReadWriteHandlerTests.java | 104 ++++++++++++------ .../elasticsearch/http/nio/NioHttpClient.java | 17 +++ .../http/nio/NioHttpServerTransportTests.java | 95 ++++++++-------- .../http/AbstractHttpServerTransport.java | 5 +- .../http/HttpHandlingSettings.java | 10 +- .../http/HttpReadTimeoutException.java | 31 ++++++ .../elasticsearch/threadpool/ThreadPool.java | 27 +++-- .../SecurityNetty4HttpServerTransport.java | 2 +- .../nio/SecurityNioHttpServerTransport.java | 2 +- 17 files changed, 263 insertions(+), 107 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/http/HttpReadTimeoutException.java diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java b/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java index 2d57faf5cb897..07333aa2eebc4 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java @@ -34,6 +34,9 @@ public WriteOperation createWriteOperation(SocketChannelContext context, Object return new FlushReadyWrite(context, (ByteBuffer[]) message, listener); } + @Override + public void channelRegistered() {} + @Override public List writeToBytes(WriteOperation writeOperation) { assert writeOperation instanceof FlushReadyWrite : "Write operation must be flush ready"; diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java b/libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java index 6b8688eccfd8c..92b276ad2d6da 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java @@ -28,6 +28,11 @@ */ public interface ReadWriteHandler { + /** + * This method is called when the channel is registered with its selector. + */ + void channelRegistered(); + /** * This method is called when a message is queued with a channel. It can be called from any thread. * This method should validate that the message is a valid type and return a write operation object diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java index 1444422f7a7f6..a926bbc9710d0 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java @@ -169,6 +169,7 @@ protected FlushOperation getPendingFlush() { @Override protected void register() throws IOException { super.register(); + readWriteHandler.channelRegistered(); if (allowChannelPredicate.test(channel) == false) { closeNow = true; } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java b/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java index e197230147c8b..c460e2147986a 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java @@ -45,7 +45,7 @@ public Runnable scheduleAtRelativeTime(Runnable task, long relativeNanos) { return delayedTask; } - Runnable pollTask(long relativeNanos) { + public Runnable pollTask(long relativeNanos) { DelayedTask task; while ((task = tasks.peek()) != null) { if (relativeNanos - task.deadline >= 0) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java index c0dc011a06c95..356cfa0bbf99d 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/http/netty4/Netty4HttpServerTransport.java @@ -45,7 +45,6 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.network.CloseableChannel; import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -59,6 +58,7 @@ import org.elasticsearch.http.AbstractHttpServerTransport; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpHandlingSettings; +import org.elasticsearch.http.HttpReadTimeoutException; import org.elasticsearch.http.HttpServerChannel; import org.elasticsearch.http.netty4.cors.Netty4CorsConfig; import org.elasticsearch.http.netty4.cors.Netty4CorsConfigBuilder; @@ -289,12 +289,9 @@ protected void stopInternal() { } @Override - protected void onException(HttpChannel channel, Exception cause) { + public void onException(HttpChannel channel, Exception cause) { if (cause instanceof ReadTimeoutException) { - if (logger.isTraceEnabled()) { - logger.trace("Http read timeout {}", channel); - } - CloseableChannel.closeChannel(channel); + super.onException(channel, new HttpReadTimeoutException(readTimeoutMillis, cause)); } else { super.onException(channel, cause); } diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java index 63e38823acb31..bc4ebe5672ec7 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/http/netty4/Netty4HttpServerTransportTests.java @@ -73,8 +73,8 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; @@ -346,7 +346,7 @@ public void dispatchBadRequest(final RestRequest request, transport.start(); final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); - AtomicBoolean channelClosed = new AtomicBoolean(false); + CountDownLatch channelClosedLatch = new CountDownLatch(1); Bootstrap clientBootstrap = new Bootstrap().channel(NioSocketChannel.class).handler(new ChannelInitializer() { @@ -357,9 +357,9 @@ protected void initChannel(SocketChannel ch) { } }).group(group); ChannelFuture connect = clientBootstrap.connect(remoteAddress.address()); - connect.channel().closeFuture().addListener(future -> channelClosed.set(true)); + connect.channel().closeFuture().addListener(future -> channelClosedLatch.countDown()); - assertBusy(() -> assertTrue("Channel should be closed due to read timeout", channelClosed.get()), 5, TimeUnit.SECONDS); + assertTrue("Channel should be closed due to read timeout", channelClosedLatch.await(1, TimeUnit.MINUTES)); } finally { group.shutdownGracefully().await(); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java index 17a5c1fb97e80..7a4fbfe42aefa 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java @@ -30,31 +30,45 @@ import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpResponseEncoder; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.http.HttpHandlingSettings; import org.elasticsearch.http.HttpPipelinedRequest; +import org.elasticsearch.http.HttpReadTimeoutException; import org.elasticsearch.http.nio.cors.NioCorsConfig; import org.elasticsearch.http.nio.cors.NioCorsHandler; import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.ReadWriteHandler; import org.elasticsearch.nio.SocketChannelContext; +import org.elasticsearch.nio.TaskScheduler; import org.elasticsearch.nio.WriteOperation; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.function.LongSupplier; public class HttpReadWriteHandler implements ReadWriteHandler { private final NettyAdaptor adaptor; private final NioHttpChannel nioHttpChannel; private final NioHttpServerTransport transport; + private final TaskScheduler taskScheduler; + private final LongSupplier nanoClock; + private final long readTimeoutNanos; + private boolean channelRegistered = false; + private boolean requestSinceReadTimeoutTrigger = false; + private int inFlightRequests = 0; public HttpReadWriteHandler(NioHttpChannel nioHttpChannel, NioHttpServerTransport transport, HttpHandlingSettings settings, - NioCorsConfig corsConfig) { + NioCorsConfig corsConfig, TaskScheduler taskScheduler, LongSupplier nanoClock) { this.nioHttpChannel = nioHttpChannel; this.transport = transport; + this.taskScheduler = taskScheduler; + this.nanoClock = nanoClock; + this.readTimeoutNanos = TimeUnit.MILLISECONDS.toNanos(settings.getReadTimeoutMillis()); List handlers = new ArrayList<>(5); HttpRequestDecoder decoder = new HttpRequestDecoder(settings.getMaxInitialLineLength(), settings.getMaxHeaderSize(), @@ -77,10 +91,21 @@ public HttpReadWriteHandler(NioHttpChannel nioHttpChannel, NioHttpServerTranspor } @Override - public int consumeReads(InboundChannelBuffer channelBuffer) throws IOException { + public void channelRegistered() { + channelRegistered = true; + if (readTimeoutNanos > 0) { + scheduleReadTimeout(); + } + } + + @Override + public int consumeReads(InboundChannelBuffer channelBuffer) { + assert channelRegistered : "channelRegistered should have been called"; int bytesConsumed = adaptor.read(channelBuffer.sliceAndRetainPagesTo(channelBuffer.getIndex())); Object message; while ((message = adaptor.pollInboundMessage()) != null) { + ++inFlightRequests; + requestSinceReadTimeoutTrigger = true; handleRequest(message); } @@ -96,6 +121,11 @@ public WriteOperation createWriteOperation(SocketChannelContext context, Object @Override public List writeToBytes(WriteOperation writeOperation) { + assert writeOperation.getObject() instanceof NioHttpResponse : "This channel only supports messages that are of type: " + + NioHttpResponse.class + ". Found type: " + writeOperation.getObject().getClass() + "."; + assert channelRegistered : "channelRegistered should have been called"; + --inFlightRequests; + assert inFlightRequests >= 0 : "Inflight requests should never drop below zero, found: " + inFlightRequests; adaptor.write(writeOperation); return pollFlushOperations(); } @@ -152,4 +182,17 @@ private void handleRequest(Object msg) { request.release(); } } + + private void maybeReadTimeout() { + if (requestSinceReadTimeoutTrigger == false && inFlightRequests == 0) { + transport.onException(nioHttpChannel, new HttpReadTimeoutException(TimeValue.nsecToMSec(readTimeoutNanos))); + } else { + requestSinceReadTimeoutTrigger = false; + scheduleReadTimeout(); + } + } + + private void scheduleReadTimeout() { + taskScheduler.scheduleAtRelativeTime(this::maybeReadTimeout, nanoClock.getAsLong() + readTimeoutNanos); + } } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java index 57936ff70c628..2730cb6d3a9b3 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpServerTransport.java @@ -211,7 +211,7 @@ public NioHttpChannel createChannel(NioSelector selector, SocketChannel channel) return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; HttpReadWriteHandler httpReadWritePipeline = new HttpReadWriteHandler(httpChannel,NioHttpServerTransport.this, - handlingSettings, corsConfig); + handlingSettings, corsConfig, selector.getTaskScheduler(), threadPool::relativeTimeInMillis); Consumer exceptionHandler = (e) -> onException(httpChannel, e); SocketChannelContext context = new BytesChannelContext(httpChannel, selector, exceptionHandler, httpReadWritePipeline, new InboundChannelBuffer(pageSupplier)); diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java index d7e61f21173bb..3d136705ef9d9 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/HttpReadWriteHandlerTests.java @@ -33,12 +33,13 @@ import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.HttpVersion; - import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpHandlingSettings; +import org.elasticsearch.http.HttpReadTimeoutException; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.http.HttpResponse; import org.elasticsearch.http.HttpTransportSettings; @@ -48,6 +49,7 @@ import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.SocketChannelContext; +import org.elasticsearch.nio.TaskScheduler; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; @@ -56,6 +58,8 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.function.BiConsumer; @@ -63,19 +67,14 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_ORIGIN; import static org.elasticsearch.http.HttpTransportSettings.SETTING_CORS_ENABLED; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_COMPRESSION_LEVEL; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_DETAILED_ERRORS_ENABLED; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CHUNK_SIZE; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_RESET_COOKIES; -import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -84,8 +83,9 @@ public class HttpReadWriteHandlerTests extends ESTestCase { private HttpReadWriteHandler handler; - private NioHttpChannel nioHttpChannel; + private NioHttpChannel channel; private NioHttpServerTransport transport; + private TaskScheduler taskScheduler; private final RequestEncoder requestEncoder = new RequestEncoder(); private final ResponseDecoder responseDecoder = new ResponseDecoder(); @@ -93,22 +93,14 @@ public class HttpReadWriteHandlerTests extends ESTestCase { @Before public void setMocks() { transport = mock(NioHttpServerTransport.class); - Settings settings = Settings.EMPTY; - ByteSizeValue maxChunkSize = SETTING_HTTP_MAX_CHUNK_SIZE.getDefault(settings); - ByteSizeValue maxHeaderSize = SETTING_HTTP_MAX_HEADER_SIZE.getDefault(settings); - ByteSizeValue maxInitialLineLength = SETTING_HTTP_MAX_INITIAL_LINE_LENGTH.getDefault(settings); - HttpHandlingSettings httpHandlingSettings = new HttpHandlingSettings(1024, - Math.toIntExact(maxChunkSize.getBytes()), - Math.toIntExact(maxHeaderSize.getBytes()), - Math.toIntExact(maxInitialLineLength.getBytes()), - SETTING_HTTP_RESET_COOKIES.getDefault(settings), - SETTING_HTTP_COMPRESSION.getDefault(settings), - SETTING_HTTP_COMPRESSION_LEVEL.getDefault(settings), - SETTING_HTTP_DETAILED_ERRORS_ENABLED.getDefault(settings), - SETTING_PIPELINING_MAX_EVENTS.getDefault(settings), - SETTING_CORS_ENABLED.getDefault(settings)); - nioHttpChannel = mock(NioHttpChannel.class); - handler = new HttpReadWriteHandler(nioHttpChannel, transport, httpHandlingSettings, NioCorsConfigBuilder.forAnyOrigin().build()); + Settings settings = Settings.builder().put(SETTING_HTTP_MAX_CONTENT_LENGTH.getKey(), new ByteSizeValue(1024)).build(); + HttpHandlingSettings httpHandlingSettings = HttpHandlingSettings.fromSettings(settings); + channel = mock(NioHttpChannel.class); + taskScheduler = mock(TaskScheduler.class); + + NioCorsConfig corsConfig = NioCorsConfigBuilder.forAnyOrigin().build(); + handler = new HttpReadWriteHandler(channel, transport, httpHandlingSettings, corsConfig, taskScheduler, System::nanoTime); + handler.channelRegistered(); } public void testSuccessfulDecodeHttpRequest() throws IOException { @@ -188,7 +180,7 @@ public void testDecodeHttpRequestContentLengthToLongGeneratesOutboundMessage() t flushOperation.getListener().accept(null, null); // Since we have keep-alive set to false, we should close the channel after the response has been // flushed - verify(nioHttpChannel).close(); + verify(channel).close(); } finally { response.release(); } @@ -335,10 +327,59 @@ public void testThatAnyOriginWorks() throws IOException { } } - private FullHttpResponse executeCorsRequest(final Settings settings, final String originValue, final String host) throws IOException { + @SuppressWarnings("unchecked") + public void testReadTimeout() throws IOException { + TimeValue timeValue = TimeValue.timeValueMillis(500); + Settings settings = Settings.builder().put(SETTING_HTTP_READ_TIMEOUT.getKey(), timeValue).build(); HttpHandlingSettings httpHandlingSettings = HttpHandlingSettings.fromSettings(settings); - NioCorsConfig nioCorsConfig = NioHttpServerTransport.buildCorsConfig(settings); - HttpReadWriteHandler handler = new HttpReadWriteHandler(nioHttpChannel, transport, httpHandlingSettings, nioCorsConfig); + DefaultFullHttpRequest nettyRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/"); + NioHttpRequest nioHttpRequest = new NioHttpRequest(nettyRequest, 0); + NioHttpResponse httpResponse = nioHttpRequest.createResponse(RestStatus.OK, BytesArray.EMPTY); + httpResponse.addHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), "0"); + + NioCorsConfig corsConfig = NioCorsConfigBuilder.forAnyOrigin().build(); + TaskScheduler taskScheduler = new TaskScheduler(); + + Iterator timeValues = Arrays.asList(0, 2, 4, 6, 8).iterator(); + handler = new HttpReadWriteHandler(channel, transport, httpHandlingSettings, corsConfig, taskScheduler, timeValues::next); + handler.channelRegistered(); + + prepareHandlerForResponse(handler); + SocketChannelContext context = mock(SocketChannelContext.class); + HttpWriteOperation writeOperation = new HttpWriteOperation(context, httpResponse, mock(BiConsumer.class)); + handler.writeToBytes(writeOperation); + + taskScheduler.pollTask(timeValue.getNanos() + 1).run(); + // There was a read. Do not close. + verify(transport, times(0)).onException(eq(channel), any(HttpReadTimeoutException.class)); + + prepareHandlerForResponse(handler); + prepareHandlerForResponse(handler); + + taskScheduler.pollTask(timeValue.getNanos() + 3).run(); + // There was a read. Do not close. + verify(transport, times(0)).onException(eq(channel), any(HttpReadTimeoutException.class)); + + handler.writeToBytes(writeOperation); + + taskScheduler.pollTask(timeValue.getNanos() + 5).run(); + // There has not been a read, however there is still an inflight request. Do not close. + verify(transport, times(0)).onException(eq(channel), any(HttpReadTimeoutException.class)); + + handler.writeToBytes(writeOperation); + + taskScheduler.pollTask(timeValue.getNanos() + 7).run(); + // No reads and no inflight requests, close + verify(transport, times(1)).onException(eq(channel), any(HttpReadTimeoutException.class)); + assertNull(taskScheduler.pollTask(timeValue.getNanos() + 9)); + } + + private FullHttpResponse executeCorsRequest(final Settings settings, final String originValue, final String host) throws IOException { + HttpHandlingSettings httpSettings = HttpHandlingSettings.fromSettings(settings); + NioCorsConfig corsConfig = NioHttpServerTransport.buildCorsConfig(settings); + HttpReadWriteHandler handler = new HttpReadWriteHandler(channel, transport, httpSettings, corsConfig, taskScheduler, + System::nanoTime); + handler.channelRegistered(); prepareHandlerForResponse(handler); DefaultFullHttpRequest httpRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/"); if (originValue != null) { @@ -360,7 +401,7 @@ private FullHttpResponse executeCorsRequest(final Settings settings, final Strin - private NioHttpRequest prepareHandlerForResponse(HttpReadWriteHandler handler) throws IOException { + private void prepareHandlerForResponse(HttpReadWriteHandler handler) throws IOException { HttpMethod method = randomBoolean() ? HttpMethod.GET : HttpMethod.HEAD; HttpVersion version = randomBoolean() ? HttpVersion.HTTP_1_0 : HttpVersion.HTTP_1_1; String uri = "http://localhost:9090/" + randomAlphaOfLength(8); @@ -385,7 +426,6 @@ private NioHttpRequest prepareHandlerForResponse(HttpReadWriteHandler handler) t assertEquals(HttpRequest.HttpVersion.HTTP_1_0, nioHttpRequest.protocolVersion()); } assertEquals(nioHttpRequest.uri(), uri); - return nioHttpRequest; } private InboundChannelBuffer toChannelBuffer(ByteBuf buf) { diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java index e3259b10b9745..634ea7b44af74 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpClient.java @@ -116,6 +116,20 @@ public final FullHttpResponse post(InetSocketAddress remoteAddress, FullHttpRequ return responses.iterator().next(); } + public final NioSocketChannel connect(InetSocketAddress remoteAddress) { + ChannelFactory factory = new ClientChannelFactory(new CountDownLatch(0), new + ArrayList<>()); + try { + NioSocketChannel nioSocketChannel = nioGroup.openChannel(remoteAddress, factory); + PlainActionFuture connectFuture = PlainActionFuture.newFuture(); + nioSocketChannel.addConnectListener(ActionListener.toBiConsumer(connectFuture)); + connectFuture.actionGet(); + return nioSocketChannel; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + private void onException(Exception e) { logger.error("Exception from http client", e); } @@ -212,6 +226,9 @@ private HttpClientHandler(NioSocketChannel channel, CountDownLatch latch, Collec adaptor.addCloseListener((v, e) -> channel.close()); } + @Override + public void channelRegistered() {} + @Override public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener) { assert message instanceof HttpRequest : "Expected type HttpRequest.class, found: " + message.getClass(); diff --git a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java index 2ffd5a64147cc..0b470fda00a6b 100644 --- a/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java +++ b/plugins/transport-nio/src/test/java/org/elasticsearch/http/nio/NioHttpServerTransportTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.MockPageCacheRecycler; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -49,6 +50,7 @@ import org.elasticsearch.http.NullDispatcher; import org.elasticsearch.http.nio.cors.NioCorsConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.nio.NioSocketChannel; import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; @@ -66,6 +68,8 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; @@ -309,52 +313,47 @@ threadPool, xContentRegistry(), dispatcher, new NioGroupFactory(settings, logger assertThat(causeReference.get(), instanceOf(TooLongFrameException.class)); } -// public void testReadTimeout() throws Exception { -// final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { -// -// @Override -// public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) { -// throw new AssertionError("Should not have received a dispatched request"); -// } -// -// @Override -// public void dispatchBadRequest(final RestRequest request, -// final RestChannel channel, -// final ThreadContext threadContext, -// final Throwable cause) { -// throw new AssertionError("Should not have received a dispatched request"); -// } -// -// }; -// -// Settings settings = Settings.builder() -// .put(HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT.getKey(), new TimeValue(randomIntBetween(100, 300))) -// .build(); -// -// -// NioEventLoopGroup group = new NioEventLoopGroup(); -// try (NioHttpServerTransport transport = -// new NioHttpServerTransport(settings, networkService, bigArrays, threadPool, xContentRegistry(), dispatcher)) { -// transport.start(); -// final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); -// -// AtomicBoolean channelClosed = new AtomicBoolean(false); -// -// Bootstrap clientBootstrap = new Bootstrap().channel(NioSocketChannel.class).handler(new ChannelInitializer() { -// -// @Override -// protected void initChannel(SocketChannel ch) { -// ch.pipeline().addLast(new ChannelHandlerAdapter() {}); -// -// } -// }).group(group); -// ChannelFuture connect = clientBootstrap.connect(remoteAddress.address()); -// connect.channel().closeFuture().addListener(future -> channelClosed.set(true)); -// -// assertBusy(() -> assertTrue("Channel should be closed due to read timeout", channelClosed.get()), 5, TimeUnit.SECONDS); -// -// } finally { -// group.shutdownGracefully().await(); -// } -// } + public void testReadTimeout() throws Exception { + final HttpServerTransport.Dispatcher dispatcher = new HttpServerTransport.Dispatcher() { + + @Override + public void dispatchRequest(final RestRequest request, final RestChannel channel, final ThreadContext threadContext) { + throw new AssertionError("Should not have received a dispatched request"); + } + + @Override + public void dispatchBadRequest(final RestRequest request, + final RestChannel channel, + final ThreadContext threadContext, + final Throwable cause) { + throw new AssertionError("Should not have received a dispatched request"); + } + + }; + + Settings settings = Settings.builder() + .put(HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT.getKey(), new TimeValue(randomIntBetween(100, 300))) + .build(); + + + try (NioHttpServerTransport transport = new NioHttpServerTransport(settings, networkService, bigArrays, pageRecycler, + threadPool, xContentRegistry(), dispatcher, new NioGroupFactory(settings, logger))) { + transport.start(); + final TransportAddress remoteAddress = randomFrom(transport.boundAddress().boundAddresses()); + + try (NioHttpClient client = new NioHttpClient()) { + NioSocketChannel channel = null; + try { + CountDownLatch channelClosedLatch = new CountDownLatch(1); + channel = client.connect(remoteAddress.address()); + channel.addCloseListener((r, t) -> channelClosedLatch.countDown()); + assertTrue("Channel should be closed due to read timeout", channelClosedLatch.await(1, TimeUnit.MINUTES)); + } finally { + if (channel != null) { + channel.close(); + } + } + } + } + } } diff --git a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java index b79a5e77309b9..ab6168cad522e 100644 --- a/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java +++ b/server/src/main/java/org/elasticsearch/http/AbstractHttpServerTransport.java @@ -249,7 +249,7 @@ static int resolvePublishPort(Settings settings, List boundAdd return publishPort; } - protected void onException(HttpChannel channel, Exception e) { + public void onException(HttpChannel channel, Exception e) { if (lifecycle.started() == false) { // just close and ignore - we are already stopped and just need to make sure we release all resources CloseableChannel.closeChannel(channel); @@ -263,6 +263,9 @@ protected void onException(HttpChannel channel, Exception e) { logger.trace(() -> new ParameterizedMessage( "connect exception caught while handling client http traffic, closing connection {}", channel), e); CloseableChannel.closeChannel(channel); + } else if (e instanceof HttpReadTimeoutException) { + logger.trace(() -> new ParameterizedMessage("http read timeout, closing connection {}", channel), e); + CloseableChannel.closeChannel(channel); } else if (e instanceof CancelledKeyException) { logger.trace(() -> new ParameterizedMessage( "cancelled key exception caught while handling client http traffic, closing connection {}", channel), e); diff --git a/server/src/main/java/org/elasticsearch/http/HttpHandlingSettings.java b/server/src/main/java/org/elasticsearch/http/HttpHandlingSettings.java index 568f2912a677c..805ebc3d95dd3 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpHandlingSettings.java +++ b/server/src/main/java/org/elasticsearch/http/HttpHandlingSettings.java @@ -29,6 +29,7 @@ import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_CONTENT_LENGTH; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_HEADER_SIZE; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_MAX_INITIAL_LINE_LENGTH; +import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_READ_TIMEOUT; import static org.elasticsearch.http.HttpTransportSettings.SETTING_HTTP_RESET_COOKIES; import static org.elasticsearch.http.HttpTransportSettings.SETTING_PIPELINING_MAX_EVENTS; @@ -43,11 +44,12 @@ public class HttpHandlingSettings { private final int compressionLevel; private final boolean detailedErrorsEnabled; private final int pipeliningMaxEvents; + private final long readTimeoutMillis; private boolean corsEnabled; public HttpHandlingSettings(int maxContentLength, int maxChunkSize, int maxHeaderSize, int maxInitialLineLength, boolean resetCookies, boolean compression, int compressionLevel, boolean detailedErrorsEnabled, - int pipeliningMaxEvents, boolean corsEnabled) { + int pipeliningMaxEvents, long readTimeoutMillis, boolean corsEnabled) { this.maxContentLength = maxContentLength; this.maxChunkSize = maxChunkSize; this.maxHeaderSize = maxHeaderSize; @@ -57,6 +59,7 @@ public HttpHandlingSettings(int maxContentLength, int maxChunkSize, int maxHeade this.compressionLevel = compressionLevel; this.detailedErrorsEnabled = detailedErrorsEnabled; this.pipeliningMaxEvents = pipeliningMaxEvents; + this.readTimeoutMillis = readTimeoutMillis; this.corsEnabled = corsEnabled; } @@ -70,6 +73,7 @@ public static HttpHandlingSettings fromSettings(Settings settings) { SETTING_HTTP_COMPRESSION_LEVEL.get(settings), SETTING_HTTP_DETAILED_ERRORS_ENABLED.get(settings), SETTING_PIPELINING_MAX_EVENTS.get(settings), + SETTING_HTTP_READ_TIMEOUT.get(settings).getMillis(), SETTING_CORS_ENABLED.get(settings)); } @@ -109,6 +113,10 @@ public int getPipeliningMaxEvents() { return pipeliningMaxEvents; } + public long getReadTimeoutMillis() { + return readTimeoutMillis; + } + public boolean isCorsEnabled() { return corsEnabled; } diff --git a/server/src/main/java/org/elasticsearch/http/HttpReadTimeoutException.java b/server/src/main/java/org/elasticsearch/http/HttpReadTimeoutException.java new file mode 100644 index 0000000000000..a4f54e92f34a6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/http/HttpReadTimeoutException.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.http; + +public class HttpReadTimeoutException extends RuntimeException { + + public HttpReadTimeoutException(long readTimeoutMillis) { + super("http read timeout after " + readTimeoutMillis + "ms"); + + } + + public HttpReadTimeoutException(long readTimeoutMillis, Exception cause) { + super("http read timeout after " + readTimeoutMillis + "ms", cause); + } +} diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 3468d3d30212b..a72b66de52845 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -227,7 +227,17 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui * timestamp, see {@link #absoluteTimeInMillis()}. */ public long relativeTimeInMillis() { - return cachedTimeThread.relativeTimeInMillis(); + return TimeValue.nsecToMSec(relativeTimeInNanos()); + } + + /** + * Returns a value of nanoseconds that may be used for relative time calculations. + * + * This method should only be used for calculating time deltas. For an epoch based + * timestamp, see {@link #absoluteTimeInMillis()}. + */ + public long relativeTimeInNanos() { + return cachedTimeThread.relativeTimeInNanos(); } /** @@ -490,30 +500,29 @@ static class CachedTimeThread extends Thread { final long interval; volatile boolean running = true; - volatile long relativeMillis; + volatile long relativeNanos; volatile long absoluteMillis; CachedTimeThread(String name, long interval) { super(name); this.interval = interval; - this.relativeMillis = TimeValue.nsecToMSec(System.nanoTime()); + this.relativeNanos = System.nanoTime(); this.absoluteMillis = System.currentTimeMillis(); setDaemon(true); } /** - * Return the current time used for relative calculations. This is - * {@link System#nanoTime()} truncated to milliseconds. + * Return the current time used for relative calculations. This is {@link System#nanoTime()}. *

    * If {@link ThreadPool#ESTIMATED_TIME_INTERVAL_SETTING} is set to 0 * then the cache is disabled and the method calls {@link System#nanoTime()} * whenever called. Typically used for testing. */ - long relativeTimeInMillis() { + long relativeTimeInNanos() { if (0 < interval) { - return relativeMillis; + return relativeNanos; } - return TimeValue.nsecToMSec(System.nanoTime()); + return System.nanoTime(); } /** @@ -534,7 +543,7 @@ long absoluteTimeInMillis() { @Override public void run() { while (running && 0 < interval) { - relativeMillis = TimeValue.nsecToMSec(System.nanoTime()); + relativeNanos = System.nanoTime(); absoluteMillis = System.currentTimeMillis(); try { Thread.sleep(interval); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransport.java index 596f9c1e6e107..043af216b8f35 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransport.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransport.java @@ -54,7 +54,7 @@ public SecurityNetty4HttpServerTransport(Settings settings, NetworkService netwo } @Override - protected void onException(HttpChannel channel, Exception e) { + public void onException(HttpChannel channel, Exception e) { securityExceptionHandler.accept(channel, e); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java index 8ecba16fa460d..b65f29eb95100 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransport.java @@ -98,7 +98,7 @@ public NioHttpChannel createChannel(NioSelector selector, SocketChannel channel) return new Page(ByteBuffer.wrap(bytes.v()), bytes::close); }; HttpReadWriteHandler httpHandler = new HttpReadWriteHandler(httpChannel,SecurityNioHttpServerTransport.this, - handlingSettings, corsConfig); + handlingSettings, corsConfig, selector.getTaskScheduler(), threadPool::relativeTimeInNanos); InboundChannelBuffer buffer = new InboundChannelBuffer(pageSupplier); Consumer exceptionHandler = (e) -> securityExceptionHandler.accept(httpChannel, e); From fc0266846f3119d443975924dd43519645c55d0b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 1 May 2019 18:07:14 +0200 Subject: [PATCH 260/260] Fix BulkProcessorRetryIT (#41700) * Now that we process the bulk requests themselves on the WRITE threadpool, they can run out of retries too like the item requests even when backoff is active * Fixes #41324 by using the same logic that checks failed item requests for their retry status for the top level bulk requests as well --- .../client/BulkProcessorRetryIT.java | 38 +++++++++++-------- .../action/bulk/BulkProcessorRetryIT.java | 38 +++++++++++-------- 2 files changed, 46 insertions(+), 30 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java index c18c4363897e7..77877c46f1aee 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java @@ -82,6 +82,7 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon @Override public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + internalPolicy.logResponse(failure); responses.add(failure); latch.countDown(); } @@ -105,16 +106,8 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) BulkItemResponse.Failure failure = bulkItemResponse.getFailure(); if (failure.getStatus() == RestStatus.TOO_MANY_REQUESTS) { if (rejectedExecutionExpected == false) { - Iterator backoffState = internalPolicy.backoffStateFor(bulkResponse); - assertNotNull("backoffState is null (indicates a bulk request got rejected without retry)", backoffState); - if (backoffState.hasNext()) { - // we're not expecting that we overwhelmed it even once when we maxed out the number of retries - throw new AssertionError("Got rejected although backoff policy would allow more retries", - failure.getCause()); - } else { - rejectedAfterAllRetries = true; - logger.debug("We maxed out the number of bulk retries and got rejected (this is ok)."); - } + assertRetriedCorrectly(internalPolicy, bulkResponse, failure.getCause()); + rejectedAfterAllRetries = true; } } else { throw new AssertionError("Unexpected failure with status: " + failure.getStatus()); @@ -123,8 +116,12 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) } } else { if (response instanceof RemoteTransportException - && ((RemoteTransportException) response).status() == RestStatus.TOO_MANY_REQUESTS && rejectedExecutionExpected) { - // ignored, we exceeded the write queue size with dispatching the initial bulk request + && ((RemoteTransportException) response).status() == RestStatus.TOO_MANY_REQUESTS) { + if (rejectedExecutionExpected == false) { + assertRetriedCorrectly(internalPolicy, response, ((Throwable) response).getCause()); + rejectedAfterAllRetries = true; + } + // ignored, we exceeded the write queue size when dispatching the initial bulk request } else { Throwable t = (Throwable) response; // we're not expecting any other errors @@ -146,6 +143,17 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) } + private void assertRetriedCorrectly(CorrelatingBackoffPolicy internalPolicy, Object bulkResponse, Throwable failure) { + Iterator backoffState = internalPolicy.backoffStateFor(bulkResponse); + assertNotNull("backoffState is null (indicates a bulk request got rejected without retry)", backoffState); + if (backoffState.hasNext()) { + // we're not expecting that we overwhelmed it even once when we maxed out the number of retries + throw new AssertionError("Got rejected although backoff policy would allow more retries", failure); + } else { + logger.debug("We maxed out the number of bulk retries and got rejected (this is ok)."); + } + } + private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) { MultiGetRequest multiGetRequest = new MultiGetRequest(); for (int i = 1; i <= numDocs; i++) { @@ -164,7 +172,7 @@ private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) { * as the last call to the backoff policy's iterator. The advantage is that this is non-invasive to the rest of the production code. */ private static class CorrelatingBackoffPolicy extends BackoffPolicy { - private final Map> correlations = new ConcurrentHashMap<>(); + private final Map> correlations = new ConcurrentHashMap<>(); // this is intentionally *not* static final. We will only ever have one instance of this class per test case and want the // thread local to be eligible for garbage collection right after the test to avoid leaks. private final ThreadLocal> iterators = new ThreadLocal<>(); @@ -175,13 +183,13 @@ private CorrelatingBackoffPolicy(BackoffPolicy delegate) { this.delegate = delegate; } - public Iterator backoffStateFor(BulkResponse response) { + public Iterator backoffStateFor(Object response) { return correlations.get(response); } // Assumption: This method is called from the same thread as the last call to the internal iterator's #hasNext() / #next() // see also Retry.AbstractRetryHandler#onResponse(). - public void logResponse(BulkResponse response) { + public void logResponse(Object response) { Iterator iterator = iterators.get(); // did we ever retry? if (iterator != null) { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index e4b6fff9fc353..e7285ff6f97ed 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -93,6 +93,7 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon @Override public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + internalPolicy.logResponse(failure); responses.add(failure); latch.countDown(); } @@ -117,16 +118,8 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) BulkItemResponse.Failure failure = bulkItemResponse.getFailure(); if (failure.getStatus() == RestStatus.TOO_MANY_REQUESTS) { if (rejectedExecutionExpected == false) { - Iterator backoffState = internalPolicy.backoffStateFor(bulkResponse); - assertNotNull("backoffState is null (indicates a bulk request got rejected without retry)", backoffState); - if (backoffState.hasNext()) { - // we're not expecting that we overwhelmed it even once when we maxed out the number of retries - throw new AssertionError("Got rejected although backoff policy would allow more retries", - failure.getCause()); - } else { - rejectedAfterAllRetries = true; - logger.debug("We maxed out the number of bulk retries and got rejected (this is ok)."); - } + assertRetriedCorrectly(internalPolicy, bulkResponse, failure.getCause()); + rejectedAfterAllRetries = true; } } else { throw new AssertionError("Unexpected failure status: " + failure.getStatus()); @@ -135,8 +128,12 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) } } else { if (response instanceof RemoteTransportException - && ((RemoteTransportException) response).status() == RestStatus.TOO_MANY_REQUESTS && rejectedExecutionExpected) { - // ignored, we exceeded the write queue size with dispatching the initial bulk request + && ((RemoteTransportException) response).status() == RestStatus.TOO_MANY_REQUESTS) { + if (rejectedExecutionExpected == false) { + assertRetriedCorrectly(internalPolicy, response, ((Throwable) response).getCause()); + rejectedAfterAllRetries = true; + } + // ignored, we exceeded the write queue size when dispatching the initial bulk request } else { Throwable t = (Throwable) response; // we're not expecting any other errors @@ -163,6 +160,17 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) } } + private void assertRetriedCorrectly(CorrelatingBackoffPolicy internalPolicy, Object bulkResponse, Throwable failure) { + Iterator backoffState = internalPolicy.backoffStateFor(bulkResponse); + assertNotNull("backoffState is null (indicates a bulk request got rejected without retry)", backoffState); + if (backoffState.hasNext()) { + // we're not expecting that we overwhelmed it even once when we maxed out the number of retries + throw new AssertionError("Got rejected although backoff policy would allow more retries", failure); + } else { + logger.debug("We maxed out the number of bulk retries and got rejected (this is ok)."); + } + } + private static void indexDocs(BulkProcessor processor, int numDocs) { for (int i = 1; i <= numDocs; i++) { processor.add(client() @@ -183,7 +191,7 @@ private static void indexDocs(BulkProcessor processor, int numDocs) { * as the last call to the backoff policy's iterator. The advantage is that this is non-invasive to the rest of the production code. */ private static class CorrelatingBackoffPolicy extends BackoffPolicy { - private final Map> correlations = new ConcurrentHashMap<>(); + private final Map> correlations = new ConcurrentHashMap<>(); // this is intentionally *not* static final. We will only ever have one instance of this class per test case and want the // thread local to be eligible for garbage collection right after the test to avoid leaks. private final ThreadLocal> iterators = new ThreadLocal<>(); @@ -194,13 +202,13 @@ private CorrelatingBackoffPolicy(BackoffPolicy delegate) { this.delegate = delegate; } - public Iterator backoffStateFor(BulkResponse response) { + public Iterator backoffStateFor(Object response) { return correlations.get(response); } // Assumption: This method is called from the same thread as the last call to the internal iterator's #hasNext() / #next() // see also Retry.AbstractRetryHandler#onResponse(). - public void logResponse(BulkResponse response) { + public void logResponse(Object response) { Iterator iterator = iterators.get(); // did we ever retry? if (iterator != null) {