From 45c6c2046729c49bb1cdacf35cadc0adc558e924 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 17 Apr 2018 08:03:41 -0400 Subject: [PATCH 01/16] Enforce translog access via engine (#29542) Today the translog of an engine is exposed and can be accessed directly. While this exposure offers much flexibility, it also causes these troubles: - Inconsistent behavior between translog method and engine method. For example, rolling a translog generation via an engine also trims unreferenced files, but translog's method does not. - An engine does not get notified when critical errors happen in translog as the access is direct. This change isolates translog of an engine and enforces all accesses to translog via the engine. --- .../org/elasticsearch/index/IndexService.java | 3 +- .../elasticsearch/index/engine/Engine.java | 62 ++++++++++++++++++- .../index/engine/InternalEngine.java | 2 +- .../seqno/GlobalCheckpointSyncAction.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 48 ++++++++++---- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/RefreshListeners.java | 10 +-- .../recovery/RecoverySourceHandler.java | 13 ++-- .../index/IndexServiceTests.java | 6 +- .../ESIndexLevelReplicationTestCase.java | 2 +- .../IndexLevelReplicationTests.java | 8 +-- .../RecoveryDuringReplicationTests.java | 6 +- .../GlobalCheckpointSyncActionTests.java | 5 +- .../index/shard/IndexShardIT.java | 11 ++-- .../index/shard/IndexShardTests.java | 25 ++++---- .../index/shard/RefreshListenersTests.java | 2 +- .../PeerRecoveryTargetServiceTests.java | 4 +- .../indices/recovery/RecoveryTests.java | 22 +++---- .../index/engine/EngineTestCase.java | 7 +++ .../index/shard/IndexShardTestCase.java | 6 ++ .../test/InternalTestCluster.java | 3 +- 21 files changed, 170 insertions(+), 79 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index db724112574a2..585406d01a6f6 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -697,8 +697,7 @@ private void maybeFSyncTranslogs() { if (indexSettings.getTranslogDurability() == Translog.Durability.ASYNC) { for (IndexShard shard : this.shards.values()) { try { - Translog translog = shard.getTranslog(); - if (translog.syncNeeded()) { + if (shard.isSyncNeeded()) { shard.sync(); } } catch (AlreadyClosedException ex) { 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 fab8cba468b56..4c782cb500418 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -66,6 +66,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogStats; import java.io.Closeable; import java.io.FileNotFoundException; @@ -510,8 +511,18 @@ public enum SearcherScope { EXTERNAL, INTERNAL } - /** returns the translog for this engine */ - public abstract Translog getTranslog(); + /** + * Returns the translog associated with this engine. + * Prefer to keep the translog package-private, so that an engine can control all accesses to the translog. + */ + abstract Translog getTranslog(); + + /** + * Checks if the underlying storage sync is required. + */ + public boolean isTranslogSyncNeeded() { + return getTranslog().syncNeeded(); + } /** * Ensures that all locations in the given stream have been written to the underlying storage. @@ -520,6 +531,36 @@ public enum SearcherScope { public abstract void syncTranslog() throws IOException; + public Closeable acquireTranslogRetentionLock() { + return getTranslog().acquireRetentionLock(); + } + + /** + * Creates a new translog snapshot from this engine for reading translog operations whose seq# at least the provided seq#. + * The caller has to close the returned snapshot after finishing the reading. + */ + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { + return getTranslog().newSnapshotFromMinSeqNo(minSeqNo); + } + + /** + * Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#. + */ + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo); + } + + public TranslogStats getTranslogStats() { + return getTranslog().stats(); + } + + /** + * Returns the last location that the translog of this engine has written into. + */ + public Translog.Location getTranslogLastWriteLocation() { + return getTranslog().getLastWriteLocation(); + } + protected final void ensureOpen(Exception suppressed) { if (isClosed.get()) { AlreadyClosedException ace = new AlreadyClosedException(shardId + " engine is closed", failedEngine.get()); @@ -546,6 +587,13 @@ public CommitStats commitStats() { */ public abstract LocalCheckpointTracker getLocalCheckpointTracker(); + /** + * Returns the latest global checkpoint value that has been persisted in the underlying storage (i.e. translog's checkpoint) + */ + public long getLastSyncedGlobalCheckpoint() { + return getTranslog().getLastSyncedGlobalCheckpoint(); + } + /** * Global stats on segments. */ @@ -810,6 +858,16 @@ public final boolean refreshNeeded() { */ public abstract void trimTranslog() throws EngineException; + /** + * Tests whether or not the translog generation should be rolled to a new generation. + * This test is based on the size of the current generation compared to the configured generation threshold size. + * + * @return {@code true} if the current generation should be rolled to a new generation + */ + public boolean shouldRollTranslogGeneration() { + return getTranslog().shouldRollGeneration(); + } + /** * Rolls the translog generation and cleans unneeded. */ 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 dcd1ba65d8950..b28a5cd59e25b 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -422,7 +422,7 @@ private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy } @Override - public Translog getTranslog() { + Translog getTranslog() { ensureOpen(); return translog; } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index 0ec03cb7a8f5e..9b55cff8cff9a 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -131,7 +131,7 @@ protected ReplicaResult shardOperationOnReplica(final Request request, final Ind private void maybeSyncTranslog(final IndexShard indexShard) throws IOException { if (indexShard.getTranslogDurability() == Translog.Durability.REQUEST && - indexShard.getTranslog().getLastSyncedGlobalCheckpoint() < indexShard.getGlobalCheckpoint()) { + indexShard.getLastSyncedGlobalCheckpoint() < indexShard.getGlobalCheckpoint()) { indexShard.sync(); } } 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 520115dc30a46..def6362e334e4 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -934,7 +934,7 @@ public FieldDataStats fieldDataStats(String... fields) { } public TranslogStats translogStats() { - return getEngine().getTranslog().stats(); + return getEngine().getTranslogStats(); } public CompletionStats completionStats(String... fields) { @@ -1331,7 +1331,7 @@ private boolean assertMaxUnsafeAutoIdInCommit() throws IOException { } protected void onNewEngine(Engine newEngine) { - refreshListeners.setTranslog(newEngine.getTranslog()); + refreshListeners.setCurrentRefreshLocationSupplier(newEngine::getTranslogLastWriteLocation); } /** @@ -1563,8 +1563,7 @@ boolean shouldRollTranslogGeneration() { final Engine engine = getEngineOrNull(); if (engine != null) { try { - final Translog translog = engine.getTranslog(); - return translog.shouldRollGeneration(); + return engine.shouldRollTranslogGeneration(); } catch (final AlreadyClosedException e) { // we are already closed, no need to flush or roll } @@ -1579,9 +1578,26 @@ public void onSettingsChanged() { } } + /** + * Acquires a lock on the translog files, preventing them from being trimmed. + */ public Closeable acquireTranslogRetentionLock() { - Engine engine = getEngine(); - return engine.getTranslog().acquireRetentionLock(); + return getEngine().acquireTranslogRetentionLock(); + } + + /** + * Creates a new translog snapshot for reading translog operations whose seq# at least the provided seq#. + * The caller has to close the returned snapshot after finishing the reading. + */ + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { + return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo); + } + + /** + * Returns the estimated number of operations in translog whose seq# at least the provided seq#. + */ + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); } public List segments(boolean verbose) { @@ -1592,10 +1608,6 @@ public void flushAndCloseEngine() throws IOException { getEngine().flushAndClose(); } - public Translog getTranslog() { - return getEngine().getTranslog(); - } - public String getHistoryUUID() { return getEngine().getHistoryUUID(); } @@ -1733,6 +1745,13 @@ public long getGlobalCheckpoint() { return replicationTracker.getGlobalCheckpoint(); } + /** + * Returns the latest global checkpoint value that has been persisted in the underlying storage (i.e. translog's checkpoint) + */ + public long getLastSyncedGlobalCheckpoint() { + return getEngine().getLastSyncedGlobalCheckpoint(); + } + /** * Get the local knowledge of the global checkpoints for all in-sync allocation IDs. * @@ -2308,6 +2327,13 @@ public void sync() throws IOException { getEngine().syncTranslog(); } + /** + * Checks if the underlying storage sync is required. + */ + public boolean isSyncNeeded() { + return getEngine().isTranslogSyncNeeded(); + } + /** * Returns the current translog durability mode */ @@ -2467,7 +2493,7 @@ final long getLastSearcherAccess() { } private void setRefreshPending(Engine engine) { - Translog.Location lastWriteLocation = engine.getTranslog().getLastWriteLocation(); + Translog.Location lastWriteLocation = engine.getTranslogLastWriteLocation(); Translog.Location location; do { location = this.pendingRefreshLocation.get(); diff --git a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java index 1e31eae7d417f..af8c9bdd0272f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -83,7 +83,7 @@ public void resync(final IndexShard indexShard, final ActionListener ActionListener resyncListener = null; try { final long startingSeqNo = indexShard.getGlobalCheckpoint() + 1; - Translog.Snapshot snapshot = indexShard.getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); + Translog.Snapshot snapshot = indexShard.newTranslogSnapshotFromMinSeqNo(startingSeqNo); resyncListener = new ActionListener() { @Override public void onResponse(final ResyncTask resyncTask) { diff --git a/server/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java b/server/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java index 17e824eb046c7..d8a51d58ad956 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java +++ b/server/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java @@ -32,6 +32,7 @@ import java.util.concurrent.Executor; import java.util.function.Consumer; import java.util.function.IntSupplier; +import java.util.function.Supplier; import static java.util.Objects.requireNonNull; @@ -153,21 +154,20 @@ public int pendingCount() { /** * Setup the translog used to find the last refreshed location. */ - public void setTranslog(Translog translog) { - this.translog = translog; + public void setCurrentRefreshLocationSupplier(Supplier currentRefreshLocationSupplier) { + this.currentRefreshLocationSupplier = currentRefreshLocationSupplier; } - // Implementation of ReferenceManager.RefreshListener that adapts Lucene's RefreshListener into Elasticsearch's refresh listeners. - private Translog translog; /** * Snapshot of the translog location before the current refresh if there is a refresh going on or null. Doesn't have to be volatile * because when it is used by the refreshing thread. */ private Translog.Location currentRefreshLocation; + private Supplier currentRefreshLocationSupplier; @Override public void beforeRefresh() throws IOException { - currentRefreshLocation = translog.getLastWriteLocation(); + currentRefreshLocation = currentRefreshLocationSupplier.get(); } @Override 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 710b4bc46e235..78f44ee723114 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -145,9 +145,6 @@ public RecoveryResponse recoverToTarget() throws IOException { }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered "); try (Closeable ignored = shard.acquireTranslogRetentionLock()) { - - final Translog translog = shard.getTranslog(); - final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && @@ -170,7 +167,7 @@ public RecoveryResponse recoverToTarget() throws IOException { requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; try { - phase1(phase1Snapshot.getIndexCommit(), translog::totalOperations); + phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -187,7 +184,7 @@ public RecoveryResponse recoverToTarget() throws IOException { try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, translog.estimateTotalOperationsFromMinSeq(startingSeqNo)); + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } @@ -210,9 +207,9 @@ public RecoveryResponse recoverToTarget() throws IOException { logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); - logger.trace("snapshot translog for recovery; current size is [{}]", translog.estimateTotalOperationsFromMinSeq(startingSeqNo)); + logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); final long targetLocalCheckpoint; - try(Translog.Snapshot snapshot = translog.newSnapshotFromMinSeqNo(startingSeqNo)) { + try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); @@ -261,7 +258,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { // the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one if (startingSeqNo - 1 <= localCheckpoint) { final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); - try (Translog.Snapshot snapshot = shard.getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { + try (Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { Translog.Operation operation; while ((operation = snapshot.next()) != null) { if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { diff --git a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java index 5000af6688f83..4dc6a859a5c5a 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -250,7 +250,7 @@ public void testAsyncFsyncActuallyWorks() throws Exception { client().prepareIndex("test", "test", "1").setSource("{\"foo\": \"bar\"}", XContentType.JSON).get(); IndexShard shard = indexService.getShard(0); assertBusy(() -> { - assertFalse(shard.getTranslog().syncNeeded()); + assertFalse(shard.isSyncNeeded()); }); } @@ -275,7 +275,7 @@ public void testRescheduleAsyncFsync() throws Exception { client().prepareIndex("test", "test", "1").setSource("{\"foo\": \"bar\"}", XContentType.JSON).get(); assertNotNull(indexService.getFsyncTask()); final IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertFalse(shard.getTranslog().syncNeeded())); + assertBusy(() -> assertFalse(shard.isSyncNeeded())); client() .admin() @@ -311,7 +311,7 @@ public void testAsyncTranslogTrimActuallyWorks() throws Exception { indexService.updateMetaData(metaData); IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertThat(shard.getTranslog().totalOperations(), equalTo(0))); + assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0))); } public void testIllegalFsyncInterval() { diff --git a/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 97fc1b528acf3..fd62318f96c3a 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -681,7 +681,7 @@ class GlobalCheckpointSync extends ReplicationAction< @Override protected PrimaryResult performOnPrimary( final IndexShard primary, final GlobalCheckpointSyncAction.Request request) throws Exception { - primary.getTranslog().sync(); + primary.sync(); return new PrimaryResult(request, new ReplicationResponse()); } diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index baa56ee9585f6..2d2aaac7bbd26 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -330,7 +330,7 @@ public void testSeqNoCollision() throws Exception { final Translog.Operation op1; final List initOperations = new ArrayList<>(initDocs); - try (Translog.Snapshot snapshot = replica2.getTranslog().newSnapshot()) { + try (Translog.Snapshot snapshot = getTranslog(replica2).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); for (int i = 0; i < initDocs; i++) { Translog.Operation op = snapshot.next(); @@ -347,7 +347,7 @@ public void testSeqNoCollision() throws Exception { shards.promoteReplicaToPrimary(replica1).get(); // wait until resync completed. shards.index(new IndexRequest(index.getName(), "type", "d2").source("{}", XContentType.JSON)); final Translog.Operation op2; - try (Translog.Snapshot snapshot = replica2.getTranslog().newSnapshot()) { + try (Translog.Snapshot snapshot = getTranslog(replica2).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 2)); op2 = snapshot.next(); assertThat(op2.seqNo(), equalTo(op1.seqNo())); @@ -362,7 +362,7 @@ public void testSeqNoCollision() throws Exception { shards.promoteReplicaToPrimary(replica2); logger.info("--> Recover replica3 from replica2"); recoverReplica(replica3, replica2); - try (Translog.Snapshot snapshot = replica3.getTranslog().newSnapshot()) { + try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); assertThat(snapshot.next(), equalTo(op2)); assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations)); @@ -468,7 +468,7 @@ private static void assertNoOpTranslogOperationForDocumentFailure( long expectedPrimaryTerm, String failureMessage) throws IOException { for (IndexShard indexShard : replicationGroup) { - try(Translog.Snapshot snapshot = indexShard.getTranslog().newSnapshot()) { + try(Translog.Snapshot snapshot = getTranslog(indexShard).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(expectedOperation)); long expectedSeqNo = 0L; Translog.Operation op = snapshot.next(); 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 c7469f2432ad3..323b0364dfb93 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -128,7 +128,7 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { shards.flush(); translogTrimmed = randomBoolean(); if (translogTrimmed) { - final Translog translog = shards.getPrimary().getTranslog(); + final Translog translog = getTranslog(shards.getPrimary()); translog.getDeletionPolicy().setRetentionAgeInMillis(0); translog.trimUnreferencedReaders(); } @@ -271,7 +271,7 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { // otherwise the deletion policy won't trim translog assertBusy(() -> { shards.syncGlobalCheckpoint(); - assertThat(newPrimary.getTranslog().getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); + assertThat(newPrimary.getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); }); newPrimary.flush(new FlushRequest()); uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10)); @@ -340,7 +340,7 @@ public void testReplicaRollbackStaleDocumentsInPeerRecovery() throws Exception { // Index more docs - move the global checkpoint >= seqno of the stale operations. goodDocs += shards.indexDocs(scaledRandomIntBetween(staleDocs, staleDocs * 5)); shards.syncGlobalCheckpoint(); - assertThat(replica.getTranslog().getLastSyncedGlobalCheckpoint(), equalTo(replica.seqNoStats().getMaxSeqNo())); + assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(replica.seqNoStats().getMaxSeqNo())); // Recover a replica again should also rollback the stale documents. shards.removeReplica(replica); replica.close("recover replica - second time", false); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java index 70813531aeb0e..596575abc3025 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncActionTests.java @@ -90,9 +90,6 @@ public void testTranslogSyncAfterGlobalCheckpointSync() throws Exception { final Translog.Durability durability = randomFrom(Translog.Durability.ASYNC, Translog.Durability.REQUEST); when(indexShard.getTranslogDurability()).thenReturn(durability); - final Translog translog = mock(Translog.class); - when(indexShard.getTranslog()).thenReturn(translog); - final long globalCheckpoint = randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), Integer.MAX_VALUE); final long lastSyncedGlobalCheckpoint; if (randomBoolean() && globalCheckpoint != SequenceNumbers.NO_OPS_PERFORMED) { @@ -104,7 +101,7 @@ public void testTranslogSyncAfterGlobalCheckpointSync() throws Exception { } when(indexShard.getGlobalCheckpoint()).thenReturn(globalCheckpoint); - when(translog.getLastSyncedGlobalCheckpoint()).thenReturn(lastSyncedGlobalCheckpoint); + when(indexShard.getLastSyncedGlobalCheckpoint()).thenReturn(lastSyncedGlobalCheckpoint); final GlobalCheckpointSyncAction action = new GlobalCheckpointSyncAction( Settings.EMPTY, 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 f7ee54b32ee84..bc34aa60c4925 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -107,6 +107,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.elasticsearch.index.shard.IndexShardTestCase.getTranslog; public class IndexShardIT extends ESSingleNodeTestCase { @@ -167,7 +168,7 @@ public void testDurableFlagHasEffect() throws Exception { IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService(resolveIndex("test")); IndexShard shard = test.getShardOrNull(0); - Translog translog = ShardUtilsTests.getShardEngine(shard).getTranslog(); + Translog translog = getTranslog(shard); Predicate needsSync = (tlog) -> { // we can't use tlog.needsSync() here since it also takes the global checkpoint into account // we explicitly want to check here if our durability checks are taken into account so we only @@ -343,7 +344,7 @@ public void testMaybeFlush() throws Exception { SourceToParse.source("test", "test", "1", new BytesArray("{}"), XContentType.JSON), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, update -> {}); assertTrue(shard.shouldPeriodicallyFlush()); - final Translog translog = shard.getEngine().getTranslog(); + final Translog translog = getTranslog(shard); assertEquals(2, translog.stats().getUncommittedOperations()); client().prepareIndex("test", "test", "2").setSource("{}", XContentType.JSON) .setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); @@ -384,7 +385,7 @@ public void testMaybeRollTranslogGeneration() throws Exception { final IndexService test = indicesService.indexService(resolveIndex("test")); final IndexShard shard = test.getShardOrNull(0); int rolls = 0; - final Translog translog = shard.getEngine().getTranslog(); + final Translog translog = getTranslog(shard); final long generation = translog.currentFileGeneration(); final int numberOfDocuments = randomIntBetween(32, 128); for (int i = 0; i < numberOfDocuments; i++) { @@ -454,11 +455,11 @@ public void testStressMaybeFlushOrRollTranslogGeneration() throws Exception { assertThat(shard.flushStats().getPeriodic(), equalTo(periodic + 1)); }; } else { - final long generation = shard.getEngine().getTranslog().currentFileGeneration(); + final long generation = getTranslog(shard).currentFileGeneration(); client().prepareIndex("test", "test", "1").setSource("{}", XContentType.JSON).get(); check = () -> assertEquals( generation + 1, - shard.getEngine().getTranslog().currentFileGeneration()); + getTranslog(shard).currentFileGeneration()); } assertBusy(check); running.set(false); 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 5506bc515f24c..b608bc9cc5081 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -72,7 +72,6 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineException; @@ -518,7 +517,7 @@ public void onFailure(Exception e) { public void testPrimaryPromotionRollsGeneration() throws Exception { final IndexShard indexShard = newStartedShard(false); - final long currentTranslogGeneration = indexShard.getTranslog().getGeneration().translogFileGeneration; + final long currentTranslogGeneration = getTranslog(indexShard).getGeneration().translogFileGeneration; // promote the replica final ShardRouting replicaRouting = indexShard.routingEntry(); @@ -556,8 +555,8 @@ public void onFailure(Exception e) { ThreadPool.Names.GENERIC, ""); latch.await(); - assertThat(indexShard.getTranslog().getGeneration().translogFileGeneration, equalTo(currentTranslogGeneration + 1)); - assertThat(TestTranslog.getCurrentTerm(indexShard.getTranslog()), equalTo(newPrimaryTerm)); + assertThat(getTranslog(indexShard).getGeneration().translogFileGeneration, equalTo(currentTranslogGeneration + 1)); + assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); closeShards(indexShard); } @@ -578,7 +577,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E true, ShardRoutingState.STARTED, replicaRouting.allocationId()); final long newPrimaryTerm = indexShard.getPrimaryTerm() + between(1, 1000); indexShard.updateShardState(primaryRouting, newPrimaryTerm, (shard, listener) -> { - assertThat(TestTranslog.getCurrentTerm(indexShard.getTranslog()), equalTo(newPrimaryTerm)); + assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); }, 0L, Collections.singleton(indexShard.routingEntry().allocationId().getId()), new IndexShardRoutingTable.Builder(indexShard.shardId()).addShard(primaryRouting).build(), @@ -669,7 +668,7 @@ public void testOperationPermitOnReplicaShards() throws Exception { } final long primaryTerm = indexShard.getPrimaryTerm(); - final long translogGen = engineClosed ? -1 : indexShard.getTranslog().getGeneration().translogFileGeneration; + final long translogGen = engineClosed ? -1 : getTranslog(indexShard).getGeneration().translogFileGeneration; final Releasable operation1; final Releasable operation2; @@ -747,7 +746,7 @@ public void onFailure(Exception e) { @Override public void onResponse(Releasable releasable) { assertThat(indexShard.getPrimaryTerm(), equalTo(newPrimaryTerm)); - assertThat(TestTranslog.getCurrentTerm(indexShard.getTranslog()), equalTo(newPrimaryTerm)); + assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); assertThat(indexShard.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); assertThat(indexShard.getGlobalCheckpoint(), equalTo(newGlobalCheckPoint)); onResponse.set(true); @@ -793,25 +792,25 @@ private void finish() { assertFalse(onResponse.get()); assertNull(onFailure.get()); assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm)); - assertThat(TestTranslog.getCurrentTerm(indexShard.getTranslog()), equalTo(primaryTerm)); + assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(primaryTerm)); Releasables.close(operation1); // our operation should still be blocked assertFalse(onResponse.get()); assertNull(onFailure.get()); assertThat(indexShard.getPrimaryTerm(), equalTo(primaryTerm)); - assertThat(TestTranslog.getCurrentTerm(indexShard.getTranslog()), equalTo(primaryTerm)); + assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(primaryTerm)); Releasables.close(operation2); barrier.await(); // now lock acquisition should have succeeded assertThat(indexShard.getPrimaryTerm(), equalTo(newPrimaryTerm)); - assertThat(TestTranslog.getCurrentTerm(indexShard.getTranslog()), equalTo(newPrimaryTerm)); + assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm)); if (engineClosed) { assertFalse(onResponse.get()); assertThat(onFailure.get(), instanceOf(AlreadyClosedException.class)); } else { assertTrue(onResponse.get()); assertNull(onFailure.get()); - assertThat(indexShard.getTranslog().getGeneration().translogFileGeneration, equalTo(translogGen + 1)); + assertThat(getTranslog(indexShard).getGeneration().translogFileGeneration, equalTo(translogGen + 1)); assertThat(indexShard.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); assertThat(indexShard.getGlobalCheckpoint(), equalTo(newGlobalCheckPoint)); } @@ -1647,7 +1646,7 @@ public void testRecoverFromStoreWithNoOps() throws IOException { assertEquals(1, newShard.recoveryState().getTranslog().totalOperations()); assertEquals(1, newShard.recoveryState().getTranslog().totalOperationsOnStart()); assertEquals(100.0f, newShard.recoveryState().getTranslog().recoveredPercent(), 0.01f); - try (Translog.Snapshot snapshot = newShard.getTranslog().newSnapshot()) { + try (Translog.Snapshot snapshot = getTranslog(newShard).newSnapshot()) { Translog.Operation operation; int numNoops = 0; while ((operation = snapshot.next()) != null) { @@ -2048,7 +2047,7 @@ public void testTranslogRecoverySyncsTranslog() throws IOException { @Override public long indexTranslogOperations(List operations, int totalTranslogOps) throws IOException { final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps); - assertFalse(replica.getTranslog().syncNeeded()); + assertFalse(replica.isSyncNeeded()); return localCheckpoint; } }, true); 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 5803bf263633d..2d1c1d4e15af8 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -133,7 +133,7 @@ indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilari (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm); engine = new InternalEngine(config); engine.recoverFromTranslog(); - listeners.setTranslog(engine.getTranslog()); + listeners.setCurrentRefreshLocationSupplier(engine::getTranslogLastWriteLocation); } @After diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index d65d40e5bcdaa..91b35594772cf 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -59,7 +59,7 @@ public void testGetStartingSeqNo() throws Exception { } flushShard(replica); replica.updateGlobalCheckpointOnReplica(initDocs - 1, "test"); - replica.getTranslog().sync(); + replica.sync(); final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null); assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(initDocs)); recoveryTarget.decRef(); @@ -81,7 +81,7 @@ public void testGetStartingSeqNo() throws Exception { // Advances the global checkpoint, a safe commit also advances { replica.updateGlobalCheckpointOnReplica(initDocs + moreDocs - 1, "test"); - replica.getTranslog().sync(); + replica.sync(); final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null); assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(initDocs + moreDocs)); recoveryTarget.decRef(); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index f46ab7ebbd603..4e9d0ccb22e11 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -61,7 +61,7 @@ public void testTranslogHistoryTransferred() throws Exception { try (ReplicationGroup shards = createGroup(0)) { shards.startPrimary(); int docs = shards.indexDocs(10); - shards.getPrimary().getTranslog().rollGeneration(); + getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); if (randomBoolean()) { docs += shards.indexDocs(10); @@ -69,7 +69,7 @@ public void testTranslogHistoryTransferred() throws Exception { shards.addReplica(); shards.startAll(); final IndexShard replica = shards.getReplicas().get(0); - assertThat(replica.getTranslog().totalOperations(), equalTo(docs)); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(docs)); } } @@ -77,7 +77,7 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { try (ReplicationGroup shards = createGroup(0)) { shards.startPrimary(); shards.indexDocs(10); - shards.getPrimary().getTranslog().rollGeneration(); + getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); shards.indexDocs(10); final IndexShard replica = shards.addReplica(); @@ -99,7 +99,7 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { releaseRecovery.countDown(); future.get(); // rolling/flushing is async - assertBusy(() -> assertThat(replica.getTranslog().totalOperations(), equalTo(0))); + assertBusy(() -> assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(0))); } } @@ -123,7 +123,7 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { // delete #1 orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id", VersionType.EXTERNAL, u -> {}); - orgReplica.getTranslog().rollGeneration(); // isolate the delete in it's own generation + getTranslog(orgReplica).rollGeneration(); // isolate the delete in it's own generation // index #0 orgReplica.applyIndexOperationOnReplica(0, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(indexName, "type", "id", new BytesArray("{}"), XContentType.JSON), u -> {}); @@ -167,7 +167,7 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { shards.recoverReplica(newReplica); shards.assertAllEqual(3); - assertThat(newReplica.getTranslog().totalOperations(), equalTo(translogOps)); + assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(translogOps)); } } @@ -184,7 +184,7 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { IndexShard replica = shards.getReplicas().get(0); final String historyUUID = replica.getHistoryUUID(); - Translog.TranslogGeneration translogGeneration = replica.getTranslog().getGeneration(); + Translog.TranslogGeneration translogGeneration = getTranslog(replica).getGeneration(); shards.removeReplica(replica); replica.close("test", false); IndexWriterConfig iwc = new IndexWriterConfig(null) @@ -219,7 +219,7 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { shards.recoverReplica(newReplica); // file based recovery should be made assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); - assertThat(newReplica.getTranslog().totalOperations(), equalTo(numDocs)); + assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); // history uuid was restored assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -238,7 +238,7 @@ public void testPeerRecoveryPersistGlobalCheckpoint() throws Exception { } final IndexShard replica = shards.addReplica(); shards.recoverReplica(replica); - assertThat(replica.getTranslog().getLastSyncedGlobalCheckpoint(), equalTo(numDocs - 1)); + assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(numDocs - 1)); } } @@ -291,7 +291,7 @@ public void testSequenceBasedRecoveryKeepsTranslog() throws Exception { final IndexShard newReplica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId()); shards.recoverReplica(newReplica); - try (Translog.Snapshot snapshot = newReplica.getTranslog().newSnapshot()) { + try (Translog.Snapshot snapshot = getTranslog(newReplica).newSnapshot()) { assertThat("Sequence based recovery should keep existing translog", snapshot, SnapshotMatchers.size(initDocs + moreDocs)); } assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(uncommittedDocs + moreDocs)); @@ -321,7 +321,7 @@ public void testShouldFlushAfterPeerRecovery() throws Exception { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false) assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false))); - assertThat(replica.getTranslog().totalOperations(), equalTo(numDocs)); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); shards.assertAllEqual(numDocs); } } 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 dea92c2927d86..8fff17900b072 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 @@ -493,4 +493,11 @@ protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, VersionType.EXTERNAL, Engine.Operation.Origin.REPLICA, startTime); } + + /** + * Exposes a translog associated with the given engine for testing purpose. + */ + public static Translog getTranslog(Engine engine) { + return engine.getTranslog(); + } } 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 0d535d9af3851..a0e1cfc334110 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 @@ -55,6 +55,7 @@ import org.elasticsearch.index.cache.query.DisabledQueryCache; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; +import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; @@ -66,6 +67,7 @@ import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; @@ -643,6 +645,10 @@ public static Engine getEngine(IndexShard indexShard) { return indexShard.getEngine(); } + public static Translog getTranslog(IndexShard shard) { + return EngineTestCase.getTranslog(getEngine(shard)); + } + public static ReplicationTracker getReplicationTracker(IndexShard indexShard) { return indexShard.getReplicationTracker(); } 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 d82b5052dbf54..12acd21903ec4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -76,6 +76,7 @@ import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; @@ -1158,7 +1159,7 @@ private void assertOpenTranslogReferences() throws Exception { for (IndexService indexService : indexServices) { for (IndexShard indexShard : indexService) { try { - indexShard.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); + IndexShardTestCase.getTranslog(indexShard).getDeletionPolicy().assertNoOpenTranslogRefs(); } catch (AlreadyClosedException ok) { // all good } From 3367948be603f253742e95f8c7b3dfaa10852934 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 17 Apr 2018 15:04:46 +0200 Subject: [PATCH 02/16] Add documentation about the include_type_name option. (#29555) This option will be useful in 7.x to prepare for upgrade to 8.0 which won't know about types anymore. --- .../mapping/removal_of_types.asciidoc | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index 070d189a0fffe..60776763ea844 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -421,3 +421,108 @@ POST _reindex ---- // NOTCONSOLE +[float] +=== Use `include_type_name=false` to prepare for upgrade to 8.0 + +Index creation, mappings and document APIs support the `include_type_name` +option. When set to `false`, this option enables the behavior that will become +default in 8.0 when types are removed. See some examples of interactions with +Elasticsearch with this option turned off: + +[float] +==== Index creation + +[source,js] +-------------------------------------------------- +PUT index?include_type_name=false +{ + "mappings": { + "properties": { <1> + "foo": { + "type": "keyword" + } + } + } +} +-------------------------------------------------- +// CONSOLE +<1> Mappings are included directly under the `mappings` key, without a type name. + +[float] +==== PUT and GET mappings + +[source,js] +-------------------------------------------------- +PUT index + +PUT index/_mappings?include_type_name=false +{ + "properties": { <1> + "foo": { + "type": "keyword" + } + } +} + +GET index/_mappings?include_type_name=false +-------------------------------------------------- +// CONSOLE +<1> Mappings are included directly under the `mappings` key, without a type name. + + +The above call returns + +[source,js] +-------------------------------------------------- +{ + "index": { + "mappings": { + "properties": { <1> + "foo": { + "type": "keyword" + } + } + } + } +} +-------------------------------------------------- +// TESTRESPONSE +<1> Mappings are included directly under the `mappings` key, without a type name. + +[float] +==== Document APIs + +Index APIs must be call with the `{index}/_doc` path for automatic generation of +the `_id` and `{index}/_doc/{id}` with explicit ids. + +[source,js] +-------------------------------------------------- +PUT index/_doc/1?include_type_name=false +{ + "foo": "bar" +} +-------------------------------------------------- +// CONSOLE + +[source,js] +-------------------------------------------------- +{ + "_index": "index", <1> + "_id": "1", + "_version": 1, + "result": "created", + "_shards": { + "total": 2, + "successful": 1, + "failed": 0 + }, + "_seq_no": 0, + "_primary_term": 1 +} +-------------------------------------------------- +// TESTRESPONSE +<1> The response does not include a `_type`. + +Likewise the <>, <>, +<> and <> APIs do not return a `_type` +key in the response when `include_type_name` is set to `false`. From 983d6c15a2da2b1bee8c63b0dd5245b12ca000a0 Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Tue, 17 Apr 2018 10:19:54 -0400 Subject: [PATCH 03/16] Add null_value support to geo_point type (#29451) Adds support for null_value attribute to the geo_point types. Closes #12998 --- .../mapping/types/geo-point.asciidoc | 5 ++ .../elasticsearch/common/geo/GeoUtils.java | 36 ++++++++++++ .../index/mapper/GeoPointFieldMapper.java | 37 +++++++++++- .../mapper/GeoPointFieldMapperTests.java | 49 ++++++++++++++++ .../index/mapper/NullValueTests.java | 2 +- .../search/geo/GeoPointParsingTests.java | 57 ++++++++++++++++++- 6 files changed, 180 insertions(+), 6 deletions(-) diff --git a/docs/reference/mapping/types/geo-point.asciidoc b/docs/reference/mapping/types/geo-point.asciidoc index 57faef2dbd7db..97f2ddb52825b 100644 --- a/docs/reference/mapping/types/geo-point.asciidoc +++ b/docs/reference/mapping/types/geo-point.asciidoc @@ -122,6 +122,11 @@ The following parameters are accepted by `geo_point` fields: ignored. If `false`, geo-points containing any more than latitude and longitude (two dimensions) values throw an exception and reject the whole document. +<>:: + + Accepts an geopoint value which is substituted for any explicit `null` values. + Defaults to `null`, which means the field is treated as missing. + ==== Using geo-points in scripts When accessing the value of a geo-point in a script, the value is returned as 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 ce0098ea9722f..57e87e06389c4 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java @@ -24,9 +24,14 @@ import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree; import org.apache.lucene.util.SloppyMath; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.unit.DistanceUnit; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.fielddata.FieldData; import org.elasticsearch.index.fielddata.GeoPointValues; @@ -36,6 +41,7 @@ import org.elasticsearch.index.fielddata.SortingNumericDoubleValues; import java.io.IOException; +import java.io.InputStream; public class GeoUtils { @@ -351,6 +357,36 @@ public static GeoPoint parseGeoPoint(XContentParser parser, GeoPoint point) thro return parseGeoPoint(parser, point, false); } + /** + * Parses the value as a geopoint. The following types of values are supported: + *

+ * Object: has to contain either lat and lon or geohash fields + *

+ * String: expected to be in "latitude, longitude" format or a geohash + *

+ * Array: two or more elements, the first element is longitude, the second is latitude, the rest is ignored if ignoreZValue is true + */ + public static GeoPoint parseGeoPoint(Object value, final boolean ignoreZValue) throws ElasticsearchParseException { + try { + XContentBuilder content = JsonXContent.contentBuilder(); + content.startObject(); + content.field("null_value", value); + content.endObject(); + + try (InputStream stream = BytesReference.bytes(content).streamInput(); + XContentParser parser = JsonXContent.jsonXContent.createParser( + NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, stream)) { + parser.nextToken(); // start object + parser.nextToken(); // field name + parser.nextToken(); // field value + return parseGeoPoint(parser, new GeoPoint(), ignoreZValue); + } + + } catch (IOException ex) { + throw new ElasticsearchParseException("error parsing geopoint", ex); + } + } + /** * Parse a {@link GeoPoint} with a {@link XContentParser}. A geopoint has one of the following forms: * diff --git a/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java index 10c0db01dc776..0cd200021701e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java @@ -60,6 +60,7 @@ public class GeoPointFieldMapper extends FieldMapper implements ArrayValueMapper public static class Names { public static final String IGNORE_MALFORMED = "ignore_malformed"; public static final ParseField IGNORE_Z_VALUE = new ParseField("ignore_z_value"); + public static final String NULL_VALUE = "null_value"; } public static class Defaults { @@ -134,7 +135,7 @@ public Mapper.Builder parse(String name, Map node, ParserContext throws MapperParsingException { Builder builder = new GeoPointFieldMapper.Builder(name); parseField(builder, name, node, parserContext); - + Object nullValue = null; for (Iterator> iterator = node.entrySet().iterator(); iterator.hasNext();) { Map.Entry entry = iterator.next(); String propName = entry.getKey(); @@ -147,9 +148,31 @@ public Mapper.Builder parse(String name, Map node, ParserContext builder.ignoreZValue(XContentMapValues.nodeBooleanValue(propNode, name + "." + Names.IGNORE_Z_VALUE.getPreferredName())); iterator.remove(); + } else if (propName.equals(Names.NULL_VALUE)) { + if (propNode == null) { + throw new MapperParsingException("Property [null_value] cannot be null."); + } + nullValue = propNode; + iterator.remove(); } } + if (nullValue != null) { + boolean ignoreZValue = builder.ignoreZValue == null ? Defaults.IGNORE_Z_VALUE.value() : builder.ignoreZValue; + boolean ignoreMalformed = builder.ignoreMalformed == null ? Defaults.IGNORE_MALFORMED.value() : builder.ignoreZValue; + GeoPoint point = GeoUtils.parseGeoPoint(nullValue, ignoreZValue); + if (ignoreMalformed == false) { + if (point.lat() > 90.0 || point.lat() < -90.0) { + throw new IllegalArgumentException("illegal latitude value [" + point.lat() + "]"); + } + if (point.lon() > 180.0 || point.lon() < -180) { + throw new IllegalArgumentException("illegal longitude value [" + point.lon() + "]"); + } + } else { + GeoUtils.normalizePoint(point); + } + builder.nullValue(point); + } return builder; } } @@ -318,7 +341,11 @@ public Mapper parse(ParseContext context) throws IOException { } } else if (token == XContentParser.Token.VALUE_STRING) { parse(context, sparse.resetFromString(context.parser().text(), ignoreZValue.value())); - } else if (token != XContentParser.Token.VALUE_NULL) { + } else if (token == XContentParser.Token.VALUE_NULL) { + if (fieldType.nullValue() != null) { + parse(context, (GeoPoint) fieldType.nullValue()); + } + } else { try { parse(context, GeoUtils.parseGeoPoint(context.parser(), sparse)); } catch (ElasticsearchParseException e) { @@ -337,11 +364,15 @@ public Mapper parse(ParseContext context) throws IOException { protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { super.doXContentBody(builder, includeDefaults, params); if (includeDefaults || ignoreMalformed.explicit()) { - builder.field(GeoPointFieldMapper.Names.IGNORE_MALFORMED, ignoreMalformed.value()); + builder.field(Names.IGNORE_MALFORMED, ignoreMalformed.value()); } if (includeDefaults || ignoreZValue.explicit()) { builder.field(Names.IGNORE_Z_VALUE.getPreferredName(), ignoreZValue.value()); } + + if (includeDefaults || fieldType().nullValue() != null) { + builder.field(Names.NULL_VALUE, fieldType().nullValue()); + } } public Explicit ignoreZValue() { 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 03cc183b906d3..0de90631a14b3 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.mapper; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.Priority; @@ -41,10 +42,12 @@ import static org.elasticsearch.common.geo.GeoHashUtils.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; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { @@ -349,4 +352,50 @@ public void testEmptyName() throws Exception { ); assertThat(e.getMessage(), containsString("name cannot be empty string")); } + + public void testNullValue() throws Exception { + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("location") + .field("type", "geo_point") + .field(NULL_VALUE, "1,2") + .endObject().endObject() + .endObject().endObject()); + + DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser() + .parse("type", new CompressedXContent(mapping)); + FieldMapper fieldMapper = defaultMapper.mappers().getMapper("location"); + assertThat(fieldMapper, instanceOf(GeoPointFieldMapper.class)); + + Object nullValue = fieldMapper.fieldType().nullValue(); + assertThat(nullValue, equalTo(new GeoPoint(1, 2))); + + ParsedDocument doc = defaultMapper.parse(SourceToParse.source("test", "type", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .nullField("location") + .endObject()), + XContentType.JSON)); + + assertThat(doc.rootDoc().getField("location"), notNullValue()); + BytesRef defaultValue = doc.rootDoc().getField("location").binaryValue(); + + doc = defaultMapper.parse(SourceToParse.source("test", "type", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .field("location", "1, 2") + .endObject()), + XContentType.JSON)); + // Shouldn't matter if we specify the value explicitly or use null value + assertThat(defaultValue, equalTo(doc.rootDoc().getField("location").binaryValue())); + + doc = defaultMapper.parse(SourceToParse.source("test", "type", "1", BytesReference + .bytes(XContentFactory.jsonBuilder() + .startObject() + .field("location", "3, 4") + .endObject()), + XContentType.JSON)); + // Shouldn't matter if we specify the value explicitly or use null value + assertThat(defaultValue, not(equalTo(doc.rootDoc().getField("location").binaryValue()))); + } + } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NullValueTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NullValueTests.java index d9502d8e8800c..f38f83b6e418f 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NullValueTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NullValueTests.java @@ -33,7 +33,7 @@ public class NullValueTests extends ESSingleNodeTestCase { public void testNullNullValue() throws Exception { IndexService indexService = createIndex("test", Settings.builder().build()); - String[] typesToTest = {"integer", "long", "double", "float", "short", "date", "ip", "keyword", "boolean", "byte"}; + String[] typesToTest = {"integer", "long", "double", "float", "short", "date", "ip", "keyword", "boolean", "byte", "geo_point"}; for (String type : typesToTest) { String mapping = Strings.toString(XContentFactory.jsonBuilder() 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 4f410dc6d2690..f3d109868ef14 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 @@ -76,14 +76,26 @@ public void testGeoPointParsing() throws IOException { GeoPoint point = GeoUtils.parseGeoPoint(objectLatLon(randomPt.lat(), randomPt.lon())); assertPointsEqual(point, randomPt); + GeoUtils.parseGeoPoint(toObject(objectLatLon(randomPt.lat(), randomPt.lon())), randomBoolean()); + assertPointsEqual(point, randomPt); + GeoUtils.parseGeoPoint(arrayLatLon(randomPt.lat(), randomPt.lon()), point); assertPointsEqual(point, randomPt); + GeoUtils.parseGeoPoint(toObject(arrayLatLon(randomPt.lat(), randomPt.lon())), randomBoolean()); + assertPointsEqual(point, randomPt); + GeoUtils.parseGeoPoint(geohash(randomPt.lat(), randomPt.lon()), point); assertCloseTo(point, randomPt.lat(), randomPt.lon()); + GeoUtils.parseGeoPoint(toObject(geohash(randomPt.lat(), randomPt.lon())), randomBoolean()); + assertCloseTo(point, randomPt.lat(), randomPt.lon()); + GeoUtils.parseGeoPoint(stringLatLon(randomPt.lat(), randomPt.lon()), point); assertCloseTo(point, randomPt.lat(), randomPt.lon()); + + GeoUtils.parseGeoPoint(toObject(stringLatLon(randomPt.lat(), randomPt.lon())), randomBoolean()); + assertCloseTo(point, randomPt.lat(), randomPt.lon()); } // Based on #5390 @@ -99,6 +111,12 @@ public void testInvalidPointEmbeddedObject() throws IOException { parser.nextToken(); Exception e = expectThrows(ElasticsearchParseException.class, () -> GeoUtils.parseGeoPoint(parser)); assertThat(e.getMessage(), is("field must be either [lat], [lon] or [geohash]")); + + XContentParser parser2 = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); + parser2.nextToken(); + e = expectThrows(ElasticsearchParseException.class, () -> + GeoUtils.parseGeoPoint(toObject(parser2), randomBoolean())); + assertThat(e.getMessage(), is("field must be either [lat], [lon] or [geohash]")); } public void testInvalidPointLatHashMix() throws IOException { @@ -109,9 +127,14 @@ public void testInvalidPointLatHashMix() throws IOException { XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); parser.nextToken(); - Exception e = expectThrows(ElasticsearchParseException.class, () -> GeoUtils.parseGeoPoint(parser)); assertThat(e.getMessage(), is("field must be either lat/lon or geohash")); + + XContentParser parser2 = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); + parser2.nextToken(); + e = expectThrows(ElasticsearchParseException.class, () -> + GeoUtils.parseGeoPoint(toObject(parser2), randomBoolean())); + assertThat(e.getMessage(), is("field must be either lat/lon or geohash")); } public void testInvalidPointLonHashMix() throws IOException { @@ -125,6 +148,12 @@ public void testInvalidPointLonHashMix() throws IOException { Exception e = expectThrows(ElasticsearchParseException.class, () -> GeoUtils.parseGeoPoint(parser)); assertThat(e.getMessage(), is("field must be either lat/lon or geohash")); + + XContentParser parser2 = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); + parser2.nextToken(); + e = expectThrows(ElasticsearchParseException.class, () -> + GeoUtils.parseGeoPoint(toObject(parser2), randomBoolean())); + assertThat(e.getMessage(), is("field must be either lat/lon or geohash")); } public void testInvalidField() throws IOException { @@ -135,9 +164,15 @@ public void testInvalidField() throws IOException { XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); parser.nextToken(); - Exception e = expectThrows(ElasticsearchParseException.class, () -> GeoUtils.parseGeoPoint(parser)); assertThat(e.getMessage(), is("field must be either [lat], [lon] or [geohash]")); + + + XContentParser parser2 = createParser(JsonXContent.jsonXContent, BytesReference.bytes(content)); + parser2.nextToken(); + e = expectThrows(ElasticsearchParseException.class, () -> + GeoUtils.parseGeoPoint(toObject(parser2), randomBoolean())); + assertThat(e.getMessage(), is("field must be either [lat], [lon] or [geohash]")); } private XContentParser objectLatLon(double lat, double lon) throws IOException { @@ -183,4 +218,22 @@ public static void assertCloseTo(final GeoPoint point, final double lat, final d assertEquals(point.lat(), lat, TOLERANCE); assertEquals(point.lon(), lon, TOLERANCE); } + + public static Object toObject(XContentParser parser) throws IOException { + XContentParser.Token token = parser.currentToken(); + if (token == XContentParser.Token.VALUE_NULL) { + return null; + } else if (token == XContentParser.Token.VALUE_STRING) { + return parser.text(); + } else if (token == XContentParser.Token.VALUE_NUMBER) { + return parser.numberValue(); + } else if (token == XContentParser.Token.START_OBJECT) { + return parser.map(); + } else if (token == XContentParser.Token.START_ARRAY) { + return parser.list(); + } else { + fail("Unexpected token " + token); + } + return null; + } } From a8c2cc6ce70a32208e22fe19328bf64399d833a9 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 17 Apr 2018 17:11:12 +0200 Subject: [PATCH 04/16] Fix dependency checks on libs when generating Eclipse configuration. (#29550) Currently this fails because the Eclipse configuration splits the main and test folders into separate projects to avoid circular dependencies. Relates #29336 --- libs/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/libs/build.gradle b/libs/build.gradle index 78eb93886243d..7f24f69eedc2e 100644 --- a/libs/build.gradle +++ b/libs/build.gradle @@ -34,6 +34,7 @@ subprojects { Project depProject = dependencyToProject(dep) if (depProject != null && false == depProject.path.equals(':libs:elasticsearch-core') + && false == isEclipse && depProject.path.startsWith(':libs')) { throw new InvalidUserDataException("projects in :libs " + "may not depend on other projects libs except " From d7be9185c80a94e5f8b06cd372d950fad9707e36 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 17 Apr 2018 17:11:27 +0200 Subject: [PATCH 05/16] MapperService to wrap a single DocumentMapper. (#29511) This refactors MapperService so that it wraps a single `DocumentMapper` rather than a `Map`. We will need follow-ups since I haven't fixed most APIs that still expose collections of types of mappers, but this is a start... --- .../metadata/MetaDataCreateIndexService.java | 14 +- .../metadata/MetaDataMappingService.java | 26 ++-- .../org/elasticsearch/index/IndexWarmer.java | 3 +- .../index/cache/bitset/BitsetFilterCache.java | 3 +- .../index/mapper/MapperService.java | 121 ++++++++---------- .../index/search/QueryParserHelper.java | 3 +- .../metadata/IndexCreationTaskTests.java | 2 +- .../index/mapper/MapperServiceTests.java | 2 +- .../indices/cluster/ClusterStateChanges.java | 2 +- 9 files changed, 86 insertions(+), 90 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 41120115c792e..690cd1fbe5a3f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -19,11 +19,9 @@ package org.elasticsearch.cluster.metadata; -import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.Version; @@ -57,7 +55,6 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.PathUtils; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -78,12 +75,11 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import java.io.IOException; import java.io.UnsupportedEncodingException; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -475,9 +471,11 @@ public ClusterState execute(ClusterState currentState) throws Exception { // now, update the mappings with the actual source Map mappingsMetaData = new HashMap<>(); - for (DocumentMapper mapper : mapperService.docMappers(true)) { - MappingMetaData mappingMd = new MappingMetaData(mapper); - mappingsMetaData.put(mapper.type(), mappingMd); + for (DocumentMapper mapper : Arrays.asList(mapperService.documentMapper(), mapperService.documentMapper(MapperService.DEFAULT_MAPPING))) { + if (mapper != null) { + MappingMetaData mappingMd = new MappingMetaData(mapper); + mappingsMetaData.put(mapper.type(), mappingMd); + } } final IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(request.index()) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index fbeca652a97ff..b8e898cf6f5e3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -19,9 +19,7 @@ package org.elasticsearch.cluster.metadata; -import com.carrotsearch.hppc.cursors.ObjectCursor; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingClusterStateUpdateRequest; @@ -49,6 +47,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -175,10 +174,13 @@ private boolean refreshIndexMapping(IndexService indexService, IndexMetaData.Bui String index = indexService.index().getName(); try { List updatedTypes = new ArrayList<>(); - for (DocumentMapper mapper : indexService.mapperService().docMappers(true)) { - final String type = mapper.type(); - if (!mapper.mappingSource().equals(builder.mapping(type).source())) { - updatedTypes.add(type); + MapperService mapperService = indexService.mapperService(); + for (DocumentMapper mapper : Arrays.asList(mapperService.documentMapper(), mapperService.documentMapper(MapperService.DEFAULT_MAPPING))) { + if (mapper != null) { + final String type = mapper.type(); + if (!mapper.mappingSource().equals(builder.mapping(type).source())) { + updatedTypes.add(type); + } } } @@ -186,8 +188,10 @@ private boolean refreshIndexMapping(IndexService indexService, IndexMetaData.Bui if (updatedTypes.isEmpty() == false) { logger.warn("[{}] re-syncing mappings with cluster state because of types [{}]", index, updatedTypes); dirty = true; - for (DocumentMapper mapper : indexService.mapperService().docMappers(true)) { - builder.putMapping(new MappingMetaData(mapper)); + for (DocumentMapper mapper : Arrays.asList(mapperService.documentMapper(), mapperService.documentMapper(MapperService.DEFAULT_MAPPING))) { + if (mapper != null) { + builder.putMapping(new MappingMetaData(mapper)); + } } } } catch (Exception e) { @@ -320,8 +324,10 @@ private ClusterState applyRequest(ClusterState currentState, PutMappingClusterSt IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(indexMetaData); // Mapping updates on a single type may have side-effects on other types so we need to // update mapping metadata on all types - for (DocumentMapper mapper : mapperService.docMappers(true)) { - indexMetaDataBuilder.putMapping(new MappingMetaData(mapper.mappingSource())); + for (DocumentMapper mapper : Arrays.asList(mapperService.documentMapper(), mapperService.documentMapper(MapperService.DEFAULT_MAPPING))) { + if (mapper != null) { + indexMetaDataBuilder.putMapping(new MappingMetaData(mapper.mappingSource())); + } } builder.put(indexMetaDataBuilder); } diff --git a/server/src/main/java/org/elasticsearch/index/IndexWarmer.java b/server/src/main/java/org/elasticsearch/index/IndexWarmer.java index f8b9d9d2ef805..e06dc5d2e8156 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexWarmer.java +++ b/server/src/main/java/org/elasticsearch/index/IndexWarmer.java @@ -121,7 +121,8 @@ private static class FieldDataWarmer implements IndexWarmer.Listener { public TerminationHandle warmReader(final IndexShard indexShard, final Engine.Searcher searcher) { final MapperService mapperService = indexShard.mapperService(); final Map warmUpGlobalOrdinals = new HashMap<>(); - for (DocumentMapper docMapper : mapperService.docMappers(false)) { + DocumentMapper docMapper = mapperService.documentMapper(); + if (docMapper != null) { for (FieldMapper fieldMapper : docMapper.mappers()) { final MappedFieldType fieldType = fieldMapper.fieldType(); final String indexName = fieldType.name(); diff --git a/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java b/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java index a59af29036b7d..95e5af0afccac 100644 --- a/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java +++ b/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java @@ -233,7 +233,8 @@ public IndexWarmer.TerminationHandle warmReader(final IndexShard indexShard, fin boolean hasNested = false; final Set warmUp = new HashSet<>(); final MapperService mapperService = indexShard.mapperService(); - for (DocumentMapper docMapper : mapperService.docMappers(false)) { + DocumentMapper docMapper = mapperService.documentMapper(); + if (docMapper != null) { if (docMapper.hasNestedObjects()) { hasNested = true; for (ObjectMapper objectMapper : docMapper.objectMappers().values()) { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 2f027c0fbb998..c72187b649713 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -57,6 +57,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; @@ -64,13 +65,12 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Collectors; import static java.util.Collections.emptyMap; -import static java.util.Collections.emptySet; import static java.util.Collections.unmodifiableMap; public class MapperService extends AbstractIndexComponent implements Closeable { @@ -121,7 +121,8 @@ public enum MergeReason { private volatile String defaultMappingSource; - private volatile Map mappers = emptyMap(); + private volatile DocumentMapper mapper; + private volatile DocumentMapper defaultMapper; private volatile FieldTypeLookup fieldTypes; private volatile Map fullPathObjectMappers = emptyMap(); @@ -166,24 +167,6 @@ public boolean hasNested() { return this.hasNested; } - /** - * returns an immutable iterator over current document mappers. - * - * @param includingDefaultMapping indicates whether the iterator should contain the {@link #DEFAULT_MAPPING} document mapper. - * As is this not really an active type, you would typically set this to false - */ - public Iterable docMappers(final boolean includingDefaultMapping) { - return () -> { - final Collection documentMappers; - if (includingDefaultMapping) { - documentMappers = mappers.values(); - } else { - documentMappers = mappers.values().stream().filter(mapper -> !DEFAULT_MAPPING.equals(mapper.type())).collect(Collectors.toList()); - } - return Collections.unmodifiableCollection(documentMappers).iterator(); - }; - } - public IndexAnalyzers getIndexAnalyzers() { return this.indexAnalyzers; } @@ -212,7 +195,13 @@ public static Map parseMapping(NamedXContentRegistry xContentReg public boolean updateMapping(IndexMetaData indexMetaData) throws IOException { assert indexMetaData.getIndex().equals(index()) : "index mismatch: expected " + index() + " but was " + indexMetaData.getIndex(); // go over and add the relevant mappings (or update them) - final Set existingMappers = new HashSet<>(mappers.keySet()); + Set existingMappers = new HashSet<>(); + if (mapper != null) { + existingMappers.add(mapper.type()); + } + if (defaultMapper != null) { + existingMappers.add(DEFAULT_MAPPING); + } final Map updatedEntries; try { // only update entries if needed @@ -314,29 +303,32 @@ private synchronized Map internalMerge(Map documentMappers = new ArrayList<>(); + DocumentMapper documentMapper = null; for (Map.Entry entry : mappings.entrySet()) { String type = entry.getKey(); if (type.equals(DEFAULT_MAPPING)) { continue; } + if (documentMapper != null) { + throw new IllegalArgumentException("Cannot put multiple mappings: " + mappings.keySet()); + } + final boolean applyDefault = // the default was already applied if we are recovering reason != MergeReason.MAPPING_RECOVERY // only apply the default mapping if we don't have the type yet - && mappers.containsKey(type) == false; + && this.mapper == null; try { - DocumentMapper documentMapper = + documentMapper = documentParser.parse(type, entry.getValue(), applyDefault ? defaultMappingSourceOrLastStored : null); - documentMappers.add(documentMapper); } catch (Exception e) { throw new MapperParsingException("Failed to parse mapping [{}]: {}", e, entry.getKey(), e.getMessage()); } } - return internalMerge(defaultMapper, defaultMappingSource, documentMappers, reason); + return internalMerge(defaultMapper, defaultMappingSource, documentMapper, reason); } static void validateTypeName(String type) { @@ -361,13 +353,12 @@ static void validateTypeName(String type) { } private synchronized Map internalMerge(@Nullable DocumentMapper defaultMapper, @Nullable String defaultMappingSource, - List documentMappers, MergeReason reason) { + DocumentMapper mapper, MergeReason reason) { boolean hasNested = this.hasNested; Map fullPathObjectMappers = this.fullPathObjectMappers; FieldTypeLookup fieldTypes = this.fieldTypes; - Map mappers = new HashMap<>(this.mappers); - Map results = new LinkedHashMap<>(documentMappers.size() + 1); + Map results = new LinkedHashMap<>(2); if (defaultMapper != null) { if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_7_0_0_alpha1)) { @@ -378,27 +369,23 @@ private synchronized Map internalMerge(@Nullable Documen "cannot have more than one type"); } assert defaultMapper.type().equals(DEFAULT_MAPPING); - mappers.put(DEFAULT_MAPPING, defaultMapper); results.put(DEFAULT_MAPPING, defaultMapper); } { - Set actualTypes = new HashSet<>(mappers.keySet()); - documentMappers.forEach(mapper -> actualTypes.add(mapper.type())); - actualTypes.remove(DEFAULT_MAPPING); - if (actualTypes.size() > 1) { + if (mapper != null && this.mapper != null && Objects.equals(this.mapper.type(), mapper.type()) == false) { throw new IllegalArgumentException( - "Rejecting mapping update to [" + index().getName() + "] as the final mapping would have more than 1 type: " + actualTypes); + "Rejecting mapping update to [" + index().getName() + "] as the final mapping would have more than 1 type: " + Arrays.asList(this.mapper.type(), mapper.type())); } } - for (DocumentMapper mapper : documentMappers) { + DocumentMapper newMapper = null; + if (mapper != null) { // check naming validateTypeName(mapper.type()); // compute the merged DocumentMapper - DocumentMapper oldMapper = mappers.get(mapper.type()); - DocumentMapper newMapper; + DocumentMapper oldMapper = this.mapper; if (oldMapper != null) { newMapper = oldMapper.merge(mapper.mapping()); } else { @@ -442,7 +429,6 @@ private synchronized Map internalMerge(@Nullable Documen } results.put(newMapper.type(), newMapper); - mappers.put(newMapper.type(), newMapper); } if (reason == MergeReason.MAPPING_UPDATE) { @@ -456,24 +442,16 @@ private synchronized Map internalMerge(@Nullable Documen } checkIndexSortCompatibility(indexSettings.getIndexSortConfig(), hasNested); - for (Map.Entry entry : mappers.entrySet()) { - if (entry.getKey().equals(DEFAULT_MAPPING)) { - continue; - } - DocumentMapper documentMapper = entry.getValue(); - // apply changes to the field types back - DocumentMapper updatedDocumentMapper = documentMapper.updateFieldType(fieldTypes.fullNameToFieldType); - if (updatedDocumentMapper != documentMapper) { + if (newMapper != null) { + DocumentMapper updatedDocumentMapper = newMapper.updateFieldType(fieldTypes.fullNameToFieldType); + if (updatedDocumentMapper != newMapper) { // update both mappers and result - entry.setValue(updatedDocumentMapper); - if (results.containsKey(updatedDocumentMapper.type())) { - results.put(updatedDocumentMapper.type(), updatedDocumentMapper); - } + newMapper = updatedDocumentMapper; + results.put(updatedDocumentMapper.type(), updatedDocumentMapper); } } // make structures immutable - mappers = Collections.unmodifiableMap(mappers); results = Collections.unmodifiableMap(results); // only need to immutably rewrap these if the previous reference was changed. @@ -486,7 +464,10 @@ private synchronized Map internalMerge(@Nullable Documen if (defaultMappingSource != null) { this.defaultMappingSource = defaultMappingSource; } - this.mappers = mappers; + if (newMapper != null) { + this.mapper = newMapper; + } + this.defaultMapper = defaultMapper; this.fieldTypes = fieldTypes; this.hasNested = hasNested; this.fullPathObjectMappers = fullPathObjectMappers; @@ -498,7 +479,7 @@ private synchronized Map internalMerge(@Nullable Documen } private boolean assertMappersShareSameFieldType() { - for (DocumentMapper mapper : docMappers(false)) { + if (mapper != null) { List fieldMappers = new ArrayList<>(); Collections.addAll(fieldMappers, mapper.mapping().metadataMappers); MapperUtils.collect(mapper.root(), new ArrayList<>(), fieldMappers); @@ -692,18 +673,20 @@ public DocumentMapper parse(String mappingType, CompressedXContent mappingSource return documentParser.parse(mappingType, mappingSource, applyDefault ? defaultMappingSource : null); } - public boolean hasMapping(String mappingType) { - return mappers.containsKey(mappingType); + /** + * Get the set of types. + * @deprecated Indices may have one type at most, use {@link #documentMapper()} instead. + */ + @Deprecated + public Set types() { + return mapper == null ? Collections.emptySet() : Collections.singleton(mapper.type()); } /** - * Return the set of concrete types that have a mapping. - * NOTE: this does not return the default mapping. + * Return the document mapper, or {@code null} if no mapping has been put yet. */ - public Collection types() { - final Set types = new HashSet<>(mappers.keySet()); - types.remove(DEFAULT_MAPPING); - return Collections.unmodifiableSet(types); + public DocumentMapper documentMapper() { + return mapper; } /** @@ -712,7 +695,13 @@ public Collection types() { * the default mapping. */ public DocumentMapper documentMapper(String type) { - return mappers.get(type); + if (mapper != null && type.equals(mapper.type())) { + return mapper; + } + if (DEFAULT_MAPPING.equals(type)) { + return defaultMapper; + } + return null; } /** @@ -720,7 +709,7 @@ public DocumentMapper documentMapper(String type) { * type has been dynamically created. */ public DocumentMapperForType documentMapperWithAutoCreate(String type) { - DocumentMapper mapper = mappers.get(type); + DocumentMapper mapper = documentMapper(type); if (mapper != null) { return new DocumentMapperForType(mapper, null); } @@ -836,7 +825,7 @@ protected Analyzer getWrappedAnalyzer(String fieldName) { /** Return a term that uniquely identifies the document, or {@code null} if the type is not allowed. */ public Term createUidTerm(String type, String id) { - if (hasMapping(type) == false) { + if (mapper == null || mapper.type().equals(type) == false) { return null; } return new Term(IdFieldMapper.NAME, Uid.encodeId(id)); diff --git a/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java b/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java index 18a124d86b35c..c3a695beff083 100644 --- a/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java +++ b/server/src/main/java/org/elasticsearch/index/search/QueryParserHelper.java @@ -89,7 +89,8 @@ public static Map parseFieldsAndWeights(List fields) { * @param field The field name to search. */ public static FieldMapper getFieldMapper(MapperService mapperService, String field) { - for (DocumentMapper mapper : mapperService.docMappers(true)) { + DocumentMapper mapper = mapperService.documentMapper(); + if (mapper != null) { FieldMapper fieldMapper = mapper.mappers().smartNameFieldMapper(field); if (fieldMapper != null) { return fieldMapper; diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java index 211ae48d04355..49df78565d315 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexCreationTaskTests.java @@ -433,7 +433,7 @@ private void setupIndicesService() throws Exception { when(docMapper.routingFieldMapper()).thenReturn(routingMapper); - when(mapper.docMappers(anyBoolean())).thenReturn(Collections.singletonList(docMapper)); + when(mapper.documentMapper()).thenReturn(docMapper); final Index index = new Index("target", "tgt1234"); final Supplier supplier = mock(Supplier.class); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java index 59ef784aea3a2..2365dec69ecb5 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -119,7 +119,7 @@ public void testIndexIntoDefaultMapping() throws Throwable { } else { throw e; } - assertFalse(indexService.mapperService().hasMapping(MapperService.DEFAULT_MAPPING)); + assertNull(indexService.mapperService().documentMapper(MapperService.DEFAULT_MAPPING)); } public void testTotalFieldsExceedsLimit() throws Throwable { diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index 3f7f0583593af..8af19aa9ac1e4 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -148,7 +148,7 @@ public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool th when(indexService.index()).thenReturn(indexMetaData.getIndex()); MapperService mapperService = mock(MapperService.class); when(indexService.mapperService()).thenReturn(mapperService); - when(mapperService.docMappers(anyBoolean())).thenReturn(Collections.emptyList()); + when(mapperService.documentMapper()).thenReturn(null); when(indexService.getIndexEventListener()).thenReturn(new IndexEventListener() {}); when(indexService.getIndexSortSupplier()).thenReturn(() -> null); return indexService; From 9c8ebb608fa7f33810d0878853b29056fbd3924f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 17 Apr 2018 18:18:21 +0200 Subject: [PATCH 06/16] Remove `flatSettings` support from request classes (#29560) As part of adding support for new API to the high-level REST client, we added support for the `flat_settings` parameter to some of our request classes. We added documentation that such flag is only ever read by the high-level REST client, but the truth is that it doesn't do anything given that settings are always parsed back into a `Settings` object, no matter whether they are returned in a flat format or not. It was a mistake to add support for this flag in the context of the high-level REST client, hence this commit removes it. --- .../org/elasticsearch/client/Request.java | 3 --- .../elasticsearch/client/RequestTests.java | 13 ---------- .../ClusterClientDocumentationIT.java | 4 ---- .../IndicesClientDocumentationIT.java | 7 +----- .../high-level/cluster/put_settings.asciidoc | 7 ------ .../indices/indices_exists.asciidoc | 3 +-- .../high-level/indices/put_settings.asciidoc | 7 ------ .../ClusterUpdateSettingsRequest.java | 24 ------------------- .../admin/indices/get/GetIndexRequest.java | 23 ------------------ .../settings/put/UpdateSettingsRequest.java | 24 ------------------- .../UpdateSettingsRequestStreamableTests.java | 2 -- 11 files changed, 2 insertions(+), 115 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java index 4e6fcdbb8dd4a..500130ed39705 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java @@ -572,7 +572,6 @@ private static Request resize(ResizeRequest resizeRequest) throws IOException { static Request clusterPutSettings(ClusterUpdateSettingsRequest clusterUpdateSettingsRequest) throws IOException { Params parameters = Params.builder(); - parameters.withFlatSettings(clusterUpdateSettingsRequest.flatSettings()); parameters.withTimeout(clusterUpdateSettingsRequest.timeout()); parameters.withMasterTimeout(clusterUpdateSettingsRequest.masterNodeTimeout()); HttpEntity entity = createEntity(clusterUpdateSettingsRequest, REQUEST_BODY_CONTENT_TYPE); @@ -603,7 +602,6 @@ static Request indicesExist(GetIndexRequest request) { params.withLocal(request.local()); params.withHuman(request.humanReadable()); params.withIndicesOptions(request.indicesOptions()); - params.withFlatSettings(request.flatSettings()); params.withIncludeDefaults(request.includeDefaults()); return new Request(HttpHead.METHOD_NAME, endpoint, params.getParams(), null); } @@ -613,7 +611,6 @@ static Request indexPutSettings(UpdateSettingsRequest updateSettingsRequest) thr parameters.withTimeout(updateSettingsRequest.timeout()); parameters.withMasterTimeout(updateSettingsRequest.masterNodeTimeout()); parameters.withIndicesOptions(updateSettingsRequest.indicesOptions()); - parameters.withFlatSettings(updateSettingsRequest.flatSettings()); parameters.withPreserveExisting(updateSettingsRequest.isPreserveExisting()); String[] indices = updateSettingsRequest.indices() == null ? Strings.EMPTY_ARRAY : updateSettingsRequest.indices(); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java index abce180546dfc..f691c60daa5da 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java @@ -272,7 +272,6 @@ public void testIndicesExist() { Map expectedParams = new HashMap<>(); setRandomIndicesOptions(getIndexRequest::indicesOptions, getIndexRequest::indicesOptions, expectedParams); setRandomLocal(getIndexRequest, expectedParams); - setRandomFlatSettings(getIndexRequest::flatSettings, expectedParams); setRandomHumanReadable(getIndexRequest, expectedParams); setRandomIncludeDefaults(getIndexRequest, expectedParams); @@ -1292,7 +1291,6 @@ private static void resizeTest(ResizeType resizeType, CheckedFunction expectedParams = new HashMap<>(); - setRandomFlatSettings(request::flatSettings, expectedParams); setRandomMasterTimeout(request, expectedParams); setRandomTimeout(request::timeout, AcknowledgedRequest.DEFAULT_ACK_TIMEOUT, expectedParams); @@ -1344,7 +1342,6 @@ public void testIndexPutSettings() throws IOException { String[] indices = randomBoolean() ? null : randomIndicesNames(0, 2); UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(indices); Map expectedParams = new HashMap<>(); - setRandomFlatSettings(updateSettingsRequest::flatSettings, expectedParams); setRandomMasterTimeout(updateSettingsRequest, expectedParams); setRandomTimeout(updateSettingsRequest::timeout, AcknowledgedRequest.DEFAULT_ACK_TIMEOUT, expectedParams); setRandomIndicesOptions(updateSettingsRequest::indicesOptions, updateSettingsRequest::indicesOptions, expectedParams); @@ -1627,16 +1624,6 @@ private static void setRandomTimeout(Consumer setter, TimeValue defaultT } } - private static void setRandomFlatSettings(Consumer setter, Map expectedParams) { - if (randomBoolean()) { - boolean flatSettings = randomBoolean(); - setter.accept(flatSettings); - if (flatSettings) { - expectedParams.put("flat_settings", String.valueOf(flatSettings)); - } - } - } - private static void setRandomMasterTimeout(MasterNodeRequest request, Map expectedParams) { if (randomBoolean()) { String masterTimeout = randomTimeValue(); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java index 0747ca757c4b9..2e7ea1650f424 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java @@ -124,10 +124,6 @@ public void testClusterPutSettings() throws IOException { request.masterNodeTimeout("1m"); // <2> // end::put-settings-request-masterTimeout - // tag::put-settings-request-flat-settings - request.flatSettings(true); // <1> - // end::put-settings-request-flat-settings - // tag::put-settings-execute ClusterUpdateSettingsResponse response = client.cluster().putSettings(request); // end::put-settings-execute diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java index e33d1e4729b0e..24c321f87f998 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java @@ -58,7 +58,6 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.ESRestHighLevelClientTestCase; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; @@ -114,8 +113,7 @@ public void testIndicesExist() throws IOException { request.local(false); // <1> request.humanReadable(true); // <2> request.includeDefaults(false); // <3> - request.flatSettings(false); // <4> - request.indicesOptions(indicesOptions); // <5> + request.indicesOptions(indicesOptions); // <4> // end::indices-exists-request-optionals // tag::indices-exists-response @@ -1433,9 +1431,6 @@ public void testIndexPutSettings() throws Exception { // end::put-settings-settings-source } - // tag::put-settings-request-flat-settings - request.flatSettings(true); // <1> - // end::put-settings-request-flat-settings // tag::put-settings-request-preserveExisting request.setPreserveExisting(false); // <1> // end::put-settings-request-preserveExisting diff --git a/docs/java-rest/high-level/cluster/put_settings.asciidoc b/docs/java-rest/high-level/cluster/put_settings.asciidoc index 74b479faa0501..dc9b1679d4717 100644 --- a/docs/java-rest/high-level/cluster/put_settings.asciidoc +++ b/docs/java-rest/high-level/cluster/put_settings.asciidoc @@ -54,13 +54,6 @@ include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-settings-setti ==== Optional Arguments The following arguments can optionally be provided: -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-settings-request-flat-settings] --------------------------------------------------- -<1> Whether the updated settings returned in the `ClusterUpdateSettings` should -be in a flat format - ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-settings-request-timeout] diff --git a/docs/java-rest/high-level/indices/indices_exists.asciidoc b/docs/java-rest/high-level/indices/indices_exists.asciidoc index 4a227db49ed8c..ee744e97ce8bd 100644 --- a/docs/java-rest/high-level/indices/indices_exists.asciidoc +++ b/docs/java-rest/high-level/indices/indices_exists.asciidoc @@ -23,8 +23,7 @@ include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[indices-exists-req <1> Whether to return local information or retrieve the state from master node <2> Return result in a format suitable for humans <3> Whether to return all default setting for each of the indices -<4> Return settings in flat format -<5> Controls how unavailable indices are resolved and how wildcard expressions are expanded +<4> Controls how unavailable indices are resolved and how wildcard expressions are expanded [[java-rest-high-indices-sync]] ==== Synchronous Execution diff --git a/docs/java-rest/high-level/indices/put_settings.asciidoc b/docs/java-rest/high-level/indices/put_settings.asciidoc index 49312da82a400..c305eeaa0965b 100644 --- a/docs/java-rest/high-level/indices/put_settings.asciidoc +++ b/docs/java-rest/high-level/indices/put_settings.asciidoc @@ -55,13 +55,6 @@ include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-settings-setti ==== Optional Arguments The following arguments can optionally be provided: -["source","java",subs="attributes,callouts,macros"] --------------------------------------------------- -include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-settings-request-flat-settings] --------------------------------------------------- -<1> Whether the updated settings returned in the `UpdateSettings` should -be in a flat format - ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- include-tagged::{doc-tests}/IndicesClientDocumentationIT.java[put-settings-request-preserveExisting] diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsRequest.java index 38d3a9d5caf54..f13c30c53503b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/settings/ClusterUpdateSettingsRequest.java @@ -58,7 +58,6 @@ public class ClusterUpdateSettingsRequest extends AcknowledgedRequest r.transientSettings = t, (p, c) -> Settings.fromXContent(p), TRANSIENT); } - private boolean flatSettings = false; private Settings transientSettings = EMPTY_SETTINGS; private Settings persistentSettings = EMPTY_SETTINGS; @@ -74,29 +73,6 @@ public ActionRequestValidationException validate() { return validationException; } - /** - * Sets the value of "flat_settings". - * Used only by the high-level REST client. - * - * @param flatSettings - * value of "flat_settings" flag to be set - * @return this request - */ - public ClusterUpdateSettingsRequest flatSettings(boolean flatSettings) { - this.flatSettings = flatSettings; - return this; - } - - /** - * Return settings in flat format. - * Used only by the high-level REST client. - * - * @return true if settings need to be returned in flat format; false otherwise. - */ - public boolean flatSettings() { - return flatSettings; - } - public Settings transientSettings() { return transientSettings; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java index 7ca9a9f11956d..02a7a8ad79fbe 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexRequest.java @@ -66,7 +66,6 @@ public static Feature fromId(byte id) { private static final Feature[] DEFAULT_FEATURES = new Feature[] { Feature.ALIASES, Feature.MAPPINGS, Feature.SETTINGS }; private Feature[] features = DEFAULT_FEATURES; private boolean humanReadable = false; - private transient boolean flatSettings = false; private transient boolean includeDefaults = false; public GetIndexRequest() { @@ -118,28 +117,6 @@ public boolean humanReadable() { return humanReadable; } - /** - * Sets the value of "flat_settings". - * Used only by the high-level REST client. - * - * @param flatSettings value of "flat_settings" flag to be set - * @return this request - */ - public GetIndexRequest flatSettings(boolean flatSettings) { - this.flatSettings = flatSettings; - return this; - } - - /** - * Return settings in flat format. - * Used only by the high-level REST client. - * - * @return true if settings need to be returned in flat format; false otherwise. - */ - public boolean flatSettings() { - return flatSettings; - } - /** * Sets the value of "include_defaults". * Used only by the high-level REST client. diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequest.java index 594564b681562..18c7d506c7275 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequest.java @@ -55,7 +55,6 @@ public class UpdateSettingsRequest extends AcknowledgedRequesttrue if settings need to be returned in flat format; false otherwise. - */ - public boolean flatSettings() { - return flatSettings; - } - @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequestStreamableTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequestStreamableTests.java index 463049a8c3c1f..114af3c13e707 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequestStreamableTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/settings/put/UpdateSettingsRequestStreamableTests.java @@ -67,7 +67,6 @@ public static UpdateSettingsRequest createTestItem() { request.timeout(randomTimeValue()); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean())); request.setPreserveExisting(randomBoolean()); - request.flatSettings(randomBoolean()); return request; } @@ -77,7 +76,6 @@ private static UpdateSettingsRequest copyRequest(UpdateSettingsRequest request) result.timeout(request.timeout()); result.indicesOptions(request.indicesOptions()); result.setPreserveExisting(request.isPreserveExisting()); - result.flatSettings(request.flatSettings()); return result; } From cfc9d12acc3ffc6748108d3ce2749d4f5bf4ed41 Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Tue, 17 Apr 2018 17:33:18 +0000 Subject: [PATCH 07/16] [TEST] test against scaled value instead of fixed epsilon in MovAvgIT When comparing doubles, fixed epsilons can fail because the absolute difference in values may be quite large, even though the relative difference is tiny (e.g. with two very large numbers). Instead, we can scale epsilon by the absolute value of the expected value. This means we are looking for a diff that is epsilon-percent away from the value, rather than just epsilon. This is basically checking the relative error using junit's assertEqual. Closes #29456, unmutes the test --- .../search/aggregations/pipeline/moving/avg/MovAvgIT.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java index 159b7e28b1269..695a5777a5302 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java @@ -68,7 +68,6 @@ import static org.hamcrest.core.IsNull.notNullValue; import static org.hamcrest.core.IsNull.nullValue; -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/29456") @ESIntegTestCase.SuiteScopeTestCase public class MovAvgIT extends ESIntegTestCase { private static final String INTERVAL_FIELD = "l_value"; @@ -1296,7 +1295,7 @@ private void assertBucketContents(Histogram.Bucket actual, Double expectedCount, } else { assertThat("[_count] movavg is null", countMovAvg, notNullValue()); assertEquals("[_count] movavg does not match expected [" + countMovAvg.value() + " vs " + expectedCount + "]", - countMovAvg.value(), expectedCount, 0.1); + countMovAvg.value(), expectedCount, 0.1 * Math.abs(countMovAvg.value())); } // This is a gap bucket @@ -1308,7 +1307,7 @@ private void assertBucketContents(Histogram.Bucket actual, Double expectedCount, } else { assertThat("[value] movavg is null", valuesMovAvg, notNullValue()); assertEquals("[value] movavg does not match expected [" + valuesMovAvg.value() + " vs " + expectedValue + "]", - valuesMovAvg.value(), expectedValue, 0.1); + valuesMovAvg.value(), expectedValue, 0.1 * Math.abs(countMovAvg.value())); } } From 7969eb7db77aa2ccf776b34dd8be68a2cd962a0b Mon Sep 17 00:00:00 2001 From: Dimitris Athanasiou Date: Tue, 17 Apr 2018 18:46:49 +0100 Subject: [PATCH 08/16] Add versions 5.6.10 and 6.2.5 --- server/src/main/java/org/elasticsearch/Version.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index be56f01fa2dc2..5b0d85b4bfaa3 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -117,6 +117,8 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_5_6_8 = new Version(V_5_6_8_ID, org.apache.lucene.util.Version.LUCENE_6_6_1); public static final int V_5_6_9_ID = 5060999; public static final Version V_5_6_9 = new Version(V_5_6_9_ID, org.apache.lucene.util.Version.LUCENE_6_6_1); + public static final int V_5_6_10_ID = 50601099; + public static final Version V_5_6_10 = new Version(V_5_6_10_ID, org.apache.lucene.util.Version.LUCENE_6_6_1); public static final int V_6_0_0_alpha1_ID = 6000001; public static final Version V_6_0_0_alpha1 = new Version(V_6_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_7_0_0); @@ -163,6 +165,8 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_6_2_3 = new Version(V_6_2_3_ID, LUCENE_7_2_1); public static final int V_6_2_4_ID = 6020499; public static final Version V_6_2_4 = new Version(V_6_2_4_ID, LUCENE_7_2_1); + public static final int V_6_2_5_ID = 6020599; + public static final Version V_6_2_5 = new Version(V_6_2_5_ID, LUCENE_7_2_1); public static final int V_6_3_0_ID = 6030099; public static final Version V_6_3_0 = new Version(V_6_3_0_ID, org.apache.lucene.util.Version.LUCENE_7_3_0); public static final int V_7_0_0_alpha1_ID = 7000001; @@ -185,6 +189,8 @@ public static Version fromId(int id) { return V_7_0_0_alpha1; case V_6_3_0_ID: return V_6_3_0; + case V_6_2_5_ID: + return V_6_2_5; case V_6_2_4_ID: return V_6_2_4; case V_6_2_3_ID: @@ -221,6 +227,8 @@ public static Version fromId(int id) { return V_6_0_0_alpha2; case V_6_0_0_alpha1_ID: return V_6_0_0_alpha1; + case V_5_6_10_ID: + return V_5_6_10; case V_5_6_9_ID: return V_5_6_9; case V_5_6_8_ID: From c833167d8419b77607cafdc6dbffac2cf5fd468a Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Tue, 17 Apr 2018 12:16:08 -0700 Subject: [PATCH 09/16] Painless Spec Documentation Clean Up (#29441) Created a flatter structure for the different sections. Cleaned up comments, keywords, and literals. Used callouts for examples where it made sense. --- docs/painless/index.asciidoc | 33 ---- docs/painless/painless-api-reference.asciidoc | 24 +-- docs/painless/painless-casting.asciidoc | 172 +++++++++++++++++ docs/painless/painless-comments.asciidoc | 51 +++++ docs/painless/painless-description.asciidoc | 2 +- ...iidoc => painless-general-syntax.asciidoc} | 40 +--- docs/painless/painless-keywords.asciidoc | 13 ++ docs/painless/painless-lang-spec.asciidoc | 73 ++------ docs/painless/painless-literals.asciidoc | 157 ++++++++++------ docs/painless/painless-operators.asciidoc | 1 + docs/painless/painless-types.asciidoc | 177 +----------------- docs/painless/painless-variables.asciidoc | 23 ++- docs/reference/redirects.asciidoc | 2 +- 13 files changed, 384 insertions(+), 384 deletions(-) create mode 100644 docs/painless/painless-casting.asciidoc create mode 100644 docs/painless/painless-comments.asciidoc rename docs/painless/{painless-syntax.asciidoc => painless-general-syntax.asciidoc} (72%) create mode 100644 docs/painless/painless-keywords.asciidoc diff --git a/docs/painless/index.asciidoc b/docs/painless/index.asciidoc index 4898ed933363b..abfd4d4f00abe 100644 --- a/docs/painless/index.asciidoc +++ b/docs/painless/index.asciidoc @@ -5,39 +5,6 @@ include::../Versions.asciidoc[] include::painless-getting-started.asciidoc[] -// include::painless-examples.asciidoc[] - -// include::painless-design.asciidoc[] - include::painless-lang-spec.asciidoc[] -include::painless-syntax.asciidoc[] - include::painless-api-reference.asciidoc[] - -//// -Proposed Outline (WIP) -Getting Started with Painless - Accessing Doc Values - Updating Fields - Working with Dates - Using Regular Expressions - Debugging Painless Scripts - -Example Scripts - Using Painless in Script Fields - Using Painless in Watches - Using Painless in Function Score Queries - Using Painless in Script Queries - Using Painless When Updating Docs - Using Painless When Reindexing - -How Painless Works - Painless Architecture - Dispatching Functions - -Painless Language Specification -Painless API -//// - -Painless API Reference diff --git a/docs/painless/painless-api-reference.asciidoc b/docs/painless/painless-api-reference.asciidoc index 1bda6d890c859..54b1f20977b61 100644 --- a/docs/painless/painless-api-reference.asciidoc +++ b/docs/painless/painless-api-reference.asciidoc @@ -1,17 +1,13 @@ -["appendix",id="painless-api-reference"] -= Painless API Reference +[[painless-api-reference]] +== Painless API Reference -Painless has a strict whitelist for methods and -classes to make sure that all painless scripts are secure and fast. Most of -these methods are exposed directly from the JRE while others are part of -Elasticsearch or Painless itself. Below is a list of all available methods -grouped under the classes on which you can call them. Clicking on the method -name takes you to the documentation for the method. - -NOTE: Methods defined in the JRE also have a `(java 9)` link which can be used -to see the method's documentation in Java 9 while clicking on the method's name -goes to the Java 8 documentation. Usually these aren't different but it is -worth going to the version that matches the version of Java you are using to -run Elasticsearch just in case. +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 Enviroment (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. include::painless-api-reference/index.asciidoc[] diff --git a/docs/painless/painless-casting.asciidoc b/docs/painless/painless-casting.asciidoc new file mode 100644 index 0000000000000..ec4f9919bd043 --- /dev/null +++ b/docs/painless/painless-casting.asciidoc @@ -0,0 +1,172 @@ +[[painless-casting]] +=== Casting + +Casting is the conversion of one type to another. Implicit casts are casts that +occur automatically, such as during an assignment operation. Explicit casts are +casts where you use the casting operator to explicitly convert one type to +another. This is necessary during operations where the cast cannot be inferred. + +To cast to a new type, precede the expression by the new type enclosed in +parentheses, for example +`(int)x`. + +The following sections specify the implicit casts that can be performed and the +explicit casts that are allowed. The only other permitted cast is casting +a single character `String` to a `char`. + +*Grammar:* +[source,ANTLR4] +---- +cast: '(' TYPE ')' expression +---- + +[[numeric-casting]] +==== Numeric Casting + +The following table shows the allowed implicit and explicit casts between +numeric types. Read the table by row. To find out if you need to explicitly +cast from type A to type B, find the row for type A and scan across to the +column for type B. + +IMPORTANT: Explicit casts between numeric types can result in some data loss. A +smaller numeric type cannot necessarily accommodate the value from a larger +numeric type. You might also lose precision when casting from integer types +to floating point types. + +|==== +| | byte | short | char | int | long | float | double +| byte | | implicit | implicit | implicit | implicit | implicit | implicit +| short | explicit | | explicit | implicit | implicit | implicit | implicit +| char | explicit | explicit | | implicit | implicit | implicit | implicit +| int | explicit | explicit | explicit | | implicit | implicit | implicit +| long | explicit | explicit | explicit | explicit | | implicit | implicit +| float | explicit | explicit | explicit | explicit | explicit | | implicit +| double | explicit | explicit | explicit | explicit | explicit | explicit | +|==== + + +Example(s) +[source,Java] +---- +int a = 1; // Declare int variable a and set it to the literal + // value 1 +long b = a; // Declare long variable b and set it to int variable + // a with an implicit cast to convert from int to long +short c = (short)b; // Declare short variable c, explicitly cast b to a + // short, and assign b to c +byte d = a; // ERROR: Casting an int to a byte requires an explicit + // cast +double e = (double)a; // Explicitly cast int variable a to a double and assign + // it to the double variable e. The explicit cast is + // allowed, but it is not necessary. +---- + +[[reference-casting]] +==== Reference Casting + +A reference type can be implicitly cast to another reference type as long as +the type being cast _from_ is a descendant of the type being cast _to_. A +reference type can be explicitly cast _to_ if the type being cast to is a +descendant of the type being cast _from_. + +*Examples:* +[source,Java] +---- +List x; // Declare List variable x +ArrayList y = new ArrayList(); // Declare ArrayList variable y and assign it a + // newly allocated ArrayList [1] +x = y; // Assign Arraylist y to List x using an + // implicit cast +y = (ArrayList)x; // Explicitly cast List x to an ArrayList and + // assign it to ArrayList y +x = (List)y; // Set List x to ArrayList y using an explicit + // cast (the explicit cast is not necessary) +y = x; // ERROR: List x cannot be implicitly cast to + // an ArrayList, an explicit cast is required +Map m = y; // ERROR: Cannot implicitly or explicitly cast [2] + // an ArrayList to a Map, no relationship + // exists between the two types. +---- +[1] `ArrayList` is a descendant of the `List` type. +[2] `Map` is unrelated to the `List` and `ArrayList` types. + +[[def-type-casting]] +==== def Type Casting +All primitive and reference types can always be implicitly cast to +`def`. While it is possible to explicitly cast to `def`, it is not necessary. + +However, it is not always possible to implicitly cast a `def` to other +primitive and reference types. An explicit cast is required if an explicit +cast would normally be required between the non-def types. + + +*Examples:* +[source,Java] +---- +def x; // Declare def variable x and set it to null +x = 3; // Set the def variable x to the literal 3 with an implicit + // cast from int to def +double a = x; // Declare double variable a and set it to def variable x, + // which contains a double +int b = x; // ERROR: Results in a run-time error because an explicit cast is + // required to cast from a double to an int +int c = (int)x; // Declare int variable c, explicitly cast def variable x to an + // int, and assign x to c +---- + +[[boxing-unboxing]] +==== Boxing and Unboxing + +Boxing is where a cast is used to convert a primitive type to its corresponding +reference type. Unboxing is the reverse, converting a reference type to the +corresponding primitive type. + +There are two places Painless performs implicit boxing and unboxing: + +* When you call methods, Painless automatically boxes and unboxes arguments +so you can specify either primitive types or their corresponding reference +types. +* When you use the `def` type, Painless automatically boxes and unboxes as +needed when converting to and from `def`. + +The casting operator does not support any way to explicitly box a primitive +type or unbox a reference type. + +If a primitive type needs to be converted to a reference type, the Painless +reference type API supports methods that can do that. However, under normal +circumstances this should not be necessary. + +*Examples:* +[source,Java] +---- +Integer x = 1; // ERROR: not a legal implicit cast +Integer y = (Integer)1; // ERROR: not a legal explicit cast +int a = new Integer(1); // ERROR: not a legal implicit cast +int b = (int)new Integer(1); // ERROR: not a legal explicit cast +---- + +[[promotion]] +==== Promotion + +Promotion is where certain operations require types to be either a minimum +numerical type or for two (or more) types to be equivalent. +The documentation for each operation that has these requirements +includes promotion tables that describe how this is handled. + +When an operation promotes a type or types, the resultant type +of the operation is the promoted type. Types can be promoted to def +at compile-time; however, at run-time, the resultant type will be the +promotion of the types the `def` is representing. + +*Examples:* +[source,Java] +---- +2 + 2.0 // Add the literal int 2 and the literal double 2.0. The literal + // 2 is promoted to a double and the resulting value is a double. + +def x = 1; // Declare def variable x and set it to the literal int 1 through + // an implicit cast +x + 2.0F // Add def variable x and the literal float 2.0. + // At compile-time the types are promoted to def. + // At run-time the types are promoted to float. +---- diff --git a/docs/painless/painless-comments.asciidoc b/docs/painless/painless-comments.asciidoc new file mode 100644 index 0000000000000..d1d2e47a143a6 --- /dev/null +++ b/docs/painless/painless-comments.asciidoc @@ -0,0 +1,51 @@ +[[painless-comments]] +=== Comments + +Painless supports both single-line and multi-line comments. Comments can be +included anywhere within a script. Use the `//` token anywhere on a line to +specify a single-line comment. All characters from the `//` token to the end +of the line are ignored. Use an opening `/*` token and a closing `*/` token +to specify a multi-line comment. Multi-line comments can start anywhere on a +line, and all characters in between the `/*` token and `*/` token are ignored. + +*Grammar* +[source,ANTLR4] +---- +SINGLE_LINE_COMMENT: '//' .*? [\n\r]; +MULTI_LINE_COMMENT: '/*' .*? '*/'; +---- + +*Examples* + +Single-line comments. + +[source,Painless] +---- +// single-line comment + +int value; // single-line comment +---- + +Multi-line comments. + +[source,Painless] +---- +/* multi- + line + comment */ + +int value; /* multi- + line + comment */ value = 0; + +int value; /* multi-line + comment */ + +/* multi-line + comment */ int value; + +int value; /* multi-line + comment */ value = 0; + +int value; /* multi-line comment */ value = 0; +---- diff --git a/docs/painless/painless-description.asciidoc b/docs/painless/painless-description.asciidoc index 874eab5632cfb..dfaf66ca26d4b 100644 --- a/docs/painless/painless-description.asciidoc +++ b/docs/painless/painless-description.asciidoc @@ -2,7 +2,7 @@ _Painless_ is a simple, secure scripting language designed specifically for use with Elasticsearch. It is the default scripting language for Elasticsearch and can safely be used for inline and stored scripts. For a detailed description of the Painless syntax and language features, see the -{painless}/painless-specification.html[Painless Language Specification]. +{painless}/painless-lang-spec.html[Painless Language Specification]. [[painless-features]] You can use Painless anywhere scripts can be used in Elasticsearch. Painless diff --git a/docs/painless/painless-syntax.asciidoc b/docs/painless/painless-general-syntax.asciidoc similarity index 72% rename from docs/painless/painless-syntax.asciidoc rename to docs/painless/painless-general-syntax.asciidoc index c68ed5168c01b..114bff80bfa70 100644 --- a/docs/painless/painless-syntax.asciidoc +++ b/docs/painless/painless-general-syntax.asciidoc @@ -1,7 +1,6 @@ -[[painless-syntax]] -=== Painless Syntax +[[painless-general-syntax]] +=== General Syntax -[float] [[control-flow]] ==== Control flow @@ -17,7 +16,6 @@ for (item : list) { } --------------------------------------------------------- -[float] [[functions]] ==== Functions @@ -32,7 +30,6 @@ if (isNegative(someVar)) { } --------------------------------------------------------- -[float] [[lambda-expressions]] ==== Lambda expressions Lambda expressions and method references work the same as in https://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html[Java]. @@ -49,7 +46,6 @@ list.sort(Integer::compare); You can make method references to functions within the script with `this`, for example `list.sort(this::mycompare)`. -[float] [[patterns]] ==== Patterns @@ -62,7 +58,6 @@ are always constants and compiled efficiently a single time. Pattern p = /[aeiou]/ --------------------------------------------------------- -[float] [[pattern-flags]] ===== Pattern flags @@ -84,34 +79,3 @@ Pattern class] using these characters: |`u` | UNICODE_CASE | `'Ɛ' ==~ /ɛ/iu` |`x` | COMMENTS (aka extended) | `'a' ==~ /a #comment/x` |======================================================================= - -[float] -[[painless-deref]] -==== Dereferences - -Like lots of languages, Painless uses `.` to reference fields and call methods: - -[source,painless] ---------------------------------------------------------- -String foo = 'foo'; -TypeWithGetterOrPublicField bar = new TypeWithGetterOrPublicField() -return foo.length() + bar.x ---------------------------------------------------------- - -Like Groovy, Painless uses `?.` to perform null-safe references, with the -result being `null` if the left hand side is `null`: - -[source,painless] ---------------------------------------------------------- -String foo = null; -return foo?.length() // Returns null ---------------------------------------------------------- - -Unlike Groovy, Painless doesn't support writing to `null` values with this -operator: - -[source,painless] ---------------------------------------------------------- -TypeWithSetterOrPublicField foo = null; -foo?.x = 'bar' // Compile error ---------------------------------------------------------- diff --git a/docs/painless/painless-keywords.asciidoc b/docs/painless/painless-keywords.asciidoc new file mode 100644 index 0000000000000..99b5b4060d24e --- /dev/null +++ b/docs/painless/painless-keywords.asciidoc @@ -0,0 +1,13 @@ +[[painless-keywords]] +=== Keywords + +The keywords in the table below are reserved for built-in language +features. These keywords cannot be used as <> or +<>. + +[cols="^1,^1,^1,^1,^1"] +|==== +| if | else | while | do | for +| in | continue | break | return | new +| try | catch | throw | this | instanceof +|==== diff --git a/docs/painless/painless-lang-spec.asciidoc b/docs/painless/painless-lang-spec.asciidoc index 6544b0ad26495..b324ad301141c 100644 --- a/docs/painless/painless-lang-spec.asciidoc +++ b/docs/painless/painless-lang-spec.asciidoc @@ -1,73 +1,34 @@ -[[painless-specification]] +[[painless-lang-spec]] == Painless Language Specification -Painless uses a Java-style syntax that is similar to Groovy. In fact, most -Painless scripts are also valid Groovy, and simple Groovy scripts are typically -valid Painless. This specification assumes you have at least a passing -familiarity with Java and related languages. - -Painless is essentially a subset of Java with some additional scripting -language features that make scripts easier to write. However, there are some -important differences, particularly with the casting model. For more detailed +Painless is a scripting language designed for security and performance. +Painless syntax is similar to Java syntax along with some additional +features such as dynamic typing, Map and List accessor shortcuts, and array +initializers. As a direct comparison to Java, there are some important +differences, especially related to the casting model. For more detailed conceptual information about the basic constructs that Java and Painless share, refer to the corresponding topics in the https://docs.oracle.com/javase/specs/jls/se8/html/index.html[Java Language Specification]. Painless scripts are parsed and compiled using the http://www.antlr.org/[ANTLR4] -and http://asm.ow2.org/[ASM] libraries. Painless scripts are compiled directly -into Java byte code and executed against a standard Java Virtual Machine. This -specification uses ANTLR4 grammar notation to describe the allowed syntax. +and http://asm.ow2.org/[ASM] libraries. Scripts are compiled directly +into Java Virtual Machine (JVM) byte code and executed against a standard JVM. +This specification uses ANTLR4 grammar notation to describe the allowed syntax. However, the actual Painless grammar is more compact than what is shown here. -[float] -[[comments]] -==== Comments - -Painless supports both single-line and multi-line comments. You can include -comments anywhere within a script. - -Single-line comments are preceded by two slashes: `// comment`. They can be -placed anywhere on a line. All characters from the two slashes to the end of -the line are ignored. - -Multi-line comments are preceded by a slash-star `/*` and closed by -star-slash `*/`. Multi-line comments can start anywhere on a line. All -characters from the opening `/*` to the closing `*/` are ignored. - -*Examples:* - -[source,Java] ----- -// single-line comment - - // single-line comment - -/* multi- - line - comment */ +include::painless-comments.asciidoc[] - /* multi-line - comment */ +include::painless-keywords.asciidoc[] - /* multi-line comment */ ----- +include::painless-literals.asciidoc[] -[float] -[[keywords]] -==== Keywords +include::painless-variables.asciidoc[] -Painless reserves the following keywords for built-in language features. -These keywords cannot be used in other contexts, such as identifiers. +include::painless-types.asciidoc[] -[cols="^1,^1,^1,^1,^1"] -|==== -| if | else | while | do | for -| in | continue | break | return | new -| try | catch | throw | this | instanceof -|==== +include::painless-casting.asciidoc[] -include::painless-literals.asciidoc[] -include::painless-variables.asciidoc[] -include::painless-types.asciidoc[] include::painless-operators.asciidoc[] + +include::painless-general-syntax.asciidoc[] diff --git a/docs/painless/painless-literals.asciidoc b/docs/painless/painless-literals.asciidoc index 43c5eb82f96a2..3f91c9299c0ad 100644 --- a/docs/painless/painless-literals.asciidoc +++ b/docs/painless/painless-literals.asciidoc @@ -1,94 +1,143 @@ -[[literals]] +[[painless-literals]] === Literals -Literals are values that you can specify directly in Painless scripts. +Use literals to specify different types of values directly in a script. [[integers]] ==== Integers -Specify integer literals in decimal, octal, or hex notation. Use the following -single letter designations to specify the primitive type: `l` for `long`, `f` -for `float`, and `d` for `double`. If not specified, the type defaults to -`int` (with the exception of certain assignments described later). +Use integer literals to specify an integer value in decimal, octal, or hex +notation of the <> `int`, `long`, `float`, +or `double`. Use the following single letter designations to specify the +<>: `l` or `L` for `long`, `f` or `F` for +`float`, and `d` or `D` for `double`. If not specified, the type defaults to +`int`. Use `0` as a prefix to specify an integer literal as octal, and use +`0x` or `0X` as a prefix to specify an integer literal as hex. -*Grammar:* +*Grammar* [source,ANTLR4] ---- INTEGER: '-'? ( '0' | [1-9] [0-9]* ) [lLfFdD]?; -OCTAL: '-'? '0' [0-7]+ [lL]?; -HEX: '-'? '0' [xX] [0-9a-fA-F]+ [lL]?; +OCTAL: '-'? '0' [0-7]+ [lL]?; +HEX: '-'? '0' [xX] [0-9a-fA-F]+ [lL]?; ---- -*Examples:* -[source,Java] +*Examples* + +Integer literals. + +[source,Painless] ---- -0 // integer literal of 0 -0D // double literal of 0.0 -1234L // long literal of 1234 --90F // float literal of -90.0 --022 // integer literal of -18 specified in octal -0xF2A // integer literal of 3882 +0 <1> +0D <2> +1234L <3> +-90f <4> +-022 <5> +0xF2A <6> ---- -[[floating-point-values]] -==== Floating Point Values +<1> `int 0` +<2> `double 0.0` +<3> `long 1234` +<4> `float -90.0` +<5> `int -18` in octal +<6> `int 3882` in hex + +[[floats]] +==== Floats -Specify floating point literals using the following single letter designations -for the primitive type: `f` for `float` and `d` for `double`. -If not specified, the type defaults to `double`. +Use floating point literals to specify a floating point value of the +<> `float` or `double`. Use the following +single letter designations to specify the <>: +`f` or `F` for `float` and `d` or `D` for `double`. If not specified, the type defaults +to `double`. -*Grammar:* +*Grammar* [source,ANTLR4] ---- -DECIMAL: '-'? ( '0' | [1-9] [0-9]* ) (DOT [0-9]+)? ( [eE] [+\-]? [0-9]+ )? [fFdD]?; +DECIMAL: '-'? ( '0' | [1-9] [0-9]* ) (DOT [0-9]+)? EXPONENT? [fFdD]?; +EXPONENT: ( [eE] [+\-]? [0-9]+ ); ---- -*Examples:* -[source,Java] +*Examples* + +Floating point literals. + +[source,Painless] ---- -0.0 // double value of 0.0 -1E6 // double value of 1000000 -0.977777 // double value of 0.97777 --126.34 // double value of -126.34 -89.9F // float value of 89.9 +0.0 <1> +1E6 <2> +0.977777 <3> +-126.34 <4> +89.9F <5> ---- +<1> `double 0.0` +<2> `double 1000000.0` in exponent notation +<3> `double 0.977777` +<4> `double -126.34` +<5> `float 89.9` + [[strings]] ==== Strings -Specify literal string with either single or double quotes. In double-quoted -literal strings, you can escape double-quotes with a backslash to include them -in the string. Similarly, you escape single quotes with a backslash in -single-quoted literal strings. Backslashes themselves also need to be -escaped with a backslash. +Use string literals to specify string values of the +<> with either single-quotes or double-quotes. +Use a `\"` token to include a double-quote as part of a double-quoted string +literal. Use a `\'` token to include a single-quote as part of a single-quoted +string literal. Use a `\\` token to include a backslash as part of any string +literal. -*Grammar:* +*Grammar* [source,ANTLR4] ---- -STRING: ( '"' ( '\\"' | '\\\\' | ~[\\"] )*? '"' ) | ( '\'' ( '\\\'' | '\\\\' | ~[\\'] )*? '\'' ); +STRING: ( '"' ( '\\"' | '\\\\' | ~[\\"] )*? '"' ) + | ( '\'' ( '\\\'' | '\\\\' | ~[\\'] )*? '\'' ); ---- -*Examples:* -[source,Java] +*Examples* + +String literals using single-quotes. + +[source,Painless] ---- -"double-quoted String literal" -'single-quoted String literal' -"\"double-quoted String with escaped double-quotes\" and backslash: \\" -'\'single-quoted String with escaped single-quotes\' and backslash \\' -"double-quoted String with non-escaped 'single-quotes'" -'single-quoted String with non-escaped "double-quotes"' +'single-quoted string literal' +'\'single-quoted string with escaped single-quotes\' and backslash \\' +'single-quoted string with non-escaped "double-quotes"' ---- -[[char]] -===== Char +String literals using double-quotes. -You cannot directly specify character literals in Painless. However, you can -cast single-character strings to char. Attempting to cast a multi-character -string to a char throws an error. +[source,Painless] +---- +"double-quoted string literal" +"\"double-quoted string with escaped double-quotes\" and backslash: \\" +"double-quoted string with non-escaped 'single-quotes'" +---- -*Examples:* -[source,Java] +[[characters]] +==== Characters + +Use the <> to convert string literals or +<> values into <> values. +<> values converted into +<> values must be exactly one character in length +or an error will occur. + +*Examples* + +Casting string literals into <> values. + +[source,Painless] ---- (char)"C" (char)'c' ----- \ No newline at end of file +---- + +Casting a <> value into a <> value. + +[source,Painless] +---- +String s = "s"; +char c = (char)s; +---- diff --git a/docs/painless/painless-operators.asciidoc b/docs/painless/painless-operators.asciidoc index 0d5135022ad90..11ee97def66ba 100644 --- a/docs/painless/painless-operators.asciidoc +++ b/docs/painless/painless-operators.asciidoc @@ -1,3 +1,4 @@ +[[painless-operators]] === Operators The following is a table of the available operators in Painless. Each operator will have further information and examples outside of the table. Many operators will have a promotion table as described by the documentation on promotion [MARK]. diff --git a/docs/painless/painless-types.asciidoc b/docs/painless/painless-types.asciidoc index 9e5077503b4a8..9d575a2069ae3 100644 --- a/docs/painless/painless-types.asciidoc +++ b/docs/painless/painless-types.asciidoc @@ -1,5 +1,5 @@ -[[types]] -=== Data Types +[[painless-types]] +=== Types Painless supports both dynamic and static types. Static types are split into _primitive types_ and _reference types_. @@ -267,176 +267,3 @@ def[] da = new def[] {i, l, f*d, s}; // Declare def array da and set it to // a def array with a size of 4 and the // values i, l, f*d, and s ---- - -[[casting]] -=== Casting - -Casting is the conversion of one type to another. Implicit casts are casts that -occur automatically, such as during an assignment operation. Explicit casts are -casts where you use the casting operator to explicitly convert one type to -another. This is necessary during operations where the cast cannot be inferred. - -To cast to a new type, precede the expression by the new type enclosed in -parentheses, for example -`(int)x`. - -The following sections specify the implicit casts that can be performed and the -explicit casts that are allowed. The only other permitted cast is casting -a single character `String` to a `char`. - -*Grammar:* -[source,ANTLR4] ----- -cast: '(' TYPE ')' expression ----- - -[[numeric-casting]] -==== Numeric Casting - -The following table shows the allowed implicit and explicit casts between -numeric types. Read the table by row. To find out if you need to explicitly -cast from type A to type B, find the row for type A and scan across to the -column for type B. - -IMPORTANT: Explicit casts between numeric types can result in some data loss. A -smaller numeric type cannot necessarily accommodate the value from a larger -numeric type. You might also lose precision when casting from integer types -to floating point types. - -|==== -| | byte | short | char | int | long | float | double -| byte | | implicit | implicit | implicit | implicit | implicit | implicit -| short | explicit | | explicit | implicit | implicit | implicit | implicit -| char | explicit | explicit | | implicit | implicit | implicit | implicit -| int | explicit | explicit | explicit | | implicit | implicit | implicit -| long | explicit | explicit | explicit | explicit | | implicit | implicit -| float | explicit | explicit | explicit | explicit | explicit | | implicit -| double | explicit | explicit | explicit | explicit | explicit | explicit | -|==== - - -Example(s) -[source,Java] ----- -int a = 1; // Declare int variable a and set it to the literal - // value 1 -long b = a; // Declare long variable b and set it to int variable - // a with an implicit cast to convert from int to long -short c = (short)b; // Declare short variable c, explicitly cast b to a - // short, and assign b to c -byte d = a; // ERROR: Casting an int to a byte requires an explicit - // cast -double e = (double)a; // Explicitly cast int variable a to a double and assign - // it to the double variable e. The explicit cast is - // allowed, but it is not necessary. ----- - -[[reference-casting]] -==== Reference Casting - -A reference type can be implicitly cast to another reference type as long as -the type being cast _from_ is a descendant of the type being cast _to_. A -reference type can be explicitly cast _to_ if the type being cast to is a -descendant of the type being cast _from_. - -*Examples:* -[source,Java] ----- -List x; // Declare List variable x -ArrayList y = new ArrayList(); // Declare ArrayList variable y and assign it a - // newly allocated ArrayList [1] -x = y; // Assign Arraylist y to List x using an - // implicit cast -y = (ArrayList)x; // Explicitly cast List x to an ArrayList and - // assign it to ArrayList y -x = (List)y; // Set List x to ArrayList y using an explicit - // cast (the explicit cast is not necessary) -y = x; // ERROR: List x cannot be implicitly cast to - // an ArrayList, an explicit cast is required -Map m = y; // ERROR: Cannot implicitly or explicitly cast [2] - // an ArrayList to a Map, no relationship - // exists between the two types. ----- -[1] `ArrayList` is a descendant of the `List` type. -[2] `Map` is unrelated to the `List` and `ArrayList` types. - -[[def-type-casting]] -==== def Type Casting -All primitive and reference types can always be implicitly cast to -`def`. While it is possible to explicitly cast to `def`, it is not necessary. - -However, it is not always possible to implicitly cast a `def` to other -primitive and reference types. An explicit cast is required if an explicit -cast would normally be required between the non-def types. - - -*Examples:* -[source,Java] ----- -def x; // Declare def variable x and set it to null -x = 3; // Set the def variable x to the literal 3 with an implicit - // cast from int to def -double a = x; // Declare double variable a and set it to def variable x, - // which contains a double -int b = x; // ERROR: Results in a run-time error because an explicit cast is - // required to cast from a double to an int -int c = (int)x; // Declare int variable c, explicitly cast def variable x to an - // int, and assign x to c ----- - -[[boxing-unboxing]] -==== Boxing and Unboxing - -Boxing is where a cast is used to convert a primitive type to its corresponding -reference type. Unboxing is the reverse, converting a reference type to the -corresponding primitive type. - -There are two places Painless performs implicit boxing and unboxing: - -* When you call methods, Painless automatically boxes and unboxes arguments -so you can specify either primitive types or their corresponding reference -types. -* When you use the `def` type, Painless automatically boxes and unboxes as -needed when converting to and from `def`. - -The casting operator does not support any way to explicitly box a primitive -type or unbox a reference type. - -If a primitive type needs to be converted to a reference type, the Painless -reference type API supports methods that can do that. However, under normal -circumstances this should not be necessary. - -*Examples:* -[source,Java] ----- -Integer x = 1; // ERROR: not a legal implicit cast -Integer y = (Integer)1; // ERROR: not a legal explicit cast -int a = new Integer(1); // ERROR: not a legal implicit cast -int b = (int)new Integer(1); // ERROR: not a legal explicit cast ----- - -[[promotion]] -==== Promotion - -Promotion is where certain operations require types to be either a minimum -numerical type or for two (or more) types to be equivalent. -The documentation for each operation that has these requirements -includes promotion tables that describe how this is handled. - -When an operation promotes a type or types, the resultant type -of the operation is the promoted type. Types can be promoted to def -at compile-time; however, at run-time, the resultant type will be the -promotion of the types the `def` is representing. - -*Examples:* -[source,Java] ----- -2 + 2.0 // Add the literal int 2 and the literal double 2.0. The literal - // 2 is promoted to a double and the resulting value is a double. - -def x = 1; // Declare def variable x and set it to the literal int 1 through - // an implicit cast -x + 2.0F // Add def variable x and the literal float 2.0. - // At compile-time the types are promoted to def. - // At run-time the types are promoted to float. ----- diff --git a/docs/painless/painless-variables.asciidoc b/docs/painless/painless-variables.asciidoc index 2177b0bb91ba8..08725b328a3c2 100644 --- a/docs/painless/painless-variables.asciidoc +++ b/docs/painless/painless-variables.asciidoc @@ -1,15 +1,15 @@ -[[variables]] +[[painless-variables]] === Variables -Variables in Painless must be declared and can be statically or <>. +Variables in Painless must be declared and can be +statically or <>. -[[variable-identifiers]] -==== Variable Identifiers +[[identifiers]] +==== Identifiers Specify variable identifiers using the following grammar. Variable identifiers -must start with a letter or underscore. You cannot use <> or -<> as identifiers. +must start with a letter or underscore. You cannot use +<> or <> as identifiers. *Grammar:* [source,ANTLR4] @@ -20,7 +20,6 @@ ID: [_a-zA-Z] [_a-zA-Z-0-9]*; *Examples:* [source,Java] ---- -_ a Z id @@ -30,8 +29,8 @@ MAP25 _map25 ---- -[[variable-declaration]] -==== Variable Declaration +[[declaration]] +==== Declaration Variables must be declared before you use them. The format is `type-name identifier-name`. To declare multiple variables of the same type, specify a @@ -56,7 +55,7 @@ int i = 10; // Declare the int variable i and set it to the int literal 10 ---- [[variable-assignment]] -==== Variable Assignment +==== Assignment Use the equals operator (`=`) to assign a value to a variable. The format is `identifier-name = value`. Any value expression can be assigned to any variable @@ -80,7 +79,7 @@ int i;   // Declare an int i i = 10;  // Set the int i to the int literal 10 ---- -Immediately assigning a value when declaring a variable. +Immediately assigning a value when declaring a variable. [source,Java] ---- diff --git a/docs/reference/redirects.asciidoc b/docs/reference/redirects.asciidoc index a17027fb3c335..1583726421aeb 100644 --- a/docs/reference/redirects.asciidoc +++ b/docs/reference/redirects.asciidoc @@ -489,7 +489,7 @@ Using `_index` in scripts has been replaced with writing `ScriptEngine` backends === Painless Syntax See the -{painless}/painless-specification.html[Painless Language Specification] +{painless}/painless-lang-spec.html[Painless Language Specification] in the guide to the {painless}/index.html[Painless Scripting Language]. [role="exclude",id="modules-scripting-painless-debugging"] From 52858ba760bf49681cc09cc0e79cca59822d5811 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Tue, 17 Apr 2018 16:04:16 -0700 Subject: [PATCH 10/16] Fix the version ID for v5.6.10. (#29570) --- server/src/main/java/org/elasticsearch/Version.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index 5b0d85b4bfaa3..d8ef145a4a57f 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -117,7 +117,7 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_5_6_8 = new Version(V_5_6_8_ID, org.apache.lucene.util.Version.LUCENE_6_6_1); public static final int V_5_6_9_ID = 5060999; public static final Version V_5_6_9 = new Version(V_5_6_9_ID, org.apache.lucene.util.Version.LUCENE_6_6_1); - public static final int V_5_6_10_ID = 50601099; + public static final int V_5_6_10_ID = 5061099; public static final Version V_5_6_10 = new Version(V_5_6_10_ID, org.apache.lucene.util.Version.LUCENE_6_6_1); public static final int V_6_0_0_alpha1_ID = 6000001; public static final Version V_6_0_0_alpha1 = From c8209fa7b122457cfaa4b9d2f9635664ccace8e1 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Tue, 17 Apr 2018 19:27:02 -0700 Subject: [PATCH 11/16] Fix the assertion message for an incorrect current version. (#29572) --- .../org/elasticsearch/test/VersionUtils.java | 2 +- .../elasticsearch/test/VersionUtilsTests.java | 20 ++++++++++++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) 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 9fde8b66a1f96..766fc80ba5605 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/VersionUtils.java @@ -54,7 +54,7 @@ static Tuple, List> resolveReleasedVersions(Version curre Version last = versions.remove(versions.size() - 1); assert last.equals(current) : "The highest version must be the current one " - + "but was [" + versions.get(versions.size() - 1) + "] and current was [" + current + "]"; + + "but was [" + last + "] and current was [" + current + "]"; if (current.revision != 0) { /* If we are in a stable branch there should be no unreleased version constants diff --git a/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java b/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java index 67a9a40f0fc1f..3c8b349792b75 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/VersionUtilsTests.java @@ -28,9 +28,9 @@ import java.util.LinkedHashSet; import java.util.List; -import static java.util.Collections.singletonList; import static java.util.stream.Collectors.toCollection; import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -305,6 +305,24 @@ public void testResolveReleasedVersionsAtNewMinorBranchIn6x() { TestNewMinorBranchIn6x.V_6_2_0))); } + public static class TestIncorrectCurrentVersion { + public static final Version V_5_3_0 = Version.fromString("5.3.0"); + public static final Version V_5_3_1 = Version.fromString("5.3.1"); + public static final Version V_5_4_0 = Version.fromString("5.4.0"); + public static final Version V_5_4_1 = Version.fromString("5.4.1"); + public static final Version CURRENT = V_5_4_1; + } + + public void testIncorrectCurrentVersion() { + Version previousVersion = TestIncorrectCurrentVersion.V_5_4_0; + AssertionError error = expectThrows(AssertionError.class, () -> + VersionUtils.resolveReleasedVersions(previousVersion, TestIncorrectCurrentVersion.class)); + + String message = error.getMessage(); + assertThat(message, containsString(TestIncorrectCurrentVersion.CURRENT.toString())); + assertThat(message, containsString(previousVersion.toString())); + } + /** * Tests that {@link Version#minimumCompatibilityVersion()} and {@link VersionUtils#allReleasedVersions()} * agree with the list of wire and index compatible versions we build in gradle. From 8b34066d8b1cb53f8876502b6cce81fdf111fd58 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 18 Apr 2018 10:54:45 +0200 Subject: [PATCH 12/16] Mutes failing MovAvgIT tests Relates #29456 --- .../search/aggregations/pipeline/moving/avg/MovAvgIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java index 695a5777a5302..43c7010d4b023 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/moving/avg/MovAvgIT.java @@ -69,6 +69,7 @@ import static org.hamcrest.core.IsNull.nullValue; @ESIntegTestCase.SuiteScopeTestCase +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/29456") public class MovAvgIT extends ESIntegTestCase { private static final String INTERVAL_FIELD = "l_value"; private static final String VALUE_FIELD = "v_value"; From a7c985797608b9d1b8f425ba1ca76ba635ff8427 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 18 Apr 2018 13:01:06 +0200 Subject: [PATCH 13/16] Fix binary doc values fetching in _search (#29567) Binary doc values are retrieved during the DocValueFetchSubPhase through an instance of ScriptDocValues. Since 6.0 ScriptDocValues instances are not allowed to reuse the object that they return (https://github.com/elastic/elasticsearch/issues/26775) but BinaryScriptDocValues doesn't follow this restriction and reuses instances of BytesRefBuilder among different documents. This results in `field` values assigned to the wrong document in the response. This commit fixes this issue by recreating the BytesRef for each value that needs to be returned. Fixes #29565 --- .../index/fielddata/ScriptDocValues.java | 16 ++++++-- .../fielddata/BinaryDVFieldDataTests.java | 37 ++++++++++--------- 2 files changed, 33 insertions(+), 20 deletions(-) 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 552ddbf9d616d..01f6bc192988c 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java @@ -633,7 +633,12 @@ public String get(int index) { public BytesRef getBytesValue() { if (size() > 0) { - return values[0].get(); + /** + * We need to make a copy here because {@link BinaryScriptDocValues} might reuse the + * returned value and the same instance might be used to + * return values from multiple documents. + **/ + return values[0].toBytesRef(); } else { return null; } @@ -658,14 +663,19 @@ public BytesRefs(SortedBinaryDocValues in) { @Override public BytesRef get(int index) { - return values[index].get(); + /** + * We need to make a copy here because {@link BinaryScriptDocValues} might reuse the + * returned value and the same instance might be used to + * return values from multiple documents. + **/ + return values[index].toBytesRef(); } public BytesRef getValue() { if (count == 0) { return new BytesRef(); } - return values[0].get(); + return values[0].toBytesRef(); } } diff --git a/server/src/test/java/org/elasticsearch/index/fielddata/BinaryDVFieldDataTests.java b/server/src/test/java/org/elasticsearch/index/fielddata/BinaryDVFieldDataTests.java index 7f407dd1c01d1..3b29d15bf3fb2 100644 --- a/server/src/test/java/org/elasticsearch/index/fielddata/BinaryDVFieldDataTests.java +++ b/server/src/test/java/org/elasticsearch/index/fielddata/BinaryDVFieldDataTests.java @@ -52,7 +52,6 @@ public void testDocValue() throws Exception { final DocumentMapper mapper = mapperService.documentMapperParser().parse("test", new CompressedXContent(mapping)); - List bytesList1 = new ArrayList<>(2); bytesList1.add(randomBytes()); bytesList1.add(randomBytes()); @@ -123,22 +122,26 @@ public void testDocValue() throws Exception { // Test whether ScriptDocValues.BytesRefs makes a deepcopy fieldData = indexFieldData.load(reader); ScriptDocValues scriptValues = fieldData.getScriptValues(); - scriptValues.setNextDocId(0); - assertEquals(2, scriptValues.size()); - assertEquals(bytesList1.get(0), scriptValues.get(0)); - assertEquals(bytesList1.get(1), scriptValues.get(1)); - - scriptValues.setNextDocId(1); - assertEquals(1, scriptValues.size()); - assertEquals(bytes1, scriptValues.get(0)); - - scriptValues.setNextDocId(2); - assertEquals(0, scriptValues.size()); - - scriptValues.setNextDocId(3); - assertEquals(2, scriptValues.size()); - assertEquals(bytesList2.get(0), scriptValues.get(0)); - assertEquals(bytesList2.get(1), scriptValues.get(1)); + Object[][] retValues = new BytesRef[4][0]; + for (int i = 0; i < 4; i++) { + scriptValues.setNextDocId(i); + retValues[i] = new BytesRef[scriptValues.size()]; + for (int j = 0; j < retValues[i].length; j++) { + retValues[i][j] = scriptValues.get(j); + } + } + assertEquals(2, retValues[0].length); + assertEquals(bytesList1.get(0), retValues[0][0]); + assertEquals(bytesList1.get(1), retValues[0][1]); + + assertEquals(1, retValues[1].length); + assertEquals(bytes1, retValues[1][0]); + + assertEquals(0, retValues[2].length); + + assertEquals(2, retValues[3].length); + assertEquals(bytesList2.get(0), retValues[3][0]); + assertEquals(bytesList2.get(1), retValues[3][1]); } private static BytesRef randomBytes() { From a548a7f2cb23bc36de66e36ed8e93337fb271bc0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 18 Apr 2018 08:43:43 -0400 Subject: [PATCH 14/16] Fix full cluster restart test recovery (#29545) The test was using a parameter on GET /_cluster/health that older nodes do not understand. Yet, we do no even need to make this call here, we can use ensure green for the index. --- .../org/elasticsearch/upgrades/FullClusterRestartIT.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index e50e0c45c8dc9..57c6ad7ff861f 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -687,8 +687,7 @@ public void testEmptyShard() throws IOException { * Tests recovery of an index with or without a translog and the * statistics we gather about that. */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/29544") - public void testRecovery() throws IOException { + public void testRecovery() throws Exception { int count; boolean shouldHaveTranslog; if (runningAgainstOldCluster) { @@ -701,7 +700,7 @@ public void testRecovery() throws IOException { indexRandomDocuments(count, true, true, i -> jsonBuilder().startObject().field("field", "value").endObject()); // make sure all recoveries are done - ensureNoInitializingShards(); + ensureGreen(index); // Explicitly flush so we're sure to have a bunch of documents in the Lucene index client().performRequest("POST", "/_flush"); if (shouldHaveTranslog) { From 9d11c7a6c152d41411927482e8b7151be0df705a Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 18 Apr 2018 15:13:24 +0200 Subject: [PATCH 15/16] Remove extra copy in ScriptDocValues.Strings This commit removes a BytesRef copy introduced in #29567 and not required. Relates #29567 --- .../index/fielddata/ScriptDocValues.java | 26 ++----------------- 1 file changed, 2 insertions(+), 24 deletions(-) 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 01f6bc192988c..3d07a0f87aa5e 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/ScriptDocValues.java @@ -631,28 +631,9 @@ public String get(int index) { return values[index].get().utf8ToString(); } - public BytesRef getBytesValue() { - if (size() > 0) { - /** - * We need to make a copy here because {@link BinaryScriptDocValues} might reuse the - * returned value and the same instance might be used to - * return values from multiple documents. - **/ - return values[0].toBytesRef(); - } else { - return null; - } - } - public String getValue() { - BytesRef value = getBytesValue(); - if (value == null) { - return null; - } else { - return value.utf8ToString(); - } + return count == 0 ? null : get(0); } - } public static final class BytesRefs extends BinaryScriptDocValues { @@ -672,10 +653,7 @@ public BytesRef get(int index) { } public BytesRef getValue() { - if (count == 0) { - return new BytesRef(); - } - return values[0].toBytesRef(); + return count == 0 ? new BytesRef() : get(0); } } From 2b47d67d9564b5fd6c15f20cca696879902147a6 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 18 Apr 2018 09:18:08 -0400 Subject: [PATCH 16/16] Remove the index thread pool (#29556) Now that single-document indexing requests are executed on the bulk thread pool the index thread pool is no longer needed. This commit removes this thread pool from Elasticsearch. --- docs/reference/cat/thread_pool.asciidoc | 2 - .../migration/migrate_7_0/settings.asciidoc | 10 ++++- docs/reference/modules/threadpool.asciidoc | 12 ++--- .../test/cat.thread_pool/10_basic.yml | 6 +-- .../action/delete/TransportDeleteAction.java | 2 +- .../action/index/TransportIndexAction.java | 2 +- .../action/update/TransportUpdateAction.java | 2 +- .../threadpool/ExecutorBuilder.java | 2 +- .../threadpool/FixedExecutorBuilder.java | 44 ++----------------- .../elasticsearch/threadpool/ThreadPool.java | 3 -- .../action/RejectionActionIT.java | 4 +- .../bulk/TransportBulkActionIngestTests.java | 2 +- .../ESIndexLevelReplicationTestCase.java | 2 +- .../index/shard/IndexShardTests.java | 24 +++++----- .../flush/SyncedFlushSingleNodeTests.java | 2 +- .../threadpool/FixedThreadPoolTests.java | 4 -- .../UpdateThreadPoolSettingsTests.java | 21 ++------- 17 files changed, 42 insertions(+), 102 deletions(-) diff --git a/docs/reference/cat/thread_pool.asciidoc b/docs/reference/cat/thread_pool.asciidoc index 63c4a3939e763..a5c71ac271b8d 100644 --- a/docs/reference/cat/thread_pool.asciidoc +++ b/docs/reference/cat/thread_pool.asciidoc @@ -22,7 +22,6 @@ node-0 flush 0 0 0 node-0 force_merge 0 0 0 node-0 generic 0 0 0 node-0 get 0 0 0 -node-0 index 0 0 0 node-0 listener 0 0 0 node-0 management 1 0 0 node-0 refresh 0 0 0 @@ -52,7 +51,6 @@ flush force_merge generic get -index listener management refresh diff --git a/docs/reference/migration/migrate_7_0/settings.asciidoc b/docs/reference/migration/migrate_7_0/settings.asciidoc index 1035bc73393ac..1556056337b37 100644 --- a/docs/reference/migration/migrate_7_0/settings.asciidoc +++ b/docs/reference/migration/migrate_7_0/settings.asciidoc @@ -5,4 +5,12 @@ ==== Percolator * The deprecated `index.percolator.map_unmapped_fields_as_string` setting has been removed in favour of - the `index.percolator.map_unmapped_fields_as_text` setting. \ No newline at end of file + the `index.percolator.map_unmapped_fields_as_text` setting. + +==== Index thread pool + +* Internally, single-document index/delete/update requests are executed as bulk + requests with a single-document payload. This means that these requests are + executed on the bulk thread pool. As such, the indexing thread pool is no + longer needed and has been removed. As such, the settings + `thread_pool.index.size` and `thread_pool.index.queue_size` have been removed. \ No newline at end of file diff --git a/docs/reference/modules/threadpool.asciidoc b/docs/reference/modules/threadpool.asciidoc index fa8522ea1cbb0..b85cda1aa3685 100644 --- a/docs/reference/modules/threadpool.asciidoc +++ b/docs/reference/modules/threadpool.asciidoc @@ -13,12 +13,6 @@ There are several thread pools, but the important ones include: For generic operations (e.g., background node discovery). Thread pool type is `scaling`. -`index`:: - For index/delete operations. Thread pool type is `fixed` - with a size of `# of available processors`, - queue_size of `200`. The maximum size for this pool - is `1 + # of available processors`. - `search`:: For count/search/suggest operations. Thread pool type is `fixed_auto_queue_size` with a size of @@ -55,13 +49,13 @@ There are several thread pools, but the important ones include: Mainly for java client executing of action when listener threaded is set to true. Thread pool type is `scaling` with a default max of `min(10, (# of available processors)/2)`. -Changing a specific thread pool can be done by setting its type-specific parameters; for example, changing the `index` +Changing a specific thread pool can be done by setting its type-specific parameters; for example, changing the `bulk` thread pool to have more threads: [source,yaml] -------------------------------------------------- thread_pool: - index: + bulk: size: 30 -------------------------------------------------- @@ -89,7 +83,7 @@ full, it will abort the request. [source,yaml] -------------------------------------------------- thread_pool: - index: + bulk: size: 30 queue_size: 1000 -------------------------------------------------- diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml index bb16ae391c46d..d7d33c15ec18a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml @@ -33,7 +33,7 @@ - do: cat.thread_pool: - thread_pool_patterns: bulk,management,flush,index,generic,force_merge + thread_pool_patterns: bulk,management,flush,generic,force_merge h: id,name,active v: true @@ -44,7 +44,6 @@ \S+\s+ flush \s+ \d+ \n \S+\s+ force_merge \s+ \d+ \n \S+\s+ generic \s+ \d+ \n - \S+\s+ index \s+ \d+ \n \S+\s+ management \s+ \d+ \n)+ $/ - do: @@ -72,12 +71,11 @@ - do: cat.thread_pool: - thread_pool_patterns: bulk,index,search + thread_pool_patterns: bulk,search size: "" - match: $body: | / #node_name name active queue rejected ^ (\S+ \s+ bulk \s+ \d+ \s+ \d+ \s+ \d+ \n - \S+ \s+ index \s+ \d+ \s+ \d+ \s+ \d+ \n \S+ \s+ search \s+ \d+ \s+ \d+ \s+ \d+ \n)+ $/ diff --git a/server/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/server/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index 3aaf4a472facf..89ed24c573a25 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/server/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -46,7 +46,7 @@ public TransportDeleteAction(Settings settings, TransportService transportServic ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TransportBulkAction bulkAction, TransportShardBulkAction shardBulkAction) { super(settings, DeleteAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, - actionFilters, indexNameExpressionResolver, DeleteRequest::new, DeleteRequest::new, ThreadPool.Names.INDEX, + actionFilters, indexNameExpressionResolver, DeleteRequest::new, DeleteRequest::new, ThreadPool.Names.BULK, bulkAction, shardBulkAction); } diff --git a/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 88a210c718019..deeb179221994 100644 --- a/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -54,7 +54,7 @@ public TransportIndexAction(Settings settings, TransportService transportService ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TransportBulkAction bulkAction, TransportShardBulkAction shardBulkAction) { super(settings, IndexAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, - actionFilters, indexNameExpressionResolver, IndexRequest::new, IndexRequest::new, ThreadPool.Names.INDEX, + actionFilters, indexNameExpressionResolver, IndexRequest::new, IndexRequest::new, ThreadPool.Names.BULK, bulkAction, shardBulkAction); } diff --git a/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index 242dfe635ec91..8428c85cdb62f 100644 --- a/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -86,7 +86,7 @@ public TransportUpdateAction(Settings settings, ThreadPool threadPool, ClusterSe @Override protected String executor() { - return ThreadPool.Names.INDEX; + return ThreadPool.Names.BULK; } @Override diff --git a/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java index 314eb1df71a4b..5404e7ac3defb 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java @@ -48,7 +48,7 @@ protected static String settingsKey(final String prefix, final String key) { } protected int applyHardSizeLimit(final Settings settings, final String name) { - if (name.equals(ThreadPool.Names.BULK) || name.equals(ThreadPool.Names.INDEX)) { + if (name.equals(ThreadPool.Names.BULK)) { return 1 + EsExecutors.numberOfProcessors(settings); } else { return Integer.MAX_VALUE; diff --git a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java index 94db6cb64e2c8..43da1044c6bd0 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java +++ b/server/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java @@ -49,20 +49,7 @@ public final class FixedExecutorBuilder extends ExecutorBuilder( sizeKey, s -> Integer.toString(size), s -> Setting.parseInt(s, 1, applyHardSizeLimit(settings, name), sizeKey), - properties); + Setting.Property.NodeScope); final String queueSizeKey = settingsKey(prefix, "queue_size"); - this.queueSizeSetting = Setting.intSetting(queueSizeKey, queueSize, properties); + this.queueSizeSetting = Setting.intSetting(queueSizeKey, queueSize, Setting.Property.NodeScope); } @Override diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 4a172e9926911..c238fb45ad54a 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -69,7 +69,6 @@ public static class Names { public static final String LISTENER = "listener"; public static final String GET = "get"; public static final String ANALYZE = "analyze"; - public static final String INDEX = "index"; public static final String BULK = "bulk"; public static final String SEARCH = "search"; public static final String MANAGEMENT = "management"; @@ -126,7 +125,6 @@ public static ThreadPoolType fromType(String type) { map.put(Names.LISTENER, ThreadPoolType.FIXED); map.put(Names.GET, ThreadPoolType.FIXED); map.put(Names.ANALYZE, ThreadPoolType.FIXED); - map.put(Names.INDEX, ThreadPoolType.FIXED); map.put(Names.BULK, ThreadPoolType.FIXED); map.put(Names.SEARCH, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE); map.put(Names.MANAGEMENT, ThreadPoolType.SCALING); @@ -172,7 +170,6 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui final int halfProcMaxAt10 = halfNumberOfProcessorsMaxTen(availableProcessors); final int genericThreadPoolMax = boundedBy(4 * availableProcessors, 128, 512); builders.put(Names.GENERIC, new ScalingExecutorBuilder(Names.GENERIC, 4, genericThreadPoolMax, TimeValue.timeValueSeconds(30))); - builders.put(Names.INDEX, new FixedExecutorBuilder(settings, Names.INDEX, availableProcessors, 200, true)); builders.put(Names.BULK, new FixedExecutorBuilder(settings, Names.BULK, availableProcessors, 200)); // now that we reuse bulk for index/delete ops builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, availableProcessors, 1000)); builders.put(Names.ANALYZE, new FixedExecutorBuilder(settings, Names.ANALYZE, 1, 16)); diff --git a/server/src/test/java/org/elasticsearch/action/RejectionActionIT.java b/server/src/test/java/org/elasticsearch/action/RejectionActionIT.java index 0aa84ad806998..a4cdc3408fb27 100644 --- a/server/src/test/java/org/elasticsearch/action/RejectionActionIT.java +++ b/server/src/test/java/org/elasticsearch/action/RejectionActionIT.java @@ -45,8 +45,8 @@ protected Settings nodeSettings(int nodeOrdinal) { .put(super.nodeSettings(nodeOrdinal)) .put("thread_pool.search.size", 1) .put("thread_pool.search.queue_size", 1) - .put("thread_pool.index.size", 1) - .put("thread_pool.index.queue_size", 1) + .put("thread_pool.bulk.size", 1) + .put("thread_pool.bulk.queue_size", 1) .put("thread_pool.get.size", 1) .put("thread_pool.get.queue_size", 1) .build(); 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 32dfbe85d426e..5cd411c71b8c8 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -124,7 +124,7 @@ class TestSingleItemBulkWriteAction extends TransportSingleItemBulkWriteAction indexShard.acquireReplicaOperationPermit(indexShard.getPrimaryTerm() + randomIntBetween(1, 100), - SequenceNumbers.UNASSIGNED_SEQ_NO, null, ThreadPool.Names.INDEX, "")); + SequenceNumbers.UNASSIGNED_SEQ_NO, null, ThreadPool.Names.BULK, "")); closeShards(indexShard); } @@ -342,7 +342,7 @@ public void onFailure(Exception e) { throw new RuntimeException(e); } }, - ThreadPool.Names.INDEX, id); + ThreadPool.Names.BULK, id); }); thread.start(); threads.add(thread); @@ -393,7 +393,7 @@ public void onFailure(Exception e) { throw new RuntimeException(e); } }, - ThreadPool.Names.INDEX, id); + ThreadPool.Names.BULK, id); }); thread.start(); delayedThreads.add(thread); @@ -589,7 +589,7 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E assertEquals(0, indexShard.getActiveOperationsCount()); if (indexShard.routingEntry().isRelocationTarget() == false) { try { - indexShard.acquireReplicaOperationPermit(primaryTerm, indexShard.getGlobalCheckpoint(), null, ThreadPool.Names.INDEX, ""); + indexShard.acquireReplicaOperationPermit(primaryTerm, indexShard.getGlobalCheckpoint(), null, ThreadPool.Names.BULK, ""); fail("shard shouldn't accept operations as replica"); } catch (IllegalStateException ignored) { @@ -608,14 +608,14 @@ public void testOperationPermitsOnPrimaryShards() throws InterruptedException, E private Releasable acquirePrimaryOperationPermitBlockingly(IndexShard indexShard) throws ExecutionException, InterruptedException { PlainActionFuture fut = new PlainActionFuture<>(); - indexShard.acquirePrimaryOperationPermit(fut, ThreadPool.Names.INDEX, ""); + indexShard.acquirePrimaryOperationPermit(fut, ThreadPool.Names.BULK, ""); return fut.get(); } private Releasable acquireReplicaOperationPermitBlockingly(IndexShard indexShard, long opPrimaryTerm) throws ExecutionException, InterruptedException { PlainActionFuture fut = new PlainActionFuture<>(); - indexShard.acquireReplicaOperationPermit(opPrimaryTerm, indexShard.getGlobalCheckpoint(), fut, ThreadPool.Names.INDEX, ""); + indexShard.acquireReplicaOperationPermit(opPrimaryTerm, indexShard.getGlobalCheckpoint(), fut, ThreadPool.Names.BULK, ""); return fut.get(); } @@ -663,7 +663,7 @@ public void testOperationPermitOnReplicaShards() throws Exception { if (shardRouting.primary() == false) { final IllegalStateException e = expectThrows(IllegalStateException.class, - () -> indexShard.acquirePrimaryOperationPermit(null, ThreadPool.Names.INDEX, "")); + () -> indexShard.acquirePrimaryOperationPermit(null, ThreadPool.Names.BULK, "")); assertThat(e, hasToString(containsString("shard " + shardRouting + " is not a primary"))); } @@ -700,7 +700,7 @@ public void onFailure(Exception e) { }; indexShard.acquireReplicaOperationPermit(primaryTerm - 1, SequenceNumbers.UNASSIGNED_SEQ_NO, onLockAcquired, - ThreadPool.Names.INDEX, ""); + ThreadPool.Names.BULK, ""); assertFalse(onResponse.get()); assertTrue(onFailure.get()); @@ -1020,7 +1020,7 @@ public void onFailure(Exception e) { latch.countDown(); } }, - ThreadPool.Names.INDEX, ""); + ThreadPool.Names.BULK, ""); }; final long firstIncrement = 1 + (randomBoolean() ? 0 : 1); @@ -1381,7 +1381,7 @@ public void onResponse(Releasable releasable) { super.onResponse(releasable); } }; - shard.acquirePrimaryOperationPermit(onLockAcquired, ThreadPool.Names.INDEX, "i_" + i); + shard.acquirePrimaryOperationPermit(onLockAcquired, ThreadPool.Names.BULK, "i_" + i); onLockAcquiredActions.add(onLockAcquired); } diff --git a/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java b/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java index 6561001ad7d86..c71ccdfba8c89 100644 --- a/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java @@ -113,7 +113,7 @@ public void testSyncFailsIfOperationIsInFlight() throws InterruptedException, Ex SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); final ShardId shardId = shard.shardId(); PlainActionFuture fut = new PlainActionFuture<>(); - shard.acquirePrimaryOperationPermit(fut, ThreadPool.Names.INDEX, ""); + shard.acquirePrimaryOperationPermit(fut, ThreadPool.Names.BULK, ""); try (Releasable operationLock = fut.get()) { SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener<>(); flushService.attemptSyncedFlush(shardId, listener); diff --git a/server/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java b/server/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java index 09019f37655d1..5ec0f30f520b9 100644 --- a/server/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java +++ b/server/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java @@ -85,10 +85,6 @@ public void testRejectedExecutionCounter() throws InterruptedException { assertThat(counter, equalTo(rejections)); assertThat(stats(threadPool, threadPoolName).getRejected(), equalTo(rejections)); - - if (threadPoolName.equals(ThreadPool.Names.INDEX)) { - assertSettingDeprecationsAndWarnings(new String[]{"thread_pool.index.queue_size", "thread_pool.index.size"}); - } } finally { terminateThreadPoolIfNeeded(threadPool); } diff --git a/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java b/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java index f0f8c70a3f9d4..31142fe9e45bc 100644 --- a/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java @@ -60,8 +60,7 @@ public void testCorrectThreadPoolTypePermittedInSettings() throws InterruptedExc } } - public void testIndexingThreadPoolsMaxSize() throws InterruptedException { - final String name = randomFrom(Names.BULK, Names.INDEX); + public void testBulkThreadPoolsMaxSize() { final int maxSize = 1 + EsExecutors.numberOfProcessors(Settings.EMPTY); final int tooBig = randomIntBetween(1 + maxSize, Integer.MAX_VALUE); @@ -74,7 +73,7 @@ public void testIndexingThreadPoolsMaxSize() throws InterruptedException { try { tp = new ThreadPool(Settings.builder() .put("node.name", "testIndexingThreadPoolsMaxSize") - .put("thread_pool." + name + ".size", tooBig) + .put("thread_pool." + Names.BULK + ".size", tooBig) .build()); } finally { terminateThreadPoolIfNeeded(tp); @@ -84,15 +83,11 @@ public void testIndexingThreadPoolsMaxSize() throws InterruptedException { assertThat( initial, hasToString(containsString( - "Failed to parse value [" + tooBig + "] for setting [thread_pool." + name + ".size] must be "))); - - if (name.equals(Names.INDEX)) { - assertSettingDeprecationsAndWarnings(new String[] { "thread_pool.index.size" }); - } + "Failed to parse value [" + tooBig + "] for setting [thread_pool." + Names.BULK + ".size] must be "))); } private static int getExpectedThreadPoolSize(Settings settings, String name, int size) { - if (name.equals(ThreadPool.Names.BULK) || name.equals(ThreadPool.Names.INDEX)) { + if (name.equals(ThreadPool.Names.BULK)) { return Math.min(size, EsExecutors.numberOfProcessors(settings)); } else { return size; @@ -120,10 +115,6 @@ public void testFixedExecutorType() throws InterruptedException { assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize)); // keep alive does not apply to fixed thread pools assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getKeepAliveTime(TimeUnit.MINUTES), equalTo(0L)); - - if (threadPoolName.equals(Names.INDEX)) { - assertSettingDeprecationsAndWarnings(new String[] { "thread_pool.index.size" }); - } } finally { terminateThreadPoolIfNeeded(threadPool); } @@ -179,10 +170,6 @@ public void testShutdownNowInterrupts() throws Exception { latch.await(3, TimeUnit.SECONDS); // if this throws then ThreadPool#shutdownNow did not interrupt assertThat(oldExecutor.isShutdown(), equalTo(true)); assertThat(oldExecutor.isTerminating() || oldExecutor.isTerminated(), equalTo(true)); - - if (threadPoolName.equals(Names.INDEX)) { - assertSettingDeprecationsAndWarnings(new String[] { "thread_pool.index.queue_size" }); - } } finally { terminateThreadPoolIfNeeded(threadPool); }